spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Gabor Somogyi (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (SPARK-21687) Spark SQL should set createTime for Hive partition
Date Mon, 08 Jan 2018 12:54:00 GMT

    [ https://issues.apache.org/jira/browse/SPARK-21687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16316253#comment-16316253
] 

Gabor Somogyi commented on SPARK-21687:
---------------------------------------

[~debugger87] How was it actually tested? I've tried to reproduce the successful scenario
with hive purely but seems like it also doesn't update the createTime field (which make sense
for me).
I've done the following:

{code:java}
CREATE TABLE partitioned_table (test_column INT) PARTITIONED BY (p INT);
INSERT OVERWRITE TABLE partitioned_table PARTITION(p=1) SELECT 1;
DESCRIBE extended partitioned_table PARTITION(p=1);

Partition(values:[1], dbName:default, tableName:partitioned_table, createTime:1515413589,
lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:test_column, type:int, comment:null),
FieldSchema(name:p, type:int, comment:null)], location:hdfs://hostname:8020/user/hive/warehouse/partitioned_table/p=1,
inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat,
compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,
parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[],
skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{totalSize=2,
numRows=1, rawDataSize=1, COLUMN_STATS_ACCURATE=true, numFiles=1, transient_lastDdlTime=1515413589})

INSERT OVERWRITE TABLE partitioned_table PARTITION(p=1) SELECT 2;
DESCRIBE extended partitioned_table PARTITION(p=1);

Partition(values:[1], dbName:default, tableName:partitioned_table, createTime:1515413589,
lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:test_column, type:int, comment:null),
FieldSchema(name:p, type:int, comment:null)], location:hdfs://hostname:8020/user/hive/warehouse/partitioned_table/p=1,
inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat,
compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,
parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[],
skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{totalSize=2,
numRows=1, rawDataSize=1, COLUMN_STATS_ACCURATE=true, numFiles=1, transient_lastDdlTime=1515413817})
{code}

As I see spark works the same way as hive. I suggest the following options:
- Drop partition and create it again
- Try to use transient_lastDdlTime (of course this gets updated not only when partition overwritten)

What I've found in the meantime is that spark doesn't show the createTime when partition described.
This can be fixed but this is a different topic and will not fix your issue.

Waiting for your thoughts...


> Spark SQL should set createTime for Hive partition
> --------------------------------------------------
>
>                 Key: SPARK-21687
>                 URL: https://issues.apache.org/jira/browse/SPARK-21687
>             Project: Spark
>          Issue Type: Improvement
>          Components: SQL
>    Affects Versions: 2.1.0, 2.2.0
>            Reporter: Chaozhong Yang
>            Priority: Minor
>
> In Spark SQL, we often use `insert overwite table t partition(p=xx)` to create partition
for partitioned table. `createTime` is an important information to manage data lifecycle,
e.g TTL.
> However, we found that Spark SQL doesn't call setCreateTime in `HiveClientImpl#toHivePartition`
as follows:
> {code:scala}
> def toHivePartition(
>       p: CatalogTablePartition,
>       ht: HiveTable): HivePartition = {
>     val tpart = new org.apache.hadoop.hive.metastore.api.Partition
>     val partValues = ht.getPartCols.asScala.map { hc =>
>       p.spec.get(hc.getName).getOrElse {
>         throw new IllegalArgumentException(
>           s"Partition spec is missing a value for column '${hc.getName}': ${p.spec}")
>       }
>     }
>     val storageDesc = new StorageDescriptor
>     val serdeInfo = new SerDeInfo
>     p.storage.locationUri.map(CatalogUtils.URIToString(_)).foreach(storageDesc.setLocation)
>     p.storage.inputFormat.foreach(storageDesc.setInputFormat)
>     p.storage.outputFormat.foreach(storageDesc.setOutputFormat)
>     p.storage.serde.foreach(serdeInfo.setSerializationLib)
>     serdeInfo.setParameters(p.storage.properties.asJava)
>     storageDesc.setSerdeInfo(serdeInfo)
>     tpart.setDbName(ht.getDbName)
>     tpart.setTableName(ht.getTableName)
>     tpart.setValues(partValues.asJava)
>     tpart.setSd(storageDesc)
>     new HivePartition(ht, tpart)
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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


Mime
View raw message