cloud-fan commented on a change in pull request #33588:
URL: https://github.com/apache/spark/pull/33588#discussion_r748088138
##########
File path:
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
##########
@@ -768,6 +772,56 @@ abstract class OrcQuerySuite extends OrcQueryTest with
SharedSparkSession {
}
}
}
+
+ test("Read/write all timestamp types") {
+ val data = (0 to 255).map { i =>
+ (new Timestamp(i), LocalDateTime.of(2019, 3, 21, 0, 2, 3, 456000000 + i))
+ }
+
+ Seq("true", "false").foreach { key =>
+ withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> key) {
+ withOrcFile(data) { file =>
+ checkAnswer(
+ spark.read.orc(file),
+ data.toDF().collect())
+ }
+ }
+ }
+ }
+
+ test("Read/write all timestamp types with non-primitive type") {
+ val data: Seq[TimestampsWithNonPrimitiveType] = (0 to 255).map { i =>
+ TimestampsWithNonPrimitiveType(
+ new Timestamp(i), LocalDateTime.of(2019, 3, 21, 0, 2, 3, 456000000 +
i))
+ }
+
+ Seq("true", "false").foreach { key =>
+ withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> key) {
+ withOrcFile(data) { file =>
+ checkAnswer(
+ spark.read.orc(file),
+ data.toDF().collect())
+ }
+ }
+ }
+ }
+
+ test("test for timestamp types: save and load case class RDD with `None`s as
orc") {
Review comment:
can we merge this into the first test? we can append nulls to the input
data.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##########
@@ -295,6 +296,16 @@ object OrcUtils extends Logging {
typeDesc.setAttribute(
CATALYST_TYPE_ATTRIBUTE_NAME, d.typeName)
Some(typeDesc)
+ case n: TimestampNTZType =>
+ val typeDesc = new
TypeDescription(TypeDescription.Category.TIMESTAMP)
Review comment:
Can we change the above two interval cases and also create
`TypeDescription` directly?
##########
File path:
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
##########
@@ -768,6 +772,56 @@ abstract class OrcQuerySuite extends OrcQueryTest with
SharedSparkSession {
}
}
}
+
+ test("Read/write all timestamp types") {
+ val data = (0 to 255).map { i =>
+ (new Timestamp(i), LocalDateTime.of(2019, 3, 21, 0, 2, 3, 456000000 + i))
+ }
+
+ Seq("true", "false").foreach { key =>
+ withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> key) {
+ withOrcFile(data) { file =>
+ checkAnswer(
+ spark.read.orc(file),
+ data.toDF().collect())
+ }
+ }
+ }
+ }
+
+ test("Read/write all timestamp types with non-primitive type") {
+ val data: Seq[TimestampsWithNonPrimitiveType] = (0 to 255).map { i =>
+ TimestampsWithNonPrimitiveType(
Review comment:
This is actually testing DataFrame creation, not ORC, we can remove it.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##########
@@ -47,15 +47,15 @@ private[sql] object OrcFileFormat {
def getQuotedSchemaString(dataType: DataType): String = dataType match {
Review comment:
is it duplicated with `OrcUtils.orcTypeDescriptionString`?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]