You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Bradford Stephens <br...@gmail.com> on 2009/06/09 19:13:35 UTC

HBase Failing on Large Loads

Hey rock stars,

I'm having problems loading large amounts of data into a table (about
120 GB, 250million rows). My Map task runs fine, but when it comes to
reducing, things start burning. 'top' inidcates that I only have ~
100M of RAM free on my datanodes, and every process starts thrashing
... even ssh and ping. Then I start to get errors like:

"org.apache.hadoop.hbase.client.RegionOfflineException: region
offline: joinedcontent,,1244513452487"

and:

"Task attempt_200906082135_0001_r_000002_0 failed to report status for
603 seconds. Killing!"

I'm running Hadoop .19.1 and HBase .19.3, with 1 master/name node and
8 regionservers. 2 x Dual Core Intel 3.2 GHz procs, 4 GB of RAM. 16
map tasks, 8 reducers. I've set the MAX_HEAP in hadoop-env to 768, and
the one in hbase-env is at its default with 1000. I've also done all
the performance enchancements in the Wiki with the file handlers, the
garbage collection, and the epoll limits.

What am I missing? :)

Cheers,
Bradford

Re: HBase Failing on Large Loads

Posted by Billy Pearson <sa...@pearsonwholesale.com>.
I thank most of your problems are coming from running to many map/reduce 
task at the same time with so
little memory and swapping and regionserver/datanodes/tasktrackers do not 
have time to check in to tell there masters that there alive still and stuff 
starts failing.

I would try 2 maps 2 reduce per machine maybe 4 with that little memory.
I run 3 mappers and 2 reducers per server with 4gb memory with 1gb heap for
hbase/datanode/tasktracker and 400mb for task.

Billy




"Bradford Stephens" 
<br...@gmail.com> wrote in message 
news:860544ed0906091715j71a644ccu17eeee29f60fd69e@mail.gmail.com...
I ran some more tests to clarify my questions from above. After the
same MR job, 5 out of 8 of my Regionservers died before I terminated
the job.  Here's what I saw in one of the HBase Regionserver logs...

Exception in createBlockOutputStream java.io.IOException: Bad connect
ack with firstBadLink 192.168.18.48:50010  (with many different
IPs...)

Then I get errors like this:

Error Recovery for block blk_-4108085472136309132_97478 in pipeline
192.168.18.49:50010, 192.168.18.48:50010, 192.168.18.16:50010: bad
datanode 192.168.18.48:50010

then things continue for a while and I get this:

Exception while reading from blk_1698571189906026963_93533 of
/hbase-0.19/joinedcontent/2018887968/content/mapfiles/3048972636250467459/data
from 192.168.18.49:50010: java.io.IOException: Premeture EOF from
inputStream

Then I start seeing stuff like this:

Error Recovery for block blk_3202913437369696154_99607 bad datanode[0]
nodes == null
2009-06-09 16:31:15,330 WARN org.apache.hadoop.hdfs.DFSClient: Could
not get block locations. Source file
"/hbase-0.19/joinedcontent/compaction.dir/2018887968/content/mapfiles/2166568776864749492/data"
- Aborting...

Exception in createBlockOutputStream java.io.IOException: Could not
read from stream

Abandoning block blk_-4592653855912358506_99607


And this...
DataStreamer Exception: java.io.IOException: Unable to create new block.


Then it eventually dies.


On Tue, Jun 9, 2009 at 11:51 AM, Bradford
Stephens<br...@gmail.com> wrote:
> I sort of need the reduce since I'm combining primary keys from a CSV
> file. Although I guess I could just use the combiner class... hrm.
>
> How do I decrease the batch size?
>
> Also, I tried to make a map-only task that used ImmutableBytesWritable
> and BatchUpdate as the output K and V, and TableOutputFormat as the
> OutputFormat -- the job fails, saying that "HbaseMapWritable cannot be
> cast to org.apache.hadoop.hbase.io.BatchUpdate". I've checked my
> Mapper multiple times, it's definitely ouputting a BatchUpdate.
>
> On Tue, Jun 9, 2009 at 10:43 AM, 
> stack<st...@duboce.net> wrote:
>> On Tue, Jun 9, 2009 at 10:13 AM, Bradford Stephens <
>> bradfordstephens@gmail.com> wrote:
>>
>>
>>> Hey rock stars,
>>>
>>
>>
>> Flattery makes us perk up for sure.
>>
>>
>>
>>>
>>> I'm having problems loading large amounts of data into a table (about
>>> 120 GB, 250million rows). My Map task runs fine, but when it comes to
>>> reducing, things start burning. 'top' inidcates that I only have ~
>>> 100M of RAM free on my datanodes, and every process starts thrashing
>>> ... even ssh and ping. Then I start to get errors like:
>>>
>>> "org.apache.hadoop.hbase.client.RegionOfflineException: region
>>> offline: joinedcontent,,1244513452487"
>>>
>>
>> See if said region is actually offline? Try getting a row from it in 
>> shell.
>>
>>
>>
>>>
>>> and:
>>>
>>> "Task attempt_200906082135_0001_r_000002_0 failed to report status for
>>> 603 seconds. Killing!"
>>
>>
>>
>> Sounds like nodes are heavily loaded.. so loaded either the task can't
>> report in... or its stuck on an hbase update so long, its taking ten 
>> minutes
>> or more to return.
>>
>> One thing to look at is disabling batching or making batches smaller. 
>> When
>> batch is big, can take a while under high-load for all row edits to go 
>> in.
>> HBase client will not return till all row commits have succeeded. Smaller
>> batches will mean more likely to return and not have the task killed 
>> because
>> takes longer than the report period to checkin.
>>
>>
>> Whats your MR job like? Your updating hbase in the reduce phase i presume
>> (TableOutputFormat?). Do you need the reduce? Can you update hbase in the
>> map step? Saves on the sort the MR framework is doing -- a sort that is
>> unnecessary given as hbase orders on insertion.
>>
>>
>> Can you try with a lighter load? Maybe a couple of smaller MR jobs rather
>> than one big one?
>>
>> St.Ack
>>
>>
>>>
>>>
>>> I'm running Hadoop .19.1 and HBase .19.3, with 1 master/name node and
>>> 8 regionservers. 2 x Dual Core Intel 3.2 GHz procs, 4 GB of RAM. 16
>>> map tasks, 8 reducers. I've set the MAX_HEAP in hadoop-env to 768, and
>>> the one in hbase-env is at its default with 1000. I've also done all
>>> the performance enchancements in the Wiki with the file handlers, the
>>> garbage collection, and the epoll limits.
>>>
>>> What am I missing? :)
>>>
>>> Cheers,
>>> Bradford
>>>
>>
>



Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
I ran some more tests to clarify my questions from above. After the
same MR job, 5 out of 8 of my Regionservers died before I terminated
the job.  Here's what I saw in one of the HBase Regionserver logs...

Exception in createBlockOutputStream java.io.IOException: Bad connect
ack with firstBadLink 192.168.18.48:50010  (with many different
IPs...)

Then I get errors like this:

Error Recovery for block blk_-4108085472136309132_97478 in pipeline
192.168.18.49:50010, 192.168.18.48:50010, 192.168.18.16:50010: bad
datanode 192.168.18.48:50010

then things continue for a while and I get this:

Exception while reading from blk_1698571189906026963_93533 of
/hbase-0.19/joinedcontent/2018887968/content/mapfiles/3048972636250467459/data
from 192.168.18.49:50010: java.io.IOException: Premeture EOF from
inputStream

Then I start seeing stuff like this:

Error Recovery for block blk_3202913437369696154_99607 bad datanode[0]
nodes == null
2009-06-09 16:31:15,330 WARN org.apache.hadoop.hdfs.DFSClient: Could
not get block locations. Source file
"/hbase-0.19/joinedcontent/compaction.dir/2018887968/content/mapfiles/2166568776864749492/data"
- Aborting...

Exception in createBlockOutputStream java.io.IOException: Could not
read from stream

Abandoning block blk_-4592653855912358506_99607


And this...
DataStreamer Exception: java.io.IOException: Unable to create new block.


Then it eventually dies.


On Tue, Jun 9, 2009 at 11:51 AM, Bradford
Stephens<br...@gmail.com> wrote:
> I sort of need the reduce since I'm combining primary keys from a CSV
> file. Although I guess I could just use the combiner class... hrm.
>
> How do I decrease the batch size?
>
> Also, I tried to make a map-only task that used ImmutableBytesWritable
> and BatchUpdate as the output K and V, and TableOutputFormat as the
> OutputFormat -- the job fails, saying that "HbaseMapWritable cannot be
> cast to org.apache.hadoop.hbase.io.BatchUpdate". I've checked my
> Mapper multiple times, it's definitely ouputting a BatchUpdate.
>
> On Tue, Jun 9, 2009 at 10:43 AM, stack<st...@duboce.net> wrote:
>> On Tue, Jun 9, 2009 at 10:13 AM, Bradford Stephens <
>> bradfordstephens@gmail.com> wrote:
>>
>>
>>> Hey rock stars,
>>>
>>
>>
>> Flattery makes us perk up for sure.
>>
>>
>>
>>>
>>> I'm having problems loading large amounts of data into a table (about
>>> 120 GB, 250million rows). My Map task runs fine, but when it comes to
>>> reducing, things start burning. 'top' inidcates that I only have ~
>>> 100M of RAM free on my datanodes, and every process starts thrashing
>>> ... even ssh and ping. Then I start to get errors like:
>>>
>>> "org.apache.hadoop.hbase.client.RegionOfflineException: region
>>> offline: joinedcontent,,1244513452487"
>>>
>>
>> See if said region is actually offline?  Try getting a row from it in shell.
>>
>>
>>
>>>
>>> and:
>>>
>>> "Task attempt_200906082135_0001_r_000002_0 failed to report status for
>>> 603 seconds. Killing!"
>>
>>
>>
>> Sounds like nodes are heavily loaded.. so loaded either the task can't
>> report in... or its stuck on an hbase update so long, its taking ten minutes
>> or more to return.
>>
>> One thing to look at is disabling batching or making batches smaller.   When
>> batch is big, can take a while under high-load for all row edits to go in.
>> HBase client will not return till all row commits have succeeded.  Smaller
>> batches will mean more likely to return and not have the task killed because
>> takes longer than the report period to checkin.
>>
>>
>> Whats your MR job like?  Your updating hbase in the reduce phase i presume
>> (TableOutputFormat?).  Do you need the reduce?  Can you update hbase in the
>> map step?   Saves on the sort the MR framework is doing -- a sort that is
>> unnecessary given as hbase orders on insertion.
>>
>>
>> Can you try with a lighter load?  Maybe a couple of smaller MR jobs rather
>> than one big one?
>>
>> St.Ack
>>
>>
>>>
>>>
>>> I'm running Hadoop .19.1 and HBase .19.3, with 1 master/name node and
>>> 8 regionservers. 2 x Dual Core Intel 3.2 GHz procs, 4 GB of RAM. 16
>>> map tasks, 8 reducers. I've set the MAX_HEAP in hadoop-env to 768, and
>>> the one in hbase-env is at its default with 1000. I've also done all
>>> the performance enchancements in the Wiki with the file handlers, the
>>> garbage collection, and the epoll limits.
>>>
>>> What am I missing? :)
>>>
>>> Cheers,
>>> Bradford
>>>
>>
>

Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
Also, there's a slight variation: "Trying to contact region server
Some server for region joinedcontent"

"Some server"? Interesting :)

On Wed, Jun 10, 2009 at 2:50 PM, Bradford
Stephens<br...@gmail.com> wrote:
> OK, I've tried all the optimizations you've suggested (still running
> with a M/R job). Still having problems like this:
>
> org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
> contact region server 192.168.18.15:60020 for region
> joinedcontent,242FEB3ED9BE0D8EF3856E9C4251464C,1244666594390, row
> '291DB5C7440B0A5BDB0C12501308C55B', but failed after 10 attempts.
> Exceptions:
> java.io.IOException: Call to /192.168.18.15:60020 failed on local
> exception: java.io.EOFException
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
>
> On Wed, Jun 10, 2009 at 12:40 AM, stack<st...@duboce.net> wrote:
>> On Tue, Jun 9, 2009 at 11:51 AM, Bradford Stephens <
>> bradfordstephens@gmail.com> wrote:
>>
>>> I sort of need the reduce since I'm combining primary keys from a CSV
>>> file. Although I guess I could just use the combiner class... hrm.
>>>
>>> How do I decrease the batch size?
>>
>>
>>
>> Below is from hbase-default.xml:
>>
>>  <property>
>>    <name>hbase.client.write.buffer</name>
>>    <value>2097152</value>
>>    <description>Size of the write buffer in bytes. A bigger buffer takes
>> more
>>    memory -- on both the client and server side since server instantiates
>>    the passed write buffer to process it -- but reduces the number of RPC.
>>    For an estimate of server-side memory-used, evaluate
>>    hbase.client.write.buffer * hbase.regionserver.handler.count
>>    </description>
>>  </property>
>>
>>
>> You upped xceivers on your datanodes and you set your
>> dfs.datanode.socket.write.timeout = 0?
>>
>>
>>
>>> Also, I tried to make a map-only task that used ImmutableBytesWritable
>>> and BatchUpdate as the output K and V, and TableOutputFormat as the
>>> OutputFormat -- the job fails, saying that "HbaseMapWritable cannot be
>>> cast to org.apache.hadoop.hbase.io.BatchUpdate". I've checked my
>>> Mapper multiple times, it's definitely ouputting a BatchUpdate.
>>>
>>
>>
>> You are using TOF as the map output?  Paste the exception.  You could try
>> making a HTable instance in your configure call and then do
>> t.commit(BatchUpdate) in your map.  Emit nothing or something simple like an
>> integer so the counters when job is done make some kind of sense.
>>
>> Tell us something about our schema.  How many column families and columns?
>>
>> St.Ack
>>
>>
>>>
>>> On Tue, Jun 9, 2009 at 10:43 AM, stack<st...@duboce.net> wrote:
>>> > On Tue, Jun 9, 2009 at 10:13 AM, Bradford Stephens <
>>> > bradfordstephens@gmail.com> wrote:
>>> >
>>> >
>>> >> Hey rock stars,
>>> >>
>>> >
>>> >
>>> > Flattery makes us perk up for sure.
>>> >
>>> >
>>> >
>>> >>
>>> >> I'm having problems loading large amounts of data into a table (about
>>> >> 120 GB, 250million rows). My Map task runs fine, but when it comes to
>>> >> reducing, things start burning. 'top' inidcates that I only have ~
>>> >> 100M of RAM free on my datanodes, and every process starts thrashing
>>> >> ... even ssh and ping. Then I start to get errors like:
>>> >>
>>> >> "org.apache.hadoop.hbase.client.RegionOfflineException: region
>>> >> offline: joinedcontent,,1244513452487"
>>> >>
>>> >
>>> > See if said region is actually offline?  Try getting a row from it in
>>> shell.
>>> >
>>> >
>>> >
>>> >>
>>> >> and:
>>> >>
>>> >> "Task attempt_200906082135_0001_r_000002_0 failed to report status for
>>> >> 603 seconds. Killing!"
>>> >
>>> >
>>> >
>>> > Sounds like nodes are heavily loaded.. so loaded either the task can't
>>> > report in... or its stuck on an hbase update so long, its taking ten
>>> minutes
>>> > or more to return.
>>> >
>>> > One thing to look at is disabling batching or making batches smaller.
>>> When
>>> > batch is big, can take a while under high-load for all row edits to go
>>> in.
>>> > HBase client will not return till all row commits have succeeded.
>>>  Smaller
>>> > batches will mean more likely to return and not have the task killed
>>> because
>>> > takes longer than the report period to checkin.
>>> >
>>> >
>>> > Whats your MR job like?  Your updating hbase in the reduce phase i
>>> presume
>>> > (TableOutputFormat?).  Do you need the reduce?  Can you update hbase in
>>> the
>>> > map step?   Saves on the sort the MR framework is doing -- a sort that is
>>> > unnecessary given as hbase orders on insertion.
>>> >
>>> >
>>> > Can you try with a lighter load?  Maybe a couple of smaller MR jobs
>>> rather
>>> > than one big one?
>>> >
>>> > St.Ack
>>> >
>>> >
>>> >>
>>> >>
>>> >> I'm running Hadoop .19.1 and HBase .19.3, with 1 master/name node and
>>> >> 8 regionservers. 2 x Dual Core Intel 3.2 GHz procs, 4 GB of RAM. 16
>>> >> map tasks, 8 reducers. I've set the MAX_HEAP in hadoop-env to 768, and
>>> >> the one in hbase-env is at its default with 1000. I've also done all
>>> >> the performance enchancements in the Wiki with the file handlers, the
>>> >> garbage collection, and the epoll limits.
>>> >>
>>> >> What am I missing? :)
>>> >>
>>> >> Cheers,
>>> >> Bradford
>>> >>
>>> >
>>>
>>
>

Re: HBase Failing on Large Loads

Posted by Andrew Purtell <ap...@apache.org>.
TableMapReduceUtil can set the number of mappers and reducers appropriate to the number of regions in the table at job start time. 

See TableMapReduceUtil#setNumMapTasks  and TableMapReduceUtil#setNumReduceTasks

   - Andy
 




________________________________
From: stack <st...@duboce.net>
To: hbase-user@hadoop.apache.org
Sent: Friday, June 12, 2009 11:59:27 AM
Subject: Re: HBase Failing on Large Loads

On Fri, Jun 12, 2009 at 11:50 AM, mike anderson <sa...@gmail.com>wrote:

>
> I'm wondering how you set up your job to run 2 maps/1 reducer per machine.
> Is this a matter of adding more region servers? I currently have 1
> regionserver and 144 regions (living on the same cluster as hadoop.
>

TableInputFormat makes as many maps as there are regions (with some
caveats).  My guess is that you only have 4 regions in you table since you
don't have that many rows? Your best bet is study of TIF#getSplits.  You
could override it to get more maps or, just trust that when you have more
data in the table, and therefore more regions, more maps will be run.

On the reduce side, I'm not sure.  Check TableOutputFormat but I'd say 1
reduce per machine is default.  In this case hbase is probably respecting
what you have configured in your hadoop-site.xml/mapred-site.xml.

St.Ack



      

Re: HBase Failing on Large Loads

Posted by Ryan Rawson <ry...@gmail.com>.
Generally you want to have # map partitions = # table regions.

Then from there, you configure in the hadoop config how many to run at the
same time per machine.

On Fri, Jun 12, 2009 at 11:59 AM, stack <st...@duboce.net> wrote:

> On Fri, Jun 12, 2009 at 11:50 AM, mike anderson <saidtherobot@gmail.com
> >wrote:
>
> >
> > I'm wondering how you set up your job to run 2 maps/1 reducer per
> machine.
> > Is this a matter of adding more region servers? I currently have 1
> > regionserver and 144 regions (living on the same cluster as hadoop.
> >
>
> TableInputFormat makes as many maps as there are regions (with some
> caveats).  My guess is that you only have 4 regions in you table since you
> don't have that many rows? Your best bet is study of TIF#getSplits.  You
> could override it to get more maps or, just trust that when you have more
> data in the table, and therefore more regions, more maps will be run.
>
> On the reduce side, I'm not sure.  Check TableOutputFormat but I'd say 1
> reduce per machine is default.  In this case hbase is probably respecting
> what you have configured in your hadoop-site.xml/mapred-site.xml.
>
> St.Ack
>

Re: HBase Failing on Large Loads

Posted by stack <st...@duboce.net>.
On Fri, Jun 12, 2009 at 11:50 AM, mike anderson <sa...@gmail.com>wrote:

>
> I'm wondering how you set up your job to run 2 maps/1 reducer per machine.
> Is this a matter of adding more region servers? I currently have 1
> regionserver and 144 regions (living on the same cluster as hadoop.
>

TableInputFormat makes as many maps as there are regions (with some
caveats).  My guess is that you only have 4 regions in you table since you
don't have that many rows? Your best bet is study of TIF#getSplits.  You
could override it to get more maps or, just trust that when you have more
data in the table, and therefore more regions, more maps will be run.

On the reduce side, I'm not sure.  Check TableOutputFormat but I'd say 1
reduce per machine is default.  In this case hbase is probably respecting
what you have configured in your hadoop-site.xml/mapred-site.xml.

St.Ack

Re: HBase Failing on Large Loads

Posted by mike anderson <sa...@gmail.com>.
I have a related question (apologies in advance if I should be branching to
a new thread). I'm also doing a large amount of hadoop MR data crunching on
documents stored in hbase. I ran a simple letter count MR job on my
documents (only 150k for now, but soon to be 2.5mil) and it took a
surprisingly long time (about 9 hours). Looking at the job tracker I noticed
I only had run 4 map tasks and 1 reduce task. Given that my cluster is 10
quad-core machines I would have assumed there to be 4 map tasks on EACH
machine for a total of 40?

I'm wondering how you set up your job to run 2 maps/1 reducer per machine.
Is this a matter of adding more region servers? I currently have 1
regionserver and 144 regions (living on the same cluster as hadoop.

Thanks.

On 6/12/09, Bradford Stephens <br...@gmail.com> wrote:
>
> Yeah, I was thinking about that. I guess data in HBase needs to be
> sorted once, so data locality is a null issue on load. So, we can have
> a separate HBase cluster and it wouldn't impact load performance any
> more than having our Hadoop data 'crunching' cluster on the same boxes
> as HBase? We plan on doing most of our MR tasks *on* documents in
> HBase, however.
>
> I'm running 2 maps and 1 reducer per machine. All the Map tasks are
> done by the time we start reducin' and loadin'.
>
> Sorry if this had too many run-on sentences :)
>
>
>
> On Fri, Jun 12, 2009 at 10:31 AM, Ryan Rawson<ry...@gmail.com> wrote:
> > yeah, sounds like it.  How many maps are you running per machine?
> >
> > You could try making the hbase and mr machines disjoint?
> >
> > On Jun 12, 2009 10:20 AM, "Bradford Stephens" <
> bradfordstephens@gmail.com>
> > wrote:
> >
> > I don't think we're using ZK, I'm on HBase-0.19.4...am I wrong? :)
> > I've already got the GC configured to do what you suggested... I'm not
> > getting very long pauses from the log file. I really think the problem
> > is resource starvation because I only have 2 total cores on each of
> > those boxes, and it's running HBase on top of Hadoop DataNodes and
> > TaskTrackers. Am I right in this thinking?
> >
> > On Thu, Jun 11, 2009 at 10:29 PM, Ryan Rawson<ry...@gmail.com> wrote:
> >
> > Since you are on a 2-4 c...
> >
>

Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
Yeah, I was thinking about that. I guess data in HBase needs to be
sorted once, so data locality is a null issue on load. So, we can have
a separate HBase cluster and it wouldn't impact load performance any
more than having our Hadoop data 'crunching' cluster on the same boxes
as HBase? We plan on doing most of our MR tasks *on* documents in
HBase, however.

I'm running 2 maps and 1 reducer per machine. All the Map tasks are
done by the time we start reducin' and loadin'.

Sorry if this had too many run-on sentences :)


On Fri, Jun 12, 2009 at 10:31 AM, Ryan Rawson<ry...@gmail.com> wrote:
> yeah, sounds like it.  How many maps are you running per machine?
>
> You could try making the hbase and mr machines disjoint?
>
> On Jun 12, 2009 10:20 AM, "Bradford Stephens" <br...@gmail.com>
> wrote:
>
> I don't think we're using ZK, I'm on HBase-0.19.4...am I wrong? :)
> I've already got the GC configured to do what you suggested... I'm not
> getting very long pauses from the log file. I really think the problem
> is resource starvation because I only have 2 total cores on each of
> those boxes, and it's running HBase on top of Hadoop DataNodes and
> TaskTrackers. Am I right in this thinking?
>
> On Thu, Jun 11, 2009 at 10:29 PM, Ryan Rawson<ry...@gmail.com> wrote: >
> Since you are on a 2-4 c...
>

Re: HBase Failing on Large Loads

Posted by Ryan Rawson <ry...@gmail.com>.
yeah, sounds like it.  How many maps are you running per machine?

You could try making the hbase and mr machines disjoint?

On Jun 12, 2009 10:20 AM, "Bradford Stephens" <br...@gmail.com>
wrote:

I don't think we're using ZK, I'm on HBase-0.19.4...am I wrong? :)
I've already got the GC configured to do what you suggested... I'm not
getting very long pauses from the log file. I really think the problem
is resource starvation because I only have 2 total cores on each of
those boxes, and it's running HBase on top of Hadoop DataNodes and
TaskTrackers. Am I right in this thinking?

On Thu, Jun 11, 2009 at 10:29 PM, Ryan Rawson<ry...@gmail.com> wrote: >
Since you are on a 2-4 c...

Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
I don't think we're using ZK, I'm on HBase-0.19.4...am I wrong? :)
I've already got the GC configured to do what you suggested... I'm not
getting very long pauses from the log file. I really think the problem
is resource starvation because I only have 2 total cores on each of
those boxes, and it's running HBase on top of Hadoop DataNodes and
TaskTrackers. Am I right in this thinking?

On Thu, Jun 11, 2009 at 10:29 PM, Ryan Rawson<ry...@gmail.com> wrote:
> Since you are on a 2-4 cpu system, you need to use:
>
> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode"
>
> What do your gc verbose log say?  are you getting huge pauses?
>
> you can up the ZK, try doing this in zoo.conf server and client:
>
> tickTime=20000
> initLimit=5
> syncLimit=2
>
> and in hbase-site.xml:
> <property>
> <name>zookeeper.session.timeout</name>
> <value>60000</value>
> </property>
>
> This will give you a much higher zookeeper time out.
>
> Let us know!
>
>
>
> On Thu, Jun 11, 2009 at 10:25 PM, Bradford Stephens <
> bradfordstephens@gmail.com> wrote:
>
>> Thanks for helping me, o people of awesomeness.
>>
>> VM settings are 1000 for HBase, and I used the GC laid out in the
>> Wiki. Also, " -server " ... basically, I did everything here :
>> http://wiki.apache.org/hadoop/PerformanceTuning , and on
>>
>> http://ryantwopointoh.blogspot.com/2009/01/performance-of-hbase-importing.html
>>
>> On Thu, Jun 11, 2009 at 8:02 PM, Ryan Rawson<ry...@gmail.com> wrote:
>> > What are you vm/gc settings?  Let's tune that!
>> >
>> > On Jun 11, 2009 7:08 PM, "Bradford Stephens" <bradfordstephens@gmail.com
>> >
>> > wrote:
>> >
>> > OK, so I discovered the ulimit wasn't changed like I thought it was,
>> > had to fool with PAM in Ubuntu.
>> >
>> > Everything's running a little better, and I cut the data size by 66%.
>> >
>> > It took a while, but one of the machines with only 2 cores failed, and
>> > I caught it in the moment. Then 2 other machiens failed a few minutes
>> > later in a cascade. I'm thinking that HBase +Hadoop takes up so much
>> > proc time that the machine gradually stops responding to heartbeat....
>> > does that seem rational?
>> >
>> > Here's the first regionserver log: http://pastebin.com/m96e06fe
>> > I wish I could attach the log of one of the regionservers that failed
>> > a few minutes later, but it's 708MB! Here's some examples of the tail:
>> >
>> >  2009-06-11 19:00:18,418 WARN
>> > org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
>> > to master for 906196 milliseconds - retrying
>> > 2009-06-11 19:00:18,419 WARN
>> > org.apache.hadoop.hbase.regionserver.HRegionServer: error getting
>> > store file index size for 944890031/url:
>> > java.io.FileNotFoundException: File does not exist:
>> >
>> hdfs://dttest01:54310/hbase-0.19/joinedcontent/944890031/url/mapfiles/2512503149715575970/index
>> >
>> > The HBase Master log is surprisingly quiet...
>> >
>> > Overall, I think HBase just isn't happy on a machine with two
>> > single-core procs, and when they start dropping like flies, everything
>> > goes to hell. Do my log files support this?
>> >
>> > Cheers,
>> > Bradford
>> >
>> > On Wed, Jun 10, 2009 at 4:01 PM, Ryan Rawson<ry...@gmail.com> wrote:
>> >
>> > Hey, > > Looks lke you h...
>> >
>>
>

Re: HBase Failing on Large Loads

Posted by Ryan Rawson <ry...@gmail.com>.
Since you are on a 2-4 cpu system, you need to use:

"-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode"

What do your gc verbose log say?  are you getting huge pauses?

you can up the ZK, try doing this in zoo.conf server and client:

tickTime=20000
initLimit=5
syncLimit=2

and in hbase-site.xml:
<property>
<name>zookeeper.session.timeout</name>
<value>60000</value>
</property>

This will give you a much higher zookeeper time out.

Let us know!



On Thu, Jun 11, 2009 at 10:25 PM, Bradford Stephens <
bradfordstephens@gmail.com> wrote:

> Thanks for helping me, o people of awesomeness.
>
> VM settings are 1000 for HBase, and I used the GC laid out in the
> Wiki. Also, " -server " ... basically, I did everything here :
> http://wiki.apache.org/hadoop/PerformanceTuning , and on
>
> http://ryantwopointoh.blogspot.com/2009/01/performance-of-hbase-importing.html
>
> On Thu, Jun 11, 2009 at 8:02 PM, Ryan Rawson<ry...@gmail.com> wrote:
> > What are you vm/gc settings?  Let's tune that!
> >
> > On Jun 11, 2009 7:08 PM, "Bradford Stephens" <bradfordstephens@gmail.com
> >
> > wrote:
> >
> > OK, so I discovered the ulimit wasn't changed like I thought it was,
> > had to fool with PAM in Ubuntu.
> >
> > Everything's running a little better, and I cut the data size by 66%.
> >
> > It took a while, but one of the machines with only 2 cores failed, and
> > I caught it in the moment. Then 2 other machiens failed a few minutes
> > later in a cascade. I'm thinking that HBase +Hadoop takes up so much
> > proc time that the machine gradually stops responding to heartbeat....
> > does that seem rational?
> >
> > Here's the first regionserver log: http://pastebin.com/m96e06fe
> > I wish I could attach the log of one of the regionservers that failed
> > a few minutes later, but it's 708MB! Here's some examples of the tail:
> >
> >  2009-06-11 19:00:18,418 WARN
> > org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
> > to master for 906196 milliseconds - retrying
> > 2009-06-11 19:00:18,419 WARN
> > org.apache.hadoop.hbase.regionserver.HRegionServer: error getting
> > store file index size for 944890031/url:
> > java.io.FileNotFoundException: File does not exist:
> >
> hdfs://dttest01:54310/hbase-0.19/joinedcontent/944890031/url/mapfiles/2512503149715575970/index
> >
> > The HBase Master log is surprisingly quiet...
> >
> > Overall, I think HBase just isn't happy on a machine with two
> > single-core procs, and when they start dropping like flies, everything
> > goes to hell. Do my log files support this?
> >
> > Cheers,
> > Bradford
> >
> > On Wed, Jun 10, 2009 at 4:01 PM, Ryan Rawson<ry...@gmail.com> wrote:
> >
> > Hey, > > Looks lke you h...
> >
>

Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
Thanks for helping me, o people of awesomeness.

VM settings are 1000 for HBase, and I used the GC laid out in the
Wiki. Also, " -server " ... basically, I did everything here :
http://wiki.apache.org/hadoop/PerformanceTuning , and on
http://ryantwopointoh.blogspot.com/2009/01/performance-of-hbase-importing.html

On Thu, Jun 11, 2009 at 8:02 PM, Ryan Rawson<ry...@gmail.com> wrote:
> What are you vm/gc settings?  Let's tune that!
>
> On Jun 11, 2009 7:08 PM, "Bradford Stephens" <br...@gmail.com>
> wrote:
>
> OK, so I discovered the ulimit wasn't changed like I thought it was,
> had to fool with PAM in Ubuntu.
>
> Everything's running a little better, and I cut the data size by 66%.
>
> It took a while, but one of the machines with only 2 cores failed, and
> I caught it in the moment. Then 2 other machiens failed a few minutes
> later in a cascade. I'm thinking that HBase +Hadoop takes up so much
> proc time that the machine gradually stops responding to heartbeat....
> does that seem rational?
>
> Here's the first regionserver log: http://pastebin.com/m96e06fe
> I wish I could attach the log of one of the regionservers that failed
> a few minutes later, but it's 708MB! Here's some examples of the tail:
>
>  2009-06-11 19:00:18,418 WARN
> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
> to master for 906196 milliseconds - retrying
> 2009-06-11 19:00:18,419 WARN
> org.apache.hadoop.hbase.regionserver.HRegionServer: error getting
> store file index size for 944890031/url:
> java.io.FileNotFoundException: File does not exist:
> hdfs://dttest01:54310/hbase-0.19/joinedcontent/944890031/url/mapfiles/2512503149715575970/index
>
> The HBase Master log is surprisingly quiet...
>
> Overall, I think HBase just isn't happy on a machine with two
> single-core procs, and when they start dropping like flies, everything
> goes to hell. Do my log files support this?
>
> Cheers,
> Bradford
>
> On Wed, Jun 10, 2009 at 4:01 PM, Ryan Rawson<ry...@gmail.com> wrote: >
> Hey, > > Looks lke you h...
>

Re: HBase Failing on Large Loads

Posted by Ryan Rawson <ry...@gmail.com>.
What are you vm/gc settings?  Let's tune that!

On Jun 11, 2009 7:08 PM, "Bradford Stephens" <br...@gmail.com>
wrote:

OK, so I discovered the ulimit wasn't changed like I thought it was,
had to fool with PAM in Ubuntu.

Everything's running a little better, and I cut the data size by 66%.

It took a while, but one of the machines with only 2 cores failed, and
I caught it in the moment. Then 2 other machiens failed a few minutes
later in a cascade. I'm thinking that HBase +Hadoop takes up so much
proc time that the machine gradually stops responding to heartbeat....
does that seem rational?

Here's the first regionserver log: http://pastebin.com/m96e06fe
I wish I could attach the log of one of the regionservers that failed
a few minutes later, but it's 708MB! Here's some examples of the tail:

 2009-06-11 19:00:18,418 WARN
org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
to master for 906196 milliseconds - retrying
2009-06-11 19:00:18,419 WARN
org.apache.hadoop.hbase.regionserver.HRegionServer: error getting
store file index size for 944890031/url:
java.io.FileNotFoundException: File does not exist:
hdfs://dttest01:54310/hbase-0.19/joinedcontent/944890031/url/mapfiles/2512503149715575970/index

The HBase Master log is surprisingly quiet...

Overall, I think HBase just isn't happy on a machine with two
single-core procs, and when they start dropping like flies, everything
goes to hell. Do my log files support this?

Cheers,
Bradford

On Wed, Jun 10, 2009 at 4:01 PM, Ryan Rawson<ry...@gmail.com> wrote: >
Hey, > > Looks lke you h...

Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
OK, so I discovered the ulimit wasn't changed like I thought it was,
had to fool with PAM in Ubuntu.

Everything's running a little better, and I cut the data size by 66%.

It took a while, but one of the machines with only 2 cores failed, and
I caught it in the moment. Then 2 other machiens failed a few minutes
later in a cascade. I'm thinking that HBase +Hadoop takes up so much
proc time that the machine gradually stops responding to heartbeat....
does that seem rational?

Here's the first regionserver log: http://pastebin.com/m96e06fe
I wish I could attach the log of one of the regionservers that failed
a few minutes later, but it's 708MB! Here's some examples of the tail:

 2009-06-11 19:00:18,418 WARN
org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
to master for 906196 milliseconds - retrying
2009-06-11 19:00:18,419 WARN
org.apache.hadoop.hbase.regionserver.HRegionServer: error getting
store file index size for 944890031/url:
java.io.FileNotFoundException: File does not exist:
hdfs://dttest01:54310/hbase-0.19/joinedcontent/944890031/url/mapfiles/2512503149715575970/index

The HBase Master log is surprisingly quiet...

Overall, I think HBase just isn't happy on a machine with two
single-core procs, and when they start dropping like flies, everything
goes to hell. Do my log files support this?

Cheers,
Bradford


On Wed, Jun 10, 2009 at 4:01 PM, Ryan Rawson<ry...@gmail.com> wrote:
> Hey,
>
> Looks lke you have some HDFS issues.
>
> Things I did to make myself stable:
>
> - run HDFS with -Xmx=2000m
> - run HDFS with 2047 xciever limit (goes into hdfs-core.xml or
> hadoop-site.xml)
> - ulimit -n 32k - also important
>
> With this I find that HDFS is very stable, I've imported hundreds of gigs.
>
> You want to make sure the HDFS xciever limit is set in the hadoop/conf
> directory, copied to every node and HDFS restarted.  Also sounds like you
> might have a cluster with multiple versions of hadoop.  Double check that!
>
> you're close!
> -ryan
>
> On Wed, Jun 10, 2009 at 3:32 PM, Bradford Stephens <
> bradfordstephens@gmail.com> wrote:
>
>> Thanks so much for all the help, everyone... things are still broken,
>> but maybe we're getting close.
>>
>> All the regionservers were dead by the time the job ended.  I see
>> quite a few error messages like this:
>>
>> (I've put the entirety of the regionserver logs on pastebin:)
>> http://pastebin.com/m2e6f9283
>> http://pastebin.com/mf97bd57
>>
>> 2009-06-10 14:47:54,994 ERROR
>> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to process
>> message: MSG_REGION_OPEN:
>> joinedcontent,1DCC1616F7C7B53B69B5536F407A64DF,1244667570521:
>> safeMode=false
>> java.lang.NullPointerException
>>
>> There's also a scattering of messages like this:
>> 2009-06-10 13:49:02,855 WARN
>> org.apache.hadoop.hbase.regionserver.HLog: IPC Server handler 1 on
>> 60020 took 3267ms appending an edit to HLog; editcount=21570
>>
>> aaand....
>>
>> 2009-06-10 14:03:27,270 INFO
>> org.apache.hadoop.hbase.regionserver.HLog: Closed
>>
>> hdfs://dttest01:54310/hbase-0.19/log_192.168.18.49_1244659862699_60020/hlog.dat.1244667757560,
>> entries=100006. New log writer:
>> /hbase-0.19/log_192.168.18.49_1244659862699_60020/hlog.dat.1244667807249
>> 2009-06-10 14:03:28,160 INFO org.apache.hadoop.hdfs.DFSClient:
>> Exception in createBlockOutputStream java.io.IOException: Bad connect
>> ack with firstBadLink 192.168.18.47:50010
>> 2009-06-10 14:03:28,160 INFO org.apache.hadoop.hdfs.DFSClient:
>> Abandoning block blk_4831127457964871573_140781
>> 2009-06-10 14:03:34,170 INFO org.apache.hadoop.hdfs.DFSClient:
>> Exception in createBlockOutputStream java.io.IOException: Could not
>> read from stream
>> 2009-06-10 14:03:34,170 INFO org.apache.hadoop.hdfs.DFSClient:
>> Abandoning block blk_-6169186743102862627_140796
>> 2009-06-10 14:03:34,485 INFO
>> org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Forced flushing
>> of joinedcontent,1F2F64F59088A3B121CFC66F7FCBA2A9,1244667654435
>> because global memcache limit of 398.7m exceeded; currently 399.0m and
>> flushing till 249.2m
>>
>> Finally, I saw this when I stopped and re-started my cluster:
>>
>> 2009-06-10 15:29:09,494 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.18.16:50010,
>> storageID=DS-486600617-192.168.18.16-50010-1241838200467,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.io.IOException: Version Mismatch
>>        at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:81)
>>        at java.lang.Thread.run(Thread.java:619)
>>
>>
>> On Wed, Jun 10, 2009 at 2:55 PM, Ryan Rawson<ry...@gmail.com> wrote:
>> > That is a client exception that is a sign of problems on the
>> > regionserver...is it still running? What do the logs look like?
>> >
>> > On Jun 10, 2009 2:51 PM, "Bradford Stephens" <bradfordstephens@gmail.com
>> >
>> > wrote:
>> >
>> > OK, I've tried all the optimizations you've suggested (still running
>> > with a M/R job). Still having problems like this:
>> >
>> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
>> > contact region server 192.168.18.15:60020 for region
>> > joinedcontent,242FEB3ED9BE0D8EF3856E9C4251464C,1244666594390, row
>> > '291DB5C7440B0A5BDB0C12501308C55B', but failed after 10 attempts.
>> > Exceptions:
>> > java.io.IOException: Call to /192.168.18.15:60020 failed on local
>> > exception: java.io.EOFException
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
>> > connection exception: java.net.ConnectException: Connection refused
>> >
>> > On Wed, Jun 10, 2009 at 12:40 AM, stack<st...@duboce.net> wrote: > On
>> Tue,
>> > Jun 9, 2009 at 11:51 AM,...
>> >
>>
>

Re: HBase Failing on Large Loads

Posted by Ryan Rawson <ry...@gmail.com>.
Hey,

Looks lke you have some HDFS issues.

Things I did to make myself stable:

- run HDFS with -Xmx=2000m
- run HDFS with 2047 xciever limit (goes into hdfs-core.xml or
hadoop-site.xml)
- ulimit -n 32k - also important

With this I find that HDFS is very stable, I've imported hundreds of gigs.

You want to make sure the HDFS xciever limit is set in the hadoop/conf
directory, copied to every node and HDFS restarted.  Also sounds like you
might have a cluster with multiple versions of hadoop.  Double check that!

you're close!
-ryan

On Wed, Jun 10, 2009 at 3:32 PM, Bradford Stephens <
bradfordstephens@gmail.com> wrote:

> Thanks so much for all the help, everyone... things are still broken,
> but maybe we're getting close.
>
> All the regionservers were dead by the time the job ended.  I see
> quite a few error messages like this:
>
> (I've put the entirety of the regionserver logs on pastebin:)
> http://pastebin.com/m2e6f9283
> http://pastebin.com/mf97bd57
>
> 2009-06-10 14:47:54,994 ERROR
> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to process
> message: MSG_REGION_OPEN:
> joinedcontent,1DCC1616F7C7B53B69B5536F407A64DF,1244667570521:
> safeMode=false
> java.lang.NullPointerException
>
> There's also a scattering of messages like this:
> 2009-06-10 13:49:02,855 WARN
> org.apache.hadoop.hbase.regionserver.HLog: IPC Server handler 1 on
> 60020 took 3267ms appending an edit to HLog; editcount=21570
>
> aaand....
>
> 2009-06-10 14:03:27,270 INFO
> org.apache.hadoop.hbase.regionserver.HLog: Closed
>
> hdfs://dttest01:54310/hbase-0.19/log_192.168.18.49_1244659862699_60020/hlog.dat.1244667757560,
> entries=100006. New log writer:
> /hbase-0.19/log_192.168.18.49_1244659862699_60020/hlog.dat.1244667807249
> 2009-06-10 14:03:28,160 INFO org.apache.hadoop.hdfs.DFSClient:
> Exception in createBlockOutputStream java.io.IOException: Bad connect
> ack with firstBadLink 192.168.18.47:50010
> 2009-06-10 14:03:28,160 INFO org.apache.hadoop.hdfs.DFSClient:
> Abandoning block blk_4831127457964871573_140781
> 2009-06-10 14:03:34,170 INFO org.apache.hadoop.hdfs.DFSClient:
> Exception in createBlockOutputStream java.io.IOException: Could not
> read from stream
> 2009-06-10 14:03:34,170 INFO org.apache.hadoop.hdfs.DFSClient:
> Abandoning block blk_-6169186743102862627_140796
> 2009-06-10 14:03:34,485 INFO
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Forced flushing
> of joinedcontent,1F2F64F59088A3B121CFC66F7FCBA2A9,1244667654435
> because global memcache limit of 398.7m exceeded; currently 399.0m and
> flushing till 249.2m
>
> Finally, I saw this when I stopped and re-started my cluster:
>
> 2009-06-10 15:29:09,494 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.18.16:50010,
> storageID=DS-486600617-192.168.18.16-50010-1241838200467,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.io.IOException: Version Mismatch
>        at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:81)
>        at java.lang.Thread.run(Thread.java:619)
>
>
> On Wed, Jun 10, 2009 at 2:55 PM, Ryan Rawson<ry...@gmail.com> wrote:
> > That is a client exception that is a sign of problems on the
> > regionserver...is it still running? What do the logs look like?
> >
> > On Jun 10, 2009 2:51 PM, "Bradford Stephens" <bradfordstephens@gmail.com
> >
> > wrote:
> >
> > OK, I've tried all the optimizations you've suggested (still running
> > with a M/R job). Still having problems like this:
> >
> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
> > contact region server 192.168.18.15:60020 for region
> > joinedcontent,242FEB3ED9BE0D8EF3856E9C4251464C,1244666594390, row
> > '291DB5C7440B0A5BDB0C12501308C55B', but failed after 10 attempts.
> > Exceptions:
> > java.io.IOException: Call to /192.168.18.15:60020 failed on local
> > exception: java.io.EOFException
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> >
> > On Wed, Jun 10, 2009 at 12:40 AM, stack<st...@duboce.net> wrote: > On
> Tue,
> > Jun 9, 2009 at 11:51 AM,...
> >
>

Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
Thanks so much for all the help, everyone... things are still broken,
but maybe we're getting close.

All the regionservers were dead by the time the job ended.  I see
quite a few error messages like this:

(I've put the entirety of the regionserver logs on pastebin:)
http://pastebin.com/m2e6f9283
http://pastebin.com/mf97bd57

2009-06-10 14:47:54,994 ERROR
org.apache.hadoop.hbase.regionserver.HRegionServer: unable to process
message: MSG_REGION_OPEN:
joinedcontent,1DCC1616F7C7B53B69B5536F407A64DF,1244667570521:
safeMode=false
java.lang.NullPointerException

There's also a scattering of messages like this:
2009-06-10 13:49:02,855 WARN
org.apache.hadoop.hbase.regionserver.HLog: IPC Server handler 1 on
60020 took 3267ms appending an edit to HLog; editcount=21570

aaand....

2009-06-10 14:03:27,270 INFO
org.apache.hadoop.hbase.regionserver.HLog: Closed
hdfs://dttest01:54310/hbase-0.19/log_192.168.18.49_1244659862699_60020/hlog.dat.1244667757560,
entries=100006. New log writer:
/hbase-0.19/log_192.168.18.49_1244659862699_60020/hlog.dat.1244667807249
2009-06-10 14:03:28,160 INFO org.apache.hadoop.hdfs.DFSClient:
Exception in createBlockOutputStream java.io.IOException: Bad connect
ack with firstBadLink 192.168.18.47:50010
2009-06-10 14:03:28,160 INFO org.apache.hadoop.hdfs.DFSClient:
Abandoning block blk_4831127457964871573_140781
2009-06-10 14:03:34,170 INFO org.apache.hadoop.hdfs.DFSClient:
Exception in createBlockOutputStream java.io.IOException: Could not
read from stream
2009-06-10 14:03:34,170 INFO org.apache.hadoop.hdfs.DFSClient:
Abandoning block blk_-6169186743102862627_140796
2009-06-10 14:03:34,485 INFO
org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Forced flushing
of joinedcontent,1F2F64F59088A3B121CFC66F7FCBA2A9,1244667654435
because global memcache limit of 398.7m exceeded; currently 399.0m and
flushing till 249.2m

Finally, I saw this when I stopped and re-started my cluster:

2009-06-10 15:29:09,494 ERROR
org.apache.hadoop.hdfs.server.datanode.DataNode:
DatanodeRegistration(192.168.18.16:50010,
storageID=DS-486600617-192.168.18.16-50010-1241838200467,
infoPort=50075, ipcPort=50020):DataXceiver
java.io.IOException: Version Mismatch
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:81)
        at java.lang.Thread.run(Thread.java:619)


On Wed, Jun 10, 2009 at 2:55 PM, Ryan Rawson<ry...@gmail.com> wrote:
> That is a client exception that is a sign of problems on the
> regionserver...is it still running? What do the logs look like?
>
> On Jun 10, 2009 2:51 PM, "Bradford Stephens" <br...@gmail.com>
> wrote:
>
> OK, I've tried all the optimizations you've suggested (still running
> with a M/R job). Still having problems like this:
>
> org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
> contact region server 192.168.18.15:60020 for region
> joinedcontent,242FEB3ED9BE0D8EF3856E9C4251464C,1244666594390, row
> '291DB5C7440B0A5BDB0C12501308C55B', but failed after 10 attempts.
> Exceptions:
> java.io.IOException: Call to /192.168.18.15:60020 failed on local
> exception: java.io.EOFException
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
> java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> connection exception: java.net.ConnectException: Connection refused
>
> On Wed, Jun 10, 2009 at 12:40 AM, stack<st...@duboce.net> wrote: > On Tue,
> Jun 9, 2009 at 11:51 AM,...
>

Re: HBase Failing on Large Loads

Posted by Ryan Rawson <ry...@gmail.com>.
That is a client exception that is a sign of problems on the
regionserver...is it still running? What do the logs look like?

On Jun 10, 2009 2:51 PM, "Bradford Stephens" <br...@gmail.com>
wrote:

OK, I've tried all the optimizations you've suggested (still running
with a M/R job). Still having problems like this:

org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
contact region server 192.168.18.15:60020 for region
joinedcontent,242FEB3ED9BE0D8EF3856E9C4251464C,1244666594390, row
'291DB5C7440B0A5BDB0C12501308C55B', but failed after 10 attempts.
Exceptions:
java.io.IOException: Call to /192.168.18.15:60020 failed on local
exception: java.io.EOFException
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused

On Wed, Jun 10, 2009 at 12:40 AM, stack<st...@duboce.net> wrote: > On Tue,
Jun 9, 2009 at 11:51 AM,...

Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
OK, I've tried all the optimizations you've suggested (still running
with a M/R job). Still having problems like this:

org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
contact region server 192.168.18.15:60020 for region
joinedcontent,242FEB3ED9BE0D8EF3856E9C4251464C,1244666594390, row
'291DB5C7440B0A5BDB0C12501308C55B', but failed after 10 attempts.
Exceptions:
java.io.IOException: Call to /192.168.18.15:60020 failed on local
exception: java.io.EOFException
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused

On Wed, Jun 10, 2009 at 12:40 AM, stack<st...@duboce.net> wrote:
> On Tue, Jun 9, 2009 at 11:51 AM, Bradford Stephens <
> bradfordstephens@gmail.com> wrote:
>
>> I sort of need the reduce since I'm combining primary keys from a CSV
>> file. Although I guess I could just use the combiner class... hrm.
>>
>> How do I decrease the batch size?
>
>
>
> Below is from hbase-default.xml:
>
>  <property>
>    <name>hbase.client.write.buffer</name>
>    <value>2097152</value>
>    <description>Size of the write buffer in bytes. A bigger buffer takes
> more
>    memory -- on both the client and server side since server instantiates
>    the passed write buffer to process it -- but reduces the number of RPC.
>    For an estimate of server-side memory-used, evaluate
>    hbase.client.write.buffer * hbase.regionserver.handler.count
>    </description>
>  </property>
>
>
> You upped xceivers on your datanodes and you set your
> dfs.datanode.socket.write.timeout = 0?
>
>
>
>> Also, I tried to make a map-only task that used ImmutableBytesWritable
>> and BatchUpdate as the output K and V, and TableOutputFormat as the
>> OutputFormat -- the job fails, saying that "HbaseMapWritable cannot be
>> cast to org.apache.hadoop.hbase.io.BatchUpdate". I've checked my
>> Mapper multiple times, it's definitely ouputting a BatchUpdate.
>>
>
>
> You are using TOF as the map output?  Paste the exception.  You could try
> making a HTable instance in your configure call and then do
> t.commit(BatchUpdate) in your map.  Emit nothing or something simple like an
> integer so the counters when job is done make some kind of sense.
>
> Tell us something about our schema.  How many column families and columns?
>
> St.Ack
>
>
>>
>> On Tue, Jun 9, 2009 at 10:43 AM, stack<st...@duboce.net> wrote:
>> > On Tue, Jun 9, 2009 at 10:13 AM, Bradford Stephens <
>> > bradfordstephens@gmail.com> wrote:
>> >
>> >
>> >> Hey rock stars,
>> >>
>> >
>> >
>> > Flattery makes us perk up for sure.
>> >
>> >
>> >
>> >>
>> >> I'm having problems loading large amounts of data into a table (about
>> >> 120 GB, 250million rows). My Map task runs fine, but when it comes to
>> >> reducing, things start burning. 'top' inidcates that I only have ~
>> >> 100M of RAM free on my datanodes, and every process starts thrashing
>> >> ... even ssh and ping. Then I start to get errors like:
>> >>
>> >> "org.apache.hadoop.hbase.client.RegionOfflineException: region
>> >> offline: joinedcontent,,1244513452487"
>> >>
>> >
>> > See if said region is actually offline?  Try getting a row from it in
>> shell.
>> >
>> >
>> >
>> >>
>> >> and:
>> >>
>> >> "Task attempt_200906082135_0001_r_000002_0 failed to report status for
>> >> 603 seconds. Killing!"
>> >
>> >
>> >
>> > Sounds like nodes are heavily loaded.. so loaded either the task can't
>> > report in... or its stuck on an hbase update so long, its taking ten
>> minutes
>> > or more to return.
>> >
>> > One thing to look at is disabling batching or making batches smaller.
>> When
>> > batch is big, can take a while under high-load for all row edits to go
>> in.
>> > HBase client will not return till all row commits have succeeded.
>>  Smaller
>> > batches will mean more likely to return and not have the task killed
>> because
>> > takes longer than the report period to checkin.
>> >
>> >
>> > Whats your MR job like?  Your updating hbase in the reduce phase i
>> presume
>> > (TableOutputFormat?).  Do you need the reduce?  Can you update hbase in
>> the
>> > map step?   Saves on the sort the MR framework is doing -- a sort that is
>> > unnecessary given as hbase orders on insertion.
>> >
>> >
>> > Can you try with a lighter load?  Maybe a couple of smaller MR jobs
>> rather
>> > than one big one?
>> >
>> > St.Ack
>> >
>> >
>> >>
>> >>
>> >> I'm running Hadoop .19.1 and HBase .19.3, with 1 master/name node and
>> >> 8 regionservers. 2 x Dual Core Intel 3.2 GHz procs, 4 GB of RAM. 16
>> >> map tasks, 8 reducers. I've set the MAX_HEAP in hadoop-env to 768, and
>> >> the one in hbase-env is at its default with 1000. I've also done all
>> >> the performance enchancements in the Wiki with the file handlers, the
>> >> garbage collection, and the epoll limits.
>> >>
>> >> What am I missing? :)
>> >>
>> >> Cheers,
>> >> Bradford
>> >>
>> >
>>
>

Re: HBase Failing on Large Loads

Posted by stack <st...@duboce.net>.
On Tue, Jun 9, 2009 at 11:51 AM, Bradford Stephens <
bradfordstephens@gmail.com> wrote:

> I sort of need the reduce since I'm combining primary keys from a CSV
> file. Although I guess I could just use the combiner class... hrm.
>
> How do I decrease the batch size?



Below is from hbase-default.xml:

  <property>
    <name>hbase.client.write.buffer</name>
    <value>2097152</value>
    <description>Size of the write buffer in bytes. A bigger buffer takes
more
    memory -- on both the client and server side since server instantiates
    the passed write buffer to process it -- but reduces the number of RPC.
    For an estimate of server-side memory-used, evaluate
    hbase.client.write.buffer * hbase.regionserver.handler.count
    </description>
  </property>


You upped xceivers on your datanodes and you set your
dfs.datanode.socket.write.timeout = 0?



> Also, I tried to make a map-only task that used ImmutableBytesWritable
> and BatchUpdate as the output K and V, and TableOutputFormat as the
> OutputFormat -- the job fails, saying that "HbaseMapWritable cannot be
> cast to org.apache.hadoop.hbase.io.BatchUpdate". I've checked my
> Mapper multiple times, it's definitely ouputting a BatchUpdate.
>


You are using TOF as the map output?  Paste the exception.  You could try
making a HTable instance in your configure call and then do
t.commit(BatchUpdate) in your map.  Emit nothing or something simple like an
integer so the counters when job is done make some kind of sense.

Tell us something about our schema.  How many column families and columns?

St.Ack


>
> On Tue, Jun 9, 2009 at 10:43 AM, stack<st...@duboce.net> wrote:
> > On Tue, Jun 9, 2009 at 10:13 AM, Bradford Stephens <
> > bradfordstephens@gmail.com> wrote:
> >
> >
> >> Hey rock stars,
> >>
> >
> >
> > Flattery makes us perk up for sure.
> >
> >
> >
> >>
> >> I'm having problems loading large amounts of data into a table (about
> >> 120 GB, 250million rows). My Map task runs fine, but when it comes to
> >> reducing, things start burning. 'top' inidcates that I only have ~
> >> 100M of RAM free on my datanodes, and every process starts thrashing
> >> ... even ssh and ping. Then I start to get errors like:
> >>
> >> "org.apache.hadoop.hbase.client.RegionOfflineException: region
> >> offline: joinedcontent,,1244513452487"
> >>
> >
> > See if said region is actually offline?  Try getting a row from it in
> shell.
> >
> >
> >
> >>
> >> and:
> >>
> >> "Task attempt_200906082135_0001_r_000002_0 failed to report status for
> >> 603 seconds. Killing!"
> >
> >
> >
> > Sounds like nodes are heavily loaded.. so loaded either the task can't
> > report in... or its stuck on an hbase update so long, its taking ten
> minutes
> > or more to return.
> >
> > One thing to look at is disabling batching or making batches smaller.
> When
> > batch is big, can take a while under high-load for all row edits to go
> in.
> > HBase client will not return till all row commits have succeeded.
>  Smaller
> > batches will mean more likely to return and not have the task killed
> because
> > takes longer than the report period to checkin.
> >
> >
> > Whats your MR job like?  Your updating hbase in the reduce phase i
> presume
> > (TableOutputFormat?).  Do you need the reduce?  Can you update hbase in
> the
> > map step?   Saves on the sort the MR framework is doing -- a sort that is
> > unnecessary given as hbase orders on insertion.
> >
> >
> > Can you try with a lighter load?  Maybe a couple of smaller MR jobs
> rather
> > than one big one?
> >
> > St.Ack
> >
> >
> >>
> >>
> >> I'm running Hadoop .19.1 and HBase .19.3, with 1 master/name node and
> >> 8 regionservers. 2 x Dual Core Intel 3.2 GHz procs, 4 GB of RAM. 16
> >> map tasks, 8 reducers. I've set the MAX_HEAP in hadoop-env to 768, and
> >> the one in hbase-env is at its default with 1000. I've also done all
> >> the performance enchancements in the Wiki with the file handlers, the
> >> garbage collection, and the epoll limits.
> >>
> >> What am I missing? :)
> >>
> >> Cheers,
> >> Bradford
> >>
> >
>

Re: HBase Failing on Large Loads

Posted by Bradford Stephens <br...@gmail.com>.
I sort of need the reduce since I'm combining primary keys from a CSV
file. Although I guess I could just use the combiner class... hrm.

How do I decrease the batch size?

Also, I tried to make a map-only task that used ImmutableBytesWritable
and BatchUpdate as the output K and V, and TableOutputFormat as the
OutputFormat -- the job fails, saying that "HbaseMapWritable cannot be
cast to org.apache.hadoop.hbase.io.BatchUpdate". I've checked my
Mapper multiple times, it's definitely ouputting a BatchUpdate.

On Tue, Jun 9, 2009 at 10:43 AM, stack<st...@duboce.net> wrote:
> On Tue, Jun 9, 2009 at 10:13 AM, Bradford Stephens <
> bradfordstephens@gmail.com> wrote:
>
>
>> Hey rock stars,
>>
>
>
> Flattery makes us perk up for sure.
>
>
>
>>
>> I'm having problems loading large amounts of data into a table (about
>> 120 GB, 250million rows). My Map task runs fine, but when it comes to
>> reducing, things start burning. 'top' inidcates that I only have ~
>> 100M of RAM free on my datanodes, and every process starts thrashing
>> ... even ssh and ping. Then I start to get errors like:
>>
>> "org.apache.hadoop.hbase.client.RegionOfflineException: region
>> offline: joinedcontent,,1244513452487"
>>
>
> See if said region is actually offline?  Try getting a row from it in shell.
>
>
>
>>
>> and:
>>
>> "Task attempt_200906082135_0001_r_000002_0 failed to report status for
>> 603 seconds. Killing!"
>
>
>
> Sounds like nodes are heavily loaded.. so loaded either the task can't
> report in... or its stuck on an hbase update so long, its taking ten minutes
> or more to return.
>
> One thing to look at is disabling batching or making batches smaller.   When
> batch is big, can take a while under high-load for all row edits to go in.
> HBase client will not return till all row commits have succeeded.  Smaller
> batches will mean more likely to return and not have the task killed because
> takes longer than the report period to checkin.
>
>
> Whats your MR job like?  Your updating hbase in the reduce phase i presume
> (TableOutputFormat?).  Do you need the reduce?  Can you update hbase in the
> map step?   Saves on the sort the MR framework is doing -- a sort that is
> unnecessary given as hbase orders on insertion.
>
>
> Can you try with a lighter load?  Maybe a couple of smaller MR jobs rather
> than one big one?
>
> St.Ack
>
>
>>
>>
>> I'm running Hadoop .19.1 and HBase .19.3, with 1 master/name node and
>> 8 regionservers. 2 x Dual Core Intel 3.2 GHz procs, 4 GB of RAM. 16
>> map tasks, 8 reducers. I've set the MAX_HEAP in hadoop-env to 768, and
>> the one in hbase-env is at its default with 1000. I've also done all
>> the performance enchancements in the Wiki with the file handlers, the
>> garbage collection, and the epoll limits.
>>
>> What am I missing? :)
>>
>> Cheers,
>> Bradford
>>
>

Re: HBase Failing on Large Loads

Posted by stack <st...@duboce.net>.
On Tue, Jun 9, 2009 at 10:13 AM, Bradford Stephens <
bradfordstephens@gmail.com> wrote:


> Hey rock stars,
>


Flattery makes us perk up for sure.



>
> I'm having problems loading large amounts of data into a table (about
> 120 GB, 250million rows). My Map task runs fine, but when it comes to
> reducing, things start burning. 'top' inidcates that I only have ~
> 100M of RAM free on my datanodes, and every process starts thrashing
> ... even ssh and ping. Then I start to get errors like:
>
> "org.apache.hadoop.hbase.client.RegionOfflineException: region
> offline: joinedcontent,,1244513452487"
>

See if said region is actually offline?  Try getting a row from it in shell.



>
> and:
>
> "Task attempt_200906082135_0001_r_000002_0 failed to report status for
> 603 seconds. Killing!"



Sounds like nodes are heavily loaded.. so loaded either the task can't
report in... or its stuck on an hbase update so long, its taking ten minutes
or more to return.

One thing to look at is disabling batching or making batches smaller.   When
batch is big, can take a while under high-load for all row edits to go in.
HBase client will not return till all row commits have succeeded.  Smaller
batches will mean more likely to return and not have the task killed because
takes longer than the report period to checkin.


Whats your MR job like?  Your updating hbase in the reduce phase i presume
(TableOutputFormat?).  Do you need the reduce?  Can you update hbase in the
map step?   Saves on the sort the MR framework is doing -- a sort that is
unnecessary given as hbase orders on insertion.


Can you try with a lighter load?  Maybe a couple of smaller MR jobs rather
than one big one?

St.Ack


>
>
> I'm running Hadoop .19.1 and HBase .19.3, with 1 master/name node and
> 8 regionservers. 2 x Dual Core Intel 3.2 GHz procs, 4 GB of RAM. 16
> map tasks, 8 reducers. I've set the MAX_HEAP in hadoop-env to 768, and
> the one in hbase-env is at its default with 1000. I've also done all
> the performance enchancements in the Wiki with the file handlers, the
> garbage collection, and the epoll limits.
>
> What am I missing? :)
>
> Cheers,
> Bradford
>