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

liyunzhang commented on HIVE-18301:
-----------------------------------

[~xuefuz],[~csun]: I read jiras about MapInput IOContext problem and enable 
MapInput rdd cache. And found the problem only happens OContext problem with 
multiple MapWorks cloned for multi-insert \[Spark Branch\] like HIVE-8920 
mentioned.
In HIVE-8920, I found the failure case is like
{code}
from (select * from dec union all select * from dec2) s
insert overwrite table dec3 select s.name, sum(s.value) group by s.name
insert overwrite table dec4 select s.name, s.value order by s.value;
{code}
I indeed saw the exception in my hive.log like
{code}
Caused by: java.lang.IllegalStateException: Invalid input path 
hdfs://localhost:8020/user/hive/warehouse/dec2/dec.txt
        at 
org.apache.hadoop.hive.ql.exec.MapOperator.getNominalPath(MapOperator.java:406)
        at 
org.apache.hadoop.hive.ql.exec.MapOperator.cleanUpInputFileChangedOp(MapOperator.java:442)
{code}

here the problem  happens on the MapInput is the union result of dec and dec2 
case. But when I modify case
{code}
from (select * from dec ) s
insert overwrite table dec3 select s.name, sum(s.value) group by s.name
insert overwrite table dec4 select s.name, s.value order by s.value;
{code}
No such exception whether in local or yarn mode.

Whether the problem only happens  in such complicated case( the rdd cache is 
the  union result of two tables)?  If only happen in such complicated case, why 
not only disable MapInput rdd cache in such case? Is there any other reason to 
disable MapInput#rdd cache? Please spend some time to view it as both of you 
have experience on it, thanks!

> Investigate to enable MapInput cache in Hive on Spark
> -----------------------------------------------------
>
>                 Key: HIVE-18301
>                 URL: https://issues.apache.org/jira/browse/HIVE-18301
>             Project: Hive
>          Issue Type: Bug
>            Reporter: liyunzhang
>            Assignee: liyunzhang
>
> Before IOContext problem is found in MapTran when spark rdd cache is enabled 
> in HIVE-8920.
> so we disabled rdd cache in MapTran at 
> [SparkPlanGenerator|https://github.com/kellyzly/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java#L202].
>   The problem is IOContext seems not initialized correctly in the spark yarn 
> client/cluster mode and caused the exception like 
> {code}
> Job aborted due to stage failure: Task 93 in stage 0.0 failed 4 times, most 
> recent failure: Lost task 93.3 in stage 0.0 (TID 616, bdpe48): 
> java.lang.RuntimeException: Error processing row: 
> java.lang.NullPointerException
>       at 
> org.apache.hadoop.hive.ql.exec.spark.SparkMapRecordHandler.processRow(SparkMapRecordHandler.java:165)
>       at 
> org.apache.hadoop.hive.ql.exec.spark.HiveMapFunctionResultList.processNextRecord(HiveMapFunctionResultList.java:48)
>       at 
> org.apache.hadoop.hive.ql.exec.spark.HiveMapFunctionResultList.processNextRecord(HiveMapFunctionResultList.java:27)
>       at 
> org.apache.hadoop.hive.ql.exec.spark.HiveBaseFunctionResultList.hasNext(HiveBaseFunctionResultList.java:85)
>       at 
> scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:42)
>       at 
> org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
>       at org.apache.spark.scheduler.Task.run(Task.scala:85)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.NullPointerException
>       at 
> org.apache.hadoop.hive.ql.exec.AbstractMapOperator.getNominalPath(AbstractMapOperator.java:101)
>       at 
> org.apache.hadoop.hive.ql.exec.MapOperator.cleanUpInputFileChangedOp(MapOperator.java:516)
>       at 
> org.apache.hadoop.hive.ql.exec.Operator.cleanUpInputFileChanged(Operator.java:1187)
>       at 
> org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:546)
>       at 
> org.apache.hadoop.hive.ql.exec.spark.SparkMapRecordHandler.processRow(SparkMapRecordHandler.java:152)
>       ... 12 more
> Driver stacktrace:
> {code}
> in yarn client/cluster mode, sometimes 
> [ExecMapperContext#currentInputPath|https://github.com/kellyzly/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapperContext.java#L109]
>  is null when rdd cach is enabled.



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

Reply via email to