Skip to content

Commit

Permalink
[SPARK-50795][SQL][FOLLOWUP] Set isParsing to false for the timestamp…
Browse files Browse the repository at this point in the history
… formatter in DESCRIBE AS JSON

### What changes were proposed in this pull request?

This PR set isParsing to false for the timestamp formatter in DESCRIBE AS JSON, because the formatter is not used for parsing datetime strings

### Why are the changes needed?

Although it does not affect the final output due to the current fmt we use now being w/o 'S' portion, it can prevent potential bugs if we store/display higher-precision timestamps.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?

Existing tests

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

Closes #50065 from yaooqinn/SPARK-50795.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
  • Loading branch information
yaooqinn committed Feb 25, 2025
1 parent 352d1ed commit 7feb911
Showing 1 changed file with 4 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.sql.execution.command

import java.time.ZoneId
import java.time.ZoneOffset

import scala.collection.mutable

Expand All @@ -31,13 +31,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, Se
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.util.{
quoteIfNeeded,
DateFormatter,
DateTimeUtils,
Iso8601TimestampFormatter,
LegacyDateFormats
}
import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, DateTimeUtils, TimestampFormatter}
import org.apache.spark.sql.classic.ClassicConversions.castToImpl
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
import org.apache.spark.sql.connector.catalog.V1Table
Expand All @@ -59,13 +53,8 @@ case class DescribeRelationJsonCommand(
nullable = false,
new MetadataBuilder().putString("comment", "JSON metadata of the table").build())()
)) extends UnaryRunnableCommand {
private lazy val timestampFormatter = new Iso8601TimestampFormatter(
pattern = "yyyy-MM-dd'T'HH:mm:ss'Z'",
zoneId = ZoneId.of("UTC"),
locale = DateFormatter.defaultLocale,
legacyFormat = LegacyDateFormats.LENIENT_SIMPLE_DATE_FORMAT,
isParsing = true
)
private lazy val timestampFormatter =
TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss'Z'", ZoneOffset.UTC, isParsing = false)

override def run(sparkSession: SparkSession): Seq[Row] = {
val jsonMap = mutable.LinkedHashMap[String, JValue]()
Expand Down

0 comments on commit 7feb911

Please sign in to comment.