hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stack <st...@duboce.net>
Subject Re: Bulk Load Sample Code
Date Fri, 12 Nov 2010 16:50:20 GMT
Look in jobtracker interface.  Figure which  jobs are running.  Are
they progressing?  Probably not.  Go to the machine that is hosting
the task.  Tail tasktracker logs.  Figure who the map is trying to
talk too.  Stack dump the map task for good measure figure where its
blocked.  If its trying to talk to hbase or hdfs, go where its
connecting.  Tail logs there and/or stack trace.

The task timesout after ten minutes.  Check logs then.  Might be a clue.

Sounds like mismatched jars or something your program is doing.

St.Ack


On Fri, Nov 12, 2010 at 8:01 AM, Shuja Rehman <shujamughal@gmail.com> wrote:
> St.Ack,
>
> Here is one new problem now. When I run the job with 1 file, everything goes
> smooth. but when i give a set of files as input, it stuck and did not doing
> anything. here is output.
>
>
> 10/11/12 07:42:54 INFO mapreduce.HFileOutputFormat: Looking up current
> regions for table org.apache.hadoop.hbase.client.HTable@55bb93
> 10/11/12 07:42:54 INFO mapreduce.HFileOutputFormat: Configuring 1 reduce
> partitions to match current region count
> 10/11/12 07:42:54 INFO mapreduce.HFileOutputFormat: Writing partition
> information to
> hdfs://app4.hsd1.wa.comcast.net./user/root/partitions_1289576574949
> 10/11/12 07:42:55 INFO util.NativeCodeLoader: Loaded the native-hadoop
> library
> 10/11/12 07:42:55 INFO zlib.ZlibFactory: Successfully loaded & initialized
> native-zlib library
> 10/11/12 07:42:55 INFO compress.CodecPool: Got brand-new compressor
> 10/11/12 07:42:55 INFO mapreduce.HFileOutputFormat: Incremental table output
> configured.
> 10/11/12 07:42:55 WARN mapred.JobClient: Use GenericOptionsParser for
> parsing the arguments. Applications should implement Tool for the same.
> 10/11/12 07:42:56 INFO input.FileInputFormat: Total input paths to process :
> 96
> 10/11/12 07:42:56 INFO mapred.JobClient: Running job: job_201011120442_0004
> 10/11/12 07:42:57 INFO mapred.JobClient:  map 0% reduce 0%
>
> Any guess why it is not proceeding forward?
>
> Thanks
>
>
> On Fri, Nov 12, 2010 at 8:04 PM, Shuja Rehman <shujamughal@gmail.com> wrote:
>
>> Thanks St.Ack
>>
>> It solved the problem.
>>
>>
>> On Fri, Nov 12, 2010 at 7:41 PM, Stack <stack@duboce.net> wrote:
>>
>>> Fix your classpath.  Add the google library.  See
>>>
>>> http://hbase.apache.org/docs/r0.89.20100924/apidocs/org/apache/hadoop/hbase/mapreduce/package-summary.html#package_description
>>> for more on classpath.
>>>
>>> St.Ack
>>>
>>>
>>> On Fri, Nov 12, 2010 at 5:07 AM, Shuja Rehman <shujamughal@gmail.com>
>>> wrote:
>>> > Hi
>>> >
>>> > I am trying to use configureIncrementalLoad() function to handle the
>>> > totalOrderPartitioning but it throws this exception.
>>> >
>>> > 10/11/12 05:02:21 INFO zookeeper.ClientCnxn: Opening socket connection
>>> to
>>> > server /10.10.10.2:2181
>>> > 10/11/12 05:02:21 INFO zookeeper.ClientCnxn: Socket connection
>>> established
>>> > to app4.hsd1.wa.comcast.net./10.10.10.2:2181, initiating session
>>> > 10/11/12 05:02:21 INFO zookeeper.ClientCnxn: Session establishment
>>> complete
>>> > on server app4.hsd1.wa.comcast.net./10.10.10.2:2181, sessionid =
>>> > 0x12c401bfdae0008, negotiated timeout = 40000
>>> > 10/11/12 05:02:21 INFO mapreduce.HFileOutputFormat: Looking up current
>>> > regions for table org.apache.hadoop.hbase.client.HTable@21e554
>>> > 10/11/12 05:02:21 INFO mapreduce.HFileOutputFormat: Configuring 1 reduce
>>> > partitions to match current region count
>>> > 10/11/12 05:02:21 INFO mapreduce.HFileOutputFormat: Writing partition
>>> > information to
>>> > hdfs://app4.hsd1.wa.comcast.net./user/root/partitions_1289566941504
>>> > Exception in thread "main" java.lang.NoClassDefFoundError:
>>> > com/google/common/base/Preconditions
>>> >        at
>>> >
>>> org.apache.hadoop.hbase.mapreduce.HFileOutputFormat.writePartitions(HFileOutputFormat.java:185)
>>> >        at
>>> >
>>> org.apache.hadoop.hbase.mapreduce.HFileOutputFormat.configureIncrementalLoad(HFileOutputFormat.java:258)
>>> >        at ParserDriver.runJob(ParserDriver.java:162)
>>> >        at ParserDriver.main(ParserDriver.java:109)
>>> >        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>> >        at
>>> >
>>> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>>> >        at
>>> >
>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>> >        at java.lang.reflect.Method.invoke(Method.java:597)
>>> >        at org.apache.hadoop.util.RunJar.main(RunJar.java:186)
>>> > Caused by: java.lang.ClassNotFoundException:
>>> > com.google.common.base.Preconditions
>>> >        at java.net.URLClassLoader$1.run(URLClassLoader.java:202)
>>> >        at java.security.AccessController.doPrivileged(Native Method)
>>> >        at java.net.URLClassLoader.findClass(URLClassLoader.java:190)
>>> >        at java.lang.ClassLoader.loadClass(ClassLoader.java:307)
>>> >        at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:301)
>>> >        at java.lang.ClassLoader.loadClass(ClassLoader.java:248)
>>> >        ... 9 more
>>> > 10/11/12 05:02:21 INFO zookeeper.ZooKeeper: Session: 0x12c401bfdae0008
>>> > closed
>>> >
>>> > Here is the code.
>>> >
>>> >  Configuration conf = HBaseConfiguration.create();
>>> >
>>> >   Job job = new Job(conf, "j");
>>> >
>>> >    HTable table = new HTable(conf, "mytab");
>>> >
>>> >    FileInputFormat.setInputPaths(job, input);
>>> >    job.setJarByClass(ParserDriver.class);
>>> >    job.setMapperClass(MyParserMapper.class);
>>> >
>>> >    job.setInputFormatClass(XmlInputFormat.class);
>>> >    job.setReducerClass(PutSortReducer.class);
>>> >    Path outPath = new Path(output);
>>> >    FileOutputFormat.setOutputPath(job, outPath);
>>> >
>>> >    job.setMapOutputValueClass(Put.class);
>>> >    job.setMapOutputKeyClass(ImmutableBytesWritable.class);
>>> > *    HFileOutputFormat.configureIncrementalLoad(job, table);*
>>> >    TableMapReduceUtil.addDependencyJars(job);
>>> >     job.waitForCompletion(true);
>>> >
>>> > I guess, there are some jar files missing. if yes then from where to get
>>> > these?
>>> >
>>> > Thanks
>>> >
>>> > On Thu, Nov 11, 2010 at 12:57 AM, Stack <stack@duboce.net> wrote:
>>> >
>>> >> On Wed, Nov 10, 2010 at 11:53 AM, Shuja Rehman <shujamughal@gmail.com>
>>> >> wrote:
>>> >> > oh! I think u have not read the full post. The essay has 3 paragraphs
>>>  :)
>>> >> >
>>> >> > *Should I need to add the following line also
>>> >> >
>>> >> >  job.setPartitionerClass(TotalOrderPartitioner.class);
>>> >> >
>>> >>
>>> >> You need to specify other than default partitioner so yes, above seems
>>> >> necessary (Be aware that if only one reducer, all may appear to work
>>> >> though your partitioner is bad... its when you have multiple reducers
>>> >> that bad partitioner will show).
>>> >>
>>> >> > which book? OReilly.Hadoop.The.Definitive.Guide.Jun.2009?
>>> >> >
>>> >>
>>> >> Yes.  Or 2nd edition, October 2010.
>>> >>
>>> >> St.Ack
>>> >>
>>> >> >
>>> >> >
>>> >> >
>>> >> > On Thu, Nov 11, 2010 at 12:49 AM, Stack <stack@duboce.net>
wrote:
>>> >> >
>>> >> >> Which two questions (you wrote an essay that looked like one
big
>>> >> >> question -- smile).
>>> >> >> St.Ack
>>> >> >>
>>> >> >> On Wed, Nov 10, 2010 at 11:44 AM, Shuja Rehman <
>>> shujamughal@gmail.com>
>>> >> >> wrote:
>>> >> >> > yeah, I tried it and it did not fails. can u answer other
2
>>> questions
>>> >> as
>>> >> >> > well?
>>> >> >> >
>>> >> >> >
>>> >> >> >
>>> >> >> > On Thu, Nov 11, 2010 at 12:15 AM, Stack <stack@duboce.net>
wrote:
>>> >> >> >
>>> >> >> >> All below looks reasonable (I did not do detailed
review of your
>>> code
>>> >> >> >> posting).  Have you tried it?  Did it fail?
>>> >> >> >> St.Ack
>>> >> >> >>
>>> >> >> >> On Wed, Nov 10, 2010 at 11:12 AM, Shuja Rehman <
>>> >> shujamughal@gmail.com>
>>> >> >> >> wrote:
>>> >> >> >> > On Wed, Nov 10, 2010 at 9:20 PM, Stack <stack@duboce.net>
>>> wrote:
>>> >> >> >> >
>>> >> >> >> >> What you need?  bulk-upload, in the scheme
of things, is a
>>> well
>>> >> >> >> >> documented feature.  Its also one that has
had some exercise
>>> and
>>> >> is
>>> >> >> >> >> known to work well.  For a 0.89 release
and trunk,
>>> documentation
>>> >> is
>>> >> >> >> >> here:
>>> http://hbase.apache.org/docs/r0.89.20100924/bulk-loads.html
>>> >> .
>>> >> >> >> >> The unit test you refer to below is good
for figuring how to
>>> run a
>>> >> >> job
>>> >> >> >> >> (Bulk-upload was redone for 0.89/trunk and
is much improved
>>> over
>>> >> what
>>> >> >> >> >> was available in 0.20.x)
>>> >> >> >> >>
>>> >> >> >> >
>>> >> >> >> > *I need to load data into hbase using Hfiles.
 *
>>> >> >> >> >
>>> >> >> >> > Ok, let me tell what I understand from all these
things.
>>> Basically
>>> >> >> there
>>> >> >> >> are
>>> >> >> >> > two ways to bulk load into hbase.
>>> >> >> >> >
>>> >> >> >> > 1- Using Command Line tools (importtsv, completebulkload
)
>>> >> >> >> > 2- Mapreduce job using HFileOutputFormat
>>> >> >> >> >
>>> >> >> >> > At the moment, I have generated the Hfiles using
>>> HFileOutputFormat
>>> >> and
>>> >> >> >> > loading these files into hbase using completebulkload
command
>>> line
>>> >> >> tool.
>>> >> >> >> > here is my basic code skeleton. Correct me if
I do anything
>>> wrong.
>>> >> >> >> >
>>> >> >> >> > Configuration conf = new Configuration();
>>> >> >> >> > Job job = new Job(conf, "myjob");
>>> >> >> >> >
>>> >> >> >> >    FileInputFormat.setInputPaths(job, input);
>>> >> >> >> >    job.setJarByClass(ParserDriver.class);
>>> >> >> >> >    job.setMapperClass(MyParserMapper.class);
>>> >> >> >> >    job.setNumReduceTasks(1);
>>> >> >> >> >    job.setInputFormatClass(XmlInputFormat.class);
>>> >> >> >> >    job.setOutputFormatClass(HFileOutputFormat.class);
>>> >> >> >> >    job.setOutputKeyClass(ImmutableBytesWritable.class);
>>> >> >> >> >    job.setOutputValueClass(Put.class);
>>> >> >> >> >    job.setReducerClass(PutSortReducer.class);
>>> >> >> >> >
>>> >> >> >> >    Path outPath = new Path(output);
>>> >> >> >> >    FileOutputFormat.setOutputPath(job, outPath);
>>> >> >> >> >          job.waitForCompletion(true);
>>> >> >> >> >
>>> >> >> >> > and here is mapper skeleton
>>> >> >> >> >
>>> >> >> >> > public class MyParserMapper   extends
>>> >> >> >> >    Mapper<LongWritable, Text, ImmutableBytesWritable,
Put> {
>>> >> >> >> >  while(true)
>>> >> >> >> >   {
>>> >> >> >> >       Put put = new Put(rowId);
>>> >> >> >> >      put.add(...);
>>> >> >> >> >      context.write(rwId, put);
>>> >> >> >> >   }
>>> >> >> >> >
>>> >> >> >> > The link says:
>>> >> >> >> > *In order to function efficiently, HFileOutputFormat
must be
>>> >> >> configured
>>> >> >> >> such
>>> >> >> >> > that each output HFile fits within a single region.
In order to
>>> do
>>> >> >> this,
>>> >> >> >> > jobs use Hadoop's TotalOrderPartitioner class
to partition the
>>> map
>>> >> >> output
>>> >> >> >> > into disjoint ranges of the key space, corresponding
to the key
>>> >> ranges
>>> >> >> of
>>> >> >> >> > the regions in the table. *"
>>> >> >> >> >
>>> >> >> >> > Now according to my configuration above  where
i need to set
>>> >> >> >> > *TotalOrderPartitioner
>>> >> >> >> > ? *Should I need to add the following line also
>>> >> >> >> >
>>> >> >> >> > job.setPartitionerClass(TotalOrderPartitioner.class);
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> > On totalorderpartition, this is a partitioner
class from
>>> hadoop.
>>> >>  The
>>> >> >> >> >> MR partitioner -- the class that dictates
which reducers get
>>> what
>>> >> map
>>> >> >> >> >> outputs -- is pluggable. The default partitioner
does a hash
>>> of
>>> >> the
>>> >> >> >> >> output key to figure which reducer.  This
won't work if you
>>> are
>>> >> >> >> >> looking to have your hfile output totally
sorted.
>>> >> >> >> >>
>>> >> >> >> >>
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> >> If you can't figure what its about, I'd suggest
you check out
>>> the
>>> >> >> >> >> hadoop book where it gets a good explication.
>>> >> >> >> >>
>>> >> >> >> >>   which book? OReilly.Hadoop.The.Definitive.Guide.Jun.2009?
>>> >> >> >> >
>>> >> >> >> > On incremental upload, the doc. suggests you
look at the output
>>> for
>>> >> >> >> >> LoadIncrementalHFiles command.  Have you
done that?  You run
>>> the
>>> >> >> >> >> command and it'll add in whatever is ready
for loading.
>>> >> >> >> >>
>>> >> >> >> >
>>> >> >> >> >   I just use the command line tool for bulk
uplaod but not seen
>>> >> >> >> > LoadIncrementalHFiles  class yet to do it through
program
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> >  ------------------------------
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> >>
>>> >> >> >> >> St.Ack
>>> >> >> >> >>
>>> >> >> >> >>
>>> >> >> >> >> On Wed, Nov 10, 2010 at 6:47 AM, Shuja Rehman
<
>>> >> shujamughal@gmail.com
>>> >> >> >
>>> >> >> >> >> wrote:
>>> >> >> >> >> > Hey Community,
>>> >> >> >> >> >
>>> >> >> >> >> > Well...it seems that nobody has experienced
with the bulk
>>> load
>>> >> >> option.
>>> >> >> >> I
>>> >> >> >> >> > have found one class which might help
to write the code for
>>> it.
>>> >> >> >> >> >
>>> >> >> >> >> >
>>> >> >> >> >>
>>> >> >> >>
>>> >> >>
>>> >>
>>> https://svn.apache.org/repos/asf/hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
>>> >> >> >> >> >
>>> >> >> >> >> > From this, you can get the idea how
to write map reduce job
>>> to
>>> >> >> output
>>> >> >> >> in
>>> >> >> >> >> > HFiles format. But There is a little
confusion about these
>>> two
>>> >> >> things
>>> >> >> >> >> >
>>> >> >> >> >> > 1-TotalOrderPartitioner
>>> >> >> >> >> > 2-configureIncrementalLoad
>>> >> >> >> >> >
>>> >> >> >> >> > Does anybody have idea about how these
things and how to
>>> >> configure
>>> >> >> it
>>> >> >> >> for
>>> >> >> >> >> > the job?
>>> >> >> >> >> >
>>> >> >> >> >> > Thanks
>>> >> >> >> >> >
>>> >> >> >> >> >
>>> >> >> >> >> >
>>> >> >> >> >> > On Wed, Nov 10, 2010 at 1:02 AM, Shuja
Rehman <
>>> >> >> shujamughal@gmail.com>
>>> >> >> >> >> wrote:
>>> >> >> >> >> >
>>> >> >> >> >> >> Hi
>>> >> >> >> >> >>
>>> >> >> >> >> >> I am trying to investigate the bulk
load option as
>>> described in
>>> >> >> the
>>> >> >> >> >> >> following link.
>>> >> >> >> >> >>
>>> >> >> >> >> >>
>>> http://hbase.apache.org/docs/r0.89.20100621/bulk-loads.html
>>> >> >> >> >> >>
>>> >> >> >> >> >> Does anybody have sample code or
have used it before?
>>> >> >> >> >> >> Can it be helpful to insert data
into existing table. In my
>>> >> >> scenario,
>>> >> >> >> I
>>> >> >> >> >> >> have one table with 1 column family
in which data will be
>>> >> inserted
>>> >> >> >> every
>>> >> >> >> >> 15
>>> >> >> >> >> >> minutes.
>>> >> >> >> >> >>
>>> >> >> >> >> >> Kindly share your experiences
>>> >> >> >> >> >>
>>> >> >> >> >> >> Thanks
>>> >> >> >> >> >> --
>>> >> >> >> >> >> Regards
>>> >> >> >> >> >> Shuja-ur-Rehman Baig
>>> >> >> >> >> >> <http://pk.linkedin.com/in/shujamughal>
>>> >> >> >> >> >>
>>> >> >> >> >> >>
>>> >> >> >> >> >
>>> >> >> >> >> >
>>> >> >> >> >> > --
>>> >> >> >> >> > Regards
>>> >> >> >> >> > Shuja-ur-Rehman Baig
>>> >> >> >> >> > <http://pk.linkedin.com/in/shujamughal>
>>> >> >> >> >> >
>>> >> >> >> >>
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> > --
>>> >> >> >> > Regards
>>> >> >> >> > Shuja-ur-Rehman Baig
>>> >> >> >> > <http://pk.linkedin.com/in/shujamughal>
>>> >> >> >> >
>>> >> >> >>
>>> >> >> >
>>> >> >> >
>>> >> >> >
>>> >> >> > --
>>> >> >> > Regards
>>> >> >> > Shuja-ur-Rehman Baig
>>> >> >> > <http://pk.linkedin.com/in/shujamughal>
>>> >> >> >
>>> >> >>
>>> >> >
>>> >> >
>>> >> >
>>> >> > --
>>> >> > Regards
>>> >> > Shuja-ur-Rehman Baig
>>> >> > <http://pk.linkedin.com/in/shujamughal>
>>> >> >
>>> >>
>>> >
>>> >
>>> >
>>> > --
>>> > Regards
>>> > Shuja-ur-Rehman Baig
>>> > <http://pk.linkedin.com/in/shujamughal>
>>> >
>>>
>>
>>
>>
>> --
>> Regards
>> Shuja-ur-Rehman Baig
>> <http://pk.linkedin.com/in/shujamughal>
>>
>>
>
>
> --
> Regards
> Shuja-ur-Rehman Baig
> <http://pk.linkedin.com/in/shujamughal>
>

Mime
View raw message