This clearly indicate the merge still happens due to the conditional task. Can 
you double check if the parameter is set (hive.merge.mapfiles). 

Also if you can also revert it back to use the old map-reduce merging (rather 
than using CombineHiveInputFormat for map-only merging) by setting 
hive.mergejob.maponly=false. 

I'm also curious why CombineHiveInputFormat failed in environment, can you also 
check your task log and see what errors are there (without changing all the 
above parameters)? 

On Sep 27, 2010, at 10:38 AM, Pradeep Kamath wrote:

> Here is the output of explain:
> 
> STAGE DEPENDENCIES:
> Stage-1 is a root stage
> Stage-4 depends on stages: Stage-1 , consists of Stage-3, Stage-2
> Stage-3
> Stage-0 depends on stages: Stage-3, Stage-2
> Stage-2
> 
> STAGE PLANS:
> Stage: Stage-1
>   Map Reduce
>     Alias -> Map Operator Tree:
>       numbers_text
>         TableScan
>           alias: numbers_text
>           Select Operator
>             expressions:
>                   expr: id
>                   type: int
>                   expr: num
>                   type: int
>             outputColumnNames: _col0, _col1
>             File Output Operator
>               compressed: false
>               GlobalTableId: 1
>               table:
>                   input format: org.apache.hadoop.mapred.TextInputFormat
>                   output format: 
> org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
>                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
>                   name: numbers_text_part
> 
> Stage: Stage-4
>   Conditional Operator
> 
> Stage: Stage-3
>   Move Operator
>     files:
>         hdfs directory: true
>         destination: 
> hdfs://wilbur21.labs.corp.sp1.yahoo.com/tmp/hive-pradeepk/hive_2010-09-27_10-37-06_724_1678373180997754320/-ext-10000
> 
> Stage: Stage-0
>   Move Operator
>     tables:
>         partition:
>           part p1
>         replace: true
>         table:
>             input format: org.apache.hadoop.mapred.TextInputFormat
>             output format: 
> org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
>             serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
>             name: numbers_text_part
> 
> Stage: Stage-2
>   Map Reduce
>     Alias -> Map Operator Tree:
>       
> hdfs://wilbur21.labs.corp.sp1.yahoo.com/tmp/hive-pradeepk/hive_2010-09-27_10-37-06_724_1678373180997754320/-ext-10002
>           File Output Operator
>             compressed: false
>             GlobalTableId: 0
>             table:
>                 input format: org.apache.hadoop.mapred.TextInputFormat
>                 output format: 
> org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
>                 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
>                 name: numbers_text_part
> 
> 
> yongqiang he wrote:
>> There is one ticket for insert overwrite local directory:
>> https://issues.apache.org/jira/browse/HIVE-1582
>> 
>> On Mon, Sep 27, 2010 at 9:31 AM, Ning Zhang <nzh...@facebook.com> wrote:
>>  
>>> Can you do explain your query after setting the parameter?
>>> 
>>> 
>>> On Sep 27, 2010, at 9:25 AM, Ashutosh Chauhan wrote:
>>> 
>>>    
>>>> I suspected the same. But, even after setting this property, second MR
>>>> job did get launched and then failed.
>>>> 
>>>> Ashutosh
>>>> On Mon, Sep 27, 2010 at 09:25, Ning Zhang <nzh...@facebook.com> wrote:
>>>>      
>>>>> I'm guessing this is due to the merge task (the 2nd MR job that merges 
>>>>> small
>>>>> files together). You can try to 'set hive.merge.mapfiles=false;' before 
>>>>> the
>>>>> query and see if it succeeded.
>>>>> If it is due to merge job, can you attach the plan and check the
>>>>> mapper/reducer task log and see what errors/exceptions are there?
>>>>> 
>>>>> On Sep 27, 2010, at 9:10 AM, Pradeep Kamath wrote:
>>>>> 
>>>>> Hi,
>>>>> 
>>>>>  Any help in debugging the issue I am seeing below will be greatly
>>>>> appreciated. Unless I am doing something wrong, this seems to be a
>>>>> regression in trunk.
>>>>> 
>>>>> 
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Pradeep
>>>>> 
>>>>> 
>>>>> 
>>>>> ________________________________
>>>>> 
>>>>> From: Pradeep Kamath [mailto:prade...@yahoo-inc.com]
>>>>> Sent: Friday, September 24, 2010 1:41 PM
>>>>> To: hive-user@hadoop.apache.org
>>>>> Subject: Insert overwrite error using hive trunk
>>>>> 
>>>>> 
>>>>> 
>>>>> Hi,
>>>>> 
>>>>>   I am trying to insert overwrite into a partitioned table reading data
>>>>> from a non partitioned table and seeing a failure in the second map reduce
>>>>> job – wonder if I am doing something wrong – any pointers appreciated (I 
>>>>> am
>>>>> using latest trunk code against hadoop 20 cluster). Details below[1].
>>>>> 
>>>>> 
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Pradeep
>>>>> 
>>>>> 
>>>>> 
>>>>> [1]
>>>>> 
>>>>> Details:
>>>>> 
>>>>> bin/hive -e "describe numbers_text;"
>>>>> 
>>>>> col_name                data_type               comment
>>>>> 
>>>>> id                      int                     None
>>>>> 
>>>>> num                     int                     None
>>>>> 
>>>>> 
>>>>> 
>>>>> bin/hive -e "describe numbers_text_part;"
>>>>> 
>>>>> col_name                data_type               comment
>>>>> 
>>>>> id                      int                     None
>>>>> 
>>>>> num                     int                     None
>>>>> 
>>>>> # Partition Information
>>>>> 
>>>>> col_name                data_type               comment
>>>>> 
>>>>> part                    string                  None
>>>>> 
>>>>> 
>>>>> 
>>>>> bin/hive -e "select * from numbers_text;"
>>>>> 
>>>>> 1       10
>>>>> 
>>>>> 2       20
>>>>> 
>>>>> 
>>>>> 
>>>>> bin/hive -e "insert overwrite table numbers_text_part partition(part='p1')
>>>>> select id, num from numbers_text;"
>>>>> 
>>>>> Total MapReduce jobs = 2
>>>>> 
>>>>> Launching Job 1 out of 2
>>>>> 
>>>>> Number of reduce tasks is set to 0 since there's no reduce operator
>>>>> 
>>>>> …
>>>>> 
>>>>> 2010-09-24 13:28:55,649 Stage-1 map = 0%,  reduce = 0%
>>>>> 
>>>>> 2010-09-24 13:28:58,687 Stage-1 map = 100%,  reduce = 0%
>>>>> 
>>>>> 2010-09-24 13:29:01,726 Stage-1 map = 100%,  reduce = 100%
>>>>> 
>>>>> Ended Job = job_201009241059_0281
>>>>> 
>>>>> Ended Job = -1897439470, job is filtered out (removed at runtime).
>>>>> 
>>>>> Launching Job 2 out of 2
>>>>> 
>>>>> Number of reduce tasks is set to 0 since there's no reduce operator
>>>>> 
>>>>> …
>>>>> 
>>>>> 2010-09-24 13:29:03,504 Stage-2 map = 100%,  reduce = 100%
>>>>> 
>>>>> Ended Job = job_201009241059_0282 with errors
>>>>> 
>>>>> FAILED: Execution Error, return code 2 from
>>>>> org.apache.hadoop.hive.ql.exec.MapRedTask
>>>>> 
>>>>> 
>>>>> 
>>>>> tail /tmp/pradeepk/hive.log:
>>>>> 
>>>>> 2010-09-24 13:29:01,888 WARN  mapred.JobClient
>>>>> (JobClient.java:configureCommandLineOptions(539)) - Use 
>>>>> GenericOptionsParser
>>>>> for parsing the arguments. Applications should implement Tool for the 
>>>>> same.
>>>>> 
>>>>> 2010-09-24 13:29:01,903 WARN  fs.FileSystem 
>>>>> (FileSystem.java:fixName(153)) -
>>>>> "wilbur21.labs.corp.sp1.yahoo.com:8020" is a deprecated filesystem name. 
>>>>> Use
>>>>> "hdfs://wilbur21.labs.corp.sp1.yahoo.com:8020/" instead.
>>>>> 
>>>>> 2010-09-24 13:29:03,512 ERROR exec.MapRedTask
>>>>> (SessionState.java:printError(277)) - Ended Job = job_201009241059_0282 
>>>>> with
>>>>> errors
>>>>> 
>>>>> 2010-09-24 13:29:03,537 ERROR ql.Driver 
>>>>> (SessionState.java:printError(277))
>>>>> - FAILED: Execution Error, return code 2 from
>>>>> org.apache.hadoop.hive.ql.exec.MapRedTask
>>>>> 
>>>>>        
>>>    
> 

Reply via email to