Skip to content

Conversation

@gaogaotiantian
Copy link
Contributor

@gaogaotiantian gaogaotiantian commented Nov 10, 2025

What changes were proposed in this pull request?

We cache the tzinfo on local machine for timestamp conversion to avoid extra latency for calling datetime.datetime.fromtimestamp()

Why are the changes needed?

In Python, a forked process on Unix (that uses glibc I believe) will have a bad lock/cache state for timezone, which result in a extremely slow datetime.datetime.from_timestamp() (2000x slower on my machine).

Does this PR introduce any user-facing change?

No

How was this patch tested?

It was tested locally by hand to confirm the timestamp result is the same and the performance is normal.

Was this patch authored or co-authored using generative AI tooling?

No

@gaogaotiantian gaogaotiantian changed the title [SPARK-54285][PYTHON] Cache timezone info to avoid expensive timestamp conversion [WIP][SPARK-54285][PYTHON] Cache timezone info to avoid expensive timestamp conversion Nov 11, 2025
@@ -400,7 +406,9 @@ def toInternal(self, dt: datetime.datetime) -> int:
def fromInternal(self, ts: int) -> datetime.datetime:
if ts is not None:
# using int to avoid precision loss in float
return datetime.datetime.fromtimestamp(ts // 1000000).replace(microsecond=ts % 1000000)
return datetime.datetime.fromtimestamp(ts // 1000000, self.tz_info).replace(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it possible that the users change the timezone between calls of this method?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is WIP for now. I just realized that there are some timezone manipulations in our tests.

BTW, the timezone config does not work in workers - there's a ticket for it (https://issues.apache.org/jira/browse/SPARK-33863). We can use this mechanism and limit it in either worker process or the conversion itself.

envVars.put("SPARK_SIMPLIFIED_TRACEBACK", "1")
}
if (sessionLocalTimeZone.isDefined) {
envVars.put("TZ", sessionLocalTimeZone.get)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nah, let's don't mix local timezone with the system timezone. Both can be different. Session timezone should only be applied when operating time-related operations within Spark.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah sure, I'll use a different env var here and deal with it in worker process.

@gaogaotiantian
Copy link
Contributor Author

Okay the tests passed! I think this is ready for review now.

@gaogaotiantian gaogaotiantian changed the title [WIP][SPARK-54285][PYTHON] Cache timezone info to avoid expensive timestamp conversion [SPARK-54285][PYTHON] Cache timezone info to avoid expensive timestamp conversion Nov 12, 2025
conf.get(PYTHON_WORKER_TRACEBACK_DUMP_INTERVAL_SECONDS)
protected val hideTraceback: Boolean = false
protected val simplifiedTraceback: Boolean = false
protected val sessionLocalTimeZone = conf.getOption("spark.sql.session.timeZone")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm, this is a session config, so it's not available here unless it's specified when the session is created?

e.g.,

spark.conf.set('spark.sql.session.timeZone', 'UTC')

# .. run UDF including timestamp type

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yea and session conf is dynamic, I think every UDF execution (not per row, but per query) should pass the session conf to the python worker.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@gaogaotiantian We can use the same way as the other configs to get the runtime config, like hideTraceback or simplifiedTraceback above.
Also, could you add some tests with the config change?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so it's not available here unless it's specified when the session is created?

it seems SESSION_LOCAL_TIMEZONE has a default value depending on driver's timezone

val SESSION_LOCAL_TIMEZONE = buildConf(SqlApiConfHelper.SESSION_LOCAL_TIMEZONE_KEY)
.doc("The ID of session local timezone in the format of either region-based zone IDs or " +
"zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. " +
"Zone offsets must be in the format '(+|-)HH', '(+|-)HH:mm' or '(+|-)HH:mm:ss', e.g '-08', " +
"'+01:00' or '-13:33:33'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. Other " +
"short names are not recommended to use because they can be ambiguous.")
.version("2.2.0")
.stringConf
.checkValue(isValidTimezone, errorClass = "TIME_ZONE", parameters = tz => Map.empty)
.createWithDefaultFunction(() => TimeZone.getDefault.getID)

+1 to add new tests with different timezone, you may refer to

def test_time_zone_against_map_in_arrow(self):
import pyarrow as pa
for tz in [
"Asia/Shanghai",
"Asia/Hong_Kong",
"America/Los_Angeles",
"Pacific/Honolulu",
"Europe/Amsterdam",
"US/Pacific",
]:

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it seems SESSION_LOCAL_TIMEZONE has a default value depending on driver's timezone

Yes, but the value is only available in SQL conf, but this is a Spark conf, so even the default value is not available here.


tzname = os.environ.get("SPARK_SESSION_LOCAL_TIMEZONE", None)
if tzname is not None:
tz = zoneinfo.ZoneInfo(tzname)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure we can change the timezone to follow the session local timezone here. I'm worrying it could introduce a breaking change.

Should we just use datetime.datetime.now().astimezone().tzinfo if it's the same behavior as the current one?

Copy link
Member

@ueshin ueshin Nov 12, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, is this for SPARK-33863?
We need a flag to control whether to apply the session local timezone or not. Maybe in a separate PR?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So we need to consider whether this should be treated as a bug. I believe Pandas UDF now uses this conf directly:

timezone = runner_conf.get("spark.sql.session.timeZone", None)

It would be weird if we have different behaviors between pandas UDF and Python UDF right? If we consider this as a bug as mentioned in 33863, this PR is just a bug fix.

Copy link
Contributor

@zhengruifeng zhengruifeng Nov 13, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 for a flag to control this.
even if this is a bug comparing with pandas udf, existing workflows might already depend on it. We need a flag to restore old behavior.

# otherwise the forked process will be extremely slow to convert the timestamp.
# This is probably a glibc issue - the forked process will have a bad cache/lock
# status for the timezone info.
tz_info = None
Copy link
Contributor

@zhengruifeng zhengruifeng Nov 13, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it possible to make timezone an optional field in TimestampType?
if it is not set, then respect the config "spark.sql.session.timeZone"
@cloud-fan @HyukjinKwon

e.g. in pyarrow,

In [29]: pa.timestamp('us', tz='America/Los_Angeles')
Out[29]: TimestampType(timestamp[us, tz=America/Los_Angeles])

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The missing timezone causes a lot of confusion when we have to convert the timezone

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TIMESTAMP WITH TIMEZONE type may be added in the future

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So where is the value set from?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if TIMESTAMP WITH TIMEZONE is supported in the future, I think the TIMEZONE should be explicitly set by users, otherwise default to "spark.sql.session.timeZone"

@gaogaotiantian
Copy link
Contributor Author

gaogaotiantian commented Nov 13, 2025

So here's my thought for this change. I think this is a bug fix, because converting timestamp based on the cluster machine time zone does not even make sense. The task could be distributed to different machines which have different local timezones, there's no way the users can rely on the "existing behavior". The existing behavior will give inconsistent result for the same data, same udf, just because they are distributed to different machines - that's clearly a bug.

We should at least try to make all workers use the same timezone - right now there's no way to do that. With this change, all the workers will respect spark.sql.session.timeZone, which is a great improvement. But that's not the end of it, because without that config, the clusters will still use the local timezone. I think an acceptable fallback behavior is to use the timezone for the driver - which will at least give a consistent result.

As for the timestamp type itself, Python suggests against the naive timestamp (timestamp without a timezone). I think we should give user a warning when they use naive timestamp and encourage them to use either utc, or an aware timestamp.

I also root for having a timestamp type that supports timezone.

@HyukjinKwon
Copy link
Member

For now, we have been using naive datetimes so far, which matches with TimestampType concept without a timezone. I think we should probably just focus on fixing the perf issue in this PR instead of fixing other bugs together.

Another thing is that spark.sql.session.timeZone takes care about the timestamp diff only in Spark operators - it doesn't affect UDF executions. The same applies to Scala/Java UDF executions as well.

@HyukjinKwon
Copy link
Member

If we want to make a change affecting spark.sql.session.timeZone, we will have to revisit other places together. For example, for now, if you do spark.createDataFrame with datetime, the behaviour changes will happen because when you create, they are created based on local times, but it will be different when you execute Python UDFs.

@gaogaotiantian
Copy link
Contributor Author

gaogaotiantian commented Nov 13, 2025

So let me try to understand the current behavior - if I do a spark.createDataFrame() with a fixed datetime, or now(), I could end up with a dataframe that has all kinds of timestamps? Just because the task is distributed to clusters with different local timezones?

This is what the docs have:

TimestampType: Timestamp with local time zone(TIMESTAMP_LTZ). It represents values comprising values of fields year, month, day, hour, minute, and second, with the session local time-zone.

It should be session local right?

@cloud-fan
Copy link
Contributor

Spark has both TimestampType (with local timezone) and TimestampNTZType (no timezone). I'm not sure how we map these two types to python objects, but in JVM, we map them to java.sql.Timestamp and LocalDateTime.

That being said, we should never rely on the local machine timezone. We should either respect the session timezone (specified by spark.sql.session.timeZone and it has a default value if not set), or the python objects should be timezone agnostic.

@gaogaotiantian
Copy link
Contributor Author

That being said, we should never rely on the local machine timezone. We should either respect the session timezone (specified by spark.sql.session.timeZone and it has a default value if not set), or the python objects should be timezone agnostic.

I totally agree with this - that's the point I'm trying to make. Local machine timezone should never affect the result of user code.

Let's talk about timestamps. It's discouraged in Python to have a real datetime object without timezone - because for any operation, it would be treated as with local time zone. I believe the actual internal storage uses an integer timestamp. When Python tries to convert an integer timestamp to a datetime by datetime.datetime.fromtimestamp, it assumes the integer to be a POSIX timestamp, so it needs a timezone to convert to a datetime. There's no real "timezone agnostic datetime" in Python - Python will assume a datetime without a timezone is local machine timezone.

That being said, I think using UTC for TimestampNTZ is the correct implementation because Python will treat the integer timestamp as UTC, that should give the correct result.

However, for timestamp with timezone, we should use either session config, or at least a consistent value for all executors (driver timezone would be a good candidate, UTC is another option).

if tracebackDumpIntervalSeconds is not None and int(tracebackDumpIntervalSeconds) > 0:
faulthandler.dump_traceback_later(int(tracebackDumpIntervalSeconds), repeat=True)

tzname = os.environ.get("SPARK_SESSION_LOCAL_TIMEZONE", None)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if this PR only affect vanilla python udf, can we move this part into wrap_udf method?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not familiar with UDFs enough to answer this question. But is it possible that users use TimestampType.from_internal() in their udf? Or it's something we don't care. I think this only affects the cases where from_internal() is called.

# using int to avoid precision loss in float
return datetime.datetime.fromtimestamp(ts // 1000000).replace(microsecond=ts % 1000000)
return datetime.datetime.fromtimestamp(ts // 1000000, self.tz_info).replace(
microsecond=ts % 1000000, tzinfo=None
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a comment on why dropping the tzinfo here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Basically for backward compatibility. We are still working on the details of the behavior so this might not be the final implementation.

@gaogaotiantian
Copy link
Contributor Author

Let me remove all the bug-fix code. Let's merge the perf improvement part then discuss the timezone issue.

@zhengruifeng
Copy link
Contributor

@gaogaotiantian please update the PR description to remove timezone awareness part

sys.exit(-1)
start_faulthandler_periodic_traceback()

# Use the local timezone to convert the timestamp
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we add a TODO comment? I think it's wrong to use local timezone, but it's the current behavior today. We should revisit it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree it's wrong to use the local timezone. I'm thinking to start a PR very soon and we can discuss the details so we don't have to leave trace in our code. There is also a JIRA that's already opened for the issue.

@gaogaotiantian
Copy link
Contributor Author

@zhengruifeng done!

@zhengruifeng
Copy link
Contributor

merged to master

@gaogaotiantian gaogaotiantian deleted the fix-timestamp-convert branch November 20, 2025 01:40
@gaogaotiantian
Copy link
Contributor Author

I believe this is a MacOS specific issue - will that make this less interesting? I think most of our users run the code on Linux clusters. I confirmed that the timestamp conversion worked fine on Linux machine.

However, having this cached timezone can help with the implementation of using spark conf settings so it's not useless work. @cloud-fan do you want me to start a PR to use session conf in workers? I think we still have disagreements in the topic itself.

@cloud-fan
Copy link
Contributor

@gaogaotiantian yes please, let's have a PR and then we can have more concrete discussions.

@zhengruifeng
Copy link
Contributor

I believe this is a MacOS specific issue

@gaogaotiantian let's make a follow-up PR to mention this in related codes

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants