This is an automated email from the ASF dual-hosted git repository.

ruifengz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new a6d17f71f8d0 [SPARK-53956][PYTHON] Support TIME in the 
try_make_timestamp function in PySpark
a6d17f71f8d0 is described below

commit a6d17f71f8d039efb278c83eae57cc7364b349f9
Author: Uros Bojanic <[email protected]>
AuthorDate: Thu Oct 23 08:19:12 2025 +0800

    [SPARK-53956][PYTHON] Support TIME in the try_make_timestamp function in 
PySpark
    
    ### What changes were proposed in this pull request?
    Implement the support for TIME type in `try_make_timestamp` function in 
PySpark API.
    
    ### Why are the changes needed?
    Expand API support for the `TryMakeTimestamp` expression.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes, the new function is now available in PySpark API.
    
    ### How was this patch tested?
    Added appropriate Python functions tests and examples.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #52666 from uros-db/python-try-make_timestamp.
    
    Authored-by: Uros Bojanic <[email protected]>
    Signed-off-by: Ruifeng Zheng <[email protected]>
---
 python/pyspark/sql/connect/functions/builtin.py |  89 ++++++++-
 python/pyspark/sql/functions/builtin.py         | 181 +++++++++++++++---
 python/pyspark/sql/tests/test_functions.py      | 242 +++++++++++++++++++++++-
 3 files changed, 473 insertions(+), 39 deletions(-)

diff --git a/python/pyspark/sql/connect/functions/builtin.py 
b/python/pyspark/sql/connect/functions/builtin.py
index 2668b7a526fd..aadbfde97876 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 9d8492f19d16..543fc5c68ffe 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 41c07a61eb1e..b7479b7b7f2d 100644
--- a/python/pyspark/sql/tests/test_functions.py
+++ b/python/pyspark/sql/tests/test_functions.py
@@ -441,19 +441,247 @@ class FunctionsTestsMixin:
         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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to