[
https://issues.apache.org/jira/browse/SPARK-54372?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18043403#comment-18043403
]
Ashrith Bandla commented on SPARK-54372:
----------------------------------------
Hi, I investigated this issue and wanted to share my findings and get community
input before proceeding:
The behavior is intentional, it comes from Hive compatibility code in
`TypeCoercionHelper.scala`:
// Hive lets you do aggregation of timestamps... for some reason
case Sum(e @ TimestampTypeExpression(), _) => Sum(Cast(e, DoubleType))
case Average(e @ TimestampTypeExpression(), _) => Average(Cast(e,
DoubleType))When `avg(<timestamp>)` is called, Spark implicitly casts the
timestamp to `DoubleType` (epoch seconds since 1970-01-01 00:00:00 UTC), then
computes the average. For the example date '1969-12-21', this produces:
- Epoch seconds: -950400 (negative because it's before Unix epoch)
- Average of single value: -950400.0 (displayed as -979200.0 due to timezone)
Why This Is Tricky to Change
I prototyped a fix that would reject `avg(timestamp)` in ANSI mode (similar to
how we reject other type mismatches). However, this turned out to be a breaking
change:
1. Spark 4.0 has ANSI mode ON by default - This would affect most users
upgrading to Spark 4.x
2. Existing golden tests rely on this behavior - Multiple SQL test files
(`window.sql`, `type-coercion-edge-cases.sql`, `udf-window.sql`) explicitly
test and expect `avg(timestamp)` to return epoch seconds as `double`
...and many other reasons.
Options for the Community to Consider, other things I could do
1. Document the current behavior- Add clear documentation that `avg(timestamp)`
returns epoch seconds as double, which may be confusing for dates before 1970
2. Add a warning- Emit a warning when `avg(timestamp)` is used, suggesting
explicit conversion
3. Change return type to timestamp - Like DuckDB, make `avg(timestamp)` return
a timestamp (the "middle" timestamp). This is also a breaking change but
arguably more intuitive
4. Won't fix - Accept this as legacy Hive behavior that users should work
around with explicit conversions
My Recommendation
Given the scope of impact, I think this needs broader community discussion
before implementation. The behavior, while surprising, has been stable and
relied upon for years.
I'm happy to implement whichever approach the community prefers. What do
maintainers think is the right path forward?
> PySpark: incorrect `avg(<timestamp>)` query result
> --------------------------------------------------
>
> Key: SPARK-54372
> URL: https://issues.apache.org/jira/browse/SPARK-54372
> Project: Spark
> Issue Type: Bug
> Components: PySpark
> Affects Versions: 4.0.1
> Environment: Platform: Ubuntu 24.04
> Linux-6.14.0-35-generic-x86_64-with-glibc2.39
> Python: 3.10.19 | packaged by conda-forge | (main, Oct 22 2025,
> 22:29:10) [GCC 14.3.0]
> openjdk version "17.0.17-internal" 2025-10-21
> OpenJDK Runtime Environment (build 17.0.17-internal+0-adhoc..src)
> OpenJDK 64-Bit Server VM (build 17.0.17-internal+0-adhoc..src, mixed mode,
> sharing)
> pyspark 4.0.1
> duckdb 1.4.2
> pandas 2.3.3
> pyarrow 22.0.0
> Reporter: asddfl
> Priority: Critical
> Labels: pull-request-available
>
> The `avg(<timestamp>)` query result of PySpark is incorrect.
> I consider the query results from PySpark should be the same as those from
> DuckDBSpark, returning a timestamp value.
> {code:python}
> import pandas as pd
> from pyspark.sql import SparkSession
> from pyspark.sql import functions as F
> from duckdb.experimental.spark.sql import SparkSession as DuckdbSparkSession
> pd_df = pd.DataFrame({
> 'c0': ['1969-12-21'],
> })
> spark = SparkSession.builder.getOrCreate()
> df = spark.createDataFrame(pd_df)
> df.createOrReplaceTempView("t0")
> print("PySpark result:")
> pyspark_result =
> spark.table("t0").groupBy("c0").agg(F.avg(F.col("c0").cast("timestamp")))
> pyspark_result.show()
> duckdb_spark = DuckdbSparkSession.builder.getOrCreate()
> df = duckdb_spark.createDataFrame(pd_df)
> df.createOrReplaceTempView("t0")
> from duckdb.experimental.spark.sql import functions as F
> print("Duckdb Spark result: ")
> duckdb_spark_result =
> duckdb_spark.table("t0").groupBy("c0").agg(F.avg(F.col("c0").cast("timestamp")))
> duckdb_spark_result.show()
> {code}
> {code:bash}
> PySpark result:
> +----------+--------------------------+
>
> | c0|avg(CAST(c0 AS TIMESTAMP))|
> +----------+--------------------------+
> |1969-12-21| -979200.0|
> +----------+--------------------------+
> Duckdb Spark result:
> ┌──────┬──────────────┐
> │ c0 │ avg(CAST(c0 AS TIMESTAMP)) │
> │ varchar │ timestamp │
> ├──────┼──────────────┤
> │ 1969-12-21 │ 1969-12-21 00:00:00 │
> └──────┴──────────────┘
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]