MaxGekk commented on code in PR #43451:
URL: https://github.com/apache/spark/pull/43451#discussion_r1365590894


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala:
##########
@@ -1087,6 +1087,27 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
     }
   }
 
+  test("SPARK-45604: schema mismatch failure error on timestamp_nz to 
array<timestamp_nz>") {
+    import testImplicits._
+
+    withTempPath { dir =>
+      val path = dir.getCanonicalPath
+      val timestamp = java.time.LocalDateTime.of(1, 2, 3, 4, 5)
+      val df1 = Seq((1, timestamp)).toDF()
+      val df2 = Seq((2, Array(timestamp))).toDF()
+      df1.write.mode("overwrite").parquet(s"$path/parquet")
+      df2.write.mode("append").parquet(s"$path/parquet")
+
+      withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") {
+        val e = intercept[SparkException] {
+          spark.read.schema(df2.schema).parquet(s"$path/parquet").collect()
+        }
+        assert(e.getCause.isInstanceOf[SparkException])
+        
assert(e.getCause.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException])

Review Comment:
   This exception should be migrated to `SparkThrowable`, and we should throw 
an exception with proper error class. Please, add a follow up ticket to 
https://issues.apache.org/jira/browse/SPARK-37935



##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java:
##########
@@ -109,15 +109,17 @@ public ParquetVectorUpdater getUpdater(ColumnDescriptor 
descriptor, DataType spa
           // For unsigned int64, it stores as plain signed int64 in Parquet 
when dictionary
           // fallbacks. We read them as decimal values.
           return new UnsignedLongUpdater();
-        } else if 
(isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
+        } else if (sparkType instanceof DatetimeType &&

Review Comment:
   `DatetimeType` also includes the `DATE` type. Does this `if` handle the date 
types too?



-- 
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]

Reply via email to