You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/03/07 14:05:37 UTC

[GitHub] [spark] wangyum commented on a change in pull request #24003: [SPARK-19678][FOLLOW-UP][SQL] Add behavior change test when table statistics are incorrect

wangyum commented on a change in pull request #24003: [SPARK-19678][FOLLOW-UP][SQL] Add behavior change test when table statistics are incorrect
URL: https://github.com/apache/spark/pull/24003#discussion_r263395794
 
 

 ##########
 File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
 ##########
 @@ -110,6 +110,41 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto
     }
   }
 
+  test("Hive serde table with incorrect statistics") {
+    withTempDir { tempDir =>
+      withTable("t1") {
+        spark.range(5).write.mode(SaveMode.Overwrite).parquet(tempDir.getCanonicalPath)
+        val dataSize = tempDir.listFiles.filter(!_.getName.endsWith(".crc")).map(_.length).sum
+        spark.sql(
+          s"""
+             |CREATE EXTERNAL TABLE t1(id BIGINT)
+             |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+             |STORED AS
+             |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+             |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+             |LOCATION '${tempDir.getCanonicalPath}'
+             |TBLPROPERTIES (
+             |'rawDataSize'='-1', 'numFiles'='0', 'totalSize'='0',
+             |'COLUMN_STATS_ACCURATE'='false', 'numRows'='-1'
+             |)""".stripMargin)
+
+        spark.sql("REFRESH TABLE t1")
+        // Before SPARK-19678, sizeInBytes should be equal to dataSize.
+        // After SPARK-19678, sizeInBytes should be equal to DEFAULT_SIZE_IN_BYTES.
+        val relation1 = spark.table("t1").queryExecution.analyzed.children.head
+        assert(relation1.stats.sizeInBytes === spark.sessionState.conf.defaultSizeInBytes)
 
 Review comment:
   Before SPARK-19678. It dataSize(397) no matter `ENABLE_FALL_BACK_TO_HDFS_FOR_STATS`  is true or false.
   The computeStats code path if disable `ENABLE_FALL_BACK_TO_HDFS_FOR_STATS`:
   https://github.com/apache/spark/blob/dc2a4d4ad478fdb0486cc0515d4fe8b402d24db4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala#L76
   
   
   After SPARK-19678. it's dataSize(397) if enable `ENABLE_FALL_BACK_TO_HDFS_FOR_STATS`, otherwise it's `DEFAULT_SIZE_IN_BYTES`.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

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