@@ -18,8 +18,11 @@ package org.apache.spark.sql.delta
18
18
19
19
// scalastyle:off import.ordering.noEmptyLine
20
20
import java .sql .Date
21
- import java .time .LocalDate
21
+ import java .sql .Timestamp
22
+ import java .time .LocalDateTime
22
23
import java .time .LocalTime
24
+ import java .time .format .DateTimeFormatter
25
+ import java .util .TimeZone
23
26
24
27
import scala .collection .JavaConverters ._
25
28
import scala .util .Try
@@ -38,8 +41,9 @@ import org.apache.iceberg.types.Types.NestedField
38
41
import org .apache .spark .SparkConf
39
42
import org .apache .spark .sql .{AnalysisException , DataFrame , QueryTest , Row }
40
43
import org .apache .spark .sql .catalyst .TableIdentifier
41
- import org .apache .spark .sql .catalyst .util .DateTimeUtils .{stringToDate , toJavaDate }
44
+ import org .apache .spark .sql .catalyst .util .DateTimeUtils .{getZoneId , microsToLocalDateTime , stringToDate , stringToTimestamp , stringToTimestampWithoutTimeZone , toJavaDate , toJavaTimestamp }
42
45
import org .apache .spark .sql .functions .{col , expr , from_json , lit , struct , substring }
46
+ import org .apache .spark .sql .internal .SQLConf
43
47
import org .apache .spark .sql .types .{Decimal , DecimalType , LongType , StringType , StructField , StructType , TimestampType }
44
48
import org .apache .spark .unsafe .types .UTF8String
45
49
// scalastyle:on import.ordering.noEmptyLine
@@ -554,12 +558,12 @@ trait CloneIcebergSuiteBase extends QueryTest
554
558
val filesRead =
555
559
getFilesRead(spark, deltaLog, predicate, checkEmptyUnusedFilters = false )
556
560
try {
557
- assert(filesRead.size == expectedFilesReadNum)
558
- assert(filesRead.map(_.partitionValues.head._2).toSet ==
559
- expectedFilesReadIndices.map(_.toString))
560
561
checkAnswer(
561
562
spark.sql(s " select * from $cloneTable where $predicate" ), df.where(predicate)
562
563
)
564
+ assert(filesRead.size == expectedFilesReadNum)
565
+ assert(filesRead.map(_.partitionValues.head._2).toSet ==
566
+ expectedFilesReadIndices.map(_.toString))
563
567
} catch {
564
568
case e : Throwable =>
565
569
throw new RuntimeException (
@@ -597,7 +601,7 @@ trait CloneIcebergSuiteBase extends QueryTest
597
601
expectedFilesReadIndices = Set (2 )
598
602
)
599
603
),
600
- mode
604
+ mode = mode
601
605
)
602
606
}
603
607
@@ -620,7 +624,7 @@ trait CloneIcebergSuiteBase extends QueryTest
620
624
expectedFilesReadIndices = Set (1 )
621
625
)
622
626
),
623
- mode
627
+ mode = mode
624
628
)
625
629
}
626
630
@@ -643,7 +647,7 @@ trait CloneIcebergSuiteBase extends QueryTest
643
647
expectedFilesReadIndices = Set (1 )
644
648
)
645
649
),
646
- mode
650
+ mode = mode
647
651
)
648
652
}
649
653
@@ -680,7 +684,7 @@ trait CloneIcebergSuiteBase extends QueryTest
680
684
expectedFilesReadIndices = Set ()
681
685
)
682
686
),
683
- mode
687
+ mode = mode
684
688
)
685
689
}
686
690
@@ -725,8 +729,157 @@ trait CloneIcebergSuiteBase extends QueryTest
725
729
expectedFilesReadIndices = Set (4 , 5 )
726
730
)
727
731
),
728
- mode
732
+ mode = mode
733
+ )
734
+ }
735
+
736
+ // Exactly on minutes
737
+ testClone(" Convert Iceberg timestamptz type - 1" ) { mode =>
738
+ testStatsConversionAndDataSkipping(
739
+ icebergDataType = " timestamp" , // spark timestamp => iceberg timestamptz
740
+ tableData = Seq (
741
+ toTimestamp(" 1908-03-15 10:1:17" )
742
+ ),
743
+ extractFunc = row => {
744
+ timestamptzExtracter(row, pattern = " yyyy-MM-dd'T'HH:mm:ssXXX" )
745
+ },
746
+ expectedStats = Seq (" 1908-03-15T10:01:17+00:00" ),
747
+ dataSkippingTestParams = Seq (
748
+ DataSkippingTestParam (
749
+ predicate = " col2 > TIMESTAMP'1908-03-15T10:01:18+00:00'" ,
750
+ expectedFilesReadNum = 0 ,
751
+ expectedFilesReadIndices = Set ()
752
+ ),
753
+ DataSkippingTestParam (
754
+ predicate = " col2 <= TIMESTAMP'1908-03-15T10:01:17+00:00'" ,
755
+ expectedFilesReadNum = 1 ,
756
+ expectedFilesReadIndices = Set (1 )
757
+ )
758
+ ),
759
+ mode = mode
760
+ )
761
+ }
762
+
763
+ // Fractional time
764
+ testClone(" Convert Iceberg timestamptz type - 2" ) { mode =>
765
+ testStatsConversionAndDataSkipping(
766
+ icebergDataType = " timestamp" , // spark timestamp => iceberg timestamptz
767
+ tableData = Seq (
768
+ toTimestamp(" 1997-12-11 5:40:19.23349" )
769
+ ),
770
+ extractFunc = row => {
771
+ timestamptzExtracter(row, pattern = " yyyy-MM-dd'T'HH:mm:ss.SSSSSXXX" )
772
+ },
773
+ expectedStats = Seq (" 1997-12-11T05:40:19.23349+00:00" ),
774
+ dataSkippingTestParams = Seq (
775
+ DataSkippingTestParam (
776
+ predicate = " col2 > TIMESTAMP'1997-12-11T05:40:19.233+00:00'" ,
777
+ expectedFilesReadNum = 1 ,
778
+ expectedFilesReadIndices = Set (1 )
779
+ ),
780
+ DataSkippingTestParam (
781
+ predicate = " col2 <= TIMESTAMP'1997-12-11T05:40:19.10+00:00'" ,
782
+ expectedFilesReadNum = 0 ,
783
+ expectedFilesReadIndices = Set ()
784
+ )
785
+ ),
786
+ mode = mode
787
+ )
788
+ }
789
+
790
+ // Customized timezone
791
+ testClone(" Convert Iceberg timestamptz type - 3" ) { mode =>
792
+ testStatsConversionAndDataSkipping(
793
+ icebergDataType = " timestamp" , // spark timestamp => iceberg timestamptz
794
+ tableData = Seq (
795
+ toTimestamp(" 2077-11-11 3:23:11.23456+02:15" )
796
+ ),
797
+ extractFunc = row => {
798
+ timestamptzExtracter(row, pattern = " yyyy-MM-dd'T'HH:mm:ss.SSSSSXXX" )
799
+ },
800
+ expectedStats = Seq (" 2077-11-11T01:08:11.23456+00:00" ),
801
+ dataSkippingTestParams = Seq (
802
+ DataSkippingTestParam (
803
+ predicate = " col2 > TIMESTAMP'2077-11-11T03:23:11.23456+02:16'" ,
804
+ expectedFilesReadNum = 1 ,
805
+ expectedFilesReadIndices = Set (1 )
806
+ ),
807
+ DataSkippingTestParam (
808
+ predicate = " col2 < TIMESTAMP'2077-11-11T03:23:11.23456+02:16'" ,
809
+ expectedFilesReadNum = 0 ,
810
+ expectedFilesReadIndices = Set ()
811
+ )
812
+ ),
813
+ mode = mode
814
+ )
815
+ }
816
+
817
+ // Exactly on minutes
818
+ testClone(" Convert Iceberg timestamp type - 1" ) { mode =>
819
+ testStatsConversionAndDataSkipping(
820
+ icebergDataType = " timestamp_ntz" , // spark timestamp_ntz => iceberg timestamp
821
+ tableData = Seq (
822
+ toTimestampNTZ(" 2024-01-02T02:04:05.123456" )
823
+ ),
824
+ extractFunc = row => {
825
+ row.get(0 ).asInstanceOf [LocalDateTime ].toString
826
+ },
827
+ expectedStats = Seq (" 2024-01-02T02:04:05.123456" ),
828
+ dataSkippingTestParams = Seq (
829
+ DataSkippingTestParam (
830
+ predicate = " col2 > TIMESTAMP'2024-01-02T02:04:04.123456'" ,
831
+ expectedFilesReadNum = 1 ,
832
+ expectedFilesReadIndices = Set (1 )
833
+ )
834
+ ),
835
+ mode = mode
836
+ )
837
+ }
838
+
839
+ // Fractional time
840
+ testClone(" Convert Iceberg timestamp type - 2" ) { mode =>
841
+ testStatsConversionAndDataSkipping(
842
+ icebergDataType = " timestamp_ntz" , // spark timestamp_ntz => iceberg timestamp
843
+ tableData = Seq (
844
+ toTimestampNTZ(" 1712-4-29T06:23:49.12" )
845
+ ),
846
+ extractFunc = row => {
847
+ row.get(0 ).asInstanceOf [LocalDateTime ].toString
848
+ .replaceAll(" 0+$" , " " ) // remove trailing zeros
849
+ },
850
+ expectedStats = Seq (" 1712-04-29T06:23:49.12" ),
851
+ dataSkippingTestParams = Seq (
852
+ DataSkippingTestParam (
853
+ predicate = " col2 > TIMESTAMP'1712-04-29T06:23:49.11'" ,
854
+ expectedFilesReadNum = 1 ,
855
+ expectedFilesReadIndices = Set (1 )
856
+ )
857
+ ),
858
+ mode = mode
859
+ )
860
+ }
861
+
862
+ private def toTimestamp (timestamp : String ): Timestamp = {
863
+ toJavaTimestamp(stringToTimestamp(UTF8String .fromString(timestamp),
864
+ getZoneId(SQLConf .get.sessionLocalTimeZone)).get)
865
+ }
866
+
867
+ private def toTimestampNTZ (timestampNTZ : String ): LocalDateTime = {
868
+ microsToLocalDateTime(
869
+ stringToTimestampWithoutTimeZone(
870
+ UTF8String .fromString(timestampNTZ)
871
+ ).get
872
+ )
873
+ }
874
+
875
+ private def timestamptzExtracter (row : Row , pattern : String ): String = {
876
+ val ts = row.getTimestamp(0 ).toLocalDateTime.atZone(
877
+ getZoneId(TimeZone .getDefault.getID)
729
878
)
879
+ ts.withZoneSameInstant(getZoneId(SQLConf .get.sessionLocalTimeZone))
880
+ .format(DateTimeFormatter .ofPattern(pattern))
881
+ .replace(" UTC" , " +00:00" )
882
+ .replace(" Z" , " +00:00" )
730
883
}
731
884
}
732
885
0 commit comments