[
https://issues.apache.org/jira/browse/SPARK-49858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17937615#comment-17937615
]
Lam Tran commented on SPARK-49858:
----------------------------------
When inferring timestamp value within the record of JSON reader, if
`inferTimestamp = "true"` and `timestampFormat` is not set, Spark will use `
org.apache.spark.sql.catalyst.util.DefaultTimestampFormatter` to get each year,
month, day,... part of a string represents a timestamp, which in turn uses
function `def parseTimestampString` of
`org.apache.spark.sql.catalyst.util.SparkDateTimeUtils` for getting those
tokens.
"23456" is considered a valid year because Spark supports the year part up to 6
digits, hence output an inferred TimestampType for the above string.
{code:java}
def isValidDigits(segment: Int, digits: Int): Boolean = {
// A Long is able to represent a timestamp within [+-]200 thousand years
val maxDigitsYear = 6
// For the nanosecond part, more than 6 digits is allowed, but will be
truncated.
segment == 6 || (segment == 0 && digits >= 4 && digits <= maxDigitsYear) ||
// For the zoneId segment(7), it's could be zero digits when it's a
region-based zone ID
(segment == 7 && digits <= 2) ||
(segment != 0 && segment != 6 && segment != 7 && digits > 0 && digits <= 2)
} {code}
I think I will try to create a PR to fix this as get back to you [~dhimmel]
> Pyspark JSON reader incorrectly considers a string of digits a timestamp and
> fails
> ----------------------------------------------------------------------------------
>
> Key: SPARK-49858
> URL: https://issues.apache.org/jira/browse/SPARK-49858
> Project: Spark
> Issue Type: Bug
> Components: SQL
> Affects Versions: 3.5.0
> Reporter: Daniel Himmelstein
> Priority: Major
> Attachments: image-2025-03-22-17-20-24-495.png,
> image-2025-03-22-17-23-17-473.png
>
>
> With pyspark 3.5.0 the reading the following JSON will fail:
> {code:python}
> from pyspark.sql import SparkSessionspark =
> SparkSession.builder.appName("timestamp_test").getOrCreate()
> data = spark.sparkContext.parallelize(['{"field" : "23456"}'])
> df = (
> spark.read.option("inferTimestamp", True)
> # .option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]")
> .json(path=data)
> )
> df.printSchema()
> df.collect()
> {code}
> The printSchema command shows that the field is parsed as a timestamp,
> causing the following error:
> {code:java}
> File
> ~/miniforge3/envs/facets/lib/python3.11/site-packages/pyspark/sql/types.py:282,
> in TimestampType.fromInternal(self, ts)
> 279 def fromInternal(self, ts: int) -> datetime.datetime:
> 280 if ts is not None:
> 281 # using int to avoid precision loss in float
> --> 282 return datetime.datetime.fromtimestamp(ts //
> 1000000).replace(microsecond=ts % 1000000)
> ValueError: year 23455 is out of range
> {code}
> If we uncomment the timestampFormat option, the command succeeds.
> I believe there are two issues:
> # that a string of digits with length > 4 is inferred to be a timestamp
> # that setting timestampFormat to the default according to [the
> documentation|https://spark.apache.org/docs/3.5.0/sql-data-sources-json.html]
> fixes the problem such that the documented default is not the actual default.
> This might be related to SPARK-45424.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]