You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Daniel Leffel <da...@gmail.com> on 2008/04/15 18:07:39 UTC

Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Hi,
I'm new to HBase and investigating it for use in our large production
environment. Largely, I'm quite impressed by the possibilities.

One bit of behavior I'm noticing is that region servers tend to quit
unexpectedly during moderately loaded map/reduce jobs without really any
helpful information in the logs as to why.

Is the behavior expected? How else should I be troubleshooting?

Thanks in advance!

Danny Leffel

Re: Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Posted by stack <st...@duboce.net>.
Daniel Leffel wrote:
> I was running 0.1.0 (hadn't noticed the new release). To rule out that
> as the issue, I've since upgraded to 0.1.1. I am still seeing the same
> behavior. Also, I'm running on top of Hadoop 0.16.1.
>
>   
Thanks.  Interspersed in the below (When you get a chance, move beyond 
0.16.1.  It was deprecated soon after release).

> Interestingly, during the latest test, the master process died too.
>
> Not sure what to be looking for in the DEBUG output, but here's a few
> interesting entries.
>
> Here's the master node when one of the region servers died:
> -------------------------------------
> 2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.Leases:
> HMaster.leaseChecker lease expired 1563117711/1563117711
> 2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.HMaster:
> 10.252.39.164:60020 lease expired
> 2008-04-15 18:09:10,201 INFO org.apache.hadoop.hbase.HMaster:
> assigning region -ROOT-,,0 to the only server 10.252.67.181:60020
> 2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
> Received MSG_REPORT_PROCESS_OPEN : -ROOT-,,0 from 10.252.67.181:60020
> 2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
> Received MSG_REPORT_OPEN : -ROOT-,,0 from 10.252.67.181:60020
> 2008-04-15 18:09:13,211 INFO org.apache.hadoop.hbase.HMaster:
> 10.252.67.181:60020 serving -ROOT-,,0
> 2008-04-15 18:09:30,963 WARN org.apache.hadoop.hbase.HMaster: Scan ROOT region
> java.net.SocketTimeoutException: timed out waiting for rpc response
>         at org.apache.hadoop.ipc.Client.call(Client.java:514)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
>         at $Proxy1.openScanner(Unknown Source)
>         at org.apache.hadoop.hbase.HMaster$BaseScanner.scanRegion(HMaster.java:227)
>         at org.apache.hadoop.hbase.HMaster$RootScanner.scanRoot(HMaster.java:540)
>         at org.apache.hadoop.hbase.HMaster$RootScanner.maintenanceScan(HMaster.java:565)
>         at org.apache.hadoop.hbase.HMaster$BaseScanner.chore(HMaster.java:207)
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:63)
>   

Here the Master is failing on its scan of the most important region, the 
-ROOT- region.  Would be interesting to see hregionserver side of things 
at this time.

> 2008-04-15 18:09:30,965 INFO org.apache.hadoop.hbase.HMaster:
> HMaster.rootScanner scanning meta region {regionname: -ROOT-,,0,
> startKey: <>, server: 10.252.67.181:60020}
> 2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster:
> HMaster.rootScanner regioninfo: {regionname: .META.,,1, startKey: <>,
> endKey: <>, encodedName: 1028785192, tableDesc: {name: .META.,
> families: {info:={name: info, max versions: 1, compression: NONE, in
> memory: false, max length: 2147483647, bloom filter: none}}}}, server:
> 10.252.67.181:60020, startCode: 1208278043501
> 2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster: Current
> assignment of .META.,,1 is not valid: storedInfo: address:
> 10.252.67.181:60020, startcode: 1208296518624, load: (requests: 0
> regions: 12), startCode: 1208278043501, storedInfo.startCode:
> 1208296518624, unassignedRegions: false, pendingRegions: false
> 2008-04-15 18:09:30,975 INFO org.apache.hadoop.hbase.HMaster:
> HMaster.rootScanner scan of meta region {regionname: -ROOT-,,0,
> startKey: <>, server: 10.252.67.181:60020} complete
>   

Here, Master is complaining that the SECOND most important region, the 
.META. region, has not been properly deployed.

Both regions were given to 10.252.67.181 to serve.

Things seem to have started working though given the state of the rest 
of our logs (You seem to be taking on updates successfully at 18:40 below).

> 2008-04-15 18:09:31,271 INFO org.apache.hadoop.hbase.HMaster:
> assigning region .META.,,1 to the only server 10.252.67.181:60020
> 2008-04-15 18:09:34,281 DEBUG org.apache.hadoop.hbase.HMaster:
> Received MSG_REPORT_PROCESS_OPEN : .META.,,1 from 10.252.67.181:60020
> 2008-04-15 18:09:35,602 INFO org.apache.hadoop.hbase.HMaster:
> HMaster.metaScanner scanning meta region {regionname: .META.,,1,
> startKey: <>, server: 10.252.67.181:60020}
> ------------------------
>
>
>
> Other curious entries from master when the regionserver died:
> -------------------------------
> 2008-04-15 18:11:20,904 DEBUG org.apache.hadoop.hbase.HLog: Splitting
> 66 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.066
> 2008-04-15 18:11:21,756 DEBUG org.apache.hadoop.hbase.HLog: Applied
> 30734 total edits
....
> 2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
> Received MSG_REPORT_PROCESS_OPEN :
> category_to_all_parents_map,4050172,1208212849307 from
> 10.252.67.181:60020
> 2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
> Received MSG_REPORT_PROCESS_OPEN :
> category_to_immediate_children_map,,1208288806024 from
> 10.252.67.181:60020
>   
Above is master cleaning up after 10.252.39.164 crashed.  Says its 
assigning all regions that were on that server to 181 -- because its 
only server running.

Something ain't to healthy in the state of Denmark.

> ---------------------------------------------
>
>
> The Exception that is thrown is almost either one of the following:
> ----------------------------------
> org.apache.hadoop.hbase.NotServingRegionException:
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,,1208222885052
> 	at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
> 	at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
> 	at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
> 	at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 	at java.lang.reflect.Method.invoke(Method.java:597)
> 	at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
>
> 	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.hbase.RemoteExceptionHandler.decodeRemoteException(RemoteExceptionHandler.java:82)
> 	at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1028)
> 	at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
> 	at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
> --------------------------------------
>   

These are usually 'Normal' (I know, we shouldn't be using exceptions as 
part of normal processing but...).  Its the fault that goes back to the 
client to tell it it needs recalibrate (The region it was asking for has 
probably been split or, less usually, it has not been deployed ... yet).

> -- or ---
>
> ---------------------------------------
> java.lang.RuntimeException: java.lang.reflect.UndeclaredThrowableException
> 	at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1050)
> 	at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
> 	at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
> 	at com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:93)
> 	at com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:1)
> 	at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.combineAndSpill(MapTask.java:522)
> 	at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.sortAndSpillToDisk(MapTask.java:493)
> 	at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:713)
> 	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:209)
> 	at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:2084)
> Caused by: java.lang.reflect.UndeclaredThrowableException
> 	at $Proxy2.findRootRegion(Unknown Source)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRootRegion(HConnectionManager.java:706)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:336)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:313)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:409)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:346)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:482)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:350)
> 	at org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
> 	at org.apache.hadoop.hbase.HTable.getRegionLocation(HTable.java:114)
> 	at org.apache.hadoop.hbase.HTable$ServerCallable.instantiateServer(HTable.java:1009)
> 	at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1024)
> 	... 9 more
> Caused by: java.net.ConnectException: Connection refused
> 	at java.net.PlainSocketImpl.socketConnect(Native Method)
> 	at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
> 	at java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
> 	at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
> 	at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
> 	at java.net.Socket.connect(Socket.java:519)
> 	at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:161)
> 	at org.apache.hadoop.ipc.Client.getConnection(Client.java:578)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:501)
> 	at org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
> 	... 22 more
> ---------------------------
>   

This is complaint about being unable to find the -ROOT- region.  If 
-ROOT- region is not deployed, then nothing will work in hbase.

>
>
> The region server does a lot of log rolling. Is this normal?
> -----------------------------------------------
> 2008-04-15 18:24:22,577 WARN org.apache.hadoop.hbase.util.Sleeper: We
> slept 37782ms, ten times longer than scheduled: 3000
> 2008-04-15 18:24:26,738 INFO org.apache.hadoop.hbase.HRegionServer:
> Rolling hlog. Number of entries: 30006
> 2008-04-15 18:24:26,749 DEBUG org.apache.hadoop.hbase.HLog: Closing
> current log writer
> hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.053
> to get a new one
> 2008-04-15 18:24:26,760 INFO org.apache.hadoop.hbase.HLog: new log
> writer created at
> hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
> 2008-04-15 18:24:33,073 INFO org.apache.hadoop.hbase.HRegionServer:
> Rolling hlog. Number of entries: 30450
> 2008-04-15 18:24:33,409 DEBUG org.apache.hadoop.hbase.HLog: Closing
> current log writer
> hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
> to get a new one
> 2008-04-15 18:24:33,419 INFO org.apache.hadoop.hbase.HLog: new log
> writer created at
> hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.055
> -------------------------------------------------
>
>
>   

If its taking on lots of edits, yes we roll frequently.


> After a lot of rolling, this happens:
> ------------------------------------------------
> 2008-04-15 18:38:59,716 INFO org.apache.hadoop.hbase.HLog: new log
> writer created at
> hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
> 2008-04-15 18:39:02,113 INFO org.apache.hadoop.hbase.HRegionServer:
> Rolling hlog. Number of entries: 30360
> 2008-04-15 18:39:02,236 DEBUG org.apache.hadoop.hbase.HLog: Closing
> current log writer
> hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
> to get a new one
> 2008-04-15 18:39:02,238 INFO org.apache.hadoop.hbase.HLog: new log
> writer created at
> hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.160
> 2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
> updates for 'IPC Server handler 1 on 60020': Memcache size 64.0m is >=
> than blocking 64.0m size
> 2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
> updates for 'IPC Server handler 7 on 60020': Memcache size 64.0m is >=
> than blocking 64.0m size
> 2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
> updates for 'IPC Server handler 8 on 60020': Memcache size 64.0m is >=
> than blocking 64.0m size
> 2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
> updates for 'IPC Server handler 9 on 60020': Memcache size 64.0m is >=
> than blocking 64.0m size
> 2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
> updates for 'IPC Server handler 4 on 60020': Memcache size 64.0m is >=
> than blocking 64.0m size
> 2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
> updates for 'IPC Server handler 3 on 60020': Memcache size 64.0m is >=
> than blocking 64.0m size
>   

Here the regionserver is trying to protect itself -- edits are coming in 
faster than it can flush them to disk.  It puts up a block on edits to 
give the server a breather so it gets a chance to catch up.  It looks 
bad in the logs because all server threads hit block at about same time 
and each makes its own log.


> 2008-04-15 18:39:05,627 DEBUG org.apache.hadoop.hbase.HRegion: Started
> memcache flush for region
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363. Size
> 64.0m
> 2008-04-15 18:39:09,544 DEBUG org.apache.hadoop.hbase.HStore: Added
> 1064767197/adjusted_q/3161297904753567399 with 0 entries, sequence id
> 64293357, and size 110.0 for 1064767197/adjusted_q
> 2008-04-15 18:39:15,489 DEBUG org.apache.hadoop.hbase.HStore: Added
> 1064767197/rule_id/5414884402918348270 with 285058 entries, sequence
> id 64293357, and size 11.8m for 1064767197/rule_id
>   

Here is the flush to free things up again.


> 2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> Unblocking updates for region
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> Server handler 1 on 60020'
> 2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> Unblocking updates for region
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> Server handler 7 on 60020'
> 2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> Unblocking updates for region
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> Server handler 8 on 60020'
> 2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> Unblocking updates for region
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> Server handler 9 on 60020'
> 2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> Unblocking updates for region
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> Server handler 4 on 60020'
> 2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> Unblocking updates for region
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> Server handler 3 on 60020'
> 2008-04-15 18:39:16,739 INFO org.apache.hadoop.hbase.HRegionServer:
> Rolling hlog. Number of entries: 33252
> 2008-04-15 18:39:45,008 INFO org.apache.hadoop.hbase.HRegionServer:
> MSG_REGION_OPEN : .META.,,1
> 2008-04-15 18:39:51,636 DEBUG org.apache.hadoop.hbase.HStore: Added
> 1064767197/category_id/5081092265153632091 with 285058 entries,
> sequence id 64293357, and size 14.5m for 1064767197/category_id
> 2008-04-15 18:40:09,041 DEBUG org.apache.hadoop.hbase.HStore: Added
> 1064767197/success_count/2540574911286829228 with 285058 entries,
> sequence id 64293357, and size 13.4m for 1064767197/success_count
> 2008-04-15 18:40:12,707 DEBUG org.apache.hadoop.hbase.HStore: Added
> 1064767197/adjusted_deviation/270946121541713279 with 0 entries,
> sequence id 64293357, and size 110.0 for 1064767197/adjusted_deviation
> 2008-04-15 18:40:30,453 DEBUG org.apache.hadoop.hbase.HStore: Added
> 1064767197/hidden_variable/493176493179028867 with 285058 entries,
> sequence id 64293357, and size 16.6m for 1064767197/hidden_variable
> ---------------------------------------------------
>
>   

Above, the block has come off and we're taking updates again.

You're using a MR job to load hbase.  Tell us more.

St.Ack

> On Tue, Apr 15, 2008 at 9:12 AM, stack <st...@duboce.net> wrote:
>   
>> You are running 0.1.1 hbase?  Please enable DEBUG logging, see http://wiki.apache.org/hadoop/Hbase/FAQ#5 for how, and send over a log.
>> Thanks,
>> St.Ack
>>
>>
>>
>>
>> Daniel Leffel wrote:
>>
>>     
>>> Hi,
>>> I'm new to HBase and investigating it for use in our large production
>>> environment. Largely, I'm quite impressed by the possibilities.
>>>
>>> One bit of behavior I'm noticing is that region servers tend to quit
>>> unexpectedly during moderately loaded map/reduce jobs without really any
>>> helpful information in the logs as to why.
>>>
>>> Is the behavior expected? How else should I be troubleshooting?
>>>
>>> Thanks in advance!
>>>
>>> Danny Leffel
>>>
>>>
>>>
>>>       
>>     


Re: Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Posted by Daniel Leffel <da...@gmail.com>.
Excellent. Looking forward to trying it out!

On Thu, Apr 24, 2008 at 2:24 PM, stack <st...@duboce.net> wrote:

> Daniel Leffel wrote:
>
> > I've continued down the path of investigation here and after upgrading
> > to
> > Hadoop 0.16.3 and Hbase 0.1.1, I've discovered the Region Server process
> > is
> > dying with an OutOfMemory exception.
> >
> > I've tried upping the HBase heap on the region server to 2 gigs.
> >
> > This is happening during a TableReduce.
> >
> > Are there any other setting I can tweak to help let the region server
> > catch
> > up before running out of memory?
> >
> >
>
> Give me a second.  I'm trying to put together a 0.1.2 candidate.  It has
> backport of HBASE-512 which seems to make a big difference regards count of
> regions any one server can carry.  Once its up, take it for a spin.
> St.Ack
>
>

Re: Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Posted by stack <st...@duboce.net>.
Daniel Leffel wrote:
> I've continued down the path of investigation here and after upgrading to
> Hadoop 0.16.3 and Hbase 0.1.1, I've discovered the Region Server process is
> dying with an OutOfMemory exception.
>
> I've tried upping the HBase heap on the region server to 2 gigs.
>
> This is happening during a TableReduce.
>
> Are there any other setting I can tweak to help let the region server catch
> up before running out of memory?
>   

Give me a second.  I'm trying to put together a 0.1.2 candidate.  It has 
backport of HBASE-512 which seems to make a big difference regards count 
of regions any one server can carry.  Once its up, take it for a spin.
St.Ack


Re: Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Posted by Daniel Leffel <da...@gmail.com>.
I've continued down the path of investigation here and after upgrading to
Hadoop 0.16.3 and Hbase 0.1.1, I've discovered the Region Server process is
dying with an OutOfMemory exception.

I've tried upping the HBase heap on the region server to 2 gigs.

This is happening during a TableReduce.

Are there any other setting I can tweak to help let the region server catch
up before running out of memory?

Thanks in advance!

Danny



On Tue, Apr 15, 2008 at 8:07 PM, stack <st...@duboce.net> wrote:

> In the just-previous mail I explained that the NotServingRegionException
> is part of 'normal' operation.  Usually the client recalibrates and away we
> go again.  Is it failing your MR upload job?
>
> St.Ack
>
>
> Daniel Leffel wrote:
>
> > Now I just got this exception:
> >
> > 2008-04-15 18:50:02,107 DEBUG org.apache.hadoop.hbase.HStore: maximum
> > sequence id for hstore 856584617/rule_id is 68555204
> > 2008-04-15 18:50:02,284 INFO org.apache.hadoop.ipc.Server: IPC Server
> > handler 8 on 60020, call
> >
> > batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> > 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@19d277e)
> > from 10.252.50.36:53955: error:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
> >        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
> >        at
> > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> >        at java.lang.reflect.Method.invoke(Method.java:597)
> >        at
> > org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> >        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> > 2008-04-15 18:50:02,286 INFO org.apache.hadoop.ipc.Server: IPC Server
> > handler 4 on 60020, call
> >
> > batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> > 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@1890c67)
> > from 10.252.114.85:43998: error:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
> >        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
> >        at
> > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> >        at java.lang.reflect.Method.invoke(Method.java:597)
> >        at
> > org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> >        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> > 2008-04-15 18:50:02,288 INFO org.apache.hadoop.ipc.Server: IPC Server
> > handler 3 on 60020, call
> >
> > batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> > 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@a801b0)
> > from 10.252.67.181:56452: error:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
> >        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
> >        at
> > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> >        at java.lang.reflect.Method.invoke(Method.java:597)
> >        at
> > org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> >        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> > 2008-04-15 18:50:02,299 INFO org.apache.hadoop.ipc.Server: IPC Server
> > handler 9 on 60020, call
> >
> > batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> > 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@5ea4c5)
> > from 10.252.50.36:53954: error:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
> >        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
> >        at
> > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> >        at java.lang.reflect.Method.invoke(Method.java:597)
> >        at
> > org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> >        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> > 2008-04-15 18:50:02,352 INFO org.apache.hadoop.ipc.Server: IPC Server
> > handler 6 on 60020, call
> >
> > batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> > 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@193b604)
> > from 10.252.114.85:43999: error:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
> >        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
> >        at
> > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> >        at java.lang.reflect.Method.invoke(Method.java:597)
> >        at
> > org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> >        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> > 2008-04-15 18:50:02,353 INFO org.apache.hadoop.ipc.Server: IPC Server
> > handler 6 on 60020, call
> >
> > batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> > 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@92e96c)
> > from 10.252.67.181:56453: error:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> > org.apache.hadoop.hbase.NotServingRegionException:
> > category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
> >        at
> > org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
> >        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
> >        at
> > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> >        at java.lang.reflect.Method.invoke(Method.java:597)
> >        at
> > org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> >        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> >
> >
> >
> > -----------------------------------
> >
> >
> >
> >
> > On Tue, Apr 15, 2008 at 3:43 PM, Daniel Leffel <da...@gmail.com>
> > wrote:
> >
> >
> > > I was running 0.1.0 (hadn't noticed the new release). To rule out that
> > >  as the issue, I've since upgraded to 0.1.1. I am still seeing the
> > > same
> > >  behavior. Also, I'm running on top of Hadoop 0.16.1.
> > >
> > >  Interestingly, during the latest test, the master process died too.
> > >
> > >  Not sure what to be looking for in the DEBUG output, but here's a few
> > >  interesting entries.
> > >
> > >  Here's the master node when one of the region servers died:
> > >  -------------------------------------
> > >  2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.Leases:
> > >  HMaster.leaseChecker lease expired 1563117711/1563117711
> > >  2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.HMaster:
> > >  10.252.39.164:60020 lease expired
> > >  2008-04-15 18:09:10,201 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region -ROOT-,,0 to the only server 10.252.67.181:60020
> > >  2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  Received MSG_REPORT_PROCESS_OPEN : -ROOT-,,0 from 10.252.67.181:60020
> > >  2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  Received MSG_REPORT_OPEN : -ROOT-,,0 from 10.252.67.181:60020
> > >  2008-04-15 18:09:13,211 INFO org.apache.hadoop.hbase.HMaster:
> > >  10.252.67.181:60020 serving -ROOT-,,0
> > >  2008-04-15 18:09:30,963 WARN org.apache.hadoop.hbase.HMaster: Scan
> > > ROOT region
> > >  java.net.SocketTimeoutException: timed out waiting for rpc response
> > >        at org.apache.hadoop.ipc.Client.call(Client.java:514)
> > >        at
> > > org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
> > >        at $Proxy1.openScanner(Unknown Source)
> > >        at
> > > org.apache.hadoop.hbase.HMaster$BaseScanner.scanRegion(HMaster.java:227)
> > >        at
> > > org.apache.hadoop.hbase.HMaster$RootScanner.scanRoot(HMaster.java:540)
> > >        at
> > > org.apache.hadoop.hbase.HMaster$RootScanner.maintenanceScan(HMaster.java:565)
> > >        at
> > > org.apache.hadoop.hbase.HMaster$BaseScanner.chore(HMaster.java:207)
> > >        at org.apache.hadoop.hbase.Chore.run(Chore.java:63)
> > >  2008-04-15 18:09:30,965 INFO org.apache.hadoop.hbase.HMaster:
> > >  HMaster.rootScanner scanning meta region {regionname: -ROOT-,,0,
> > >  startKey: <>, server: 10.252.67.181:60020}
> > >  2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  HMaster.rootScanner regioninfo: {regionname: .META.,,1, startKey: <>,
> > >  endKey: <>, encodedName: 1028785192, tableDesc: {name: .META.,
> > >  families: {info:={name: info, max versions: 1, compression: NONE, in
> > >  memory: false, max length: 2147483647, bloom filter: none}}}},
> > > server:
> > >  10.252.67.181:60020, startCode: 1208278043501
> > >  2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster:
> > > Current
> > >  assignment of .META.,,1 is not valid: storedInfo: address:
> > >  10.252.67.181:60020, startcode: 1208296518624, load: (requests: 0
> > >  regions: 12), startCode: 1208278043501, storedInfo.startCode:
> > >  1208296518624, unassignedRegions: false, pendingRegions: false
> > >  2008-04-15 18:09:30,975 INFO org.apache.hadoop.hbase.HMaster:
> > >  HMaster.rootScanner scan of meta region {regionname: -ROOT-,,0,
> > >  startKey: <>, server: 10.252.67.181:60020} complete
> > >  2008-04-15 18:09:31,271 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region .META.,,1 to the only server 10.252.67.181:60020
> > >  2008-04-15 18:09:34,281 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  Received MSG_REPORT_PROCESS_OPEN : .META.,,1 from 10.252.67.181:60020
> > >  2008-04-15 18:09:35,602 INFO org.apache.hadoop.hbase.HMaster:
> > >  HMaster.metaScanner scanning meta region {regionname: .META.,,1,
> > >  startKey: <>, server: 10.252.67.181:60020}
> > >  ------------------------
> > >
> > >
> > >
> > >  Other curious entries from master when the regionserver died:
> > >  -------------------------------
> > >  2008-04-15 18:11:20,904 DEBUG org.apache.hadoop.hbase.HLog: Splitting
> > >  66 of 70:
> > > hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.066
> > >  2008-04-15 18:11:21,756 DEBUG org.apache.hadoop.hbase.HLog: Applied
> > >  30734 total edits
> > >  2008-04-15 18:11:21,763 DEBUG org.apache.hadoop.hbase.HLog: Splitting
> > >  67 of 70:
> > > hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.067
> > >  2008-04-15 18:11:23,608 DEBUG org.apache.hadoop.hbase.HLog: Applied
> > >  30060 total edits
> > >  2008-04-15 18:11:23,641 DEBUG org.apache.hadoop.hbase.HLog: Splitting
> > >  68 of 70:
> > > hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.068
> > >  2008-04-15 18:11:24,421 DEBUG org.apache.hadoop.hbase.HLog: Applied
> > >  30008 total edits
> > >  2008-04-15 18:11:24,424 DEBUG org.apache.hadoop.hbase.HLog: Splitting
> > >  69 of 70:
> > > hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.069
> > >  2008-04-15 18:11:24,424 INFO org.apache.hadoop.hbase.HLog: Skipping
> > >  org.apache.hadoop.fs.FileStatus@593db81a because zero length
> > >  2008-04-15 18:11:25,563 INFO org.apache.hadoop.hbase.HLog: log file
> > >  splitting completed for
> > >
> > >  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020
> > >  2008-04-15 18:11:25,563 DEBUG org.apache.hadoop.hbase.HMaster:
> > > process
> > >  server shutdown scanning root region on 10.252.67.181
> > >  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster:
> > > process
> > >  server shutdown scanning root region on 10.252.67.181 finished
> > > HMaster
> > >  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  numberOfMetaRegions: 1, onlineMetaRegions.size(): 1
> > >  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster:
> > > process
> > >  server shutdown scanning .META.,,1 on 10.252.67.181:60020 HMaster
> > >  attempt 0
> > >  2008-04-15 18:11:25,580 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 was on
> > >  shutdown server <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,583 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_to_all_parents,6597500,1207947088224 was on shutdown server
> > >  <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,585 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_to_all_parents_map,11429436,1208214299908 was on shutdown
> > >  server <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,587 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_to_all_parents_map,13948388,1208214507790 was on shutdown
> > >  server <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,588 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_to_all_parents_map,2039541,1208212644065 was on shutdown
> > >  server <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,591 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_to_all_parents_map,4050172,1208212849307 was on shutdown
> > >  server <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,594 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_to_all_parents_map,8265783,1208211269830 was on shutdown
> > >  server <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,595 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_to_all_parents_map,8665350,1208211269830 was on shutdown
> > >  server <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,596 INFO org.apache.hadoop.hbase.HMaster:
> > >  category_to_immediate_children_map,,1208288806024 was on shutdown
> > >  server <10.252.39.164:60020> (or server is null --
> > >  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
> > >  in meta and clearing pendingRegions
> > >  2008-04-15 18:11:25,597 DEBUG org.apache.hadoop.hbase.HMaster:
> > > process
> > >  server shutdown finished scanning .META.,,1 on 10.252.67.181:60020
> > >  HMaster
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region category_to_all_parents_map,13948388,1208214507790
> > > to
> > >  the only server 10.252.67.181:60020
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 to the
> > >  only server 10.252.67.181:60020
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region category_to_all_parents_map,4050172,1208212849307 to
> > >  the only server 10.252.67.181:60020
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region category_to_immediate_children_map,,1208288806024 to
> > >  the only server 10.252.67.181:60020
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region category_to_all_parents_map,2039541,1208212644065 to
> > >  the only server 10.252.67.181:60020
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region category_to_all_parents,6597500,1207947088224 to the
> > >  only server 10.252.67.181:60020
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region category_to_all_parents_map,8265783,1208211269830 to
> > >  the only server 10.252.67.181:60020
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region category_to_all_parents_map,11429436,1208214299908
> > > to
> > >  the only server 10.252.67.181:60020
> > >  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
> > >  assigning region category_to_all_parents_map,8665350,1208211269830 to
> > >  the only server 10.252.67.181:60020
> > >  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  Received MSG_REPORT_PROCESS_OPEN :
> > >  category_to_all_parents_map,13948388,1208214507790 from
> > >  10.252.67.181:60020
> > >  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  Received MSG_REPORT_PROCESS_OPEN :
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 from
> > >  10.252.67.181:60020
> > >  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  Received MSG_REPORT_PROCESS_OPEN :
> > >  category_to_all_parents_map,4050172,1208212849307 from
> > >  10.252.67.181:60020
> > >  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
> > >  Received MSG_REPORT_PROCESS_OPEN :
> > >  category_to_immediate_children_map,,1208288806024 from
> > >  10.252.67.181:60020
> > >  ---------------------------------------------
> > >
> > >
> > >  The Exception that is thrown is almost either one of the following:
> > >  ----------------------------------
> > >  org.apache.hadoop.hbase.NotServingRegionException:
> > >  org.apache.hadoop.hbase.NotServingRegionException:
> > >  category_rule_pricebin_statistics,,1208222885052
> > >        at
> > > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
> > >        at
> > > org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
> > >        at
> > > org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
> > >        at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source)
> > >        at
> > > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> > >        at java.lang.reflect.Method.invoke(Method.java:597)
> > >        at
> > > org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> > >        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> > >
> > >        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.hbase.RemoteExceptionHandler.decodeRemoteException(RemoteExceptionHandler.java:82)
> > >        at
> > > org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1028)
> > >        at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
> > >        at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
> > >  --------------------------------------
> > >
> > >  -- or ---
> > >
> > >  ---------------------------------------
> > >  java.lang.RuntimeException:
> > > java.lang.reflect.UndeclaredThrowableException
> > >        at
> > > org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1050)
> > >        at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
> > >        at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
> > >        at
> > > com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:93)
> > >        at
> > > com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:1)
> > >        at
> > > org.apache.hadoop.mapred.MapTask$MapOutputBuffer.combineAndSpill(MapTask.java:522)
> > >        at
> > > org.apache.hadoop.mapred.MapTask$MapOutputBuffer.sortAndSpillToDisk(MapTask.java:493)
> > >        at
> > > org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:713)
> > >        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:209)
> > >        at
> > > org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:2084)
> > >  Caused by: java.lang.reflect.UndeclaredThrowableException
> > >        at $Proxy2.findRootRegion(Unknown Source)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRootRegion(HConnectionManager.java:706)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:336)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:313)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:409)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:346)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:482)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:350)
> > >        at
> > > org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
> > >        at
> > > org.apache.hadoop.hbase.HTable.getRegionLocation(HTable.java:114)
> > >        at
> > > org.apache.hadoop.hbase.HTable$ServerCallable.instantiateServer(HTable.java:1009)
> > >        at
> > > org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1024)
> > >        ... 9 more
> > >  Caused by: java.net.ConnectException: Connection refused
> > >        at java.net.PlainSocketImpl.socketConnect(Native Method)
> > >        at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
> > >        at
> > > java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
> > >        at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
> > >        at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
> > >        at java.net.Socket.connect(Socket.java:519)
> > >        at
> > > org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:161)
> > >        at org.apache.hadoop.ipc.Client.getConnection(Client.java:578)
> > >        at org.apache.hadoop.ipc.Client.call(Client.java:501)
> > >        at
> > > org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
> > >        ... 22 more
> > >  ---------------------------
> > >
> > >
> > >
> > >  The region server does a lot of log rolling. Is this normal?
> > >  -----------------------------------------------
> > >  2008-04-15 18:24:22,577 WARN org.apache.hadoop.hbase.util.Sleeper: We
> > >  slept 37782ms, ten times longer than scheduled: 3000
> > >  2008-04-15 18:24:26,738 INFO org.apache.hadoop.hbase.HRegionServer:
> > >  Rolling hlog. Number of entries: 30006
> > >  2008-04-15 18:24:26,749 DEBUG org.apache.hadoop.hbase.HLog: Closing
> > >  current log writer
> > >
> > >  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.053
> > >  to get a new one
> > >  2008-04-15 18:24:26,760 INFO org.apache.hadoop.hbase.HLog: new log
> > >  writer created at
> > >
> > >  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
> > >  2008-04-15 18:24:33,073 INFO org.apache.hadoop.hbase.HRegionServer:
> > >  Rolling hlog. Number of entries: 30450
> > >  2008-04-15 18:24:33,409 DEBUG org.apache.hadoop.hbase.HLog: Closing
> > >  current log writer
> > >
> > >  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
> > >  to get a new one
> > >  2008-04-15 18:24:33,419 INFO org.apache.hadoop.hbase.HLog: new log
> > >  writer created at
> > >
> > >  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.055
> > >  -------------------------------------------------
> > >
> > >
> > >  After a lot of rolling, this happens:
> > >  ------------------------------------------------
> > >  2008-04-15 18:38:59,716 INFO org.apache.hadoop.hbase.HLog: new log
> > >  writer created at
> > >
> > >  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
> > >  2008-04-15 18:39:02,113 INFO org.apache.hadoop.hbase.HRegionServer:
> > >  Rolling hlog. Number of entries: 30360
> > >  2008-04-15 18:39:02,236 DEBUG org.apache.hadoop.hbase.HLog: Closing
> > >  current log writer
> > >
> > >  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
> > >  to get a new one
> > >  2008-04-15 18:39:02,238 INFO org.apache.hadoop.hbase.HLog: new log
> > >  writer created at
> > >
> > >  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.160
> > >  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion:
> > > Blocking
> > >  updates for 'IPC Server handler 1 on 60020': Memcache size 64.0m is
> > > >=
> > >  than blocking 64.0m size
> > >  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion:
> > > Blocking
> > >  updates for 'IPC Server handler 7 on 60020': Memcache size 64.0m is
> > > >=
> > >  than blocking 64.0m size
> > >  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion:
> > > Blocking
> > >  updates for 'IPC Server handler 8 on 60020': Memcache size 64.0m is
> > > >=
> > >  than blocking 64.0m size
> > >  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion:
> > > Blocking
> > >  updates for 'IPC Server handler 9 on 60020': Memcache size 64.0m is
> > > >=
> > >  than blocking 64.0m size
> > >  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion:
> > > Blocking
> > >  updates for 'IPC Server handler 4 on 60020': Memcache size 64.0m is
> > > >=
> > >  than blocking 64.0m size
> > >  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion:
> > > Blocking
> > >  updates for 'IPC Server handler 3 on 60020': Memcache size 64.0m is
> > > >=
> > >  than blocking 64.0m size
> > >  2008-04-15 18:39:05,627 DEBUG org.apache.hadoop.hbase.HRegion:
> > > Started
> > >  memcache flush for region
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363. Size
> > >  64.0m
> > >  2008-04-15 18:39:09,544 DEBUG org.apache.hadoop.hbase.HStore: Added
> > >  1064767197/adjusted_q/3161297904753567399 with 0 entries, sequence id
> > >  64293357, and size 110.0 for 1064767197/adjusted_q
> > >  2008-04-15 18:39:15,489 DEBUG org.apache.hadoop.hbase.HStore: Added
> > >  1064767197/rule_id/5414884402918348270 with 285058 entries, sequence
> > >  id 64293357, and size 11.8m for 1064767197/rule_id
> > >  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> > >  Unblocking updates for region
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> > >  Server handler 1 on 60020'
> > >  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> > >  Unblocking updates for region
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> > >  Server handler 7 on 60020'
> > >  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> > >  Unblocking updates for region
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> > >  Server handler 8 on 60020'
> > >  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> > >  Unblocking updates for region
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> > >  Server handler 9 on 60020'
> > >  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> > >  Unblocking updates for region
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> > >  Server handler 4 on 60020'
> > >  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
> > >  Unblocking updates for region
> > >  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
> > >  Server handler 3 on 60020'
> > >  2008-04-15 18:39:16,739 INFO org.apache.hadoop.hbase.HRegionServer:
> > >  Rolling hlog. Number of entries: 33252
> > >  2008-04-15 18:39:45,008 INFO org.apache.hadoop.hbase.HRegionServer:
> > >  MSG_REGION_OPEN : .META.,,1
> > >  2008-04-15 18:39:51,636 DEBUG org.apache.hadoop.hbase.HStore: Added
> > >  1064767197/category_id/5081092265153632091 with 285058 entries,
> > >  sequence id 64293357, and size 14.5m for 1064767197/category_id
> > >  2008-04-15 18:40:09,041 DEBUG org.apache.hadoop.hbase.HStore: Added
> > >  1064767197/success_count/2540574911286829228 with 285058 entries,
> > >  sequence id 64293357, and size 13.4m for 1064767197/success_count
> > >  2008-04-15 18:40:12,707 DEBUG org.apache.hadoop.hbase.HStore: Added
> > >  1064767197/adjusted_deviation/270946121541713279 with 0 entries,
> > >  sequence id 64293357, and size 110.0 for
> > > 1064767197/adjusted_deviation
> > >  2008-04-15 18:40:30,453 DEBUG org.apache.hadoop.hbase.HStore: Added
> > >  1064767197/hidden_variable/493176493179028867 with 285058 entries,
> > >  sequence id 64293357, and size 16.6m for 1064767197/hidden_variable
> > >  ---------------------------------------------------
> > >
> > >
> > >
> > >
> > >  On Tue, Apr 15, 2008 at 9:12 AM, stack <st...@duboce.net> wrote:
> > >  > You are running 0.1.1 hbase?  Please enable DEBUG logging, see
> > > http://wiki.apache.org/hadoop/Hbase/FAQ#5 for how, and send over a
> > > log.
> > >  > Thanks,
> > >  > St.Ack
> > >  >
> > >  >
> > >  >
> > >  >
> > >  > Daniel Leffel wrote:
> > >  >
> > >  > > Hi,
> > >  > > I'm new to HBase and investigating it for use in our large
> > > production
> > >  > > environment. Largely, I'm quite impressed by the possibilities.
> > >  > >
> > >  > > One bit of behavior I'm noticing is that region servers tend to
> > > quit
> > >  > > unexpectedly during moderately loaded map/reduce jobs without
> > > really any
> > >  > > helpful information in the logs as to why.
> > >  > >
> > >  > > Is the behavior expected? How else should I be troubleshooting?
> > >  > >
> > >  > > Thanks in advance!
> > >  > >
> > >  > > Danny Leffel
> > >  > >
> > >  > >
> > >  > >
> > >  >
> > >  >
> > >
> > >
> > >
> >
>

Re: Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Posted by stack <st...@duboce.net>.
In the just-previous mail I explained that the NotServingRegionException 
is part of 'normal' operation.  Usually the client recalibrates and away 
we go again.  Is it failing your MR upload job?
St.Ack


Daniel Leffel wrote:
> Now I just got this exception:
>
> 2008-04-15 18:50:02,107 DEBUG org.apache.hadoop.hbase.HStore: maximum
> sequence id for hstore 856584617/rule_id is 68555204
> 2008-04-15 18:50:02,284 INFO org.apache.hadoop.ipc.Server: IPC Server
> handler 8 on 60020, call
> batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@19d277e)
> from 10.252.50.36:53955: error:
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
>         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
>         at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> 2008-04-15 18:50:02,286 INFO org.apache.hadoop.ipc.Server: IPC Server
> handler 4 on 60020, call
> batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@1890c67)
> from 10.252.114.85:43998: error:
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
>         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
>         at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> 2008-04-15 18:50:02,288 INFO org.apache.hadoop.ipc.Server: IPC Server
> handler 3 on 60020, call
> batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@a801b0)
> from 10.252.67.181:56452: error:
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
>         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
>         at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> 2008-04-15 18:50:02,299 INFO org.apache.hadoop.ipc.Server: IPC Server
> handler 9 on 60020, call
> batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@5ea4c5)
> from 10.252.50.36:53954: error:
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
>         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
>         at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> 2008-04-15 18:50:02,352 INFO org.apache.hadoop.ipc.Server: IPC Server
> handler 6 on 60020, call
> batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@193b604)
> from 10.252.114.85:43999: error:
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
>         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
>         at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
> 2008-04-15 18:50:02,353 INFO org.apache.hadoop.ipc.Server: IPC Server
> handler 6 on 60020, call
> batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
> 9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@92e96c)
> from 10.252.67.181:56453: error:
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
> org.apache.hadoop.hbase.NotServingRegionException:
> category_rule_pricebin_statistics,2332627_1_-11,1208293443363
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
>         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
>         at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
>
>
>
> -----------------------------------
>
>
>
>
> On Tue, Apr 15, 2008 at 3:43 PM, Daniel Leffel <da...@gmail.com> wrote:
>   
>> I was running 0.1.0 (hadn't noticed the new release). To rule out that
>>  as the issue, I've since upgraded to 0.1.1. I am still seeing the same
>>  behavior. Also, I'm running on top of Hadoop 0.16.1.
>>
>>  Interestingly, during the latest test, the master process died too.
>>
>>  Not sure what to be looking for in the DEBUG output, but here's a few
>>  interesting entries.
>>
>>  Here's the master node when one of the region servers died:
>>  -------------------------------------
>>  2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.Leases:
>>  HMaster.leaseChecker lease expired 1563117711/1563117711
>>  2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.HMaster:
>>  10.252.39.164:60020 lease expired
>>  2008-04-15 18:09:10,201 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region -ROOT-,,0 to the only server 10.252.67.181:60020
>>  2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
>>  Received MSG_REPORT_PROCESS_OPEN : -ROOT-,,0 from 10.252.67.181:60020
>>  2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
>>  Received MSG_REPORT_OPEN : -ROOT-,,0 from 10.252.67.181:60020
>>  2008-04-15 18:09:13,211 INFO org.apache.hadoop.hbase.HMaster:
>>  10.252.67.181:60020 serving -ROOT-,,0
>>  2008-04-15 18:09:30,963 WARN org.apache.hadoop.hbase.HMaster: Scan ROOT region
>>  java.net.SocketTimeoutException: timed out waiting for rpc response
>>         at org.apache.hadoop.ipc.Client.call(Client.java:514)
>>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
>>         at $Proxy1.openScanner(Unknown Source)
>>         at org.apache.hadoop.hbase.HMaster$BaseScanner.scanRegion(HMaster.java:227)
>>         at org.apache.hadoop.hbase.HMaster$RootScanner.scanRoot(HMaster.java:540)
>>         at org.apache.hadoop.hbase.HMaster$RootScanner.maintenanceScan(HMaster.java:565)
>>         at org.apache.hadoop.hbase.HMaster$BaseScanner.chore(HMaster.java:207)
>>         at org.apache.hadoop.hbase.Chore.run(Chore.java:63)
>>  2008-04-15 18:09:30,965 INFO org.apache.hadoop.hbase.HMaster:
>>  HMaster.rootScanner scanning meta region {regionname: -ROOT-,,0,
>>  startKey: <>, server: 10.252.67.181:60020}
>>  2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster:
>>  HMaster.rootScanner regioninfo: {regionname: .META.,,1, startKey: <>,
>>  endKey: <>, encodedName: 1028785192, tableDesc: {name: .META.,
>>  families: {info:={name: info, max versions: 1, compression: NONE, in
>>  memory: false, max length: 2147483647, bloom filter: none}}}}, server:
>>  10.252.67.181:60020, startCode: 1208278043501
>>  2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster: Current
>>  assignment of .META.,,1 is not valid: storedInfo: address:
>>  10.252.67.181:60020, startcode: 1208296518624, load: (requests: 0
>>  regions: 12), startCode: 1208278043501, storedInfo.startCode:
>>  1208296518624, unassignedRegions: false, pendingRegions: false
>>  2008-04-15 18:09:30,975 INFO org.apache.hadoop.hbase.HMaster:
>>  HMaster.rootScanner scan of meta region {regionname: -ROOT-,,0,
>>  startKey: <>, server: 10.252.67.181:60020} complete
>>  2008-04-15 18:09:31,271 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region .META.,,1 to the only server 10.252.67.181:60020
>>  2008-04-15 18:09:34,281 DEBUG org.apache.hadoop.hbase.HMaster:
>>  Received MSG_REPORT_PROCESS_OPEN : .META.,,1 from 10.252.67.181:60020
>>  2008-04-15 18:09:35,602 INFO org.apache.hadoop.hbase.HMaster:
>>  HMaster.metaScanner scanning meta region {regionname: .META.,,1,
>>  startKey: <>, server: 10.252.67.181:60020}
>>  ------------------------
>>
>>
>>
>>  Other curious entries from master when the regionserver died:
>>  -------------------------------
>>  2008-04-15 18:11:20,904 DEBUG org.apache.hadoop.hbase.HLog: Splitting
>>  66 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.066
>>  2008-04-15 18:11:21,756 DEBUG org.apache.hadoop.hbase.HLog: Applied
>>  30734 total edits
>>  2008-04-15 18:11:21,763 DEBUG org.apache.hadoop.hbase.HLog: Splitting
>>  67 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.067
>>  2008-04-15 18:11:23,608 DEBUG org.apache.hadoop.hbase.HLog: Applied
>>  30060 total edits
>>  2008-04-15 18:11:23,641 DEBUG org.apache.hadoop.hbase.HLog: Splitting
>>  68 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.068
>>  2008-04-15 18:11:24,421 DEBUG org.apache.hadoop.hbase.HLog: Applied
>>  30008 total edits
>>  2008-04-15 18:11:24,424 DEBUG org.apache.hadoop.hbase.HLog: Splitting
>>  69 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.069
>>  2008-04-15 18:11:24,424 INFO org.apache.hadoop.hbase.HLog: Skipping
>>  org.apache.hadoop.fs.FileStatus@593db81a because zero length
>>  2008-04-15 18:11:25,563 INFO org.apache.hadoop.hbase.HLog: log file
>>  splitting completed for
>>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020
>>  2008-04-15 18:11:25,563 DEBUG org.apache.hadoop.hbase.HMaster: process
>>  server shutdown scanning root region on 10.252.67.181
>>  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster: process
>>  server shutdown scanning root region on 10.252.67.181 finished HMaster
>>  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster:
>>  numberOfMetaRegions: 1, onlineMetaRegions.size(): 1
>>  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster: process
>>  server shutdown scanning .META.,,1 on 10.252.67.181:60020 HMaster
>>  attempt 0
>>  2008-04-15 18:11:25,580 INFO org.apache.hadoop.hbase.HMaster:
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 was on
>>  shutdown server <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,583 INFO org.apache.hadoop.hbase.HMaster:
>>  category_to_all_parents,6597500,1207947088224 was on shutdown server
>>  <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,585 INFO org.apache.hadoop.hbase.HMaster:
>>  category_to_all_parents_map,11429436,1208214299908 was on shutdown
>>  server <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,587 INFO org.apache.hadoop.hbase.HMaster:
>>  category_to_all_parents_map,13948388,1208214507790 was on shutdown
>>  server <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,588 INFO org.apache.hadoop.hbase.HMaster:
>>  category_to_all_parents_map,2039541,1208212644065 was on shutdown
>>  server <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,591 INFO org.apache.hadoop.hbase.HMaster:
>>  category_to_all_parents_map,4050172,1208212849307 was on shutdown
>>  server <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,594 INFO org.apache.hadoop.hbase.HMaster:
>>  category_to_all_parents_map,8265783,1208211269830 was on shutdown
>>  server <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,595 INFO org.apache.hadoop.hbase.HMaster:
>>  category_to_all_parents_map,8665350,1208211269830 was on shutdown
>>  server <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,596 INFO org.apache.hadoop.hbase.HMaster:
>>  category_to_immediate_children_map,,1208288806024 was on shutdown
>>  server <10.252.39.164:60020> (or server is null --
>>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>>  in meta and clearing pendingRegions
>>  2008-04-15 18:11:25,597 DEBUG org.apache.hadoop.hbase.HMaster: process
>>  server shutdown finished scanning .META.,,1 on 10.252.67.181:60020
>>  HMaster
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region category_to_all_parents_map,13948388,1208214507790 to
>>  the only server 10.252.67.181:60020
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 to the
>>  only server 10.252.67.181:60020
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region category_to_all_parents_map,4050172,1208212849307 to
>>  the only server 10.252.67.181:60020
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region category_to_immediate_children_map,,1208288806024 to
>>  the only server 10.252.67.181:60020
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region category_to_all_parents_map,2039541,1208212644065 to
>>  the only server 10.252.67.181:60020
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region category_to_all_parents,6597500,1207947088224 to the
>>  only server 10.252.67.181:60020
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region category_to_all_parents_map,8265783,1208211269830 to
>>  the only server 10.252.67.181:60020
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region category_to_all_parents_map,11429436,1208214299908 to
>>  the only server 10.252.67.181:60020
>>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>>  assigning region category_to_all_parents_map,8665350,1208211269830 to
>>  the only server 10.252.67.181:60020
>>  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
>>  Received MSG_REPORT_PROCESS_OPEN :
>>  category_to_all_parents_map,13948388,1208214507790 from
>>  10.252.67.181:60020
>>  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
>>  Received MSG_REPORT_PROCESS_OPEN :
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 from
>>  10.252.67.181:60020
>>  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
>>  Received MSG_REPORT_PROCESS_OPEN :
>>  category_to_all_parents_map,4050172,1208212849307 from
>>  10.252.67.181:60020
>>  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
>>  Received MSG_REPORT_PROCESS_OPEN :
>>  category_to_immediate_children_map,,1208288806024 from
>>  10.252.67.181:60020
>>  ---------------------------------------------
>>
>>
>>  The Exception that is thrown is almost either one of the following:
>>  ----------------------------------
>>  org.apache.hadoop.hbase.NotServingRegionException:
>>  org.apache.hadoop.hbase.NotServingRegionException:
>>  category_rule_pricebin_statistics,,1208222885052
>>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
>>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
>>         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
>>         at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source)
>>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>         at java.lang.reflect.Method.invoke(Method.java:597)
>>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
>>
>>         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.hbase.RemoteExceptionHandler.decodeRemoteException(RemoteExceptionHandler.java:82)
>>         at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1028)
>>         at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
>>         at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
>>  --------------------------------------
>>
>>  -- or ---
>>
>>  ---------------------------------------
>>  java.lang.RuntimeException: java.lang.reflect.UndeclaredThrowableException
>>         at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1050)
>>         at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
>>         at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
>>         at com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:93)
>>         at com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:1)
>>         at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.combineAndSpill(MapTask.java:522)
>>         at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.sortAndSpillToDisk(MapTask.java:493)
>>         at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:713)
>>         at org.apache.hadoop.mapred.MapTask.run(MapTask.java:209)
>>         at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:2084)
>>  Caused by: java.lang.reflect.UndeclaredThrowableException
>>         at $Proxy2.findRootRegion(Unknown Source)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRootRegion(HConnectionManager.java:706)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:336)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:313)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:409)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:346)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:482)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:350)
>>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
>>         at org.apache.hadoop.hbase.HTable.getRegionLocation(HTable.java:114)
>>         at org.apache.hadoop.hbase.HTable$ServerCallable.instantiateServer(HTable.java:1009)
>>         at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1024)
>>         ... 9 more
>>  Caused by: java.net.ConnectException: Connection refused
>>         at java.net.PlainSocketImpl.socketConnect(Native Method)
>>         at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
>>         at java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>>         at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>>         at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>>         at java.net.Socket.connect(Socket.java:519)
>>         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:161)
>>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:578)
>>         at org.apache.hadoop.ipc.Client.call(Client.java:501)
>>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
>>         ... 22 more
>>  ---------------------------
>>
>>
>>
>>  The region server does a lot of log rolling. Is this normal?
>>  -----------------------------------------------
>>  2008-04-15 18:24:22,577 WARN org.apache.hadoop.hbase.util.Sleeper: We
>>  slept 37782ms, ten times longer than scheduled: 3000
>>  2008-04-15 18:24:26,738 INFO org.apache.hadoop.hbase.HRegionServer:
>>  Rolling hlog. Number of entries: 30006
>>  2008-04-15 18:24:26,749 DEBUG org.apache.hadoop.hbase.HLog: Closing
>>  current log writer
>>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.053
>>  to get a new one
>>  2008-04-15 18:24:26,760 INFO org.apache.hadoop.hbase.HLog: new log
>>  writer created at
>>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
>>  2008-04-15 18:24:33,073 INFO org.apache.hadoop.hbase.HRegionServer:
>>  Rolling hlog. Number of entries: 30450
>>  2008-04-15 18:24:33,409 DEBUG org.apache.hadoop.hbase.HLog: Closing
>>  current log writer
>>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
>>  to get a new one
>>  2008-04-15 18:24:33,419 INFO org.apache.hadoop.hbase.HLog: new log
>>  writer created at
>>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.055
>>  -------------------------------------------------
>>
>>
>>  After a lot of rolling, this happens:
>>  ------------------------------------------------
>>  2008-04-15 18:38:59,716 INFO org.apache.hadoop.hbase.HLog: new log
>>  writer created at
>>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
>>  2008-04-15 18:39:02,113 INFO org.apache.hadoop.hbase.HRegionServer:
>>  Rolling hlog. Number of entries: 30360
>>  2008-04-15 18:39:02,236 DEBUG org.apache.hadoop.hbase.HLog: Closing
>>  current log writer
>>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
>>  to get a new one
>>  2008-04-15 18:39:02,238 INFO org.apache.hadoop.hbase.HLog: new log
>>  writer created at
>>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.160
>>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>>  updates for 'IPC Server handler 1 on 60020': Memcache size 64.0m is >=
>>  than blocking 64.0m size
>>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>>  updates for 'IPC Server handler 7 on 60020': Memcache size 64.0m is >=
>>  than blocking 64.0m size
>>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>>  updates for 'IPC Server handler 8 on 60020': Memcache size 64.0m is >=
>>  than blocking 64.0m size
>>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>>  updates for 'IPC Server handler 9 on 60020': Memcache size 64.0m is >=
>>  than blocking 64.0m size
>>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>>  updates for 'IPC Server handler 4 on 60020': Memcache size 64.0m is >=
>>  than blocking 64.0m size
>>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>>  updates for 'IPC Server handler 3 on 60020': Memcache size 64.0m is >=
>>  than blocking 64.0m size
>>  2008-04-15 18:39:05,627 DEBUG org.apache.hadoop.hbase.HRegion: Started
>>  memcache flush for region
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363. Size
>>  64.0m
>>  2008-04-15 18:39:09,544 DEBUG org.apache.hadoop.hbase.HStore: Added
>>  1064767197/adjusted_q/3161297904753567399 with 0 entries, sequence id
>>  64293357, and size 110.0 for 1064767197/adjusted_q
>>  2008-04-15 18:39:15,489 DEBUG org.apache.hadoop.hbase.HStore: Added
>>  1064767197/rule_id/5414884402918348270 with 285058 entries, sequence
>>  id 64293357, and size 11.8m for 1064767197/rule_id
>>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>>  Unblocking updates for region
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>>  Server handler 1 on 60020'
>>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>>  Unblocking updates for region
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>>  Server handler 7 on 60020'
>>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>>  Unblocking updates for region
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>>  Server handler 8 on 60020'
>>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>>  Unblocking updates for region
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>>  Server handler 9 on 60020'
>>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>>  Unblocking updates for region
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>>  Server handler 4 on 60020'
>>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>>  Unblocking updates for region
>>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>>  Server handler 3 on 60020'
>>  2008-04-15 18:39:16,739 INFO org.apache.hadoop.hbase.HRegionServer:
>>  Rolling hlog. Number of entries: 33252
>>  2008-04-15 18:39:45,008 INFO org.apache.hadoop.hbase.HRegionServer:
>>  MSG_REGION_OPEN : .META.,,1
>>  2008-04-15 18:39:51,636 DEBUG org.apache.hadoop.hbase.HStore: Added
>>  1064767197/category_id/5081092265153632091 with 285058 entries,
>>  sequence id 64293357, and size 14.5m for 1064767197/category_id
>>  2008-04-15 18:40:09,041 DEBUG org.apache.hadoop.hbase.HStore: Added
>>  1064767197/success_count/2540574911286829228 with 285058 entries,
>>  sequence id 64293357, and size 13.4m for 1064767197/success_count
>>  2008-04-15 18:40:12,707 DEBUG org.apache.hadoop.hbase.HStore: Added
>>  1064767197/adjusted_deviation/270946121541713279 with 0 entries,
>>  sequence id 64293357, and size 110.0 for 1064767197/adjusted_deviation
>>  2008-04-15 18:40:30,453 DEBUG org.apache.hadoop.hbase.HStore: Added
>>  1064767197/hidden_variable/493176493179028867 with 285058 entries,
>>  sequence id 64293357, and size 16.6m for 1064767197/hidden_variable
>>  ---------------------------------------------------
>>
>>
>>
>>
>>  On Tue, Apr 15, 2008 at 9:12 AM, stack <st...@duboce.net> wrote:
>>  > You are running 0.1.1 hbase?  Please enable DEBUG logging, see http://wiki.apache.org/hadoop/Hbase/FAQ#5 for how, and send over a log.
>>  > Thanks,
>>  > St.Ack
>>  >
>>  >
>>  >
>>  >
>>  > Daniel Leffel wrote:
>>  >
>>  > > Hi,
>>  > > I'm new to HBase and investigating it for use in our large production
>>  > > environment. Largely, I'm quite impressed by the possibilities.
>>  > >
>>  > > One bit of behavior I'm noticing is that region servers tend to quit
>>  > > unexpectedly during moderately loaded map/reduce jobs without really any
>>  > > helpful information in the logs as to why.
>>  > >
>>  > > Is the behavior expected? How else should I be troubleshooting?
>>  > >
>>  > > Thanks in advance!
>>  > >
>>  > > Danny Leffel
>>  > >
>>  > >
>>  > >
>>  >
>>  >
>>
>>     


Re: Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Posted by Daniel Leffel <da...@gmail.com>.
Now I just got this exception:

2008-04-15 18:50:02,107 DEBUG org.apache.hadoop.hbase.HStore: maximum
sequence id for hstore 856584617/rule_id is 68555204
2008-04-15 18:50:02,284 INFO org.apache.hadoop.ipc.Server: IPC Server
handler 8 on 60020, call
batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@19d277e)
from 10.252.50.36:53955: error:
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
        at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
2008-04-15 18:50:02,286 INFO org.apache.hadoop.ipc.Server: IPC Server
handler 4 on 60020, call
batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@1890c67)
from 10.252.114.85:43998: error:
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
        at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
2008-04-15 18:50:02,288 INFO org.apache.hadoop.ipc.Server: IPC Server
handler 3 on 60020, call
batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@a801b0)
from 10.252.67.181:56452: error:
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
        at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
2008-04-15 18:50:02,299 INFO org.apache.hadoop.ipc.Server: IPC Server
handler 9 on 60020, call
batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@5ea4c5)
from 10.252.50.36:53954: error:
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
        at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
2008-04-15 18:50:02,352 INFO org.apache.hadoop.ipc.Server: IPC Server
handler 6 on 60020, call
batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@193b604)
from 10.252.114.85:43999: error:
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
        at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
2008-04-15 18:50:02,353 INFO org.apache.hadoop.ipc.Server: IPC Server
handler 6 on 60020, call
batchUpdate(category_rule_pricebin_statistics,2332627_1_-11,1208293443363,
9223372036854775807, org.apache.hadoop.hbase.io.BatchUpdate@92e96c)
from 10.252.67.181:56453: error:
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
        at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
        at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)



-----------------------------------




On Tue, Apr 15, 2008 at 3:43 PM, Daniel Leffel <da...@gmail.com> wrote:
> I was running 0.1.0 (hadn't noticed the new release). To rule out that
>  as the issue, I've since upgraded to 0.1.1. I am still seeing the same
>  behavior. Also, I'm running on top of Hadoop 0.16.1.
>
>  Interestingly, during the latest test, the master process died too.
>
>  Not sure what to be looking for in the DEBUG output, but here's a few
>  interesting entries.
>
>  Here's the master node when one of the region servers died:
>  -------------------------------------
>  2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.Leases:
>  HMaster.leaseChecker lease expired 1563117711/1563117711
>  2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.HMaster:
>  10.252.39.164:60020 lease expired
>  2008-04-15 18:09:10,201 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region -ROOT-,,0 to the only server 10.252.67.181:60020
>  2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
>  Received MSG_REPORT_PROCESS_OPEN : -ROOT-,,0 from 10.252.67.181:60020
>  2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
>  Received MSG_REPORT_OPEN : -ROOT-,,0 from 10.252.67.181:60020
>  2008-04-15 18:09:13,211 INFO org.apache.hadoop.hbase.HMaster:
>  10.252.67.181:60020 serving -ROOT-,,0
>  2008-04-15 18:09:30,963 WARN org.apache.hadoop.hbase.HMaster: Scan ROOT region
>  java.net.SocketTimeoutException: timed out waiting for rpc response
>         at org.apache.hadoop.ipc.Client.call(Client.java:514)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
>         at $Proxy1.openScanner(Unknown Source)
>         at org.apache.hadoop.hbase.HMaster$BaseScanner.scanRegion(HMaster.java:227)
>         at org.apache.hadoop.hbase.HMaster$RootScanner.scanRoot(HMaster.java:540)
>         at org.apache.hadoop.hbase.HMaster$RootScanner.maintenanceScan(HMaster.java:565)
>         at org.apache.hadoop.hbase.HMaster$BaseScanner.chore(HMaster.java:207)
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:63)
>  2008-04-15 18:09:30,965 INFO org.apache.hadoop.hbase.HMaster:
>  HMaster.rootScanner scanning meta region {regionname: -ROOT-,,0,
>  startKey: <>, server: 10.252.67.181:60020}
>  2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster:
>  HMaster.rootScanner regioninfo: {regionname: .META.,,1, startKey: <>,
>  endKey: <>, encodedName: 1028785192, tableDesc: {name: .META.,
>  families: {info:={name: info, max versions: 1, compression: NONE, in
>  memory: false, max length: 2147483647, bloom filter: none}}}}, server:
>  10.252.67.181:60020, startCode: 1208278043501
>  2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster: Current
>  assignment of .META.,,1 is not valid: storedInfo: address:
>  10.252.67.181:60020, startcode: 1208296518624, load: (requests: 0
>  regions: 12), startCode: 1208278043501, storedInfo.startCode:
>  1208296518624, unassignedRegions: false, pendingRegions: false
>  2008-04-15 18:09:30,975 INFO org.apache.hadoop.hbase.HMaster:
>  HMaster.rootScanner scan of meta region {regionname: -ROOT-,,0,
>  startKey: <>, server: 10.252.67.181:60020} complete
>  2008-04-15 18:09:31,271 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region .META.,,1 to the only server 10.252.67.181:60020
>  2008-04-15 18:09:34,281 DEBUG org.apache.hadoop.hbase.HMaster:
>  Received MSG_REPORT_PROCESS_OPEN : .META.,,1 from 10.252.67.181:60020
>  2008-04-15 18:09:35,602 INFO org.apache.hadoop.hbase.HMaster:
>  HMaster.metaScanner scanning meta region {regionname: .META.,,1,
>  startKey: <>, server: 10.252.67.181:60020}
>  ------------------------
>
>
>
>  Other curious entries from master when the regionserver died:
>  -------------------------------
>  2008-04-15 18:11:20,904 DEBUG org.apache.hadoop.hbase.HLog: Splitting
>  66 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.066
>  2008-04-15 18:11:21,756 DEBUG org.apache.hadoop.hbase.HLog: Applied
>  30734 total edits
>  2008-04-15 18:11:21,763 DEBUG org.apache.hadoop.hbase.HLog: Splitting
>  67 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.067
>  2008-04-15 18:11:23,608 DEBUG org.apache.hadoop.hbase.HLog: Applied
>  30060 total edits
>  2008-04-15 18:11:23,641 DEBUG org.apache.hadoop.hbase.HLog: Splitting
>  68 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.068
>  2008-04-15 18:11:24,421 DEBUG org.apache.hadoop.hbase.HLog: Applied
>  30008 total edits
>  2008-04-15 18:11:24,424 DEBUG org.apache.hadoop.hbase.HLog: Splitting
>  69 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.069
>  2008-04-15 18:11:24,424 INFO org.apache.hadoop.hbase.HLog: Skipping
>  org.apache.hadoop.fs.FileStatus@593db81a because zero length
>  2008-04-15 18:11:25,563 INFO org.apache.hadoop.hbase.HLog: log file
>  splitting completed for
>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020
>  2008-04-15 18:11:25,563 DEBUG org.apache.hadoop.hbase.HMaster: process
>  server shutdown scanning root region on 10.252.67.181
>  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster: process
>  server shutdown scanning root region on 10.252.67.181 finished HMaster
>  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster:
>  numberOfMetaRegions: 1, onlineMetaRegions.size(): 1
>  2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster: process
>  server shutdown scanning .META.,,1 on 10.252.67.181:60020 HMaster
>  attempt 0
>  2008-04-15 18:11:25,580 INFO org.apache.hadoop.hbase.HMaster:
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 was on
>  shutdown server <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,583 INFO org.apache.hadoop.hbase.HMaster:
>  category_to_all_parents,6597500,1207947088224 was on shutdown server
>  <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,585 INFO org.apache.hadoop.hbase.HMaster:
>  category_to_all_parents_map,11429436,1208214299908 was on shutdown
>  server <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,587 INFO org.apache.hadoop.hbase.HMaster:
>  category_to_all_parents_map,13948388,1208214507790 was on shutdown
>  server <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,588 INFO org.apache.hadoop.hbase.HMaster:
>  category_to_all_parents_map,2039541,1208212644065 was on shutdown
>  server <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,591 INFO org.apache.hadoop.hbase.HMaster:
>  category_to_all_parents_map,4050172,1208212849307 was on shutdown
>  server <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,594 INFO org.apache.hadoop.hbase.HMaster:
>  category_to_all_parents_map,8265783,1208211269830 was on shutdown
>  server <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,595 INFO org.apache.hadoop.hbase.HMaster:
>  category_to_all_parents_map,8665350,1208211269830 was on shutdown
>  server <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,596 INFO org.apache.hadoop.hbase.HMaster:
>  category_to_immediate_children_map,,1208288806024 was on shutdown
>  server <10.252.39.164:60020> (or server is null --
>  org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
>  in meta and clearing pendingRegions
>  2008-04-15 18:11:25,597 DEBUG org.apache.hadoop.hbase.HMaster: process
>  server shutdown finished scanning .META.,,1 on 10.252.67.181:60020
>  HMaster
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region category_to_all_parents_map,13948388,1208214507790 to
>  the only server 10.252.67.181:60020
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 to the
>  only server 10.252.67.181:60020
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region category_to_all_parents_map,4050172,1208212849307 to
>  the only server 10.252.67.181:60020
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region category_to_immediate_children_map,,1208288806024 to
>  the only server 10.252.67.181:60020
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region category_to_all_parents_map,2039541,1208212644065 to
>  the only server 10.252.67.181:60020
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region category_to_all_parents,6597500,1207947088224 to the
>  only server 10.252.67.181:60020
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region category_to_all_parents_map,8265783,1208211269830 to
>  the only server 10.252.67.181:60020
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region category_to_all_parents_map,11429436,1208214299908 to
>  the only server 10.252.67.181:60020
>  2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
>  assigning region category_to_all_parents_map,8665350,1208211269830 to
>  the only server 10.252.67.181:60020
>  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
>  Received MSG_REPORT_PROCESS_OPEN :
>  category_to_all_parents_map,13948388,1208214507790 from
>  10.252.67.181:60020
>  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
>  Received MSG_REPORT_PROCESS_OPEN :
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 from
>  10.252.67.181:60020
>  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
>  Received MSG_REPORT_PROCESS_OPEN :
>  category_to_all_parents_map,4050172,1208212849307 from
>  10.252.67.181:60020
>  2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
>  Received MSG_REPORT_PROCESS_OPEN :
>  category_to_immediate_children_map,,1208288806024 from
>  10.252.67.181:60020
>  ---------------------------------------------
>
>
>  The Exception that is thrown is almost either one of the following:
>  ----------------------------------
>  org.apache.hadoop.hbase.NotServingRegionException:
>  org.apache.hadoop.hbase.NotServingRegionException:
>  category_rule_pricebin_statistics,,1208222885052
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
>         at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
>         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
>         at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)
>
>         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.hbase.RemoteExceptionHandler.decodeRemoteException(RemoteExceptionHandler.java:82)
>         at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1028)
>         at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
>         at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
>  --------------------------------------
>
>  -- or ---
>
>  ---------------------------------------
>  java.lang.RuntimeException: java.lang.reflect.UndeclaredThrowableException
>         at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1050)
>         at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
>         at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
>         at com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:93)
>         at com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:1)
>         at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.combineAndSpill(MapTask.java:522)
>         at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.sortAndSpillToDisk(MapTask.java:493)
>         at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:713)
>         at org.apache.hadoop.mapred.MapTask.run(MapTask.java:209)
>         at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:2084)
>  Caused by: java.lang.reflect.UndeclaredThrowableException
>         at $Proxy2.findRootRegion(Unknown Source)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRootRegion(HConnectionManager.java:706)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:336)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:313)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:409)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:346)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:482)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:350)
>         at org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
>         at org.apache.hadoop.hbase.HTable.getRegionLocation(HTable.java:114)
>         at org.apache.hadoop.hbase.HTable$ServerCallable.instantiateServer(HTable.java:1009)
>         at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1024)
>         ... 9 more
>  Caused by: java.net.ConnectException: Connection refused
>         at java.net.PlainSocketImpl.socketConnect(Native Method)
>         at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
>         at java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>         at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>         at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>         at java.net.Socket.connect(Socket.java:519)
>         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:161)
>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:578)
>         at org.apache.hadoop.ipc.Client.call(Client.java:501)
>         at org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
>         ... 22 more
>  ---------------------------
>
>
>
>  The region server does a lot of log rolling. Is this normal?
>  -----------------------------------------------
>  2008-04-15 18:24:22,577 WARN org.apache.hadoop.hbase.util.Sleeper: We
>  slept 37782ms, ten times longer than scheduled: 3000
>  2008-04-15 18:24:26,738 INFO org.apache.hadoop.hbase.HRegionServer:
>  Rolling hlog. Number of entries: 30006
>  2008-04-15 18:24:26,749 DEBUG org.apache.hadoop.hbase.HLog: Closing
>  current log writer
>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.053
>  to get a new one
>  2008-04-15 18:24:26,760 INFO org.apache.hadoop.hbase.HLog: new log
>  writer created at
>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
>  2008-04-15 18:24:33,073 INFO org.apache.hadoop.hbase.HRegionServer:
>  Rolling hlog. Number of entries: 30450
>  2008-04-15 18:24:33,409 DEBUG org.apache.hadoop.hbase.HLog: Closing
>  current log writer
>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
>  to get a new one
>  2008-04-15 18:24:33,419 INFO org.apache.hadoop.hbase.HLog: new log
>  writer created at
>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.055
>  -------------------------------------------------
>
>
>  After a lot of rolling, this happens:
>  ------------------------------------------------
>  2008-04-15 18:38:59,716 INFO org.apache.hadoop.hbase.HLog: new log
>  writer created at
>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
>  2008-04-15 18:39:02,113 INFO org.apache.hadoop.hbase.HRegionServer:
>  Rolling hlog. Number of entries: 30360
>  2008-04-15 18:39:02,236 DEBUG org.apache.hadoop.hbase.HLog: Closing
>  current log writer
>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
>  to get a new one
>  2008-04-15 18:39:02,238 INFO org.apache.hadoop.hbase.HLog: new log
>  writer created at
>  hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.160
>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>  updates for 'IPC Server handler 1 on 60020': Memcache size 64.0m is >=
>  than blocking 64.0m size
>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>  updates for 'IPC Server handler 7 on 60020': Memcache size 64.0m is >=
>  than blocking 64.0m size
>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>  updates for 'IPC Server handler 8 on 60020': Memcache size 64.0m is >=
>  than blocking 64.0m size
>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>  updates for 'IPC Server handler 9 on 60020': Memcache size 64.0m is >=
>  than blocking 64.0m size
>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>  updates for 'IPC Server handler 4 on 60020': Memcache size 64.0m is >=
>  than blocking 64.0m size
>  2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
>  updates for 'IPC Server handler 3 on 60020': Memcache size 64.0m is >=
>  than blocking 64.0m size
>  2008-04-15 18:39:05,627 DEBUG org.apache.hadoop.hbase.HRegion: Started
>  memcache flush for region
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363. Size
>  64.0m
>  2008-04-15 18:39:09,544 DEBUG org.apache.hadoop.hbase.HStore: Added
>  1064767197/adjusted_q/3161297904753567399 with 0 entries, sequence id
>  64293357, and size 110.0 for 1064767197/adjusted_q
>  2008-04-15 18:39:15,489 DEBUG org.apache.hadoop.hbase.HStore: Added
>  1064767197/rule_id/5414884402918348270 with 285058 entries, sequence
>  id 64293357, and size 11.8m for 1064767197/rule_id
>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>  Unblocking updates for region
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>  Server handler 1 on 60020'
>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>  Unblocking updates for region
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>  Server handler 7 on 60020'
>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>  Unblocking updates for region
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>  Server handler 8 on 60020'
>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>  Unblocking updates for region
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>  Server handler 9 on 60020'
>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>  Unblocking updates for region
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>  Server handler 4 on 60020'
>  2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
>  Unblocking updates for region
>  category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
>  Server handler 3 on 60020'
>  2008-04-15 18:39:16,739 INFO org.apache.hadoop.hbase.HRegionServer:
>  Rolling hlog. Number of entries: 33252
>  2008-04-15 18:39:45,008 INFO org.apache.hadoop.hbase.HRegionServer:
>  MSG_REGION_OPEN : .META.,,1
>  2008-04-15 18:39:51,636 DEBUG org.apache.hadoop.hbase.HStore: Added
>  1064767197/category_id/5081092265153632091 with 285058 entries,
>  sequence id 64293357, and size 14.5m for 1064767197/category_id
>  2008-04-15 18:40:09,041 DEBUG org.apache.hadoop.hbase.HStore: Added
>  1064767197/success_count/2540574911286829228 with 285058 entries,
>  sequence id 64293357, and size 13.4m for 1064767197/success_count
>  2008-04-15 18:40:12,707 DEBUG org.apache.hadoop.hbase.HStore: Added
>  1064767197/adjusted_deviation/270946121541713279 with 0 entries,
>  sequence id 64293357, and size 110.0 for 1064767197/adjusted_deviation
>  2008-04-15 18:40:30,453 DEBUG org.apache.hadoop.hbase.HStore: Added
>  1064767197/hidden_variable/493176493179028867 with 285058 entries,
>  sequence id 64293357, and size 16.6m for 1064767197/hidden_variable
>  ---------------------------------------------------
>
>
>
>
>  On Tue, Apr 15, 2008 at 9:12 AM, stack <st...@duboce.net> wrote:
>  > You are running 0.1.1 hbase?  Please enable DEBUG logging, see http://wiki.apache.org/hadoop/Hbase/FAQ#5 for how, and send over a log.
>  > Thanks,
>  > St.Ack
>  >
>  >
>  >
>  >
>  > Daniel Leffel wrote:
>  >
>  > > Hi,
>  > > I'm new to HBase and investigating it for use in our large production
>  > > environment. Largely, I'm quite impressed by the possibilities.
>  > >
>  > > One bit of behavior I'm noticing is that region servers tend to quit
>  > > unexpectedly during moderately loaded map/reduce jobs without really any
>  > > helpful information in the logs as to why.
>  > >
>  > > Is the behavior expected? How else should I be troubleshooting?
>  > >
>  > > Thanks in advance!
>  > >
>  > > Danny Leffel
>  > >
>  > >
>  > >
>  >
>  >
>

Re: Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Posted by Daniel Leffel <da...@gmail.com>.
I was running 0.1.0 (hadn't noticed the new release). To rule out that
as the issue, I've since upgraded to 0.1.1. I am still seeing the same
behavior. Also, I'm running on top of Hadoop 0.16.1.

Interestingly, during the latest test, the master process died too.

Not sure what to be looking for in the DEBUG output, but here's a few
interesting entries.

Here's the master node when one of the region servers died:
-------------------------------------
2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.Leases:
HMaster.leaseChecker lease expired 1563117711/1563117711
2008-04-15 18:09:08,232 INFO org.apache.hadoop.hbase.HMaster:
10.252.39.164:60020 lease expired
2008-04-15 18:09:10,201 INFO org.apache.hadoop.hbase.HMaster:
assigning region -ROOT-,,0 to the only server 10.252.67.181:60020
2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
Received MSG_REPORT_PROCESS_OPEN : -ROOT-,,0 from 10.252.67.181:60020
2008-04-15 18:09:13,211 DEBUG org.apache.hadoop.hbase.HMaster:
Received MSG_REPORT_OPEN : -ROOT-,,0 from 10.252.67.181:60020
2008-04-15 18:09:13,211 INFO org.apache.hadoop.hbase.HMaster:
10.252.67.181:60020 serving -ROOT-,,0
2008-04-15 18:09:30,963 WARN org.apache.hadoop.hbase.HMaster: Scan ROOT region
java.net.SocketTimeoutException: timed out waiting for rpc response
        at org.apache.hadoop.ipc.Client.call(Client.java:514)
        at org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
        at $Proxy1.openScanner(Unknown Source)
        at org.apache.hadoop.hbase.HMaster$BaseScanner.scanRegion(HMaster.java:227)
        at org.apache.hadoop.hbase.HMaster$RootScanner.scanRoot(HMaster.java:540)
        at org.apache.hadoop.hbase.HMaster$RootScanner.maintenanceScan(HMaster.java:565)
        at org.apache.hadoop.hbase.HMaster$BaseScanner.chore(HMaster.java:207)
        at org.apache.hadoop.hbase.Chore.run(Chore.java:63)
2008-04-15 18:09:30,965 INFO org.apache.hadoop.hbase.HMaster:
HMaster.rootScanner scanning meta region {regionname: -ROOT-,,0,
startKey: <>, server: 10.252.67.181:60020}
2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster:
HMaster.rootScanner regioninfo: {regionname: .META.,,1, startKey: <>,
endKey: <>, encodedName: 1028785192, tableDesc: {name: .META.,
families: {info:={name: info, max versions: 1, compression: NONE, in
memory: false, max length: 2147483647, bloom filter: none}}}}, server:
10.252.67.181:60020, startCode: 1208278043501
2008-04-15 18:09:30,974 DEBUG org.apache.hadoop.hbase.HMaster: Current
assignment of .META.,,1 is not valid: storedInfo: address:
10.252.67.181:60020, startcode: 1208296518624, load: (requests: 0
regions: 12), startCode: 1208278043501, storedInfo.startCode:
1208296518624, unassignedRegions: false, pendingRegions: false
2008-04-15 18:09:30,975 INFO org.apache.hadoop.hbase.HMaster:
HMaster.rootScanner scan of meta region {regionname: -ROOT-,,0,
startKey: <>, server: 10.252.67.181:60020} complete
2008-04-15 18:09:31,271 INFO org.apache.hadoop.hbase.HMaster:
assigning region .META.,,1 to the only server 10.252.67.181:60020
2008-04-15 18:09:34,281 DEBUG org.apache.hadoop.hbase.HMaster:
Received MSG_REPORT_PROCESS_OPEN : .META.,,1 from 10.252.67.181:60020
2008-04-15 18:09:35,602 INFO org.apache.hadoop.hbase.HMaster:
HMaster.metaScanner scanning meta region {regionname: .META.,,1,
startKey: <>, server: 10.252.67.181:60020}
------------------------



Other curious entries from master when the regionserver died:
-------------------------------
2008-04-15 18:11:20,904 DEBUG org.apache.hadoop.hbase.HLog: Splitting
66 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.066
2008-04-15 18:11:21,756 DEBUG org.apache.hadoop.hbase.HLog: Applied
30734 total edits
2008-04-15 18:11:21,763 DEBUG org.apache.hadoop.hbase.HLog: Splitting
67 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.067
2008-04-15 18:11:23,608 DEBUG org.apache.hadoop.hbase.HLog: Applied
30060 total edits
2008-04-15 18:11:23,641 DEBUG org.apache.hadoop.hbase.HLog: Splitting
68 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.068
2008-04-15 18:11:24,421 DEBUG org.apache.hadoop.hbase.HLog: Applied
30008 total edits
2008-04-15 18:11:24,424 DEBUG org.apache.hadoop.hbase.HLog: Splitting
69 of 70: hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020/hlog.dat.069
2008-04-15 18:11:24,424 INFO org.apache.hadoop.hbase.HLog: Skipping
org.apache.hadoop.fs.FileStatus@593db81a because zero length
2008-04-15 18:11:25,563 INFO org.apache.hadoop.hbase.HLog: log file
splitting completed for
hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208296519099_60020
2008-04-15 18:11:25,563 DEBUG org.apache.hadoop.hbase.HMaster: process
server shutdown scanning root region on 10.252.67.181
2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster: process
server shutdown scanning root region on 10.252.67.181 finished HMaster
2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster:
numberOfMetaRegions: 1, onlineMetaRegions.size(): 1
2008-04-15 18:11:25,572 DEBUG org.apache.hadoop.hbase.HMaster: process
server shutdown scanning .META.,,1 on 10.252.67.181:60020 HMaster
attempt 0
2008-04-15 18:11:25,580 INFO org.apache.hadoop.hbase.HMaster:
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 was on
shutdown server <10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,583 INFO org.apache.hadoop.hbase.HMaster:
category_to_all_parents,6597500,1207947088224 was on shutdown server
<10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,585 INFO org.apache.hadoop.hbase.HMaster:
category_to_all_parents_map,11429436,1208214299908 was on shutdown
server <10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,587 INFO org.apache.hadoop.hbase.HMaster:
category_to_all_parents_map,13948388,1208214507790 was on shutdown
server <10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,588 INFO org.apache.hadoop.hbase.HMaster:
category_to_all_parents_map,2039541,1208212644065 was on shutdown
server <10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,591 INFO org.apache.hadoop.hbase.HMaster:
category_to_all_parents_map,4050172,1208212849307 was on shutdown
server <10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,594 INFO org.apache.hadoop.hbase.HMaster:
category_to_all_parents_map,8265783,1208211269830 was on shutdown
server <10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,595 INFO org.apache.hadoop.hbase.HMaster:
category_to_all_parents_map,8665350,1208211269830 was on shutdown
server <10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,596 INFO org.apache.hadoop.hbase.HMaster:
category_to_immediate_children_map,,1208288806024 was on shutdown
server <10.252.39.164:60020> (or server is null --
org.apache.hadoop.hbase.HRegionServer@d402dd --). Marking unassigned
in meta and clearing pendingRegions
2008-04-15 18:11:25,597 DEBUG org.apache.hadoop.hbase.HMaster: process
server shutdown finished scanning .META.,,1 on 10.252.67.181:60020
HMaster
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region category_to_all_parents_map,13948388,1208214507790 to
the only server 10.252.67.181:60020
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 to the
only server 10.252.67.181:60020
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region category_to_all_parents_map,4050172,1208212849307 to
the only server 10.252.67.181:60020
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region category_to_immediate_children_map,,1208288806024 to
the only server 10.252.67.181:60020
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region category_to_all_parents_map,2039541,1208212644065 to
the only server 10.252.67.181:60020
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region category_to_all_parents,6597500,1207947088224 to the
only server 10.252.67.181:60020
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region category_to_all_parents_map,8265783,1208211269830 to
the only server 10.252.67.181:60020
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region category_to_all_parents_map,11429436,1208214299908 to
the only server 10.252.67.181:60020
2008-04-15 18:11:26,372 INFO org.apache.hadoop.hbase.HMaster:
assigning region category_to_all_parents_map,8665350,1208211269830 to
the only server 10.252.67.181:60020
2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
Received MSG_REPORT_PROCESS_OPEN :
category_to_all_parents_map,13948388,1208214507790 from
10.252.67.181:60020
2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
Received MSG_REPORT_PROCESS_OPEN :
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 from
10.252.67.181:60020
2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
Received MSG_REPORT_PROCESS_OPEN :
category_to_all_parents_map,4050172,1208212849307 from
10.252.67.181:60020
2008-04-15 18:11:30,130 DEBUG org.apache.hadoop.hbase.HMaster:
Received MSG_REPORT_PROCESS_OPEN :
category_to_immediate_children_map,,1208288806024 from
10.252.67.181:60020
---------------------------------------------


The Exception that is thrown is almost either one of the following:
----------------------------------
org.apache.hadoop.hbase.NotServingRegionException:
org.apache.hadoop.hbase.NotServingRegionException:
category_rule_pricebin_statistics,,1208222885052
	at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1639)
	at org.apache.hadoop.hbase.HRegionServer.getRegion(HRegionServer.java:1611)
	at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1460)
	at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
	at java.lang.reflect.Method.invoke(Method.java:597)
	at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:899)

	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.hbase.RemoteExceptionHandler.decodeRemoteException(RemoteExceptionHandler.java:82)
	at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1028)
	at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
	at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
--------------------------------------

-- or ---

---------------------------------------
java.lang.RuntimeException: java.lang.reflect.UndeclaredThrowableException
	at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1050)
	at org.apache.hadoop.hbase.HTable.commit(HTable.java:763)
	at org.apache.hadoop.hbase.HTable.commit(HTable.java:744)
	at com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:93)
	at com.rexee.bandito.hadoop.categoryhierarcycreation.CreateMapOfCategoryToImmediateChildren$Reduce.reduce(CreateMapOfCategoryToImmediateChildren.java:1)
	at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.combineAndSpill(MapTask.java:522)
	at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.sortAndSpillToDisk(MapTask.java:493)
	at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:713)
	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:209)
	at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:2084)
Caused by: java.lang.reflect.UndeclaredThrowableException
	at $Proxy2.findRootRegion(Unknown Source)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRootRegion(HConnectionManager.java:706)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:336)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:313)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:409)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:346)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegionInMeta(HConnectionManager.java:482)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.locateRegion(HConnectionManager.java:350)
	at org.apache.hadoop.hbase.HConnectionManager$TableServers.relocateRegion(HConnectionManager.java:318)
	at org.apache.hadoop.hbase.HTable.getRegionLocation(HTable.java:114)
	at org.apache.hadoop.hbase.HTable$ServerCallable.instantiateServer(HTable.java:1009)
	at org.apache.hadoop.hbase.HTable.getRegionServerWithRetries(HTable.java:1024)
	... 9 more
Caused by: java.net.ConnectException: Connection refused
	at java.net.PlainSocketImpl.socketConnect(Native Method)
	at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
	at java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
	at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
	at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
	at java.net.Socket.connect(Socket.java:519)
	at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:161)
	at org.apache.hadoop.ipc.Client.getConnection(Client.java:578)
	at org.apache.hadoop.ipc.Client.call(Client.java:501)
	at org.apache.hadoop.hbase.ipc.HbaseRPC$Invoker.invoke(HbaseRPC.java:210)
	... 22 more
---------------------------



The region server does a lot of log rolling. Is this normal?
-----------------------------------------------
2008-04-15 18:24:22,577 WARN org.apache.hadoop.hbase.util.Sleeper: We
slept 37782ms, ten times longer than scheduled: 3000
2008-04-15 18:24:26,738 INFO org.apache.hadoop.hbase.HRegionServer:
Rolling hlog. Number of entries: 30006
2008-04-15 18:24:26,749 DEBUG org.apache.hadoop.hbase.HLog: Closing
current log writer
hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.053
to get a new one
2008-04-15 18:24:26,760 INFO org.apache.hadoop.hbase.HLog: new log
writer created at
hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
2008-04-15 18:24:33,073 INFO org.apache.hadoop.hbase.HRegionServer:
Rolling hlog. Number of entries: 30450
2008-04-15 18:24:33,409 DEBUG org.apache.hadoop.hbase.HLog: Closing
current log writer
hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.054
to get a new one
2008-04-15 18:24:33,419 INFO org.apache.hadoop.hbase.HLog: new log
writer created at
hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.055
-------------------------------------------------


After a lot of rolling, this happens:
------------------------------------------------
2008-04-15 18:38:59,716 INFO org.apache.hadoop.hbase.HLog: new log
writer created at
hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
2008-04-15 18:39:02,113 INFO org.apache.hadoop.hbase.HRegionServer:
Rolling hlog. Number of entries: 30360
2008-04-15 18:39:02,236 DEBUG org.apache.hadoop.hbase.HLog: Closing
current log writer
hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.159
to get a new one
2008-04-15 18:39:02,238 INFO org.apache.hadoop.hbase.HLog: new log
writer created at
hdfs://domU-12-31-38-00-8E-14:50001/hbase/log_10.252.39.164_1208297751167_60020/hlog.dat.160
2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
updates for 'IPC Server handler 1 on 60020': Memcache size 64.0m is >=
than blocking 64.0m size
2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
updates for 'IPC Server handler 7 on 60020': Memcache size 64.0m is >=
than blocking 64.0m size
2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
updates for 'IPC Server handler 8 on 60020': Memcache size 64.0m is >=
than blocking 64.0m size
2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
updates for 'IPC Server handler 9 on 60020': Memcache size 64.0m is >=
than blocking 64.0m size
2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
updates for 'IPC Server handler 4 on 60020': Memcache size 64.0m is >=
than blocking 64.0m size
2008-04-15 18:39:05,626 INFO org.apache.hadoop.hbase.HRegion: Blocking
updates for 'IPC Server handler 3 on 60020': Memcache size 64.0m is >=
than blocking 64.0m size
2008-04-15 18:39:05,627 DEBUG org.apache.hadoop.hbase.HRegion: Started
memcache flush for region
category_rule_pricebin_statistics,2332627_1_-11,1208293443363. Size
64.0m
2008-04-15 18:39:09,544 DEBUG org.apache.hadoop.hbase.HStore: Added
1064767197/adjusted_q/3161297904753567399 with 0 entries, sequence id
64293357, and size 110.0 for 1064767197/adjusted_q
2008-04-15 18:39:15,489 DEBUG org.apache.hadoop.hbase.HStore: Added
1064767197/rule_id/5414884402918348270 with 285058 entries, sequence
id 64293357, and size 11.8m for 1064767197/rule_id
2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
Unblocking updates for region
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
Server handler 1 on 60020'
2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
Unblocking updates for region
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
Server handler 7 on 60020'
2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
Unblocking updates for region
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
Server handler 8 on 60020'
2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
Unblocking updates for region
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
Server handler 9 on 60020'
2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
Unblocking updates for region
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
Server handler 4 on 60020'
2008-04-15 18:39:15,641 INFO org.apache.hadoop.hbase.HRegion:
Unblocking updates for region
category_rule_pricebin_statistics,2332627_1_-11,1208293443363 'IPC
Server handler 3 on 60020'
2008-04-15 18:39:16,739 INFO org.apache.hadoop.hbase.HRegionServer:
Rolling hlog. Number of entries: 33252
2008-04-15 18:39:45,008 INFO org.apache.hadoop.hbase.HRegionServer:
MSG_REGION_OPEN : .META.,,1
2008-04-15 18:39:51,636 DEBUG org.apache.hadoop.hbase.HStore: Added
1064767197/category_id/5081092265153632091 with 285058 entries,
sequence id 64293357, and size 14.5m for 1064767197/category_id
2008-04-15 18:40:09,041 DEBUG org.apache.hadoop.hbase.HStore: Added
1064767197/success_count/2540574911286829228 with 285058 entries,
sequence id 64293357, and size 13.4m for 1064767197/success_count
2008-04-15 18:40:12,707 DEBUG org.apache.hadoop.hbase.HStore: Added
1064767197/adjusted_deviation/270946121541713279 with 0 entries,
sequence id 64293357, and size 110.0 for 1064767197/adjusted_deviation
2008-04-15 18:40:30,453 DEBUG org.apache.hadoop.hbase.HStore: Added
1064767197/hidden_variable/493176493179028867 with 285058 entries,
sequence id 64293357, and size 16.6m for 1064767197/hidden_variable
---------------------------------------------------


On Tue, Apr 15, 2008 at 9:12 AM, stack <st...@duboce.net> wrote:
> You are running 0.1.1 hbase?  Please enable DEBUG logging, see http://wiki.apache.org/hadoop/Hbase/FAQ#5 for how, and send over a log.
> Thanks,
> St.Ack
>
>
>
>
> Daniel Leffel wrote:
>
> > Hi,
> > I'm new to HBase and investigating it for use in our large production
> > environment. Largely, I'm quite impressed by the possibilities.
> >
> > One bit of behavior I'm noticing is that region servers tend to quit
> > unexpectedly during moderately loaded map/reduce jobs without really any
> > helpful information in the logs as to why.
> >
> > Is the behavior expected? How else should I be troubleshooting?
> >
> > Thanks in advance!
> >
> > Danny Leffel
> >
> >
> >
>
>

Re: Hbase master

Posted by stack <st...@duboce.net>.
Yes (HBase Master is a lightly loaded server application).
St.Ack

Cosmin Lehene wrote:
> Hi, 
>
> Can the Hbase master be a Region Server as well (i.e. Be included in
> regionservers file)? In the docs it sais it¹s the Hadoop slaves file
> synonym.
>
> Thanks, 
> Cosmin
>
>
>   


Hbase master

Posted by Cosmin Lehene <cl...@adobe.com>.
Hi, 

Can the Hbase master be a Region Server as well (i.e. Be included in
regionservers file)? In the docs it sais it¹s the Hadoop slaves file
synonym.

Thanks, 
Cosmin


Re: Region Server Processes Exit Unexpectedly during Moderate Load MapReduce

Posted by stack <st...@duboce.net>.
You are running 0.1.1 hbase?  Please enable DEBUG logging, see 
http://wiki.apache.org/hadoop/Hbase/FAQ#5 for how, and send over a log.
Thanks,
St.Ack

Daniel Leffel wrote:
> Hi,
> I'm new to HBase and investigating it for use in our large production
> environment. Largely, I'm quite impressed by the possibilities.
>
> One bit of behavior I'm noticing is that region servers tend to quit
> unexpectedly during moderately loaded map/reduce jobs without really any
> helpful information in the logs as to why.
>
> Is the behavior expected? How else should I be troubleshooting?
>
> Thanks in advance!
>
> Danny Leffel
>
>