[ https://issues.apache.org/jira/browse/SPARK-27497?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Hyukjin Kwon reassigned SPARK-27497: ------------------------------------ Assignee: Bruce Robbins > Spark wipes out bucket spec in metastore when updating table stats > ------------------------------------------------------------------ > > Key: SPARK-27497 > URL: https://issues.apache.org/jira/browse/SPARK-27497 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 2.4.0, 3.0.0 > Reporter: Bruce Robbins > Assignee: Bruce Robbins > Priority: Major > Fix For: 2.4.6, 3.1.0 > > > The bucket spec gets wiped out after Spark writes to a Hive-bucketed table > that has the following characteristics: > - table is created by Hive (or even Spark, if you use HQL DDL) > - table is stored in Parquet format > - table has at least one Hive-created data file already > Also, spark.sql.hive.convertMetastoreParquet has to be set to true (the > default). > For example, do the following in Hive: > {noformat} > hive> create table sourcetable as select 1 a, 3 b, 7 c; > hive> drop table hivebucket1; > hive> create table hivebucket1 (a int, b int, c int) clustered by (a, b) > sorted by (a, b asc) into 10 buckets stored as parquet; > hive> insert into hivebucket1 select * from sourcetable; > hive> show create table hivebucket1; > OK > CREATE TABLE `hivebucket1`( > `a` int, > `b` int, > `c` int) > CLUSTERED BY ( > a, > b) > SORTED BY ( > a ASC, > b ASC) > INTO 10 BUCKETS > 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 > 'file:/Users/brobbins/github/spark_upstream/spark-warehouse/hivebucket1' > TBLPROPERTIES ( > 'COLUMN_STATS_ACCURATE'='true', > 'numFiles'='1', > 'numRows'='1', > 'rawDataSize'='3', > 'totalSize'='352', > 'transient_lastDdlTime'='1555542971') > Time taken: 0.056 seconds, Fetched: 26 row(s) > hive> > {noformat} > Then in spark-shell, do the following: > {noformat} > scala> sql("insert into hivebucket1 select 1, 3, 7") > 19/04/17 10:49:30 WARN ObjectStore: Failed to get database global_temp, > returning NoSuchObjectException > res0: org.apache.spark.sql.DataFrame = [] > {noformat} > Note: At this point, I would have expected Spark to throw an > {{AnalysisException}} with the message "Output Hive table > `default`.`hivebucket1` is bucketed...". However, I am ignoring that for now > and may open a separate Jira (SPARK-27498). > Return to some Hive CLI and note that the bucket specification is gone from > the table definition: > {noformat} > hive> show create table hivebucket1; > OK > CREATE TABLE `hivebucket1`( > `a` int, > `b` int, > `c` int) > 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 > '<location>' > TBLPROPERTIES ( > 'COLUMN_STATS_ACCURATE'='false', > 'SORTBUCKETCOLSPREFIX'='TRUE', > 'numFiles'='2', > 'numRows'='-1', > 'rawDataSize'='-1', > 'totalSize'='1144', > 'transient_lastDdlTime'='1555523374') > Time taken: 1.619 seconds, Fetched: 20 row(s) > hive> > {noformat} > This information is lost when Spark attempts to update table stats. > HiveClientImpl.toHiveTable drops the bucket specification. toHiveTable drops > the bucket information because {{table.provider}} is None instead of "hive". > {{table.provider}} is not "hive" because Spark bypassed the serdes and used > the built-in parquet code path (by default, > spark.sql.hive.convertMetastoreParquet is true). -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org