wangyum commented on a change in pull request #26804:
URL: https://github.com/apache/spark/pull/26804#discussion_r561444625



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
##########
@@ -127,6 +127,9 @@ class ParquetFileFormat
       conf.setEnum(ParquetOutputFormat.JOB_SUMMARY_LEVEL, JobSummaryLevel.NONE)
     }
 
+    // PARQUET-1580: Disables page-level CRC checksums by default.

Review comment:
       It will change the data order, please seem 
https://github.com/apache/spark/pull/26804#discussion_r561044576.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
##########
@@ -127,6 +127,9 @@ class ParquetFileFormat
       conf.setEnum(ParquetOutputFormat.JOB_SUMMARY_LEVEL, JobSummaryLevel.NONE)
     }
 
+    // PARQUET-1580: Disables page-level CRC checksums by default.

Review comment:
       It will change the data order, please see 
https://github.com/apache/spark/pull/26804#discussion_r561044576.

##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
##########
@@ -225,7 +226,9 @@ class StreamSuite extends StreamTest {
 
     val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load()
     Seq("", "parquet").foreach { useV1Source =>
-      withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> useV1Source) {
+      withSQLConf(
+        SQLConf.USE_V1_SOURCE_LIST.key -> useV1Source,
+        ParquetOutputFormat.PAGE_WRITE_CHECKSUM_ENABLED -> "false") {

Review comment:
       Thank you @gszadovszky The size is different if enable the CRC write: 
   ```
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-0ae44ddf-40bb-4ba5-84af-ec8cec037847-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-166cdfbf-b19d-4d55-b4ea-fbad6bcac9df-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  462 Jan 21 22:20 
part-00001-23a0376d-1c51-480d-b7c6-a2d9a07de0e3-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-3c52b355-290a-4dd4-aad3-4bb2960ba3b8-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-4486173e-d650-4548-8da4-b95ae0305d8c-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-4c3786f4-2702-4f58-9604-c3deed68bc86-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-71bf8d51-95b0-43a8-969b-c28630f90066-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-78776231-370d-45c6-8520-67b94c33c697-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-a2a811aa-a495-4439-9daf-8c4b2cb258d5-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-bf745883-0b3a-4383-8669-7464833bfea8-c000.snappy.parquet
   -rw-r--r--  1 yumwang  wheel  463 Jan 21 22:20 
part-00001-d3a65d34-cd1a-434d-a86c-8ee0203b3bac-c000.snappy.parquet
   yumwang@LM-SHC-16508156 1611238822602 % parquet-tools cat 
part-00001-23a0376d-1c51-480d-b7c6-a2d9a07de0e3-c000.snappy.parquet
   a = 2
   ```
   and we will order the file by size:
   
https://github.com/apache/spark/blob/8ed23ed499ec7745a8e9bdc4c4fb3200fdb6c3c8/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L609
   
   Not sure if it caused by int overflow:
   https://github.com/apache/parquet-mr/pull/647#discussion_r561914480




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

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