-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-36346][SQL] Support TimestampNTZ type in Orc file source #33588
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
3344bb8
ef516ec
a1aa093
0bd85eb
cd80e28
3cb69f0
42fa8e4
db374b9
ab8d985
7d19c1e
ae3e2de
693c620
b0acc90
3c69b9c
fc08870
528f7ad
3549dba
282c648
0e0d44c
91331c7
cffe76b
7c273d5
9243f3b
108c924
625bd9f
691a9a8
8e66926
51d7651
3b70990
2c213c3
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.orc | |
| import java.io.File | ||
| import java.nio.charset.StandardCharsets | ||
| import java.sql.Timestamp | ||
| import java.time.{LocalDateTime, ZoneOffset} | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
|
|
@@ -768,6 +769,67 @@ abstract class OrcQuerySuite extends OrcQueryTest with SharedSparkSession { | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("Read/write all timestamp types") { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. let's add test for
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. OK |
||
| val data = (0 to 255).map { i => | ||
| (new Timestamp(i), LocalDateTime.of(2019, 3, 21, 0, 2, 3, 456000000 + i)) | ||
| } :+ (null, null) | ||
|
|
||
| withOrcFile(data) { file => | ||
| withAllOrcReaders { | ||
| checkAnswer(spark.read.orc(file), data.toDF().collect()) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-36346: can't read TimestampLTZ as TimestampNTZ") { | ||
| val data = (1 to 10).map { i => | ||
| val ts = new Timestamp(i) | ||
| Row(ts) | ||
| } | ||
| val answer = (1 to 10).map { i => | ||
| // The second parameter is `nanoOfSecond`, while java.sql.Timestamp accepts milliseconds | ||
| // as input. So here we multiple the `nanoOfSecond` by NANOS_PER_MILLIS | ||
| val ts = LocalDateTime.ofEpochSecond(0, i * 1000000, ZoneOffset.UTC) | ||
| Row(ts) | ||
| } | ||
| val actualSchema = StructType(Seq(StructField("time", TimestampType, false))) | ||
| val providedSchema = StructType(Seq(StructField("time", TimestampNTZType, false))) | ||
|
|
||
| withTempPath { file => | ||
| val df = spark.createDataFrame(sparkContext.parallelize(data), actualSchema) | ||
| df.write.orc(file.getCanonicalPath) | ||
| withAllOrcReaders { | ||
| val msg = intercept[SparkException] { | ||
| spark.read.schema(providedSchema).orc(file.getCanonicalPath).collect() | ||
| }.getMessage | ||
| assert(msg.contains("Unable to convert timestamp of Orc to data type 'timestamp_ntz'")) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-36346: read TimestampNTZ as TimestampLTZ") { | ||
| val data = (1 to 10).map { i => | ||
| // The second parameter is `nanoOfSecond`, while java.sql.Timestamp accepts milliseconds | ||
| // as input. So here we multiple the `nanoOfSecond` by NANOS_PER_MILLIS | ||
| val ts = LocalDateTime.ofEpochSecond(0, i * 1000000, ZoneOffset.UTC) | ||
| Row(ts) | ||
| } | ||
| val answer = (1 to 10).map { i => | ||
| val ts = new java.sql.Timestamp(i) | ||
| Row(ts) | ||
| } | ||
| val actualSchema = StructType(Seq(StructField("time", TimestampNTZType, false))) | ||
| val providedSchema = StructType(Seq(StructField("time", TimestampType, false))) | ||
|
|
||
| withTempPath { file => | ||
| val df = spark.createDataFrame(sparkContext.parallelize(data), actualSchema) | ||
| df.write.orc(file.getCanonicalPath) | ||
| withAllOrcReaders { | ||
| checkAnswer(spark.read.schema(providedSchema).orc(file.getCanonicalPath), answer) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| class OrcV1QuerySuite extends OrcQuerySuite { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -143,6 +143,13 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor | |
| spark.read.orc(file.getAbsolutePath) | ||
| } | ||
|
|
||
| def withAllOrcReaders(code: => Unit): Unit = { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi, All. Instead of |
||
| // test the row-based reader | ||
| withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false")(code) | ||
| // test the vectorized reader | ||
| withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "true")(code) | ||
| } | ||
|
|
||
| /** | ||
| * Takes a sequence of products `data` to generate multi-level nested | ||
| * dataframes as new test data. It tests both non-nested and nested dataframes | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.