@@ -50,14 +50,15 @@ case class AllDataTypesWithNonPrimitiveType(
5050 shortField : Short ,
5151 byteField : Byte ,
5252 booleanField : Boolean ,
53- timestampField : Timestamp ,
54- timestampNTZField : LocalDateTime ,
5553 array : Seq [Int ],
5654 arrayContainsNull : Seq [Option [Int ]],
5755 map : Map [Int , Long ],
5856 mapValueContainsNull : Map [Int , Option [Long ]],
5957 data : (Seq [Int ], (Int , String )))
6058
59+ case class TimestampsWithNonPrimitiveType (
60+ timestampField : Timestamp , timestampNTZField : LocalDateTime )
61+
6162case class BinaryData (binaryData : Array [Byte ])
6263
6364case class Contact (name : String , phone : String )
@@ -69,8 +70,7 @@ abstract class OrcQueryTest extends OrcTest {
6970
7071 test(" Read/write All Types" ) {
7172 val data = (0 to 255 ).map { i =>
72- (s " $i" , i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0 ,
73- new Timestamp (i), LocalDateTime .of(2019 , 3 , 21 , 0 , 2 , 3 , 456000000 + i))
73+ (s " $i" , i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0 )
7474 }
7575
7676 withOrcFile(data) { file =>
@@ -91,7 +91,6 @@ abstract class OrcQueryTest extends OrcTest {
9191 val data : Seq [AllDataTypesWithNonPrimitiveType ] = (0 to 255 ).map { i =>
9292 AllDataTypesWithNonPrimitiveType (
9393 s " $i" , i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0 ,
94- new Timestamp (i), LocalDateTime .of(2019 , 3 , 21 , 0 , 2 , 3 , 456000000 + i),
9594 0 until i,
9695 (0 until i).map(Option (_).filter(_ % 3 == 0 )),
9796 (0 until i).map(i => i -> i.toLong).toMap,
@@ -177,15 +176,13 @@ abstract class OrcQueryTest extends OrcTest {
177176 Option .empty[Long ],
178177 Option .empty[Float ],
179178 Option .empty[Double ],
180- Option .empty[Boolean ],
181- Option .empty[Timestamp ],
182- Option .empty[LocalDateTime ]
179+ Option .empty[Boolean ]
183180 ) :: Nil
184181
185182 withOrcFile(data) { file =>
186183 checkAnswer(
187184 spark.read.orc(file),
188- Row (Seq .fill(7 )(null ): _* ))
185+ Row (Seq .fill(5 )(null ): _* ))
189186 }
190187 }
191188
@@ -720,6 +717,44 @@ abstract class OrcQuerySuite extends OrcQueryTest with SharedSparkSession {
720717 }
721718 }
722719 }
720+
721+ test(" Read/write all timestamp types" ) {
722+ val data = (0 to 255 ).map { i =>
723+ (new Timestamp (i), LocalDateTime .of(2019 , 3 , 21 , 0 , 2 , 3 , 456000000 + i))
724+ }
725+
726+ withOrcFile(data) { file =>
727+ checkAnswer(
728+ spark.read.orc(file),
729+ data.toDF().collect())
730+ }
731+ }
732+
733+ test(" Read/write all timestamp types with non-primitive type" ) {
734+ val data : Seq [TimestampsWithNonPrimitiveType ] = (0 to 255 ).map { i =>
735+ TimestampsWithNonPrimitiveType (
736+ new Timestamp (i), LocalDateTime .of(2019 , 3 , 21 , 0 , 2 , 3 , 456000000 + i))
737+ }
738+
739+ withOrcFile(data) { file =>
740+ checkAnswer(
741+ spark.read.orc(file),
742+ data.toDF().collect())
743+ }
744+ }
745+
746+ test(" test for timestamp types: save and load case class RDD with `None`s as orc" ) {
747+ val data = (
748+ Option .empty[Timestamp ],
749+ Option .empty[LocalDateTime ]
750+ ) :: Nil
751+
752+ withOrcFile(data) { file =>
753+ checkAnswer(
754+ spark.read.orc(file),
755+ Row (Seq .fill(2 )(null ): _* ))
756+ }
757+ }
723758}
724759
725760class OrcV1QuerySuite extends OrcQuerySuite {
0 commit comments