You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-user@hadoop.apache.org by 张鹏 <pe...@xiaomi.com> on 2014/10/16 05:38:48 UTC

Re: Textfile table but some map task try to use SequenceFile reader

Thanks Gogate for help

I changed the hive.input.format to org.apache.hadoop.hive.ql.io.HiveInputFormat, it works.

So I think CombineHiveInputFormat in hive 0.9 works with hadoop 2.0.0, but has incompatibility problem with hadoop 2.4. Maybe some related code changed between 2.0 and 2.4

Answer your questions, Gogate
1. test1 is sequence file table test2 is text file table, and deviceid field's type is string
2. I didn't found hive.query.result.fileformat in hive-default.xml.template in hive 0.9, and not set it explicitly.

发件人: Suhas Gogate <vg...@pivotal.io>>
答复: <us...@hive.apache.org>>
日期: Wed, 15 Oct 2014 18:04:30 -0700
至: <us...@hive.apache.org>>
抄送: "user@hadoop.apache.org<ma...@hadoop.apache.org>" <us...@hadoop.apache.org>>
主题: Re: Textfile table but some map task try to use SequenceFile reader

Peng,

1. Can you share the DDL (create table) for both test1 and test2?  Are they stored as textfile or sequence file?
2. Job1's output is intermediate data and I assume by default hive stores it as textfiles. Have you explicitly set hive.query.result.fileformat  as sequencefile?
3. Also I assume Hive should be using MultipleInputs class in hadoop, if it has to join two different tables (paths) with each having different file formats.

If you can provide the table schema and sample data to reproduce this problem with test1 and test2, I can try.. but I would suggest to upgrade hive to latest :)  or check if you can match intermediate results format match your test2 table format...

--Suhas

On Wed, Oct 15, 2014 at 8:39 AM, 张鹏 <pe...@xiaomi.com>> wrote:
Furthure found:
Works:  select deviceid from (select deviceid  from test1)t1 left outer join (select deviceid from test2)t2 on t1.deviceid=t2. deviceid;
Failed:  select deviceid from (select distinct deviceid  from test1)t1 left outer join (select deviceid from test2)t2 on t1.deviceid=t2. deviceid;

Difference: The second query has two mr jobs, first count distinct id for test1, and then join with test2.
job1's output file's format is sequence file, and map task with job1's output has other paths of test2 file in split, but there's only one input format "SeuqenceFileInputFormat"

So is there any change between hadoop 2.4 and 2.0 for combine split can cause this happen?

Thanks for help.


发件人: Peng <pe...@xiaomi.com>>
日期: Wed, 15 Oct 2014 23:17:51 +0800
至: <us...@hive.apache.org>>
主题: Textfile table but some map task try to use SequenceFile reader

We upgraded from hadoop-2.0.0 to hadoop-2.4.0, without upgrading hive and still using hive 0.9(not recompiled with hadoop2.4)

Normal queries work well, like count and udf, but some queries with JOIN failed.
I found some map tasks failed because HIVE treat inputs type wrong. Input table stored as textfile scan but some map splits format type is SequenceFileInputFormat, ant others are TextInputFormat.

I know Hive 0.9 is very old,but I can't figure out what difference between hadoop 2.4 and 2.0 caused this weired result.

Thanks for help.
Failed map task log as below:


2014-10-15 22:44:41,320 INFO [main] org.apache.hadoop.hive.ql.exec.ReduceSinkOperator: 5 finished. closing...
2014-10-15 22:44:41,320 INFO [main] org.apache.hadoop.hive.ql.exec.ReduceSinkOperator: 5 forwarded 0 rows
2014-10-15 22:44:41,320 INFO [main] org.apache.hadoop.hive.ql.exec.SelectOperator: 4 Close done
2014-10-15 22:44:41,320 INFO [main] org.apache.hadoop.hive.ql.exec.FilterOperator: 3 Close done
2014-10-15 22:44:41,320 INFO [main] org.apache.hadoop.hive.ql.exec.TableScanOperator: 2 Close done
2014-10-15 22:44:41,321 INFO [main] org.apache.hadoop.hive.ql.exec.MapOperator: DESERIALIZE_ERRORS:0
2014-10-15 22:44:41,321 INFO [main] org.apache.hadoop.hive.ql.exec.TableScanOperator: 0 finished. closing...
2014-10-15 22:44:41,321 INFO [main] org.apache.hadoop.hive.ql.exec.TableScanOperator: 0 forwarded 99481 rows
2014-10-15 22:44:41,321 INFO [main] org.apache.hadoop.hive.ql.exec.ReduceSinkOperator: 1 finished. closing...
2014-10-15 22:44:41,321 INFO [main] org.apache.hadoop.hive.ql.exec.ReduceSinkOperator: 1 forwarded 0 rows
2014-10-15 22:44:41,321 INFO [main] org.apache.hadoop.hive.ql.exec.TableScanOperator: 0 Close done
2014-10-15 22:44:41,321 INFO [main] org.apache.hadoop.hive.ql.exec.MapOperator: 10 Close done
2014-10-15 22:44:41,321 INFO [main] ExecMapper: ExecMapper: processed 99481 rows: used memory = 187570680
2014-10-15 22:44:41,327 WARN [main] org.apache.hadoop.mapred.YarnChild: Exception running child : java.io.IOException: java.lang.reflect.InvocationTargetException
at org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97)
at org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57)
at org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:350)
at org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.next(HadoopShimsSecure.java:229)
at org.apache.hadoop.mapred.MapTask$TrackedRecordReader.moveToNext(MapTask.java:198)
at org.apache.hadoop.mapred.MapTask$TrackedRecordReader.next(MapTask.java:184)
at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:52)
at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:430)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:342)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:167)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1589)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:162)
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
at org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:336)
... 11 more
Caused by: java.io.IOException: hdfs://cluster/new_user/createdate=2013-05-21/2013-05-21_204 not a SequenceFile
at org.apache.hadoop.io.SequenceFile$Reader.init(SequenceFile.java:1854)
at org.apache.hadoop.io.SequenceFile$Reader.initialize(SequenceFile.java:1814)
at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1763)
at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1777)
at org.apache.hadoop.mapred.SequenceFileRecordReader.<init>(SequenceFileRecordReader.java:51)
at org.apache.hadoop.mapred.SequenceFileInputFormat.getRecordReader(SequenceFileInputFormat.java:64)
at org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:65)
... 16 more