hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jean-Marc Spaggiari <jean-m...@spaggiari.org>
Subject Re: HFileOutputFormat2 + HBase 1.0.0
Date Wed, 29 Apr 2015 00:49:38 GMT
Which HBase version are you running that with?

2015-04-28 20:42 GMT-04:00 张铎 <palomino219@gmail.com>:

> Then this is a Map-Only job? Try this?
> job.setNumReduceTasks(0);
>
> This is my code to setup a Map-Only job which generates HFiles for bulkload
>
> job.setMapperClass(ImportMapper.class);
> job.setNumReduceTasks(0);
> job.setOutputKeyClass(ImmutableBytesWritable.class);
> job.setOutputValueClass(Cell.class);
> job.setOutputFormatClass(HFileOutputFormat2.class);
>
> And I found that there are Serializers for KeyValue and Put... I found this
> in HFileOutputFormat2
>
>     conf.setStrings("io.serializations", conf.get("io.serializations"),
>         MutationSerialization.class.getName(),
> ResultSerialization.class.getName(),
>         KeyValueSerialization.class.getName());
>
> 2015-04-29 8:31 GMT+08:00 Jean-Marc Spaggiari <jean-marc@spaggiari.org>:
>
> > Hum. My goal was to output the Puts directly from the Mapper, so seems
> that
> > this is not doable, right? You will always need to have the reducers to
> > create as many files has you have regions. Since I have just one, I was
> > trying to save some steps and generate the output directly from the
> mapper
> > side. So I guess this is not doable.
> >
> > I will try to the Reducer then...
> >
> > 2015-04-28 20:27 GMT-04:00 张铎 <palomino219@gmail.com>:
> >
> > > HFileOutputFormat2 is used at the final output step, i.e, reduce output
> > in
> > > MR job or map output in Map-Only job. It uses ImmutableBytesWritable
> and
> > > Cell as key and value.
> > > So I think your problem is not related to HFileOutputFormat2? If you
> want
> > > to use KeyValue or Put at the shuffle step(the output types of Mapper
> and
> > > input types of Reducer), you should implement Serializers for them by
> > > yourself.
> > > And I suggest to use general Writable classes at the shuffle step, and
> > > convert them to ImmutableBytesWritable and Cell in Reducer then collect
> > > them out.
> > >
> > > Thanks.
> > >
> > > 2015-04-29 7:49 GMT+08:00 Jean-Marc Spaggiari <jean-marc@spaggiari.org
> >:
> > >
> > > > ImmutableBytesWritable works because it implements
> WritableComparable..
> > > The
> > > > others don't. So make sense.
> > > >
> > > > Now question is. Should Put implement it too? If not, how are we
> > > expecting
> > > > HFileOutputFormat2 to work with MR? Or at least Writable?
> > > >
> > > > 2015-04-28 18:43 GMT-04:00 Jean-Marc Spaggiari <
> > jean-marc@spaggiari.org
> > > >:
> > > >
> > > > > Hi all,
> > > > >
> > > > > Quick question. I'm trying to do a very simple MR job just doing
> > > > > nothing... Just to try to get it run.
> > > > >
> > > > > But as soon as I set the output value to be KeyValue or Put, I get
> > > > > exception from the MR framework.
> > > > >
> > > > > The exception is the following:
> > > > > java.lang.Exception: java.lang.NullPointerException
> > > > >     at
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.mapred.LocalJobRunner$Job.runTasks(LocalJobRunner.java:462)
> > > > >     at
> > > > >
> > >
> org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:522)
> > > > > Caused by: java.lang.NullPointerException
> > > > >     at
> > > > >
> > org.apache.hadoop.mapred.MapTask$MapOutputBuffer.init(MapTask.java:988)
> > > > >     at
> > > > >
> > >
> org.apache.hadoop.mapred.MapTask.createSortingCollector(MapTask.java:391)
> > > > >     at org.apache.hadoop.mapred.MapTask.access$100(MapTask.java:80)
> > > > >     at
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.mapred.MapTask$NewOutputCollector.<init>(MapTask.java:675)
> > > > >     at
> > org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:747)
> > > > >     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:340)
> > > > >     at
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.mapred.LocalJobRunner$Job$MapTaskRunnable.run(LocalJobRunner.java:243)
> > > > >     at
> > > > >
> > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> > > > >     at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> > > > >     at
> > > > >
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > > >     at
> > > > >
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > > >     at java.lang.Thread.run(Thread.java:745)
> > > > >
> > > > > If you look into the code, that means that Hadoop is not able to
> > > > serialize
> > > > > KeyValue nor Put and so is not able to usethem in the Mapper class.
> > > > >
> > > > > To validate, I tried this:
> > > > >       SerializationFactory serializationFactory = new
> > > > > SerializationFactory(conf);
> > > > >       System.out.println
> > > > > (serializationFactory.getSerializer(KeyValue.class));
> > > > >       System.out.println
> > > (serializationFactory.getSerializer(Put.class));
> > > > >       System.out.println
> > > > (serializationFactory.getSerializer(Cell.class));
> > > > > And they all return null. Which is consistent with the exception.
> > > > >
> > > > > So you don't even need to run MR to get it fails. Just a small main
> > > with
> > > > > those 4 lines.
> > > > >
> > > > > Am I missing something? Like, doing some initialization to help
> > Hadoop
> > > to
> > > > > serialize those classes?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > JM
> > > > >
> > > >
> > >
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message