This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-paimon.git
The following commit(s) were added to refs/heads/master by this push:
new 717345501 [spark] Fix the ClassCastException when creating timestamp
data field in spark (#1551)
717345501 is described below
commit 717345501e48c8eb6d8481982ead259b3cd7f476
Author: miomiocat <[email protected]>
AuthorDate: Thu Jul 13 09:47:44 2023 +0800
[spark] Fix the ClassCastException when creating timestamp data field in
spark (#1551)
---
paimon-spark/paimon-spark-2/pom.xml | 4 ++++
.../src/main/java/org/apache/paimon/spark/SparkRow.java | 6 ++++++
2 files changed, 10 insertions(+)
diff --git a/paimon-spark/paimon-spark-2/pom.xml
b/paimon-spark/paimon-spark-2/pom.xml
index d78fe9e3f..5425ff5bf 100644
--- a/paimon-spark/paimon-spark-2/pom.xml
+++ b/paimon-spark/paimon-spark-2/pom.xml
@@ -80,6 +80,10 @@ under the License.
<groupId>org.apache.xbean</groupId>
<artifactId>xbean-asm6-shaded</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.glassfish.jersey.media</groupId>
+ <artifactId>jersey-media-json-jackson</artifactId>
+ </exclusion>
</exclusions>
</dependency>
diff --git
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java
index 5e4d04c8d..ecd1472ef 100644
---
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java
+++
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java
@@ -36,8 +36,10 @@ import org.apache.spark.sql.Row;
import java.io.Serializable;
import java.sql.Date;
+import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
+import java.time.ZoneId;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@@ -161,6 +163,10 @@ public class SparkRow implements InternalRow, Serializable
{
private static Timestamp toPaimonTimestamp(Object object) {
if (object instanceof java.sql.Timestamp) {
return Timestamp.fromSQLTimestamp((java.sql.Timestamp) object);
+ } else if (object instanceof java.time.Instant) {
+ LocalDateTime localDateTime =
+ LocalDateTime.ofInstant((Instant) object,
ZoneId.systemDefault());
+ return Timestamp.fromLocalDateTime(localDateTime);
} else {
return Timestamp.fromLocalDateTime((LocalDateTime) object);
}