Skip to content
Permalink
Browse files

[SPARK-29963][SQL][TESTS] Check formatting timestamps up to microseco…

…nd precision by JSON/CSV datasource

### What changes were proposed in this pull request?
In the PR, I propose to add tests from the commit 47cb1f3 for Spark 2.4 that check formatting of timestamp strings for various seconds fractions.

### Why are the changes needed?
To make sure that current behavior is the same as in Spark 2.4

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

### How was this patch tested?
By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`.

Closes #26601 from MaxGekk/format-timestamp-micros-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
  • Loading branch information
MaxGekk authored and HyukjinKwon committed Nov 20, 2019
1 parent e753aa3 commit 40b8a08b8b7f71685c5703605fd8a8b101de32f7
@@ -193,4 +193,44 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
}
}
}

test("formatting timestamp strings up to microsecond precision") {
DateTimeTestUtils.outstandingZoneIds.foreach { zoneId =>
def check(pattern: String, input: String, expected: String): Unit = {
val formatter = TimestampFormatter(pattern, zoneId)
val timestamp = DateTimeUtils.stringToTimestamp(
UTF8String.fromString(input), zoneId).get
val actual = formatter.format(timestamp)
assert(actual === expected)
}

check(
"yyyy-MM-dd HH:mm:ss.SSSSSSS", "2019-10-14T09:39:07.123456",
"2019-10-14 09:39:07.1234560")
check(
"yyyy-MM-dd HH:mm:ss.SSSSSS", "1960-01-01T09:39:07.123456",
"1960-01-01 09:39:07.123456")
check(
"yyyy-MM-dd HH:mm:ss.SSSSS", "0001-10-14T09:39:07.1",
"0001-10-14 09:39:07.10000")
check(
"yyyy-MM-dd HH:mm:ss.SSSS", "9999-12-31T23:59:59.999",
"9999-12-31 23:59:59.9990")
check(
"yyyy-MM-dd HH:mm:ss.SSS", "1970-01-01T00:00:00.0101",
"1970-01-01 00:00:00.010")
check(
"yyyy-MM-dd HH:mm:ss.SS", "2019-10-14T09:39:07.09",
"2019-10-14 09:39:07.09")
check(
"yyyy-MM-dd HH:mm:ss.S", "2019-10-14T09:39:07.2",
"2019-10-14 09:39:07.2")
check(
"yyyy-MM-dd HH:mm:ss.S", "2019-10-14T09:39:07",
"2019-10-14 09:39:07.0")
check(
"yyyy-MM-dd HH:mm:ss", "2019-10-14T09:39:07.123456",
"2019-10-14 09:39:07")
}
}
}
@@ -637,4 +637,11 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
df.select(from_json($"value", schema, options)),
Row(Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.123456"))))
}

test("to_json - timestamp in micros") {
val s = "2019-11-18 11:56:00.123456"
val df = Seq(java.sql.Timestamp.valueOf(s)).toDF("t").select(
to_json(struct($"t"), Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss.SSSSSS")))
checkAnswer(df, Row(s"""{"t":"$s"}"""))
}
}
@@ -2180,4 +2180,19 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData {
checkAnswer(readback, Row(Timestamp.valueOf(t)))
}
}

test("Roundtrip in reading and writing timestamps in microsecond precision") {
withTempPath { path =>
val timestamp = Timestamp.valueOf("2019-11-18 11:56:00.123456")
Seq(timestamp).toDF("t")
.write
.option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSSSSS")
.csv(path.getAbsolutePath)
val readback = spark.read
.schema("t timestamp")
.option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSSSSS")
.csv(path.getAbsolutePath)
checkAnswer(readback, Row(timestamp))
}
}
}

0 comments on commit 40b8a08

Please sign in to comment.
You can’t perform that action at this time.