From 1c1984514f1e41496f21acbd4269e2bee7b05a06 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Fri, 17 Oct 2025 14:37:24 +0200 Subject: [PATCH 1/6] Initial commit --- .../pyspark/sql/connect/functions/builtin.py | 80 ++++- python/pyspark/sql/functions/builtin.py | 120 ++++++-- python/pyspark/sql/tests/test_functions.py | 277 ++++++++++++++++++ 3 files changed, 450 insertions(+), 27 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 2668b7a526fdd..1f58a7b9835ac 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -3945,6 +3945,7 @@ def make_time(hour: "ColumnOrName", minute: "ColumnOrName", second: "ColumnOrNam make_time.__doc__ = pysparkfuncs.make_time.__doc__ +@overload def make_timestamp( years: "ColumnOrName", months: "ColumnOrName", @@ -3952,16 +3953,79 @@ def make_timestamp( hours: "ColumnOrName", mins: "ColumnOrName", secs: "ColumnOrName", - timezone: Optional["ColumnOrName"] = None, + timezone: Optional["ColumnOrName"] = None ) -> Column: - if timezone is not None: - return _invoke_function_over_columns( - "make_timestamp", years, months, days, hours, mins, secs, timezone - ) + ... + + +@overload +def make_timestamp( + *, + date: "ColumnOrName", + time: "ColumnOrName", + timezone: Optional["ColumnOrName"] = None +) -> Column: + ... + + +def 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, + *, + date: Optional["ColumnOrName"] = None, + time: Optional["ColumnOrName"] = None, + timezone: Optional["ColumnOrName"] = None +) -> Column: + 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( + "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( + "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( - "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( + "make_timestamp", + cast("ColumnOrName", date), + cast("ColumnOrName", time), + cast("ColumnOrName", timezone) + ) + else: + return _invoke_function_over_columns( + "make_timestamp", + cast("ColumnOrName", date), + cast("ColumnOrName", time) + ) make_timestamp.__doc__ = pysparkfuncs.make_timestamp.__doc__ diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 99fb7ee1ec5e8..c6e33ddd302f7 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -24790,7 +24790,7 @@ def make_time(hour: "ColumnOrName", minute: "ColumnOrName", second: "ColumnOrNam return _invoke_function_over_columns("make_time", hour, minute, second) -@_try_remote_functions +@overload def make_timestamp( years: "ColumnOrName", months: "ColumnOrName", @@ -24798,33 +24798,82 @@ def make_timestamp( hours: "ColumnOrName", mins: "ColumnOrName", secs: "ColumnOrName", - timezone: Optional["ColumnOrName"] = None, + timezone: Optional["ColumnOrName"] = None +) -> Column: + ... + + +@overload +def make_timestamp( + *, + date: "ColumnOrName", + time: "ColumnOrName", + timezone: Optional["ColumnOrName"] = None +) -> Column: + ... + + +def 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, + *, + date: Optional["ColumnOrName"] = None, + time: Optional["ColumnOrName"] = None, + timezone: Optional["ColumnOrName"] = None ) -> Column: """ - Create timestamp from years, months, days, hours, mins, secs and timezone fields. + Create timestamp from years, months, days, hours, mins, secs, and timezone fields. + Alternatively, create timestamp from date, time, 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. .. versionadded:: 3.5.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. + 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. timezone : :class:`~pyspark.sql.Column` or column name, optional The time zone identifier. For example, CET, UTC and etc. @@ -24848,7 +24897,7 @@ def 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']], @@ -24862,11 +24911,11 @@ def 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']], - ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887]], + ... ['year', 'month', 'day', 'hour', 'min', 'sec']) >>> df.select( ... sf.make_timestamp(df.year, df.month, df.day, 'hour', df.min, df.sec) ... ).show(truncate=False) @@ -24876,6 +24925,39 @@ def make_timestamp( |2014-12-28 06:30:45.887 | +------------------------------------------------+ + Example 3: 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.make_timestamp(date=df.date, time=df.time, timezone=df.tz) + ... ).show(truncate=False) + +------------------------------+ + |make_timestamp(date, time, tz)| + +------------------------------+ + |2014-12-27 21:30:45.887 | + +------------------------------+ + + Example 4: 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.make_timestamp(date=df.date, time=df.time)).show(truncate=False) + +--------------------------+ + |make_timestamp(date, time)| + +--------------------------+ + |2014-12-28 06:30:45.887 | + +--------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ if timezone is not None: diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 41c07a61eb1e3..08d6b813a7ced 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -849,6 +849,283 @@ def test_make_time(self): self.assertIsInstance(row_from_name[0], datetime.time) self.assertEqual(row_from_name[0], result) + def test_make_timestamp(self): + """Comprehensive test cases for make_timestamp with various arguments and edge cases.""" + + # 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, 1, 30) + result_frac_no_tz = datetime.datetime(2024, 5, 22, 10, 30, 45, 123000) + result_frac_with_tz = datetime.datetime(2024, 5, 22, 1, 30, 45, 123000) + + # Test 1A: Basic 6 positional arguments (years, months, days, hours, mins, secs). + actual = df.select( + F.make_timestamp(df.year, df.month, df.day, df.hour, df.minute, df.second) + ) + assertDataFrameEqual(actual, [Row(result_no_tz)]) + + # Test 1B: Basic 7 positional arguments (years, months, days, hours, mins, secs, timezone). + actual = df.select( + F.make_timestamp(df.year, df.month, df.day, df.hour, df.minute, df.second, df.timezone) + ) + assertDataFrameEqual(actual, [Row(result_with_tz)]) + + # Test 2A: Basic 6 keyword arguments (years, months, days, hours, mins, secs). + actual = df.select( + F.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.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.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.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.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.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.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.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.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.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.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: Edge case - Maximum valid positional argument values. + df_max = self.spark.createDataFrame( + [(2024, 12, 31, 23, 59, 59)], + ["year", "month", "day", "hour", "minute", "second"] + ) + expected_max = datetime.datetime(2024, 12, 31, 23, 59, 59) + actual = df_max.select( + F.make_timestamp( + df_max.year, df_max.month, df_max.day, df_max.hour, df_max.minute, df_max.second + ) + ) + assertDataFrameEqual(actual, [Row(expected_max)]) + + # Test 9: Edge case - Minimum valid positional argument values. + df_min = self.spark.createDataFrame( + [(1, 1, 1, 0, 0, 0)], ["year", "month", "day", "hour", "minute", "second"] + ) + expected_min = datetime.datetime(1, 1, 1, 0, 0, 0) + actual = df_min.select( + F.make_timestamp( + df_min.year, df_min.month, df_min.day, df_min.hour, df_min.minute, df_min.second + ) + ) + assertDataFrameEqual(actual, [Row(expected_min)]) + + # Test 10: Mixed positional and keyword (should work for valid combinations). + actual = df.select( + F.make_timestamp( + df.year, df.month, df.day, hours=df.hour, mins=df.minute, secs=df.second + ) + ) + assertDataFrameEqual(actual, [Row(result_no_tz)]) + + # Test 11A: Using literal values for positional arguments (without timezone). + actual = self.spark.range(1).select( + F.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 11B: Using literal values for positional arguments (with timezone). + actual = self.spark.range(1).select( + F.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 12A: Using literal values for date/time arguments (without timezone). + actual = self.spark.range(1).select( + F.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 12B: Using literal values for date/time arguments (with timezone). + actual = self.spark.range(1).select( + F.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 13: Mixing timestamp and date/time keyword arguments - should raise Exception. + with self.assertRaises(PySparkValueError) as context: + df_dt.select( + F.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.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 14: Incomplete keyword arguments - should raise Exception for None values. + with self.assertRaises(Exception): + F.make_timestamp(years=df.year) + with self.assertRaises(Exception): + F.make_timestamp(secs=df.second) + with self.assertRaises(Exception): + F.make_timestamp(years=df.year, months=df.month, days=df.day) + with self.assertRaises(Exception): + F.make_timestamp(days=df.day, timezone=df.timezone) + with self.assertRaises(Exception): + F.make_timestamp(hours=df.hour, mins=df.minute, secs=df.second, timezone=df.timezone) + with self.assertRaises(Exception): + F.make_timestamp(date=df_dt.date) + with self.assertRaises(Exception): + F.make_timestamp(time=df_dt.time, timezone=df_dt.timezone) + with self.assertRaises(Exception): + F.make_timestamp(timezone=df.timezone) + with self.assertRaises(Exception): + F.make_timestamp(timezone=df_dt.timezone) + def test_make_timestamp_ntz(self): """Comprehensive test cases for make_timestamp_ntz with various arguments and edge cases.""" From 34be4523f5265c040862d0e9f4ab1d732851b1ad Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Mon, 20 Oct 2025 13:25:05 +0200 Subject: [PATCH 2/6] Fix failing tests --- .../pyspark/sql/connect/functions/builtin.py | 36 +++++--- python/pyspark/sql/functions/builtin.py | 80 +++++++++++++--- python/pyspark/sql/tests/test_functions.py | 92 ++++++------------- 3 files changed, 116 insertions(+), 92 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 1f58a7b9835ac..06685bb0bb89a 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -3953,17 +3953,31 @@ def make_timestamp( hours: "ColumnOrName", mins: "ColumnOrName", secs: "ColumnOrName", - timezone: Optional["ColumnOrName"] = None ) -> Column: ... @overload def make_timestamp( - *, - date: "ColumnOrName", - time: "ColumnOrName", - timezone: Optional["ColumnOrName"] = None + years: "ColumnOrName", + months: "ColumnOrName", + days: "ColumnOrName", + hours: "ColumnOrName", + mins: "ColumnOrName", + secs: "ColumnOrName", + timezone: "ColumnOrName", +) -> Column: + ... + + +@overload +def make_timestamp(*, date: "ColumnOrName", time: "ColumnOrName") -> Column: + ... + + +@overload +def make_timestamp( + *, date: "ColumnOrName", time: "ColumnOrName", timezone: "ColumnOrName" ) -> Column: ... @@ -3975,10 +3989,10 @@ def make_timestamp( hours: Optional["ColumnOrName"] = None, mins: Optional["ColumnOrName"] = None, secs: Optional["ColumnOrName"] = None, + timezone: Optional["ColumnOrName"] = None, *, date: Optional["ColumnOrName"] = None, time: Optional["ColumnOrName"] = None, - timezone: Optional["ColumnOrName"] = None ) -> Column: if years is not None: if any(arg is not None for arg in [date, time]): @@ -3995,7 +4009,7 @@ def make_timestamp( cast("ColumnOrName", hours), cast("ColumnOrName", mins), cast("ColumnOrName", secs), - cast("ColumnOrName", timezone) + cast("ColumnOrName", timezone), ) else: return _invoke_function_over_columns( @@ -4005,7 +4019,7 @@ def make_timestamp( cast("ColumnOrName", days), cast("ColumnOrName", hours), cast("ColumnOrName", mins), - cast("ColumnOrName", secs) + cast("ColumnOrName", secs), ) else: if any(arg is not None for arg in [years, months, days, hours, mins, secs]): @@ -4018,13 +4032,11 @@ def make_timestamp( "make_timestamp", cast("ColumnOrName", date), cast("ColumnOrName", time), - cast("ColumnOrName", timezone) + cast("ColumnOrName", timezone), ) else: return _invoke_function_over_columns( - "make_timestamp", - cast("ColumnOrName", date), - cast("ColumnOrName", time) + "make_timestamp", cast("ColumnOrName", date), cast("ColumnOrName", time) ) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index c6e33ddd302f7..faa0a2c2c6b39 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -24798,17 +24798,31 @@ def make_timestamp( hours: "ColumnOrName", mins: "ColumnOrName", secs: "ColumnOrName", - timezone: Optional["ColumnOrName"] = None ) -> Column: ... @overload def make_timestamp( - *, - date: "ColumnOrName", - time: "ColumnOrName", - timezone: Optional["ColumnOrName"] = None + years: "ColumnOrName", + months: "ColumnOrName", + days: "ColumnOrName", + hours: "ColumnOrName", + mins: "ColumnOrName", + secs: "ColumnOrName", + timezone: "ColumnOrName", +) -> Column: + ... + + +@overload +def make_timestamp(*, date: "ColumnOrName", time: "ColumnOrName") -> Column: + ... + + +@overload +def make_timestamp( + *, date: "ColumnOrName", time: "ColumnOrName", timezone: "ColumnOrName" ) -> Column: ... @@ -24820,14 +24834,14 @@ def make_timestamp( hours: Optional["ColumnOrName"] = None, mins: Optional["ColumnOrName"] = None, secs: Optional["ColumnOrName"] = None, + timezone: Optional["ColumnOrName"] = None, *, date: Optional["ColumnOrName"] = None, time: Optional["ColumnOrName"] = None, - timezone: Optional["ColumnOrName"] = None ) -> Column: """ - Create timestamp from years, months, days, hours, mins, secs, and timezone fields. - Alternatively, create timestamp from date, time, and timezone fields. + Create timestamp from years, months, days, hours, mins, secs, and (optional) timezone fields. + Alternatively, create timestamp from date, time, and (optional) 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. @@ -24960,14 +24974,50 @@ def make_timestamp( >>> spark.conf.unset("spark.sql.session.timeZone") """ - if timezone is not None: - return _invoke_function_over_columns( - "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( + "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( + "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( - "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( + "make_timestamp", + cast("ColumnOrName", date), + cast("ColumnOrName", time), + cast("ColumnOrName", timezone), + ) + else: + return _invoke_function_over_columns( + "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 08d6b813a7ced..00cccb4b48f41 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -855,27 +855,27 @@ def test_make_timestamp(self): # 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"] + ["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"] + ["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") + 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") + 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, 1, 30) + result_with_tz = datetime.datetime(2024, 5, 22, 10, 30) result_frac_no_tz = datetime.datetime(2024, 5, 22, 10, 30, 45, 123000) - result_frac_with_tz = datetime.datetime(2024, 5, 22, 1, 30, 45, 123000) + result_frac_with_tz = datetime.datetime(2024, 5, 22, 10, 30, 45, 123000) # Test 1A: Basic 6 positional arguments (years, months, days, hours, mins, secs). actual = df.select( @@ -897,7 +897,7 @@ def test_make_timestamp(self): days=df.day, hours=df.hour, mins=df.minute, - secs=df.second + secs=df.second, ) ) assertDataFrameEqual(actual, [Row(result_no_tz)]) @@ -911,15 +911,13 @@ def test_make_timestamp(self): hours=df.hour, mins=df.minute, secs=df.second, - timezone=df.timezone + timezone=df.timezone, ) ) assertDataFrameEqual(actual, [Row(result_with_tz)]) # Test 3A: Alternative 2 keyword arguments (date, time). - actual = df_dt.select( - F.make_timestamp(date=df_dt.date, time=df_dt.time) - ) + actual = df_dt.select(F.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). @@ -936,7 +934,7 @@ def test_make_timestamp(self): df_frac.day, df_frac.hour, df_frac.minute, - df_frac.second + df_frac.second, ) ) assertDataFrameEqual(actual, [Row(result_frac_no_tz)]) @@ -950,7 +948,7 @@ def test_make_timestamp(self): df_frac.hour, df_frac.minute, df_frac.second, - df_frac.timezone + df_frac.timezone, ) ) assertDataFrameEqual(actual, [Row(result_frac_with_tz)]) @@ -963,7 +961,7 @@ def test_make_timestamp(self): days=df_frac.day, hours=df_frac.hour, mins=df_frac.minute, - secs=df_frac.second + secs=df_frac.second, ) ) assertDataFrameEqual(actual, [Row(result_frac_no_tz)]) @@ -977,34 +975,26 @@ def test_make_timestamp(self): hours=df_frac.hour, mins=df_frac.minute, secs=df_frac.second, - timezone=df_frac.timezone + 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.make_timestamp( - date=df_dt_frac.date, - time=df_dt_frac.time - ) - ) + actual = df_dt_frac.select(F.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.make_timestamp( - date=df_dt_frac.date, - time=df_dt_frac.time, - timezone=df_dt_frac.timezone + 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"] + [(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( @@ -1014,37 +1004,12 @@ def test_make_timestamp(self): df_leap.day, df_leap.hour, df_leap.minute, - df_leap.second + df_leap.second, ) ) assertDataFrameEqual(actual, [Row(expected_leap)]) - # Test 8: Edge case - Maximum valid positional argument values. - df_max = self.spark.createDataFrame( - [(2024, 12, 31, 23, 59, 59)], - ["year", "month", "day", "hour", "minute", "second"] - ) - expected_max = datetime.datetime(2024, 12, 31, 23, 59, 59) - actual = df_max.select( - F.make_timestamp( - df_max.year, df_max.month, df_max.day, df_max.hour, df_max.minute, df_max.second - ) - ) - assertDataFrameEqual(actual, [Row(expected_max)]) - - # Test 9: Edge case - Minimum valid positional argument values. - df_min = self.spark.createDataFrame( - [(1, 1, 1, 0, 0, 0)], ["year", "month", "day", "hour", "minute", "second"] - ) - expected_min = datetime.datetime(1, 1, 1, 0, 0, 0) - actual = df_min.select( - F.make_timestamp( - df_min.year, df_min.month, df_min.day, df_min.hour, df_min.minute, df_min.second - ) - ) - assertDataFrameEqual(actual, [Row(expected_min)]) - - # Test 10: Mixed positional and keyword (should work for valid combinations). + # Test 8: Mixed positional and keyword (should work for valid combinations). actual = df.select( F.make_timestamp( df.year, df.month, df.day, hours=df.hour, mins=df.minute, secs=df.second @@ -1052,15 +1017,13 @@ def test_make_timestamp(self): ) assertDataFrameEqual(actual, [Row(result_no_tz)]) - # Test 11A: Using literal values for positional arguments (without timezone). + # Test 9A: Using literal values for positional arguments (without timezone). actual = self.spark.range(1).select( - F.make_timestamp( - F.lit(2024), F.lit(5), F.lit(22), F.lit(10), F.lit(30), F.lit(0) - ) + F.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 11B: Using literal values for positional arguments (with timezone). + # Test 9B: Using literal values for positional arguments (with timezone). actual = self.spark.range(1).select( F.make_timestamp( F.lit(2024), F.lit(5), F.lit(22), F.lit(10), F.lit(30), F.lit(0), F.lit("CET") @@ -1068,28 +1031,27 @@ def test_make_timestamp(self): ) assertDataFrameEqual(actual, [Row(result_with_tz)]) - # Test 12A: Using literal values for date/time arguments (without timezone). + # Test 10A: Using literal values for date/time arguments (without timezone). actual = self.spark.range(1).select( F.make_timestamp( - date=F.lit(datetime.date(2024, 5, 22)), - time=F.lit(datetime.time(10, 30, 0)) + date=F.lit(datetime.date(2024, 5, 22)), time=F.lit(datetime.time(10, 30, 0)) ) ) assertDataFrameEqual(actual, [Row(result_no_tz)]) - # Test 12B: Using literal values for date/time arguments (with timezone). + # Test 10B: Using literal values for date/time arguments (with timezone). actual = self.spark.range(1).select( F.make_timestamp( date=F.lit(datetime.date(2024, 5, 22)), time=F.lit(datetime.time(10, 30, 0)), - timezone=F.lit("CET") + timezone=F.lit("CET"), ) ) assertDataFrameEqual(actual, [Row(result_with_tz)]) # Error handling tests. - # Test 13: Mixing timestamp and date/time keyword arguments - should raise Exception. + # Test 11: Mixing timestamp and date/time keyword arguments - should raise Exception. with self.assertRaises(PySparkValueError) as context: df_dt.select( F.make_timestamp(years=df.year, date=df_dt.date, time=df_dt.time) @@ -1106,7 +1068,7 @@ def test_make_timestamp(self): self.assertIn("CANNOT_SET_TOGETHER", error_msg) self.assertIn("years|months|days|hours|mins|secs and date|time", error_msg) - # Test 14: Incomplete keyword arguments - should raise Exception for None values. + # Test 12: Incomplete keyword arguments - should raise Exception for None values. with self.assertRaises(Exception): F.make_timestamp(years=df.year) with self.assertRaises(Exception): From 1a7ff086f06e6f5e6e5215a7f8872a717a22d462 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Mon, 20 Oct 2025 18:23:33 +0200 Subject: [PATCH 3/6] Fix tz results --- python/pyspark/sql/tests/test_functions.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 00cccb4b48f41..546f09996cdf2 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -873,9 +873,9 @@ def test_make_timestamp(self): ) # 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, 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, 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( From 1c86c06c98ee3348a58d99fa29723d88b2aed0ff Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Tue, 21 Oct 2025 14:38:59 +0200 Subject: [PATCH 4/6] Use positional args --- python/pyspark/sql/functions/builtin.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index faa0a2c2c6b39..1064bae05996e 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -24816,14 +24816,12 @@ def make_timestamp( @overload -def make_timestamp(*, date: "ColumnOrName", time: "ColumnOrName") -> Column: +def make_timestamp(date: "ColumnOrName", time: "ColumnOrName") -> Column: ... @overload -def make_timestamp( - *, date: "ColumnOrName", time: "ColumnOrName", timezone: "ColumnOrName" -) -> Column: +def make_timestamp(date: "ColumnOrName", time: "ColumnOrName", timezone: "ColumnOrName") -> Column: ... @@ -24835,7 +24833,6 @@ def make_timestamp( mins: Optional["ColumnOrName"] = None, secs: Optional["ColumnOrName"] = None, timezone: Optional["ColumnOrName"] = None, - *, date: Optional["ColumnOrName"] = None, time: Optional["ColumnOrName"] = None, ) -> Column: @@ -24880,6 +24877,8 @@ def make_timestamp( 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. 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. @@ -24888,8 +24887,6 @@ def make_timestamp( The time to represent, in valid TIME format. Required when creating timestamps from date and time components. Must be used with date parameter only. - timezone : :class:`~pyspark.sql.Column` or column name, optional - The time zone identifier. For example, CET, UTC and etc. Returns ------- From 775c8b0a56a5b87233c5ab64766d31b1f7ba5df8 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Wed, 22 Oct 2025 08:36:59 +0200 Subject: [PATCH 5/6] Fix Python linter issues --- python/pyspark/sql/connect/functions/builtin.py | 1 - python/pyspark/sql/functions/builtin.py | 6 ++++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 06685bb0bb89a..37c87a4d01c7c 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -3990,7 +3990,6 @@ def make_timestamp( mins: Optional["ColumnOrName"] = None, secs: Optional["ColumnOrName"] = None, timezone: Optional["ColumnOrName"] = None, - *, date: Optional["ColumnOrName"] = None, time: Optional["ColumnOrName"] = None, ) -> Column: diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 1064bae05996e..9315fbcb7a2f9 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -24816,12 +24816,14 @@ def make_timestamp( @overload -def make_timestamp(date: "ColumnOrName", time: "ColumnOrName") -> Column: +def make_timestamp(*, date: "ColumnOrName", time: "ColumnOrName") -> Column: ... @overload -def make_timestamp(date: "ColumnOrName", time: "ColumnOrName", timezone: "ColumnOrName") -> Column: +def make_timestamp( + *, date: "ColumnOrName", time: "ColumnOrName", timezone: "ColumnOrName" +) -> Column: ... From ce85a40f918f2cb7d1dae1bb17b2c53f08d99b75 Mon Sep 17 00:00:00 2001 From: Uros Bojanic Date: Wed, 22 Oct 2025 14:29:44 +0200 Subject: [PATCH 6/6] Fix function def --- python/pyspark/sql/functions/builtin.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 9315fbcb7a2f9..61a950dd4b85c 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -24827,6 +24827,7 @@ def make_timestamp( ... +@_try_remote_functions def make_timestamp( years: Optional["ColumnOrName"] = None, months: Optional["ColumnOrName"] = None,