You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Apache Spark (JIRA)" <ji...@apache.org> on 2015/05/02 13:23:06 UTC

[jira] [Assigned] (SPARK-7270) StringType dynamic partition cast to DecimalType in Spark Sql Hive

     [ https://issues.apache.org/jira/browse/SPARK-7270?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Apache Spark reassigned SPARK-7270:
-----------------------------------

    Assignee: Apache Spark

> StringType dynamic partition cast to DecimalType in Spark Sql Hive 
> -------------------------------------------------------------------
>
>                 Key: SPARK-7270
>                 URL: https://issues.apache.org/jira/browse/SPARK-7270
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>            Reporter: Feixiang Yan
>            Assignee: Apache Spark
>
> Create a hive table with two partitons,the first type is bigint and the second type is string.When insert overwrite the table with one static partiton and one dynamic partiton, the second StringType dynamic partition will be cast to DecimalType.
> {noformat}
> desc test;                                                                 
> OK
> a                   	string              	None                
> b                   	bigint              	None                
> c                   	string              	None                
> 	 	 
> # Partition Information	 	 
> # col_name            	data_type           	comment             
> 	 	 
> b                   	bigint              	None                
> c                   	string              	NoneĀ·
> {noformat}
> when run following hive sql in HiveContext
> {noformat}sqlContext.sql("insert overwrite table test partition (b=1,c) select 'a','c' from ptest"){noformat}
> get the result of partition is
> {noformat}test[1,__HIVE_DEFAULT_PARTITION__]{noformat}
> spark log
> {noformat}15/04/30 10:38:09 WARN HiveConf: DEPRECATED: hive.metastore.ds.retry.* no longer has any effect.  Use hive.hmshandler.retry.* instead
> 15/04/30 10:38:09 INFO ParseDriver: Parsing command: insert overwrite table test partition (b=1,c) select 'a','c' from ptest
> 15/04/30 10:38:09 INFO ParseDriver: Parse Completed
> 15/04/30 10:38:09 WARN HiveConf: DEPRECATED: hive.metastore.ds.retry.* no longer has any effect.  Use hive.hmshandler.retry.* instead
> 15/04/30 10:38:10 INFO HiveMetaStore: 0: Opening raw store with implemenation class:org.apache.hadoop.hive.metastore.ObjectStore
> 15/04/30 10:38:10 INFO ObjectStore: ObjectStore, initialize called
> 15/04/30 10:38:10 INFO Persistence: Property datanucleus.cache.level2 unknown - will be ignored
> 15/04/30 10:38:10 INFO Persistence: Property hive.metastore.integral.jdo.pushdown unknown - will be ignored
> 15/04/30 10:38:10 WARN Connection: BoneCP specified but not present in CLASSPATH (or one of dependencies)
> 15/04/30 10:38:10 WARN Connection: BoneCP specified but not present in CLASSPATH (or one of dependencies)
> 15/04/30 10:38:11 WARN HiveConf: DEPRECATED: hive.metastore.ds.retry.* no longer has any effect.  Use hive.hmshandler.retry.* instead
> 15/04/30 10:38:11 INFO ObjectStore: Setting MetaStore object pin classes with hive.metastore.cache.pinobjtypes="Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order"
> 15/04/30 10:38:11 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MFieldSchema" is tagged as "embedded-only" so does not have its own datastore table.
> 15/04/30 10:38:11 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MOrder" is tagged as "embedded-only" so does not have its own datastore table.
> 15/04/30 10:38:12 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MFieldSchema" is tagged as "embedded-only" so does not have its own datastore table.
> 15/04/30 10:38:12 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MOrder" is tagged as "embedded-only" so does not have its own datastore table.
> 15/04/30 10:38:12 INFO Query: Reading in results for query "org.datanucleus.store.rdbms.query.SQLQuery@0" since the connection used is closing
> 15/04/30 10:38:12 INFO ObjectStore: Initialized ObjectStore
> 15/04/30 10:38:12 INFO HiveMetaStore: Added admin role in metastore
> 15/04/30 10:38:12 INFO HiveMetaStore: Added public role in metastore
> 15/04/30 10:38:12 INFO HiveMetaStore: No user is added in admin role, since config is empty
> 15/04/30 10:38:12 INFO SessionState: No Tez session required at this point. hive.execution.engine=mr.
> 15/04/30 10:38:13 INFO HiveMetaStore: 0: get_table : db=default tbl=test
> 15/04/30 10:38:13 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=get_table : db=default tbl=test	
> 15/04/30 10:38:13 INFO HiveMetaStore: 0: get_partitions : db=default tbl=test
> 15/04/30 10:38:13 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=get_partitions : db=default tbl=test	
> 15/04/30 10:38:13 INFO HiveMetaStore: 0: get_table : db=default tbl=ptest
> 15/04/30 10:38:13 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=get_table : db=default tbl=ptest	
> 15/04/30 10:38:13 INFO HiveMetaStore: 0: get_partitions : db=default tbl=ptest
> 15/04/30 10:38:13 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=get_partitions : db=default tbl=ptest	
> 15/04/30 10:38:13 INFO deprecation: mapred.map.tasks is deprecated. Instead, use mapreduce.job.maps
> 15/04/30 10:38:13 INFO MemoryStore: ensureFreeSpace(451930) called with curMem=0, maxMem=2291041566
> 15/04/30 10:38:13 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 441.3 KB, free 2.1 GB)
> 15/04/30 10:38:13 INFO MemoryStore: ensureFreeSpace(71321) called with curMem=451930, maxMem=2291041566
> 15/04/30 10:38:13 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 69.6 KB, free 2.1 GB)
> 15/04/30 10:38:13 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.134.72.169:45859 (size: 69.6 KB, free: 2.1 GB)
> 15/04/30 10:38:13 INFO BlockManagerMaster: Updated info of block broadcast_0_piece0
> 15/04/30 10:38:13 INFO SparkContext: Created broadcast 0 from broadcast at TableReader.scala:68
> 15/04/30 10:38:13 INFO deprecation: mapred.output.compress is deprecated. Instead, use mapreduce.output.fileoutputformat.compress
> 15/04/30 10:38:13 INFO deprecation: mapred.output.compression.codec is deprecated. Instead, use mapreduce.output.fileoutputformat.compress.codec
> 15/04/30 10:38:13 INFO deprecation: mapred.output.compression.type is deprecated. Instead, use mapreduce.output.fileoutputformat.compress.type
> 15/04/30 10:38:14 INFO deprecation: mapred.job.id is deprecated. Instead, use mapreduce.job.id
> 15/04/30 10:38:14 INFO deprecation: mapred.tip.id is deprecated. Instead, use mapreduce.task.id
> 15/04/30 10:38:14 INFO deprecation: mapred.task.id is deprecated. Instead, use mapreduce.task.attempt.id
> 15/04/30 10:38:14 INFO deprecation: mapred.task.is.map is deprecated. Instead, use mapreduce.task.ismap
> 15/04/30 10:38:14 INFO deprecation: mapred.task.partition is deprecated. Instead, use mapreduce.task.partition
> 15/04/30 10:38:14 INFO GPLNativeCodeLoader: Loaded native gpl library
> 15/04/30 10:38:14 INFO LzoCodec: Successfully loaded & initialized native-lzo library [hadoop-lzo rev 7041408c0d57cb3b6f51d004772ccf5073ecc95e]
> 15/04/30 10:38:14 INFO FileInputFormat: Total input paths to process : 1
> 15/04/30 10:38:14 INFO SparkContext: Starting job: runJob at InsertIntoHiveTable.scala:93
> 15/04/30 10:38:14 INFO DAGScheduler: Got job 0 (runJob at InsertIntoHiveTable.scala:93) with 1 output partitions (allowLocal=false)
> 15/04/30 10:38:14 INFO DAGScheduler: Final stage: Stage 0(runJob at InsertIntoHiveTable.scala:93)
> 15/04/30 10:38:14 INFO DAGScheduler: Parents of final stage: List()
> 15/04/30 10:38:14 INFO DAGScheduler: Missing parents: List()
> 15/04/30 10:38:14 INFO DAGScheduler: Submitting Stage 0 (MapPartitionsRDD[5] at mapPartitions at basicOperators.scala:43), which has no missing parents
> 15/04/30 10:38:14 INFO MemoryStore: ensureFreeSpace(125560) called with curMem=523251, maxMem=2291041566
> 15/04/30 10:38:14 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 122.6 KB, free 2.1 GB)
> 15/04/30 10:38:14 INFO MemoryStore: ensureFreeSpace(82648) called with curMem=648811, maxMem=2291041566
> 15/04/30 10:38:14 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 80.7 KB, free 2.1 GB)
> 15/04/30 10:38:14 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.134.72.169:45859 (size: 80.7 KB, free: 2.1 GB)
> 15/04/30 10:38:14 INFO BlockManagerMaster: Updated info of block broadcast_1_piece0
> 15/04/30 10:38:14 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:838
> 15/04/30 10:38:14 INFO DAGScheduler: Submitting 1 missing tasks from Stage 0 (MapPartitionsRDD[5] at mapPartitions at basicOperators.scala:43)
> 15/04/30 10:38:14 INFO YarnClientClusterScheduler: Adding task set 0.0 with 1 tasks
> 15/04/30 10:38:14 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, rsync.slave006.yarn.hadoop.sjs.sogou-op.org, NODE_LOCAL, 1794 bytes)
> 15/04/30 10:38:14 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on rsync.slave006.yarn.hadoop.sjs.sogou-op.org:55678 (size: 80.7 KB, free: 5.3 GB)
> 15/04/30 10:38:16 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on rsync.slave006.yarn.hadoop.sjs.sogou-op.org:55678 (size: 69.6 KB, free: 5.3 GB)
> 15/04/30 10:38:17 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 3152 ms on rsync.slave006.yarn.hadoop.sjs.sogou-op.org (1/1)
> 15/04/30 10:38:17 INFO DAGScheduler: Stage 0 (runJob at InsertIntoHiveTable.scala:93) finished in 3.162 s
> 15/04/30 10:38:17 INFO YarnClientClusterScheduler: Removed TaskSet 0.0, whose tasks have all completed, from pool 
> 15/04/30 10:38:17 INFO DAGScheduler: Job 0 finished: runJob at InsertIntoHiveTable.scala:93, took 3.369777 s
> 15/04/30 10:38:17 INFO HiveMetaStore: 0: partition_name_has_valid_characters
> 15/04/30 10:38:17 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=partition_name_has_valid_characters	
> 15/04/30 10:38:17 INFO HiveMetaStore: 0: partition_name_has_valid_characters
> 15/04/30 10:38:17 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=partition_name_has_valid_characters	
> 15/04/30 10:38:17 WARN UserGroupInformation: No groups available for user root
> 15/04/30 10:38:17 WARN UserGroupInformation: No groups available for user root
> 15/04/30 10:38:17 WARN HiveConf: DEPRECATED: hive.metastore.ds.retry.* no longer has any effect.  Use hive.hmshandler.retry.* instead
> 15/04/30 10:38:17 INFO HiveMetaStore: 0: get_table : db=default tbl=test
> 15/04/30 10:38:17 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=get_table : db=default tbl=test	
> 15/04/30 10:38:17 INFO HiveMetaStore: 0: get_partition_with_auth : db=default tbl=test[1,__HIVE_DEFAULT_PARTITION__]
> 15/04/30 10:38:17 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=get_partition_with_auth : db=default tbl=test[1,__HIVE_DEFAULT_PARTITION__]	
> 15/04/30 10:38:17 INFO Hive: Replacing src:hdfs://yarncluster/tmp/hive-root/hive_2015-04-30_10-38-13_846_3096248751564356035-1/-ext-10000/c=__HIVE_DEFAULT_PARTITION__;dest: hdfs://yarncluster/user/root/hive/warehouse/test/b=1/c=__HIVE_DEFAULT_PARTITION__;Status:true
> 15/04/30 10:38:17 INFO HiveMetaStore: 0: get_partition_with_auth : db=default tbl=test[1,__HIVE_DEFAULT_PARTITION__]
> 15/04/30 10:38:17 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=get_partition_with_auth : db=default tbl=test[1,__HIVE_DEFAULT_PARTITION__]	
> 15/04/30 10:38:17 INFO HiveMetaStore: 0: append_partition : db=default tbl=test[1,__HIVE_DEFAULT_PARTITION__]
> 15/04/30 10:38:17 INFO audit: ugi=root	ip=unknown-ip-addr	cmd=append_partition : db=default tbl=test[1,__HIVE_DEFAULT_PARTITION__]	
> 15/04/30 10:38:17 WARN log: Updating partition stats fast for: test
> 15/04/30 10:38:17 WARN log: Updated size to 10
> 15/04/30 10:38:17 INFO Hive: New loading path = hdfs://yarncluster/tmp/hive-root/hive_2015-04-30_10-38-13_846_3096248751564356035-1/-ext-10000/c=__HIVE_DEFAULT_PARTITION__ with partSpec {b=1, c=__HIVE_DEFAULT_PARTITION__}
> res0: org.apache.spark.sql.SchemaRDD = 
> SchemaRDD[0] at RDD at SchemaRDD.scala:108
> == Query Plan ==
> == Physical Plan ==
> InsertIntoHiveTable (MetastoreRelation default, test, None), Map(b -> Some(1), c -> None), true
>  Project [a AS _c0#0,CAST(CAST(c AS _c1#1, DecimalType()), LongType) AS _c1#8L]
>   HiveTableScan [], (MetastoreRelation default, ptest, None), None
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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