This is an automated email from the ASF dual-hosted git repository.

kejia pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
     new a7fb09eb7 [VL] Enable timestamp in parquet write (#6428)
a7fb09eb7 is described below

commit a7fb09eb723f559cfeb453f6686c0c07207a95f6
Author: JiaKe <[email protected]>
AuthorDate: Fri Jul 26 08:54:09 2024 +0800

    [VL] Enable timestamp in parquet write (#6428)
---
 .../main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala | 3 +--
 .../apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala | 1 -
 .../test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala  | 4 ++++
 .../test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala  | 4 ++++
 4 files changed, 9 insertions(+), 3 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
index 933947301..990991c71 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
@@ -224,7 +224,6 @@ object VeloxBackendSettings extends BackendSettingsApi {
       val unsupportedTypes = fields.flatMap {
         field =>
           field.dataType match {
-            case _: TimestampType => Some("TimestampType")
             case _: StructType => Some("StructType")
             case _: ArrayType => Some("ArrayType")
             case _: MapType => Some("MapType")
@@ -293,7 +292,7 @@ object VeloxBackendSettings extends BackendSettingsApi {
     fields.map {
       field =>
         field.dataType match {
-          case _: TimestampType | _: StructType | _: ArrayType | _: MapType => 
return false
+          case _: StructType | _: ArrayType | _: MapType => return false
           case _ =>
         }
     }
diff --git 
a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala
 
b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala
index 9793df2ab..85b3f32a7 100644
--- 
a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala
+++ 
b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala
@@ -484,7 +484,6 @@ class VeloxParquetDataTypeValidationSuite extends 
VeloxWholeStageTransformerSuit
           val df = spark.read
             .format("parquet")
             .load(data_path)
-            .drop("timestamp")
             .drop("array")
             .drop("struct")
             .drop("map")
diff --git 
a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
 
b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
index 57346f493..52efa4b25 100644
--- 
a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
+++ 
b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
@@ -731,12 +731,14 @@ class VeloxTestSettings extends BackendTestSettings {
   enableSuite[GlutenParquetRebaseDatetimeV1Suite]
     // Velox doesn't write file metadata into parquet file.
     .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys")
+    .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key")
     // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some
     .excludeByPrefix("SPARK-31159")
     .excludeByPrefix("SPARK-35427")
   enableSuite[GlutenParquetRebaseDatetimeV2Suite]
     // Velox doesn't write file metadata into parquet file.
     .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys")
+    .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key")
     // jar path and ignore PARQUET_REBASE_MODE_IN_READ
     .excludeByPrefix("SPARK-31159")
     .excludeByPrefix("SPARK-35427")
@@ -767,6 +769,8 @@ class VeloxTestSettings extends BackendTestSettings {
   enableSuite[GlutenFileMetadataStructRowIndexSuite]
   enableSuite[GlutenParquetV1AggregatePushDownSuite]
   enableSuite[GlutenParquetV2AggregatePushDownSuite]
+    // TODO: Timestamp columns stats will lost if using int64 in parquet 
writer.
+    .exclude("aggregate push down - different data types")
   enableSuite[GlutenOrcV1AggregatePushDownSuite]
     .exclude("nested column: Count(nested sub-field) not push down")
   enableSuite[GlutenOrcV2AggregatePushDownSuite]
diff --git 
a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
 
b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
index 9716a7c14..71348c823 100644
--- 
a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
+++ 
b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
@@ -738,12 +738,14 @@ class VeloxTestSettings extends BackendTestSettings {
   enableSuite[GlutenParquetRebaseDatetimeV1Suite]
     // Velox doesn't write file metadata into parquet file.
     .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys")
+    .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key")
     // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some
     .excludeByPrefix("SPARK-31159")
     .excludeByPrefix("SPARK-35427")
   enableSuite[GlutenParquetRebaseDatetimeV2Suite]
     // Velox doesn't write file metadata into parquet file.
     .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys")
+    .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key")
     // jar path and ignore PARQUET_REBASE_MODE_IN_READ
     .excludeByPrefix("SPARK-31159")
     .excludeByPrefix("SPARK-35427")
@@ -773,6 +775,8 @@ class VeloxTestSettings extends BackendTestSettings {
   enableSuite[GlutenFileMetadataStructSuite]
   enableSuite[GlutenParquetV1AggregatePushDownSuite]
   enableSuite[GlutenParquetV2AggregatePushDownSuite]
+    // TODO: Timestamp columns stats will lost if using int64 in parquet 
writer.
+    .exclude("aggregate push down - different data types")
   enableSuite[GlutenOrcV1AggregatePushDownSuite]
     .exclude("nested column: Count(nested sub-field) not push down")
   enableSuite[GlutenOrcV2AggregatePushDownSuite]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to