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

Adam Szita commented on HIVE-18030:
-----------------------------------

The root cause of the problem is that HCatalog fails to figure out that the 
jobContext of the PoS job indeed has backend context. (This then causes to 
program to proceed in unwanted code paths)

This feature was working previously by {{mapred.task.id}} property being set 
for Pig on MR/Tez jobs. In Spark mode this property is not used so I had to 
extend to condition used in {{checkJobContextIfRunningFromBackend()}} so it 
recognises PoS jobs as backend too.

[~xuefuz], [~kellyzly] can you take a look on this patch please?

> HCatalog can't be used with Pig on Spark
> ----------------------------------------
>
>                 Key: HIVE-18030
>                 URL: https://issues.apache.org/jira/browse/HIVE-18030
>             Project: Hive
>          Issue Type: Bug
>          Components: HCatalog
>            Reporter: Adam Szita
>            Assignee: Adam Szita
>         Attachments: HIVE-18030.0.patch
>
>
> When using Pig on Spark in cluster mode, all queries containing HCatalog 
> access are failing:
> {code}
> 2017-11-03 12:39:19,268 [dispatcher-event-loop-19] INFO  
> org.apache.spark.storage.BlockManagerInfo - Added broadcast_6_piece0 in 
> memory on <<host_not_shown>>:<<port_not_shown>> (size: 83.0 KB, free: 408.5 
> MB)
> 2017-11-03 12:39:19,277 [task-result-getter-0] WARN  
> org.apache.spark.scheduler.TaskSetManager - Lost task 0.0 in stage 0.0 (TID 
> 0, <<host_not_shown>>, executor 2): java.lang.NullPointerException
>       at org.apache.hadoop.security.Credentials.addAll(Credentials.java:401)
>       at org.apache.hadoop.security.Credentials.addAll(Credentials.java:388)
>       at 
> org.apache.hive.hcatalog.pig.HCatLoader.setLocation(HCatLoader.java:128)
>       at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.mergeSplitSpecificConf(PigInputFormat.java:147)
>       at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat$RecordReaderFactory.<init>(PigInputFormat.java:115)
>       at 
> org.apache.pig.backend.hadoop.executionengine.spark.running.PigInputFormatSpark$SparkRecordReaderFactory.<init>(PigInputFormatSpark.java:126)
>       at 
> org.apache.pig.backend.hadoop.executionengine.spark.running.PigInputFormatSpark.createRecordReader(PigInputFormatSpark.java:70)
>       at 
> org.apache.spark.rdd.NewHadoopRDD$$anon$1.liftedTree1$1(NewHadoopRDD.scala:180)
>       at 
> org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(NewHadoopRDD.scala:179)
>       at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:134)
>       at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:69)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
>       at org.apache.spark.scheduler.Task.run(Task.scala:108)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:338)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>       at java.lang.Thread.run(Thread.java:748)
> {code}



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

Reply via email to