You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "Sun Rui (JIRA)" <ji...@apache.org> on 2013/11/26 11:12:35 UTC

[jira] [Created] (HIVE-5891) Alias conflict when merging multiple mapjoin tasks into their common child mapred task

Sun Rui created HIVE-5891:
-----------------------------

             Summary: Alias conflict when merging multiple mapjoin tasks into their common child mapred task
                 Key: HIVE-5891
                 URL: https://issues.apache.org/jira/browse/HIVE-5891
             Project: Hive
          Issue Type: Bug
          Components: Query Processor
    Affects Versions: 0.12.0
            Reporter: Sun Rui


Use the following test case with HIVE 0.12:

{quote}
create table src(key int, value string);
load data local inpath '/home/ray/hive-0.11.0/src/data/files/kv1.txt' overwrite into table src;
select * from (
  select c.key from
    (select a.key from src a join src b on a.key=b.key group by a.key) tmp
    join src c on tmp.key=c.key
  union all
  select c.key from
    (select a.key from src a join src b on a.key=b.key group by a.key) tmp
    join src c on tmp.key=c.key
) x;
{quote}

We will get a NullPointerException from Union Operator:

{quote}
java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"_col0":0}
	at org.apache.hadoop.hive.ql.exec.mr.ExecMapper.map(ExecMapper.java:175)
	at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:50)
	at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:436)
	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:372)
	at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:212)
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"_col0":0}
	at org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:544)
	at org.apache.hadoop.hive.ql.exec.mr.ExecMapper.map(ExecMapper.java:157)
	... 4 more
Caused by: java.lang.NullPointerException
	at org.apache.hadoop.hive.ql.exec.UnionOperator.processOp(UnionOperator.java:120)
	at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:504)
	at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:842)
	at org.apache.hadoop.hive.ql.exec.SelectOperator.processOp(SelectOperator.java:88)
	at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:504)
	at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:842)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:652)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:655)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.checkAndGenObject(CommonJoinOperator.java:758)
	at org.apache.hadoop.hive.ql.exec.MapJoinOperator.processOp(MapJoinOperator.java:220)
	at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:504)
	at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:842)
	at org.apache.hadoop.hive.ql.exec.TableScanOperator.processOp(TableScanOperator.java:91)
	at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:504)
	at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:842)
	at org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:534)
	... 5 more
{quote}
  
The root cause is in CommonJoinTaskDispatcher.mergeMapJoinTaskIntoItsChildMapRedTask().
{panel}
  +--------------+      +--------------+
  | MapJoin task |      | MapJoin task |
  +--------------+      +--------------+
             \             /
              \           /
             +--------------+
             |  Union task  |
             +--------------+
{panel}
             
CommonJoinTaskDispatcher merges the two MapJoin tasks into their common child: Union task. The two MapJoin tasks
have the same alias name for their big tables: $INTNAME, which is the name of the temporary table of a join stream.
The aliasToWork map uses alias as key, so eventually only the MapJoin operator tree of one MapJoin task is saved
into the aliasToWork map of the Union task, while the MapJoin operator tree of another MapJoin task is lost.
As a result, Union operator won't be initialized because not all of its parents gets intialized (The Union operator
itself indicates it has two parents, but actually it has only 1 parent because another parent is lost).

This issue does not exist in HIVE 0.11 and thus is a regression bug in HIVE 0.12.

The propsed solution is to use the query ID as prefix for the join stream name to avoid conflict and add sanity check
code in CommonJoinTaskDispatcher that merge of a MapJoin task into its child MapRed task is skipped if there is any
alias conflict. Please review the patch. I am not sure if the patch properly handles the case of DemuxOperator.



--
This message was sent by Atlassian JIRA
(v6.1#6144)