You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "majdyz (via GitHub)" <gi...@apache.org> on 2023/10/19 08:09:43 UTC

[PR] [SPARK-45604] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader [spark]

majdyz opened a new pull request, #43451:
URL: https://github.com/apache/spark/pull/43451

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   
   Currently, the read logical type is not checked while converting physical types INT64 into DateTime. One valid scenario where this can break is where the physical type is `timestamp_ntz`, and the logical type is `array<timestamp_ntz>`, since the logical type check does not happen, this conversion is allowed. However, the vectorized reader does not support this and will produce NPE on on-heap memory mode and SEGFAULT on off-heap memory mode (segfault can be prevented by having an explicit boundary check on OffHeapColumnVector, but this is outside of the scope of this PR).
   
   
   ### Why are the changes needed?
   Prevent NPE or Segfault from happening.
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   No.
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   A new test is added in `ParquetSchemaSuite`.
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   No.
   


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45604] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader [spark]

Posted by "majdyz (via GitHub)" <gi...@apache.org>.
majdyz commented on PR #43451:
URL: https://github.com/apache/spark/pull/43451#issuecomment-1772218760

   @MaxGekk yes, and I believe the previous branches 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #43451:
URL: https://github.com/apache/spark/pull/43451#issuecomment-1772159682

   @majdyz Does Spark 3.4.x have the same issue?


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #43451:
URL: https://github.com/apache/spark/pull/43451#issuecomment-1774003072

   +1, LGTM. Merging to master/3.5/3.4.
   Thank you, @majdyz.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45604] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader [spark]

Posted by "majdyz (via GitHub)" <gi...@apache.org>.
majdyz commented on code in PR #43451:
URL: https://github.com/apache/spark/pull/43451#discussion_r1365683194


##########
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:
   Added a subtask here: https://issues.apache.org/jira/browse/SPARK-45608



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45604] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader [spark]

Posted by "majdyz (via GitHub)" <gi...@apache.org>.
majdyz commented on code in PR #43451:
URL: https://github.com/apache/spark/pull/43451#discussion_r1365780160


##########
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:
   Good point. I've limited this check only to Timestamp & TimestampNtz now.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk closed pull request #43451: [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader
URL: https://github.com/apache/spark/pull/43451


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org