You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by schubert zhang <zs...@gmail.com> on 2009/03/21 07:31:33 UTC

Fwd: RegionServer failure and recovery take a long time

Hi Jean Daniel,

I want your help for this issue. I attach the log files, please help analyse
it. Thanks.

Test env.
  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info', VERSIONS => 1,
COMPRESSION => 'BLOCK', BLOCKCACHE => true}

All test data is generated randomly by a program.

HRegionServer Failure (2009-03-21 08:27:41,090):
  After about 8 hours running, my node-5 rangeserver failure and the
HRegionServer Shutdown  .
  It seems caused by DFSClient exceptions. (I cannot make clear what
happened on HDFS, but it seems the HDFS is ok.)

Then I start HRegionServer at this node-5. (2009-03-21 10:53:42,747):
  After the HRegionServer started, regions were reassign. I can see the
reassign on WebGUI of HBase, since some regions are now on this node.
  But following things are blocked for a long time:
  (1) The HBase client application cannot insert data for a long time (until
2009/03/21 11:11:27, its about 18 minutes).  It is RetriesExhaustedException
exception on application side (MapReduce Job).
  (2) Some regions cannot be accessed (I cannot scan/get rows in these
regions.) The exception is NotServingRegionException when getRegion.
  (3) I check the history of the region of (2) from the WebGUI. I can see in
the history, it is assigned at 11:04:15. It is so later.
The history is:
at, 21 Mar 2009 11:10:39openRegion opened on server : nd1-rack0-cloud Sat,
21 Mar 2009 11:04:15assignmentRegion assigned to server
10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on server :
nd1-rack0-cloud Sat,
21 Mar 2009 06:47:57assignmentRegion assigned to server
10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on server :
nd5-rack0-cloud Sat,
21 Mar 2009 06:27:21assignmentRegion assigned to server
10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on server :
nd5-rack0-cloud Sat,
21 Mar 2009 06:24:53assignmentRegion assigned to server
10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on server :
nd3-rack0-cloud Sat,
21 Mar 2009 06:24:13assignmentRegion assigned to server
10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on server :
nd4-rack0-cloud Sat,
21 Mar 2009 06:19:02assignmentRegion assigned to server
10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on server :
nd5-rack0-cloud Sat,
21 Mar 2009 05:59:36assignmentRegion assigned to server
10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on server :
nd3-rack0-cloud Sat,
21 Mar 2009 03:50:12assignmentRegion assigned to server
10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split from:
TESTA,13576334163@2009-03-21
00:35:57.526,1237569164012<http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012>


And following is exception when I scan a rowkey range.

org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to contact
region server 10.24.1.12:60020 for region
TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
'13576334163@2009-03-2100:35:57.526', but failed after 5 attempts.
Exceptions:
org.apache.hadoop.hbase.NotServingRegionException:
org.apache.hadoop.hbase.NotServingRegionException:
TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
        at
org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown Source)
        at
org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
Source)
        at sun.reflect.GeneratedMethodAccessor6.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(Unknown Source)
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
Source)

org.apache.hadoop.hbase.NotServingRegionException:
org.apache.hadoop.hbase.NotServingRegionException:
TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
        at
org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown Source)
        at
org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
Source)
        at sun.reflect.GeneratedMethodAccessor6.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(Unknown Source)
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
Source)

I will send the log files to you email address.

Re: RegionServer failure and recovery take a long time

Posted by Jean-Daniel Cryans <jd...@apache.org>.
Glad to hear that!

J-D

On Sun, Mar 22, 2009 at 10:32 AM, schubert zhang <zs...@gmail.com> wrote:
> Hi Jean-Daniel,
> It is delightful.
> Now, my 6-nodes cluster has been running for 12.5 hours with heavy-write,
> without obvious problem.
> I have applied your patch. And also add the gc options to hadoop.
>
> Thank you for your help.
> Schubert
>
> On Sun, Mar 22, 2009 at 8:50 AM, schubert zhang <zs...@gmail.com> wrote:
>
>> Jean Daniel,
>> I have done following:
>> (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the GC
>> detailedly.
>> (2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
>> want change to 200MB.
>> (3)  increase the lease period of master to 180 sec.
>>
>> But I have not applied the patch yet.
>>
>> But there are still following problems:
>>
>> (1) After about one hour, one region server down by:
>>
>> 009-03-22 03:36:06,858 INFO org.apache.hadoop.hbase.regionserver.HRegion:
>> Blocking updates for 'IPC Server handler 3 on 60020' on region
>> CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size
>> 128.0m is >= than blocking 128.0m size
>> 2009-03-22 03:36:07,114 INFO org.apache.hadoop.hbase.regionserver.HRegion:
>> Blocking updates for 'IPC Server handler 7 on 60020' on region
>> CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size
>> 128.0m is >= than blocking 128.0m size
>> 2009-03-22 03:36:08,384 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer
>> Exception: java.io.IOException: Unable to create new block.
>>         at
>> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
>> Source)
>>         at
>> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown Source)
>>         at
>> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
>> Source)
>>
>> 2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Error
>> Recovery for block blk_2520317265353988797_3939 bad datanode[0] nodes ==
>> null
>> 2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Could not
>> get block locations. Source file
>> "/hbase/CDR/81441797/wap/mapfiles/2481397829486034048/data" - Aborting...
>> 2009-03-22 03:36:08,419 FATAL
>> org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Replay of hlog
>> required. Forcing server shutdown
>> org.apache.hadoop.hbase.DroppedSnapshotException: region:
>> CDR,13876322158@2009-03-22 03:08:29.315,1237664116065
>>         at
>> org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(Unknown
>> Source)
>>         at org.apache.hadoop.hbase.regionserver.HRegion.flushcache(Unknown
>> Source)
>>         at
>> org.apache.hadoop.hbase.regionserver.MemcacheFlusher.flushRegion(Unknown
>> Source)
>>         at org.apache.hadoop.hbase.regionserver.MemcacheFlusher.run(Unknown
>> Source)
>> Caused by: java.io.IOException: Could not read from stream
>>         at org.apache.hadoop.net.SocketInputStream.read(Unknown Source)
>>         at java.io.DataInputStream.readByte(DataInputStream.java:248)
>>         at org.apache.hadoop.io.WritableUtils.readVLong(Unknown Source)
>>         at org.apache.hadoop.io.WritableUtils.readVInt(Unknown Source)
>>         at org.apache.hadoop.io.Text.readString(Unknown Source)
>>         at
>> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(Unknown
>> Source)
>>         at
>> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
>> Source)
>>         at
>> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown Source)
>>         at
>> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
>> Source)
>> 2009-03-22 03:36:08,422 INFO
>> org.apache.hadoop.hbase.regionserver.HRegionServer: Dump of metrics:
>> request=0.0, regions=2, stores=2, storefiles=216, storefileIndexSize=0,
>> memcacheSize=255, usedHeap=475, maxHeap=2991
>> 2009-03-22 03:36:08,422 INFO
>> org.apache.hadoop.hbase.regionserver.MemcacheFlusher:
>> regionserver/0:0:0:0:0:0:0:0:60020.cacheFlusher exiting
>>
>> (2) After about 6 hours, another region server down the old known issue:
>>
>> 2009-03-22 08:01:20,373 WARN org.apache.hadoop.hbase.util.Sleeper: We slept
>> 2048090ms, ten times longer than scheduled: 10000
>> 2009-03-22 08:01:20,404 WARN org.apache.hadoop.hbase.util.Sleeper: We slept
>> 2041927ms, ten times longer than scheduled: 3000
>> 2009-03-22 08:01:20,960 WARN
>> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report to
>> master for 2042572 milliseconds - retrying
>> 2009-03-22 08:01:22,469 INFO org.apache.hadoop.hdfs.DFSClient:
>> org.apache.hadoop.ipc.RemoteException:
>> org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException: No lease on
>> /hbase/log_10.24.1.20_1237676058534_60020/hlog.dat.1237678010758 File does
>> not exist. Holder DFSClient_315506831 does not have any open files.
>>         at
>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
>> Source)
>>         at
>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
>> Source)
>>         at
>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(Unknown
>> Source)
>>         at org.apache.hadoop.hdfs.server.namenode.NameNode.addBlock(Unknown
>> Source)
>>         at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
>>         at
>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>         at java.lang.reflect.Method.invoke(Method.java:597)
>>         at org.apache.hadoop.ipc.RPC$Server.call(Unknown Source)
>>         at org.apache.hadoop.ipc.Server$Handler.run(Unknown Source)
>>
>>         at org.apache.hadoop.ipc.Client.call(Unknown Source)
>>
>> 2042572 ms is really too long.
>>
>> Schubert
>>
>>
>> On Sun, Mar 22, 2009 at 3:01 AM, schubert zhang <zs...@gmail.com> wrote:
>>
>>> Yes, I missed " ". Thank you.
>>>
>>>
>>> On Sun, Mar 22, 2009 at 2:17 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:
>>>
>>>> Put the options between " "
>>>>
>>>> J-D
>>>>
>>>> On Sat, Mar 21, 2009 at 2:15 PM, schubert zhang <zs...@gmail.com>
>>>> wrote:
>>>> > It's strange when I add -XX:+UseConcMarkSweepGC
>>>> > -XX:+CMSIncrementalMode./hbase/bin/../conf/hbase-env.sh:
>>>> > line 37: export: `-XX:+CMSIncrementalMode': not a valid identifer
>>>> >
>>>> > My jdk version is jdk-6u6-linux-x64, I will try the
>>>> > latest jdk-6u12-linux-x64 now.
>>>> >
>>>> > Schubert
>>>> >
>>>> > On Sun, Mar 22, 2009 at 1:40 AM, Jean-Daniel Cryans <
>>>> jdcryans@apache.org>wrote:
>>>> >
>>>> >> Schubert,
>>>> >>
>>>> >> It's based on the 0.19 branch in svn but it should patch with no
>>>> >> problem. In this state the patch is still just a test I'm doing but,
>>>> >> unless you write to thousands of regions at the same time when the
>>>> >> region server fails, there should be no problem. If it does cause you
>>>> >> trouble, please leave a comment in the jira. As you can see, it was
>>>> >> able to process a huge amount of logs without any problem. Also this
>>>> >> process is only done on the master which never receives any load so
>>>> >> it's even safer.
>>>> >>
>>>> >> J-D
>>>> >>
>>>> >> On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com>
>>>> wrote:
>>>> >> > Jean Daniel,
>>>> >> > Thanks for you kindness.
>>>> >> > Yes, I want more machines, and we will get them soon. :-)
>>>> >> > My application is write-heavy very much. Since my cluster is really
>>>> >> small, I
>>>> >> > will slow down the inserts now.
>>>> >> >
>>>> >> > One more questions about you patch HBASE-1008: It is really helpful
>>>> for
>>>> >> me.
>>>> >> > Does this patch take more memory? It seems not based on 0.19.1. Can
>>>> it be
>>>> >> > applied on 0.19.1?
>>>> >> >
>>>> >> > Schubert
>>>> >> >
>>>> >> > On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <
>>>> >> jdcryans@apache.org>wrote:
>>>> >> >
>>>> >> >> Schubert,
>>>> >> >>
>>>> >> >> I have no problem at all with your english since my first language
>>>> is
>>>> >> >> french and I must doing loads of grammatical errors too ;)
>>>> >> >>
>>>> >> >> Regards the heap, make sure that 300MB fits your need in mem or you
>>>> >> might
>>>> >> >> OOME.
>>>> >> >>
>>>> >> >> Increasing the lease period is a good idea, I have done the same.
>>>> Our
>>>> >> >> jobs take 13 hours so it avoids many restarts.
>>>> >> >>
>>>> >> >> Swappinnes at 0 => no swap at all... so if your system needs to
>>>> swap
>>>> >> >> you might be in trouble. The advantage I see in a very low
>>>> swappiness
>>>> >> >> value (but not 0) is that it will only swap if ultimately
>>>> necessary.
>>>> >> >>
>>>> >> >> On a final note, using the blocking caching feature is a bit of
>>>> risk
>>>> >> >> in versions < 0.20. It does make random reads a lot faster (most of
>>>> >> >> the time) but the eviction of blocks produces a lot of garbage. The
>>>> >> >> guys from Stream.com are implementing something better at this very
>>>> >> >> moment.
>>>> >> >>
>>>> >> >> You may also want more machines :P. 6 is a very small number, we
>>>> >> >> usually see a lot more stability passed 10. Or instead you might
>>>> want
>>>> >> >> to slow down the inserts... It's good to be realist regards what
>>>> >> >> stress you put on the cluster VS the actual resources.
>>>> >> >>
>>>> >> >> J-D
>>>> >> >>
>>>> >> >> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <
>>>> zsongbo@gmail.com>
>>>> >> >> wrote:
>>>> >> >> > Hi Jean Daniel,
>>>> >> >> > Your help is so great. Thank you very much.
>>>> >> >> >
>>>> >> >> > After reading of the HBase Troubleshooting:
>>>> >> >> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also
>>>> doubt
>>>> >> >> > about garbage collector and have added the
>>>> -XX:+UseConcMarkSweepGC
>>>> >> option
>>>> >> >> 4
>>>> >> >> > hours ago. I checked the regionserves just now, one was shutdown
>>>> as
>>>> >> the
>>>> >> >> same
>>>> >> >> > cause. But its better than before.
>>>> >> >> >
>>>> >> >> > Now, I will do following turning according your guide:
>>>> >> >> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and
>>>> study
>>>> >> the
>>>> >> >> GC
>>>> >> >> > detailedly.
>>>> >> >> > (2)  decrease the heap size of mapreduce child, now I am using
>>>> 1024MB.
>>>> >> I
>>>> >> >> > want change to 300MB.
>>>> >> >> > (3)  increase the lease period of master to 180 sec.
>>>> >> >> > (4)  apply the you great patch.
>>>> >> >> >
>>>> >> >> > By the way, to avoid swap, I had changed the vm.swappiness = 0
>>>> now
>>>> >> (you
>>>> >> >> had
>>>> >> >> > tell me 20 in another email), do think it is ok?
>>>> >> >> >
>>>> >> >> > Thank you again. My english is not good, please bear with me.
>>>> >> >> >
>>>> >> >> > Schubert
>>>> >> >> >
>>>> >> >> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
>>>> >> >> jdcryans@apache.org>wrote:
>>>> >> >> >
>>>> >> >> >> Schubert,
>>>> >> >> >>
>>>> >> >> >> Yeah that's the good old problem with the garbage collector. In
>>>> your
>>>> >> >> >> logs I see a lot of :
>>>> >> >> >>
>>>> >> >> >> 2009-03-21 05:59:06,498 WARN
>>>> org.apache.hadoop.hbase.util.Sleeper: We
>>>> >> >> >> slept 144233ms, ten times longer than scheduled: 3000
>>>> >> >> >> 2009-03-21 05:59:06,600 WARN
>>>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to
>>>> report
>>>> >> >> >> to master for 144335 milliseconds - retrying
>>>> >> >> >> 2009-03-21 05:59:06,512 WARN
>>>> org.apache.hadoop.hbase.util.Sleeper: We
>>>> >> >> >> slept 143279ms, ten times longer than scheduled: 10000
>>>> >> >> >> 2009-03-21 05:59:06,701 INFO
>>>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
>>>> >> >> >> MSG_CALL_SERVER_STARTUP: safeMode=false
>>>> >> >> >>
>>>> >> >> >> That usually means that the garbage collector blocked all
>>>> threads to
>>>> >> >> >> do it's stuff. But, when it happens, it takes more time than the
>>>> >> lease
>>>> >> >> >> the master maintains on the region servers (120 sec) so the
>>>> master
>>>> >> >> >> considers this region server as dead. Then the log splitting
>>>> takes
>>>> >> >> >> over on the master which is a very very long process. During
>>>> that
>>>> >> >> >> time, sometimes more than 10 minutes, the regions from that
>>>> region
>>>> >> >> >> server are unavailable. If the cluster is small, that make
>>>> things
>>>> >> even
>>>> >> >> >> far worse.
>>>> >> >> >>
>>>> >> >> >> We had these kinds of error on our cluster during the last weeks
>>>> and
>>>> >> >> >> here is how I solved it:
>>>> >> >> >>
>>>> >> >> >> - Regards the log splitting, I suggest you take a look at this
>>>> issue
>>>> >> >> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a
>>>> patch I
>>>> >> >> >> made to speed up the process. See if it helps you.
>>>> >> >> >>
>>>> >> >> >> - Regards the garbage collector, I found that the options
>>>> >> >> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really
>>>> really
>>>> >> >> >> helpful. See
>>>> >> >> >>
>>>> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
>>>> >> >> >> for more information. Set this in the hbase-env.sh file on the
>>>> line
>>>> >> >> >> export HBASE_OPTS=...
>>>> >> >> >>
>>>> >> >> >> - Finally, to make sure that the garbage collection is fast,
>>>> check if
>>>> >> >> >> there is swap. If so, set lower heaps for the MR child processes
>>>> in
>>>> >> >> >> hadoop-site.xml (the mapred.child.java.opts property).
>>>> >> >> >>
>>>> >> >> >> J-D
>>>> >> >> >>
>>>> >> >> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <
>>>> zsongbo@gmail.com>
>>>> >> >> wrote:
>>>> >> >> >> > Hi Jean Daniel,
>>>> >> >> >> >
>>>> >> >> >> > I want your help for this issue. I attach the log files,
>>>> please
>>>> >> help
>>>> >> >> >> analyse
>>>> >> >> >> > it. Thanks.
>>>> >> >> >> >
>>>> >> >> >> > Test env.
>>>> >> >> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info',
>>>> >> VERSIONS
>>>> >> >> =>
>>>> >> >> >> 1,
>>>> >> >> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
>>>> >> >> >> >
>>>> >> >> >> > All test data is generated randomly by a program.
>>>> >> >> >> >
>>>> >> >> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
>>>> >> >> >> >  After about 8 hours running, my node-5 rangeserver failure
>>>> and the
>>>> >> >> >> > HRegionServer Shutdown  .
>>>> >> >> >> >  It seems caused by DFSClient exceptions. (I cannot make clear
>>>> what
>>>> >> >> >> > happened on HDFS, but it seems the HDFS is ok.)
>>>> >> >> >> >
>>>> >> >> >> > Then I start HRegionServer at this node-5. (2009-03-21
>>>> >> 10:53:42,747):
>>>> >> >> >> >  After the HRegionServer started, regions were reassign. I can
>>>> see
>>>> >> the
>>>> >> >> >> > reassign on WebGUI of HBase, since some regions are now on
>>>> this
>>>> >> node.
>>>> >> >> >> >  But following things are blocked for a long time:
>>>> >> >> >> >  (1) The HBase client application cannot insert data for a
>>>> long
>>>> >> time
>>>> >> >> >> (until
>>>> >> >> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
>>>> >> >> >> RetriesExhaustedException
>>>> >> >> >> > exception on application side (MapReduce Job).
>>>> >> >> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows
>>>> in
>>>> >> these
>>>> >> >> >> > regions.) The exception is NotServingRegionException when
>>>> >> getRegion.
>>>> >> >> >> >  (3) I check the history of the region of (2) from the WebGUI.
>>>> I
>>>> >> can
>>>> >> >> see
>>>> >> >> >> in
>>>> >> >> >> > the history, it is assigned at 11:04:15. It is so later.
>>>> >> >> >> > The history is:
>>>> >> >> >> > at, 21 Mar 2009 11:10:39openRegion opened on server :
>>>> >> nd1-rack0-cloud
>>>> >> >> >> Sat,
>>>> >> >> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
>>>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on
>>>> >> server
>>>> >> >> :
>>>> >> >> >> > nd1-rack0-cloud Sat,
>>>> >> >> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
>>>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on
>>>> >> server
>>>> >> >> :
>>>> >> >> >> > nd5-rack0-cloud Sat,
>>>> >> >> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
>>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on
>>>> >> server
>>>> >> >> :
>>>> >> >> >> > nd5-rack0-cloud Sat,
>>>> >> >> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
>>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on
>>>> >> server
>>>> >> >> :
>>>> >> >> >> > nd3-rack0-cloud Sat,
>>>> >> >> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
>>>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on
>>>> >> server
>>>> >> >> :
>>>> >> >> >> > nd4-rack0-cloud Sat,
>>>> >> >> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
>>>> >> >> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on
>>>> >> server
>>>> >> >> :
>>>> >> >> >> > nd5-rack0-cloud Sat,
>>>> >> >> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
>>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on
>>>> >> server
>>>> >> >> :
>>>> >> >> >> > nd3-rack0-cloud Sat,
>>>> >> >> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
>>>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split
>>>> from:
>>>> >> >> >> > TESTA,13576334163@2009-03-21
>>>> >> >> >> > 00:35:57.526,1237569164012<
>>>> >> >> >>
>>>> >> >>
>>>> >>
>>>> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
>>>> >> >> >> >
>>>> >> >> >> >
>>>> >> >> >> >
>>>> >> >> >> > And following is exception when I scan a rowkey range.
>>>> >> >> >> >
>>>> >> >> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException:
>>>> Trying to
>>>> >> >> >> contact
>>>> >> >> >> > region server 10.24.1.12:60020 for region
>>>> >> >> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
>>>> >> >> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5
>>>> attempts.
>>>> >> >> >> > Exceptions:
>>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>>>> >> >> >> >        at
>>>> >> >> >> >
>>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>>>> >> >> >> Source)
>>>> >> >> >> >        at
>>>> >> >> >> >
>>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>>>> >> >> >> > Source)
>>>> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>>>> >> >> >> Source)
>>>> >> >> >> >        at
>>>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>>>> >> >> >> > Source)
>>>> >> >> >> >
>>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>>>> >> >> >> >        at
>>>> >> >> >> >
>>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>>>> >> >> >> Source)
>>>> >> >> >> >        at
>>>> >> >> >> >
>>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>>>> >> >> >> > Source)
>>>> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>>>> >> >> >> Source)
>>>> >> >> >> >        at
>>>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>>>> >> >> >> > Source)
>>>> >> >> >> >
>>>> >> >> >> > I will send the log files to you email address.
>>>> >> >> >> >
>>>> >> >> >>
>>>> >> >> >
>>>> >> >>
>>>> >> >
>>>> >>
>>>> >
>>>>
>>>
>>>
>>
>

Re: RegionServer failure and recovery take a long time

Posted by schubert zhang <zs...@gmail.com>.
Hi Jean-Daniel,
It is delightful.
Now, my 6-nodes cluster has been running for 12.5 hours with heavy-write,
without obvious problem.
I have applied your patch. And also add the gc options to hadoop.

Thank you for your help.
Schubert

On Sun, Mar 22, 2009 at 8:50 AM, schubert zhang <zs...@gmail.com> wrote:

> Jean Daniel,
> I have done following:
> (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the GC
> detailedly.
> (2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
> want change to 200MB.
> (3)  increase the lease period of master to 180 sec.
>
> But I have not applied the patch yet.
>
> But there are still following problems:
>
> (1) After about one hour, one region server down by:
>
> 009-03-22 03:36:06,858 INFO org.apache.hadoop.hbase.regionserver.HRegion:
> Blocking updates for 'IPC Server handler 3 on 60020' on region
> CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size
> 128.0m is >= than blocking 128.0m size
> 2009-03-22 03:36:07,114 INFO org.apache.hadoop.hbase.regionserver.HRegion:
> Blocking updates for 'IPC Server handler 7 on 60020' on region
> CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size
> 128.0m is >= than blocking 128.0m size
> 2009-03-22 03:36:08,384 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer
> Exception: java.io.IOException: Unable to create new block.
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
> Source)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown Source)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
> Source)
>
> 2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Error
> Recovery for block blk_2520317265353988797_3939 bad datanode[0] nodes ==
> null
> 2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Could not
> get block locations. Source file
> "/hbase/CDR/81441797/wap/mapfiles/2481397829486034048/data" - Aborting...
> 2009-03-22 03:36:08,419 FATAL
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Replay of hlog
> required. Forcing server shutdown
> org.apache.hadoop.hbase.DroppedSnapshotException: region:
> CDR,13876322158@2009-03-22 03:08:29.315,1237664116065
>         at
> org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(Unknown
> Source)
>         at org.apache.hadoop.hbase.regionserver.HRegion.flushcache(Unknown
> Source)
>         at
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher.flushRegion(Unknown
> Source)
>         at org.apache.hadoop.hbase.regionserver.MemcacheFlusher.run(Unknown
> Source)
> Caused by: java.io.IOException: Could not read from stream
>         at org.apache.hadoop.net.SocketInputStream.read(Unknown Source)
>         at java.io.DataInputStream.readByte(DataInputStream.java:248)
>         at org.apache.hadoop.io.WritableUtils.readVLong(Unknown Source)
>         at org.apache.hadoop.io.WritableUtils.readVInt(Unknown Source)
>         at org.apache.hadoop.io.Text.readString(Unknown Source)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(Unknown
> Source)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
> Source)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown Source)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
> Source)
> 2009-03-22 03:36:08,422 INFO
> org.apache.hadoop.hbase.regionserver.HRegionServer: Dump of metrics:
> request=0.0, regions=2, stores=2, storefiles=216, storefileIndexSize=0,
> memcacheSize=255, usedHeap=475, maxHeap=2991
> 2009-03-22 03:36:08,422 INFO
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher:
> regionserver/0:0:0:0:0:0:0:0:60020.cacheFlusher exiting
>
> (2) After about 6 hours, another region server down the old known issue:
>
> 2009-03-22 08:01:20,373 WARN org.apache.hadoop.hbase.util.Sleeper: We slept
> 2048090ms, ten times longer than scheduled: 10000
> 2009-03-22 08:01:20,404 WARN org.apache.hadoop.hbase.util.Sleeper: We slept
> 2041927ms, ten times longer than scheduled: 3000
> 2009-03-22 08:01:20,960 WARN
> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report to
> master for 2042572 milliseconds - retrying
> 2009-03-22 08:01:22,469 INFO org.apache.hadoop.hdfs.DFSClient:
> org.apache.hadoop.ipc.RemoteException:
> org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException: No lease on
> /hbase/log_10.24.1.20_1237676058534_60020/hlog.dat.1237678010758 File does
> not exist. Holder DFSClient_315506831 does not have any open files.
>         at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
> Source)
>         at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
> Source)
>         at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(Unknown
> Source)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.addBlock(Unknown
> Source)
>         at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
>         at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(Unknown Source)
>         at org.apache.hadoop.ipc.Server$Handler.run(Unknown Source)
>
>         at org.apache.hadoop.ipc.Client.call(Unknown Source)
>
> 2042572 ms is really too long.
>
> Schubert
>
>
> On Sun, Mar 22, 2009 at 3:01 AM, schubert zhang <zs...@gmail.com> wrote:
>
>> Yes, I missed " ". Thank you.
>>
>>
>> On Sun, Mar 22, 2009 at 2:17 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:
>>
>>> Put the options between " "
>>>
>>> J-D
>>>
>>> On Sat, Mar 21, 2009 at 2:15 PM, schubert zhang <zs...@gmail.com>
>>> wrote:
>>> > It's strange when I add -XX:+UseConcMarkSweepGC
>>> > -XX:+CMSIncrementalMode./hbase/bin/../conf/hbase-env.sh:
>>> > line 37: export: `-XX:+CMSIncrementalMode': not a valid identifer
>>> >
>>> > My jdk version is jdk-6u6-linux-x64, I will try the
>>> > latest jdk-6u12-linux-x64 now.
>>> >
>>> > Schubert
>>> >
>>> > On Sun, Mar 22, 2009 at 1:40 AM, Jean-Daniel Cryans <
>>> jdcryans@apache.org>wrote:
>>> >
>>> >> Schubert,
>>> >>
>>> >> It's based on the 0.19 branch in svn but it should patch with no
>>> >> problem. In this state the patch is still just a test I'm doing but,
>>> >> unless you write to thousands of regions at the same time when the
>>> >> region server fails, there should be no problem. If it does cause you
>>> >> trouble, please leave a comment in the jira. As you can see, it was
>>> >> able to process a huge amount of logs without any problem. Also this
>>> >> process is only done on the master which never receives any load so
>>> >> it's even safer.
>>> >>
>>> >> J-D
>>> >>
>>> >> On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com>
>>> wrote:
>>> >> > Jean Daniel,
>>> >> > Thanks for you kindness.
>>> >> > Yes, I want more machines, and we will get them soon. :-)
>>> >> > My application is write-heavy very much. Since my cluster is really
>>> >> small, I
>>> >> > will slow down the inserts now.
>>> >> >
>>> >> > One more questions about you patch HBASE-1008: It is really helpful
>>> for
>>> >> me.
>>> >> > Does this patch take more memory? It seems not based on 0.19.1. Can
>>> it be
>>> >> > applied on 0.19.1?
>>> >> >
>>> >> > Schubert
>>> >> >
>>> >> > On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <
>>> >> jdcryans@apache.org>wrote:
>>> >> >
>>> >> >> Schubert,
>>> >> >>
>>> >> >> I have no problem at all with your english since my first language
>>> is
>>> >> >> french and I must doing loads of grammatical errors too ;)
>>> >> >>
>>> >> >> Regards the heap, make sure that 300MB fits your need in mem or you
>>> >> might
>>> >> >> OOME.
>>> >> >>
>>> >> >> Increasing the lease period is a good idea, I have done the same.
>>> Our
>>> >> >> jobs take 13 hours so it avoids many restarts.
>>> >> >>
>>> >> >> Swappinnes at 0 => no swap at all... so if your system needs to
>>> swap
>>> >> >> you might be in trouble. The advantage I see in a very low
>>> swappiness
>>> >> >> value (but not 0) is that it will only swap if ultimately
>>> necessary.
>>> >> >>
>>> >> >> On a final note, using the blocking caching feature is a bit of
>>> risk
>>> >> >> in versions < 0.20. It does make random reads a lot faster (most of
>>> >> >> the time) but the eviction of blocks produces a lot of garbage. The
>>> >> >> guys from Stream.com are implementing something better at this very
>>> >> >> moment.
>>> >> >>
>>> >> >> You may also want more machines :P. 6 is a very small number, we
>>> >> >> usually see a lot more stability passed 10. Or instead you might
>>> want
>>> >> >> to slow down the inserts... It's good to be realist regards what
>>> >> >> stress you put on the cluster VS the actual resources.
>>> >> >>
>>> >> >> J-D
>>> >> >>
>>> >> >> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <
>>> zsongbo@gmail.com>
>>> >> >> wrote:
>>> >> >> > Hi Jean Daniel,
>>> >> >> > Your help is so great. Thank you very much.
>>> >> >> >
>>> >> >> > After reading of the HBase Troubleshooting:
>>> >> >> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also
>>> doubt
>>> >> >> > about garbage collector and have added the
>>> -XX:+UseConcMarkSweepGC
>>> >> option
>>> >> >> 4
>>> >> >> > hours ago. I checked the regionserves just now, one was shutdown
>>> as
>>> >> the
>>> >> >> same
>>> >> >> > cause. But its better than before.
>>> >> >> >
>>> >> >> > Now, I will do following turning according your guide:
>>> >> >> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and
>>> study
>>> >> the
>>> >> >> GC
>>> >> >> > detailedly.
>>> >> >> > (2)  decrease the heap size of mapreduce child, now I am using
>>> 1024MB.
>>> >> I
>>> >> >> > want change to 300MB.
>>> >> >> > (3)  increase the lease period of master to 180 sec.
>>> >> >> > (4)  apply the you great patch.
>>> >> >> >
>>> >> >> > By the way, to avoid swap, I had changed the vm.swappiness = 0
>>> now
>>> >> (you
>>> >> >> had
>>> >> >> > tell me 20 in another email), do think it is ok?
>>> >> >> >
>>> >> >> > Thank you again. My english is not good, please bear with me.
>>> >> >> >
>>> >> >> > Schubert
>>> >> >> >
>>> >> >> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
>>> >> >> jdcryans@apache.org>wrote:
>>> >> >> >
>>> >> >> >> Schubert,
>>> >> >> >>
>>> >> >> >> Yeah that's the good old problem with the garbage collector. In
>>> your
>>> >> >> >> logs I see a lot of :
>>> >> >> >>
>>> >> >> >> 2009-03-21 05:59:06,498 WARN
>>> org.apache.hadoop.hbase.util.Sleeper: We
>>> >> >> >> slept 144233ms, ten times longer than scheduled: 3000
>>> >> >> >> 2009-03-21 05:59:06,600 WARN
>>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to
>>> report
>>> >> >> >> to master for 144335 milliseconds - retrying
>>> >> >> >> 2009-03-21 05:59:06,512 WARN
>>> org.apache.hadoop.hbase.util.Sleeper: We
>>> >> >> >> slept 143279ms, ten times longer than scheduled: 10000
>>> >> >> >> 2009-03-21 05:59:06,701 INFO
>>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
>>> >> >> >> MSG_CALL_SERVER_STARTUP: safeMode=false
>>> >> >> >>
>>> >> >> >> That usually means that the garbage collector blocked all
>>> threads to
>>> >> >> >> do it's stuff. But, when it happens, it takes more time than the
>>> >> lease
>>> >> >> >> the master maintains on the region servers (120 sec) so the
>>> master
>>> >> >> >> considers this region server as dead. Then the log splitting
>>> takes
>>> >> >> >> over on the master which is a very very long process. During
>>> that
>>> >> >> >> time, sometimes more than 10 minutes, the regions from that
>>> region
>>> >> >> >> server are unavailable. If the cluster is small, that make
>>> things
>>> >> even
>>> >> >> >> far worse.
>>> >> >> >>
>>> >> >> >> We had these kinds of error on our cluster during the last weeks
>>> and
>>> >> >> >> here is how I solved it:
>>> >> >> >>
>>> >> >> >> - Regards the log splitting, I suggest you take a look at this
>>> issue
>>> >> >> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a
>>> patch I
>>> >> >> >> made to speed up the process. See if it helps you.
>>> >> >> >>
>>> >> >> >> - Regards the garbage collector, I found that the options
>>> >> >> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really
>>> really
>>> >> >> >> helpful. See
>>> >> >> >>
>>> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
>>> >> >> >> for more information. Set this in the hbase-env.sh file on the
>>> line
>>> >> >> >> export HBASE_OPTS=...
>>> >> >> >>
>>> >> >> >> - Finally, to make sure that the garbage collection is fast,
>>> check if
>>> >> >> >> there is swap. If so, set lower heaps for the MR child processes
>>> in
>>> >> >> >> hadoop-site.xml (the mapred.child.java.opts property).
>>> >> >> >>
>>> >> >> >> J-D
>>> >> >> >>
>>> >> >> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <
>>> zsongbo@gmail.com>
>>> >> >> wrote:
>>> >> >> >> > Hi Jean Daniel,
>>> >> >> >> >
>>> >> >> >> > I want your help for this issue. I attach the log files,
>>> please
>>> >> help
>>> >> >> >> analyse
>>> >> >> >> > it. Thanks.
>>> >> >> >> >
>>> >> >> >> > Test env.
>>> >> >> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info',
>>> >> VERSIONS
>>> >> >> =>
>>> >> >> >> 1,
>>> >> >> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
>>> >> >> >> >
>>> >> >> >> > All test data is generated randomly by a program.
>>> >> >> >> >
>>> >> >> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
>>> >> >> >> >  After about 8 hours running, my node-5 rangeserver failure
>>> and the
>>> >> >> >> > HRegionServer Shutdown  .
>>> >> >> >> >  It seems caused by DFSClient exceptions. (I cannot make clear
>>> what
>>> >> >> >> > happened on HDFS, but it seems the HDFS is ok.)
>>> >> >> >> >
>>> >> >> >> > Then I start HRegionServer at this node-5. (2009-03-21
>>> >> 10:53:42,747):
>>> >> >> >> >  After the HRegionServer started, regions were reassign. I can
>>> see
>>> >> the
>>> >> >> >> > reassign on WebGUI of HBase, since some regions are now on
>>> this
>>> >> node.
>>> >> >> >> >  But following things are blocked for a long time:
>>> >> >> >> >  (1) The HBase client application cannot insert data for a
>>> long
>>> >> time
>>> >> >> >> (until
>>> >> >> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
>>> >> >> >> RetriesExhaustedException
>>> >> >> >> > exception on application side (MapReduce Job).
>>> >> >> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows
>>> in
>>> >> these
>>> >> >> >> > regions.) The exception is NotServingRegionException when
>>> >> getRegion.
>>> >> >> >> >  (3) I check the history of the region of (2) from the WebGUI.
>>> I
>>> >> can
>>> >> >> see
>>> >> >> >> in
>>> >> >> >> > the history, it is assigned at 11:04:15. It is so later.
>>> >> >> >> > The history is:
>>> >> >> >> > at, 21 Mar 2009 11:10:39openRegion opened on server :
>>> >> nd1-rack0-cloud
>>> >> >> >> Sat,
>>> >> >> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd1-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd5-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd5-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd3-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd4-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd5-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd3-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split
>>> from:
>>> >> >> >> > TESTA,13576334163@2009-03-21
>>> >> >> >> > 00:35:57.526,1237569164012<
>>> >> >> >>
>>> >> >>
>>> >>
>>> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> > And following is exception when I scan a rowkey range.
>>> >> >> >> >
>>> >> >> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException:
>>> Trying to
>>> >> >> >> contact
>>> >> >> >> > region server 10.24.1.12:60020 for region
>>> >> >> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
>>> >> >> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5
>>> attempts.
>>> >> >> >> > Exceptions:
>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>>> >> >> >> >        at
>>> >> >> >> >
>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>>> >> >> >> Source)
>>> >> >> >> >        at
>>> >> >> >> >
>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>>> >> >> >> > Source)
>>> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>>> >> >> >> Source)
>>> >> >> >> >        at
>>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>>> >> >> >> > Source)
>>> >> >> >> >
>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>>> >> >> >> >        at
>>> >> >> >> >
>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>>> >> >> >> Source)
>>> >> >> >> >        at
>>> >> >> >> >
>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>>> >> >> >> > Source)
>>> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>>> >> >> >> Source)
>>> >> >> >> >        at
>>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>>> >> >> >> > Source)
>>> >> >> >> >
>>> >> >> >> > I will send the log files to you email address.
>>> >> >> >> >
>>> >> >> >>
>>> >> >> >
>>> >> >>
>>> >> >
>>> >>
>>> >
>>>
>>
>>
>

Re: RegionServer failure and recovery take a long time

Posted by schubert zhang <zs...@gmail.com>.
1. Regards the "Unable to create new block" (NotReplicatedYetException) in
my first log, I cannot double check the logs, since the logs before 3/21 are
removed.  I can confirm that the disk space for HDFS is enough, and the heap
size of Namenode is also enough, but the memory in RegionServer/DataNode
is tight.
But the "Unable to create new block" issue still happen sometimes,
the phenomenon is not same as that in the first log.

In my today's namenode log, there are some following exception.
2009-03-23 10:07:57,465 INFO org.apache.hadoop.ipc.Server: IPC Server
handler 2 on 9000, call
addBlock(/hbase/log_10.24.1.18_1237686636736_60020/hlog.dat.1237774027436,
DFSClient_629567488) from 10.24.1.18:59685: error:
org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException: Not
replicated
yet:/hbase/log_10.24.1.18_1237686636736_60020/hlog.dat.1237774027436
org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException: Not
replicated
yet:/hbase/log_10.24.1.18_1237686636736_60020/hlog.dat.1237774027436
at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(Unknown
Source)
at org.apache.hadoop.hdfs.server.namenode.NameNode.addBlock(Unknown Source)
at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:597)
at org.apache.hadoop.ipc.RPC$Server.call(Unknown Source)
at org.apache.hadoop.ipc.Server$Handler.run(Unknown Source)
2009-03-23 10:07:57,552 INFO org.apache.hadoop.hdfs.StateChange: BLOCK*
NameSystem.addStoredBlock: blockMap updated: 10.24.1.12:50010 is added to
blk_8246919716767617786_109126 size 1048576
2009-03-23 10:07:57,552 INFO org.apache.hadoop.hdfs.StateChange: BLOCK*
NameSystem.addStoredBlock: blockMap updated: 10.24.1.12:50010 is added to
blk_8246919716767617786_109126 size 1048576
2009-03-23 10:07:57,554 INFO org.apache.hadoop.hdfs.StateChange: BLOCK*
NameSystem.allocateBlock:
/hbase/log_10.24.1.16_1237686658208_60020/hlog.dat.1237774044443.
blk_45871727940505900_109126
2009-03-23 10:07:57,688 INFO org.apache.hadoop.hdfs.StateChange: BLOCK*
NameSystem.addStoredBlock: blockMap updated: 10.24.1.12:50010 is added to
blk_2378060095065607252_109126 size 1048576
2009-03-23 10:07:57,688 INFO org.apache.hadoop.hdfs.StateChange: BLOCK*
NameSystem.addStoredBlock: blockMap updated: 10.24.1.14:50010 is added to
blk_2378060095065607252_109126 size 1048576
2009-03-23 10:07:57,689 INFO org.apache.hadoop.hdfs.StateChange: BLOCK*
NameSystem.allocateBlock:
/hbase/log_10.24.1.14_1237686648061_60020/hlog.dat.1237774036841.
blk_8448212226292209521_109126
2009-03-23 10:07:57,869 INFO org.apache.hadoop.ipc.Server: IPC Server
handler 1 on 9000, call
addBlock(/hbase/log_10.24.1.18_1237686636736_60020/hlog.dat.1237774027436,
DFSClient_629567488) from 10.24.1.18:59685: error:
org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException: Not
replicated
yet:/hbase/log_10.24.1.18_1237686636736_60020/hlog.dat.1237774027436
org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException: Not
replicated
yet:/hbase/log_10.24.1.18_1237686636736_60020/hlog.dat.1237774027436
at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(Unknown
Source)
at org.apache.hadoop.hdfs.server.namenode.NameNode.addBlock(Unknown Source)
at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:597)
at org.apache.hadoop.ipc.RPC$Server.call(Unknown Source)
at org.apache.hadoop.ipc.Server$Handler.run(Unknown Source)
2009-03-23 10:07:57,944 INFO org.apache.hadoop.hdfs.StateChange: BLOCK*
NameSystem.addStoredBlock: blockMap updated: 10.24.1.18:50010 is added to
blk_1270075611008480481_109121 size 1048576

I cannot find useful info in datanode's logs at the time point. But I find
something else, for examples:

2009-03-23 10:08:09,321 WARN
org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(
10.24.1.20:50010, storageID=DS-2136798339-10.24.1.20-50010-1237686444430,
infoPort=50075, ipcPort=50020):Failed to transfer
blk_-4099352067684877111_109151 to 10.24.1.18:50010 got
java.net.SocketException: Original Exception : java.io.IOException:
Connection reset by peer
        at sun.nio.ch.FileChannelImpl.transferTo0(Native Method)
        at
sun.nio.ch.FileChannelImpl.transferToDirectly(FileChannelImpl.java:418)
        at sun.nio.ch.FileChannelImpl.transferTo(FileChannelImpl.java:519)
        at org.apache.hadoop.net.SocketOutputStream.transferToFully(Unknown
Source)
        at
org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(Unknown
Source)
        at
org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(Unknown Source)
        at
org.apache.hadoop.hdfs.server.datanode.DataNode$DataTransfer.run(Unknown
Source)
        at java.lang.Thread.run(Thread.java:619)
Caused by: java.io.IOException: Connection reset by peer
        ... 8 more

and.

2009-03-23 10:10:17,313 ERROR
org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(
10.24.1.20:50010, storageID=DS-2136798339-10.24.1.20-50010-1237686444430,
infoPort=50075, ipcPort=50020):DataXceiver
org.apache.hadoop.hdfs.server.datanode.BlockAlreadyExistsException: Block
blk_-6347382571494739349_109326 is valid, and cannot be written to.
        at
org.apache.hadoop.hdfs.server.datanode.FSDataset.writeToBlock(Unknown
Source)
        at
org.apache.hadoop.hdfs.server.datanode.BlockReceiver.<init>(Unknown Source)
        at
org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(Unknown
Source)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(Unknown
Source)
        at java.lang.Thread.run(Thread.java:619)


2. Yes, In my previous setting, the heap of hadoop and hbase is 3GB, I want
to reconfigure them less in next round test. And I want to and more RAM in
next week. I remember I had encounter OOME when I set hbase heap as 1GB.

3. Yes, I was using autoFlush=flase before today, since I want to get high
performance. But when there are mange regions in the table, aufoFlush lose
its effectiveness since my row are random. Now, I set autoFlush=true, it
work well.

Thank you.
Schubert
On Mon, Mar 23, 2009 at 11:11 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:

> 2048 sec is sure long...
>
> I see in the first log a "Unable to create new block"... Can you take
> a look in your datanodes to see why? No more space left? Too much open
> files? Heap filled on the Namenode?
>
> I see you have a max heap for the region servers of 3GB, the heap will
> be bigger when the GC is called instead of a heap like 1GB. Sometimes,
> less is better. See if you can at least set this to 1.5GB.
>
> Oh but wait, 216 store files on the region server when it closed?!?
> Obviously the region server was overran by the updates. This reminds
> me of http://issues.apache.org/jira/browse/HBASE-1058
>
> Ok so my final advice would be to slow the updates and lower the heap.
> A good way to do the first one would be to turn the autoFlush on
> HTable on (unless you didn't change that value to false).
>
> J-D
>
> On Sat, Mar 21, 2009 at 8:50 PM, schubert zhang <zs...@gmail.com> wrote:
> > Jean Daniel,
> > I have done following:
> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the
> GC
> > detailedly.
> > (2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
> > want change to 200MB.
> > (3)  increase the lease period of master to 180 sec.
> >
> > But I have not applied the patch yet.
> >
> > But there are still following problems:
> >
> > (1) After about one hour, one region server down by:
> >
> > 009-03-22 03:36:06,858 INFO org.apache.hadoop.hbase.regionserver.HRegion:
> > Blocking updates for 'IPC Server handler 3 on 60020' on region
> > CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size
> 128.0m
> > is >= than blocking 128.0m size
> > 2009-03-22 03:36:07,114 INFO
> org.apache.hadoop.hbase.regionserver.HRegion:
> > Blocking updates for 'IPC Server handler 7 on 60020' on region
> > CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size
> 128.0m
> > is >= than blocking 128.0m size
> > 2009-03-22 03:36:08,384 WARN org.apache.hadoop.hdfs.DFSClient:
> DataStreamer
> > Exception: java.io.IOException: Unable to create new block.
> >        at
> >
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
> > Source)
> >        at
> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown
> Source)
> >        at
> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
> > Source)
> >
> > 2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Error
> > Recovery for block blk_2520317265353988797_3939 bad datanode[0] nodes ==
> > null
> > 2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Could not
> get
> > block locations. Source file
> > "/hbase/CDR/81441797/wap/mapfiles/2481397829486034048/data" - Aborting...
> > 2009-03-22 03:36:08,419 FATAL
> > org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Replay of hlog
> > required. Forcing server shutdown
> > org.apache.hadoop.hbase.DroppedSnapshotException: region:
> > CDR,13876322158@2009-03-22 03:08:29.315,1237664116065
> >        at
> > org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(Unknown
> > Source)
> >        at org.apache.hadoop.hbase.regionserver.HRegion.flushcache(Unknown
> > Source)
> >        at
> > org.apache.hadoop.hbase.regionserver.MemcacheFlusher.flushRegion(Unknown
> > Source)
> >        at
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher.run(Unknown
> > Source)
> > Caused by: java.io.IOException: Could not read from stream
> >        at org.apache.hadoop.net.SocketInputStream.read(Unknown Source)
> >        at java.io.DataInputStream.readByte(DataInputStream.java:248)
> >        at org.apache.hadoop.io.WritableUtils.readVLong(Unknown Source)
> >        at org.apache.hadoop.io.WritableUtils.readVInt(Unknown Source)
> >        at org.apache.hadoop.io.Text.readString(Unknown Source)
> >        at
> >
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(Unknown
> > Source)
> >        at
> >
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
> > Source)
> >        at
> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown
> Source)
> >        at
> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
> > Source)
> > 2009-03-22 03:36:08,422 INFO
> > org.apache.hadoop.hbase.regionserver.HRegionServer: Dump of metrics:
> > request=0.0, regions=2, stores=2, storefiles=216, storefileIndexSize=0,
> > memcacheSize=255, usedHeap=475, maxHeap=2991
> > 2009-03-22 03:36:08,422 INFO
> > org.apache.hadoop.hbase.regionserver.MemcacheFlusher:
> > regionserver/0:0:0:0:0:0:0:0:60020.cacheFlusher exiting
> >
> > (2) After about 6 hours, another region server down the old known issue:
> >
> > 2009-03-22 08:01:20,373 WARN org.apache.hadoop.hbase.util.Sleeper: We
> slept
> > 2048090ms, ten times longer than scheduled: 10000
> > 2009-03-22 08:01:20,404 WARN org.apache.hadoop.hbase.util.Sleeper: We
> slept
> > 2041927ms, ten times longer than scheduled: 3000
> > 2009-03-22 08:01:20,960 WARN
> > org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report to
> > master for 2042572 milliseconds - retrying
> > 2009-03-22 08:01:22,469 INFO org.apache.hadoop.hdfs.DFSClient:
> > org.apache.hadoop.ipc.RemoteException:
> > org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException: No lease on
> > /hbase/log_10.24.1.20_1237676058534_60020/hlog.dat.1237678010758 File
> does
> > not exist. Holder DFSClient_315506831 does not have any open files.
> >        at
> > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
> > Source)
> >        at
> > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
> > Source)
> >        at
> >
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(Unknown
> > Source)
> >        at
> org.apache.hadoop.hdfs.server.namenode.NameNode.addBlock(Unknown
> > Source)
> >        at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
> >        at
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> >        at java.lang.reflect.Method.invoke(Method.java:597)
> >        at org.apache.hadoop.ipc.RPC$Server.call(Unknown Source)
> >        at org.apache.hadoop.ipc.Server$Handler.run(Unknown Source)
> >
> >        at org.apache.hadoop.ipc.Client.call(Unknown Source)
> >
> > 2042572 ms is really too long.
> >
> > Schubert
> >
> > On Sun, Mar 22, 2009 at 3:01 AM, schubert zhang <zs...@gmail.com>
> wrote:
> >
> >> Yes, I missed " ". Thank you.
> >>
> >>
> >> On Sun, Mar 22, 2009 at 2:17 AM, Jean-Daniel Cryans <
> jdcryans@apache.org>wrote:
> >>
> >>> Put the options between " "
> >>>
> >>> J-D
> >>>
> >>> On Sat, Mar 21, 2009 at 2:15 PM, schubert zhang <zs...@gmail.com>
> >>> wrote:
> >>> > It's strange when I add -XX:+UseConcMarkSweepGC
> >>> > -XX:+CMSIncrementalMode./hbase/bin/../conf/hbase-env.sh:
> >>> > line 37: export: `-XX:+CMSIncrementalMode': not a valid identifer
> >>> >
> >>> > My jdk version is jdk-6u6-linux-x64, I will try the
> >>> > latest jdk-6u12-linux-x64 now.
> >>> >
> >>> > Schubert
> >>> >
> >>> > On Sun, Mar 22, 2009 at 1:40 AM, Jean-Daniel Cryans <
> >>> jdcryans@apache.org>wrote:
> >>> >
> >>> >> Schubert,
> >>> >>
> >>> >> It's based on the 0.19 branch in svn but it should patch with no
> >>> >> problem. In this state the patch is still just a test I'm doing but,
> >>> >> unless you write to thousands of regions at the same time when the
> >>> >> region server fails, there should be no problem. If it does cause
> you
> >>> >> trouble, please leave a comment in the jira. As you can see, it was
> >>> >> able to process a huge amount of logs without any problem. Also this
> >>> >> process is only done on the master which never receives any load so
> >>> >> it's even safer.
> >>> >>
> >>> >> J-D
> >>> >>
> >>> >> On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com>
> >>> wrote:
> >>> >> > Jean Daniel,
> >>> >> > Thanks for you kindness.
> >>> >> > Yes, I want more machines, and we will get them soon. :-)
> >>> >> > My application is write-heavy very much. Since my cluster is
> really
> >>> >> small, I
> >>> >> > will slow down the inserts now.
> >>> >> >
> >>> >> > One more questions about you patch HBASE-1008: It is really
> helpful
> >>> for
> >>> >> me.
> >>> >> > Does this patch take more memory? It seems not based on 0.19.1.
> Can
> >>> it be
> >>> >> > applied on 0.19.1?
> >>> >> >
> >>> >> > Schubert
> >>> >> >
> >>> >> > On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <
> >>> >> jdcryans@apache.org>wrote:
> >>> >> >
> >>> >> >> Schubert,
> >>> >> >>
> >>> >> >> I have no problem at all with your english since my first
> language
> >>> is
> >>> >> >> french and I must doing loads of grammatical errors too ;)
> >>> >> >>
> >>> >> >> Regards the heap, make sure that 300MB fits your need in mem or
> you
> >>> >> might
> >>> >> >> OOME.
> >>> >> >>
> >>> >> >> Increasing the lease period is a good idea, I have done the same.
> >>> Our
> >>> >> >> jobs take 13 hours so it avoids many restarts.
> >>> >> >>
> >>> >> >> Swappinnes at 0 => no swap at all... so if your system needs to
> swap
> >>> >> >> you might be in trouble. The advantage I see in a very low
> >>> swappiness
> >>> >> >> value (but not 0) is that it will only swap if ultimately
> necessary.
> >>> >> >>
> >>> >> >> On a final note, using the blocking caching feature is a bit of
> risk
> >>> >> >> in versions < 0.20. It does make random reads a lot faster (most
> of
> >>> >> >> the time) but the eviction of blocks produces a lot of garbage.
> The
> >>> >> >> guys from Stream.com are implementing something better at this
> very
> >>> >> >> moment.
> >>> >> >>
> >>> >> >> You may also want more machines :P. 6 is a very small number, we
> >>> >> >> usually see a lot more stability passed 10. Or instead you might
> >>> want
> >>> >> >> to slow down the inserts... It's good to be realist regards what
> >>> >> >> stress you put on the cluster VS the actual resources.
> >>> >> >>
> >>> >> >> J-D
> >>> >> >>
> >>> >> >> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <
> zsongbo@gmail.com
> >>> >
> >>> >> >> wrote:
> >>> >> >> > Hi Jean Daniel,
> >>> >> >> > Your help is so great. Thank you very much.
> >>> >> >> >
> >>> >> >> > After reading of the HBase Troubleshooting:
> >>> >> >> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also
> doubt
> >>> >> >> > about garbage collector and have added the
> -XX:+UseConcMarkSweepGC
> >>> >> option
> >>> >> >> 4
> >>> >> >> > hours ago. I checked the regionserves just now, one was
> shutdown
> >>> as
> >>> >> the
> >>> >> >> same
> >>> >> >> > cause. But its better than before.
> >>> >> >> >
> >>> >> >> > Now, I will do following turning according your guide:
> >>> >> >> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and
> >>> study
> >>> >> the
> >>> >> >> GC
> >>> >> >> > detailedly.
> >>> >> >> > (2)  decrease the heap size of mapreduce child, now I am using
> >>> 1024MB.
> >>> >> I
> >>> >> >> > want change to 300MB.
> >>> >> >> > (3)  increase the lease period of master to 180 sec.
> >>> >> >> > (4)  apply the you great patch.
> >>> >> >> >
> >>> >> >> > By the way, to avoid swap, I had changed the vm.swappiness = 0
> now
> >>> >> (you
> >>> >> >> had
> >>> >> >> > tell me 20 in another email), do think it is ok?
> >>> >> >> >
> >>> >> >> > Thank you again. My english is not good, please bear with me.
> >>> >> >> >
> >>> >> >> > Schubert
> >>> >> >> >
> >>> >> >> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
> >>> >> >> jdcryans@apache.org>wrote:
> >>> >> >> >
> >>> >> >> >> Schubert,
> >>> >> >> >>
> >>> >> >> >> Yeah that's the good old problem with the garbage collector.
> In
> >>> your
> >>> >> >> >> logs I see a lot of :
> >>> >> >> >>
> >>> >> >> >> 2009-03-21 05:59:06,498 WARN
> >>> org.apache.hadoop.hbase.util.Sleeper: We
> >>> >> >> >> slept 144233ms, ten times longer than scheduled: 3000
> >>> >> >> >> 2009-03-21 05:59:06,600 WARN
> >>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to
> >>> report
> >>> >> >> >> to master for 144335 milliseconds - retrying
> >>> >> >> >> 2009-03-21 05:59:06,512 WARN
> >>> org.apache.hadoop.hbase.util.Sleeper: We
> >>> >> >> >> slept 143279ms, ten times longer than scheduled: 10000
> >>> >> >> >> 2009-03-21 05:59:06,701 INFO
> >>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
> >>> >> >> >> MSG_CALL_SERVER_STARTUP: safeMode=false
> >>> >> >> >>
> >>> >> >> >> That usually means that the garbage collector blocked all
> threads
> >>> to
> >>> >> >> >> do it's stuff. But, when it happens, it takes more time than
> the
> >>> >> lease
> >>> >> >> >> the master maintains on the region servers (120 sec) so the
> >>> master
> >>> >> >> >> considers this region server as dead. Then the log splitting
> >>> takes
> >>> >> >> >> over on the master which is a very very long process. During
> that
> >>> >> >> >> time, sometimes more than 10 minutes, the regions from that
> >>> region
> >>> >> >> >> server are unavailable. If the cluster is small, that make
> things
> >>> >> even
> >>> >> >> >> far worse.
> >>> >> >> >>
> >>> >> >> >> We had these kinds of error on our cluster during the last
> weeks
> >>> and
> >>> >> >> >> here is how I solved it:
> >>> >> >> >>
> >>> >> >> >> - Regards the log splitting, I suggest you take a look at this
> >>> issue
> >>> >> >> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a
> >>> patch I
> >>> >> >> >> made to speed up the process. See if it helps you.
> >>> >> >> >>
> >>> >> >> >> - Regards the garbage collector, I found that the options
> >>> >> >> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really
> >>> really
> >>> >> >> >> helpful. See
> >>> >> >> >>
> >>> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
> >>> >> >> >> for more information. Set this in the hbase-env.sh file on the
> >>> line
> >>> >> >> >> export HBASE_OPTS=...
> >>> >> >> >>
> >>> >> >> >> - Finally, to make sure that the garbage collection is fast,
> >>> check if
> >>> >> >> >> there is swap. If so, set lower heaps for the MR child
> processes
> >>> in
> >>> >> >> >> hadoop-site.xml (the mapred.child.java.opts property).
> >>> >> >> >>
> >>> >> >> >> J-D
> >>> >> >> >>
> >>> >> >> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <
> >>> zsongbo@gmail.com>
> >>> >> >> wrote:
> >>> >> >> >> > Hi Jean Daniel,
> >>> >> >> >> >
> >>> >> >> >> > I want your help for this issue. I attach the log files,
> please
> >>> >> help
> >>> >> >> >> analyse
> >>> >> >> >> > it. Thanks.
> >>> >> >> >> >
> >>> >> >> >> > Test env.
> >>> >> >> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info',
> >>> >> VERSIONS
> >>> >> >> =>
> >>> >> >> >> 1,
> >>> >> >> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
> >>> >> >> >> >
> >>> >> >> >> > All test data is generated randomly by a program.
> >>> >> >> >> >
> >>> >> >> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
> >>> >> >> >> >  After about 8 hours running, my node-5 rangeserver failure
> and
> >>> the
> >>> >> >> >> > HRegionServer Shutdown  .
> >>> >> >> >> >  It seems caused by DFSClient exceptions. (I cannot make
> clear
> >>> what
> >>> >> >> >> > happened on HDFS, but it seems the HDFS is ok.)
> >>> >> >> >> >
> >>> >> >> >> > Then I start HRegionServer at this node-5. (2009-03-21
> >>> >> 10:53:42,747):
> >>> >> >> >> >  After the HRegionServer started, regions were reassign. I
> can
> >>> see
> >>> >> the
> >>> >> >> >> > reassign on WebGUI of HBase, since some regions are now on
> this
> >>> >> node.
> >>> >> >> >> >  But following things are blocked for a long time:
> >>> >> >> >> >  (1) The HBase client application cannot insert data for a
> long
> >>> >> time
> >>> >> >> >> (until
> >>> >> >> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
> >>> >> >> >> RetriesExhaustedException
> >>> >> >> >> > exception on application side (MapReduce Job).
> >>> >> >> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows
> in
> >>> >> these
> >>> >> >> >> > regions.) The exception is NotServingRegionException when
> >>> >> getRegion.
> >>> >> >> >> >  (3) I check the history of the region of (2) from the
> WebGUI.
> >>> I
> >>> >> can
> >>> >> >> see
> >>> >> >> >> in
> >>> >> >> >> > the history, it is assigned at 11:04:15. It is so later.
> >>> >> >> >> > The history is:
> >>> >> >> >> > at, 21 Mar 2009 11:10:39openRegion opened on server :
> >>> >> nd1-rack0-cloud
> >>> >> >> >> Sat,
> >>> >> >> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
> >>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened
> on
> >>> >> server
> >>> >> >> :
> >>> >> >> >> > nd1-rack0-cloud Sat,
> >>> >> >> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
> >>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened
> on
> >>> >> server
> >>> >> >> :
> >>> >> >> >> > nd5-rack0-cloud Sat,
> >>> >> >> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
> >>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened
> on
> >>> >> server
> >>> >> >> :
> >>> >> >> >> > nd5-rack0-cloud Sat,
> >>> >> >> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
> >>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened
> on
> >>> >> server
> >>> >> >> :
> >>> >> >> >> > nd3-rack0-cloud Sat,
> >>> >> >> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
> >>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened
> on
> >>> >> server
> >>> >> >> :
> >>> >> >> >> > nd4-rack0-cloud Sat,
> >>> >> >> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
> >>> >> >> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened
> on
> >>> >> server
> >>> >> >> :
> >>> >> >> >> > nd5-rack0-cloud Sat,
> >>> >> >> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
> >>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened
> on
> >>> >> server
> >>> >> >> :
> >>> >> >> >> > nd3-rack0-cloud Sat,
> >>> >> >> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
> >>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split
> >>> from:
> >>> >> >> >> > TESTA,13576334163@2009-03-21
> >>> >> >> >> > 00:35:57.526,1237569164012<
> >>> >> >> >>
> >>> >> >>
> >>> >>
> >>>
> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
> >>> >> >> >> >
> >>> >> >> >> >
> >>> >> >> >> >
> >>> >> >> >> > And following is exception when I scan a rowkey range.
> >>> >> >> >> >
> >>> >> >> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException:
> >>> Trying to
> >>> >> >> >> contact
> >>> >> >> >> > region server 10.24.1.12:60020 for region
> >>> >> >> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
> >>> >> >> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5
> >>> attempts.
> >>> >> >> >> > Exceptions:
> >>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >>> >> >> >> >        at
> >>> >> >> >> >
> >>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> >>> >> >> >> Source)
> >>> >> >> >> >        at
> >>> >> >> >> >
> >>> >>
> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> >>> >> >> >> > Source)
> >>> >> >> >> >        at
> sun.reflect.GeneratedMethodAccessor6.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(Unknown
> >>> >> >> >> Source)
> >>> >> >> >> >        at
> >>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> >>> >> >> >> > Source)
> >>> >> >> >> >
> >>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >>> >> >> >> >        at
> >>> >> >> >> >
> >>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> >>> >> >> >> Source)
> >>> >> >> >> >        at
> >>> >> >> >> >
> >>> >>
> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> >>> >> >> >> > Source)
> >>> >> >> >> >        at
> sun.reflect.GeneratedMethodAccessor6.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(Unknown
> >>> >> >> >> Source)
> >>> >> >> >> >        at
> >>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> >>> >> >> >> > Source)
> >>> >> >> >> >
> >>> >> >> >> > I will send the log files to you email address.
> >>> >> >> >> >
> >>> >> >> >>
> >>> >> >> >
> >>> >> >>
> >>> >> >
> >>> >>
> >>> >
> >>>
> >>
> >>
> >
>

Re: RegionServer failure and recovery take a long time

Posted by Jean-Daniel Cryans <jd...@apache.org>.
2048 sec is sure long...

I see in the first log a "Unable to create new block"... Can you take
a look in your datanodes to see why? No more space left? Too much open
files? Heap filled on the Namenode?

I see you have a max heap for the region servers of 3GB, the heap will
be bigger when the GC is called instead of a heap like 1GB. Sometimes,
less is better. See if you can at least set this to 1.5GB.

Oh but wait, 216 store files on the region server when it closed?!?
Obviously the region server was overran by the updates. This reminds
me of http://issues.apache.org/jira/browse/HBASE-1058

Ok so my final advice would be to slow the updates and lower the heap.
A good way to do the first one would be to turn the autoFlush on
HTable on (unless you didn't change that value to false).

J-D

On Sat, Mar 21, 2009 at 8:50 PM, schubert zhang <zs...@gmail.com> wrote:
> Jean Daniel,
> I have done following:
> (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the GC
> detailedly.
> (2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
> want change to 200MB.
> (3)  increase the lease period of master to 180 sec.
>
> But I have not applied the patch yet.
>
> But there are still following problems:
>
> (1) After about one hour, one region server down by:
>
> 009-03-22 03:36:06,858 INFO org.apache.hadoop.hbase.regionserver.HRegion:
> Blocking updates for 'IPC Server handler 3 on 60020' on region
> CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size 128.0m
> is >= than blocking 128.0m size
> 2009-03-22 03:36:07,114 INFO org.apache.hadoop.hbase.regionserver.HRegion:
> Blocking updates for 'IPC Server handler 7 on 60020' on region
> CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size 128.0m
> is >= than blocking 128.0m size
> 2009-03-22 03:36:08,384 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer
> Exception: java.io.IOException: Unable to create new block.
>        at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
> Source)
>        at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown Source)
>        at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
> Source)
>
> 2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Error
> Recovery for block blk_2520317265353988797_3939 bad datanode[0] nodes ==
> null
> 2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Could not get
> block locations. Source file
> "/hbase/CDR/81441797/wap/mapfiles/2481397829486034048/data" - Aborting...
> 2009-03-22 03:36:08,419 FATAL
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Replay of hlog
> required. Forcing server shutdown
> org.apache.hadoop.hbase.DroppedSnapshotException: region:
> CDR,13876322158@2009-03-22 03:08:29.315,1237664116065
>        at
> org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(Unknown
> Source)
>        at org.apache.hadoop.hbase.regionserver.HRegion.flushcache(Unknown
> Source)
>        at
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher.flushRegion(Unknown
> Source)
>        at org.apache.hadoop.hbase.regionserver.MemcacheFlusher.run(Unknown
> Source)
> Caused by: java.io.IOException: Could not read from stream
>        at org.apache.hadoop.net.SocketInputStream.read(Unknown Source)
>        at java.io.DataInputStream.readByte(DataInputStream.java:248)
>        at org.apache.hadoop.io.WritableUtils.readVLong(Unknown Source)
>        at org.apache.hadoop.io.WritableUtils.readVInt(Unknown Source)
>        at org.apache.hadoop.io.Text.readString(Unknown Source)
>        at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(Unknown
> Source)
>        at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
> Source)
>        at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown Source)
>        at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
> Source)
> 2009-03-22 03:36:08,422 INFO
> org.apache.hadoop.hbase.regionserver.HRegionServer: Dump of metrics:
> request=0.0, regions=2, stores=2, storefiles=216, storefileIndexSize=0,
> memcacheSize=255, usedHeap=475, maxHeap=2991
> 2009-03-22 03:36:08,422 INFO
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher:
> regionserver/0:0:0:0:0:0:0:0:60020.cacheFlusher exiting
>
> (2) After about 6 hours, another region server down the old known issue:
>
> 2009-03-22 08:01:20,373 WARN org.apache.hadoop.hbase.util.Sleeper: We slept
> 2048090ms, ten times longer than scheduled: 10000
> 2009-03-22 08:01:20,404 WARN org.apache.hadoop.hbase.util.Sleeper: We slept
> 2041927ms, ten times longer than scheduled: 3000
> 2009-03-22 08:01:20,960 WARN
> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report to
> master for 2042572 milliseconds - retrying
> 2009-03-22 08:01:22,469 INFO org.apache.hadoop.hdfs.DFSClient:
> org.apache.hadoop.ipc.RemoteException:
> org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException: No lease on
> /hbase/log_10.24.1.20_1237676058534_60020/hlog.dat.1237678010758 File does
> not exist. Holder DFSClient_315506831 does not have any open files.
>        at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
> Source)
>        at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
> Source)
>        at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(Unknown
> Source)
>        at org.apache.hadoop.hdfs.server.namenode.NameNode.addBlock(Unknown
> Source)
>        at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
>        at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>        at java.lang.reflect.Method.invoke(Method.java:597)
>        at org.apache.hadoop.ipc.RPC$Server.call(Unknown Source)
>        at org.apache.hadoop.ipc.Server$Handler.run(Unknown Source)
>
>        at org.apache.hadoop.ipc.Client.call(Unknown Source)
>
> 2042572 ms is really too long.
>
> Schubert
>
> On Sun, Mar 22, 2009 at 3:01 AM, schubert zhang <zs...@gmail.com> wrote:
>
>> Yes, I missed " ". Thank you.
>>
>>
>> On Sun, Mar 22, 2009 at 2:17 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:
>>
>>> Put the options between " "
>>>
>>> J-D
>>>
>>> On Sat, Mar 21, 2009 at 2:15 PM, schubert zhang <zs...@gmail.com>
>>> wrote:
>>> > It's strange when I add -XX:+UseConcMarkSweepGC
>>> > -XX:+CMSIncrementalMode./hbase/bin/../conf/hbase-env.sh:
>>> > line 37: export: `-XX:+CMSIncrementalMode': not a valid identifer
>>> >
>>> > My jdk version is jdk-6u6-linux-x64, I will try the
>>> > latest jdk-6u12-linux-x64 now.
>>> >
>>> > Schubert
>>> >
>>> > On Sun, Mar 22, 2009 at 1:40 AM, Jean-Daniel Cryans <
>>> jdcryans@apache.org>wrote:
>>> >
>>> >> Schubert,
>>> >>
>>> >> It's based on the 0.19 branch in svn but it should patch with no
>>> >> problem. In this state the patch is still just a test I'm doing but,
>>> >> unless you write to thousands of regions at the same time when the
>>> >> region server fails, there should be no problem. If it does cause you
>>> >> trouble, please leave a comment in the jira. As you can see, it was
>>> >> able to process a huge amount of logs without any problem. Also this
>>> >> process is only done on the master which never receives any load so
>>> >> it's even safer.
>>> >>
>>> >> J-D
>>> >>
>>> >> On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com>
>>> wrote:
>>> >> > Jean Daniel,
>>> >> > Thanks for you kindness.
>>> >> > Yes, I want more machines, and we will get them soon. :-)
>>> >> > My application is write-heavy very much. Since my cluster is really
>>> >> small, I
>>> >> > will slow down the inserts now.
>>> >> >
>>> >> > One more questions about you patch HBASE-1008: It is really helpful
>>> for
>>> >> me.
>>> >> > Does this patch take more memory? It seems not based on 0.19.1. Can
>>> it be
>>> >> > applied on 0.19.1?
>>> >> >
>>> >> > Schubert
>>> >> >
>>> >> > On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <
>>> >> jdcryans@apache.org>wrote:
>>> >> >
>>> >> >> Schubert,
>>> >> >>
>>> >> >> I have no problem at all with your english since my first language
>>> is
>>> >> >> french and I must doing loads of grammatical errors too ;)
>>> >> >>
>>> >> >> Regards the heap, make sure that 300MB fits your need in mem or you
>>> >> might
>>> >> >> OOME.
>>> >> >>
>>> >> >> Increasing the lease period is a good idea, I have done the same.
>>> Our
>>> >> >> jobs take 13 hours so it avoids many restarts.
>>> >> >>
>>> >> >> Swappinnes at 0 => no swap at all... so if your system needs to swap
>>> >> >> you might be in trouble. The advantage I see in a very low
>>> swappiness
>>> >> >> value (but not 0) is that it will only swap if ultimately necessary.
>>> >> >>
>>> >> >> On a final note, using the blocking caching feature is a bit of risk
>>> >> >> in versions < 0.20. It does make random reads a lot faster (most of
>>> >> >> the time) but the eviction of blocks produces a lot of garbage. The
>>> >> >> guys from Stream.com are implementing something better at this very
>>> >> >> moment.
>>> >> >>
>>> >> >> You may also want more machines :P. 6 is a very small number, we
>>> >> >> usually see a lot more stability passed 10. Or instead you might
>>> want
>>> >> >> to slow down the inserts... It's good to be realist regards what
>>> >> >> stress you put on the cluster VS the actual resources.
>>> >> >>
>>> >> >> J-D
>>> >> >>
>>> >> >> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <zsongbo@gmail.com
>>> >
>>> >> >> wrote:
>>> >> >> > Hi Jean Daniel,
>>> >> >> > Your help is so great. Thank you very much.
>>> >> >> >
>>> >> >> > After reading of the HBase Troubleshooting:
>>> >> >> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
>>> >> >> > about garbage collector and have added the -XX:+UseConcMarkSweepGC
>>> >> option
>>> >> >> 4
>>> >> >> > hours ago. I checked the regionserves just now, one was shutdown
>>> as
>>> >> the
>>> >> >> same
>>> >> >> > cause. But its better than before.
>>> >> >> >
>>> >> >> > Now, I will do following turning according your guide:
>>> >> >> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and
>>> study
>>> >> the
>>> >> >> GC
>>> >> >> > detailedly.
>>> >> >> > (2)  decrease the heap size of mapreduce child, now I am using
>>> 1024MB.
>>> >> I
>>> >> >> > want change to 300MB.
>>> >> >> > (3)  increase the lease period of master to 180 sec.
>>> >> >> > (4)  apply the you great patch.
>>> >> >> >
>>> >> >> > By the way, to avoid swap, I had changed the vm.swappiness = 0 now
>>> >> (you
>>> >> >> had
>>> >> >> > tell me 20 in another email), do think it is ok?
>>> >> >> >
>>> >> >> > Thank you again. My english is not good, please bear with me.
>>> >> >> >
>>> >> >> > Schubert
>>> >> >> >
>>> >> >> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
>>> >> >> jdcryans@apache.org>wrote:
>>> >> >> >
>>> >> >> >> Schubert,
>>> >> >> >>
>>> >> >> >> Yeah that's the good old problem with the garbage collector. In
>>> your
>>> >> >> >> logs I see a lot of :
>>> >> >> >>
>>> >> >> >> 2009-03-21 05:59:06,498 WARN
>>> org.apache.hadoop.hbase.util.Sleeper: We
>>> >> >> >> slept 144233ms, ten times longer than scheduled: 3000
>>> >> >> >> 2009-03-21 05:59:06,600 WARN
>>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to
>>> report
>>> >> >> >> to master for 144335 milliseconds - retrying
>>> >> >> >> 2009-03-21 05:59:06,512 WARN
>>> org.apache.hadoop.hbase.util.Sleeper: We
>>> >> >> >> slept 143279ms, ten times longer than scheduled: 10000
>>> >> >> >> 2009-03-21 05:59:06,701 INFO
>>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
>>> >> >> >> MSG_CALL_SERVER_STARTUP: safeMode=false
>>> >> >> >>
>>> >> >> >> That usually means that the garbage collector blocked all threads
>>> to
>>> >> >> >> do it's stuff. But, when it happens, it takes more time than the
>>> >> lease
>>> >> >> >> the master maintains on the region servers (120 sec) so the
>>> master
>>> >> >> >> considers this region server as dead. Then the log splitting
>>> takes
>>> >> >> >> over on the master which is a very very long process. During that
>>> >> >> >> time, sometimes more than 10 minutes, the regions from that
>>> region
>>> >> >> >> server are unavailable. If the cluster is small, that make things
>>> >> even
>>> >> >> >> far worse.
>>> >> >> >>
>>> >> >> >> We had these kinds of error on our cluster during the last weeks
>>> and
>>> >> >> >> here is how I solved it:
>>> >> >> >>
>>> >> >> >> - Regards the log splitting, I suggest you take a look at this
>>> issue
>>> >> >> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a
>>> patch I
>>> >> >> >> made to speed up the process. See if it helps you.
>>> >> >> >>
>>> >> >> >> - Regards the garbage collector, I found that the options
>>> >> >> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really
>>> really
>>> >> >> >> helpful. See
>>> >> >> >>
>>> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
>>> >> >> >> for more information. Set this in the hbase-env.sh file on the
>>> line
>>> >> >> >> export HBASE_OPTS=...
>>> >> >> >>
>>> >> >> >> - Finally, to make sure that the garbage collection is fast,
>>> check if
>>> >> >> >> there is swap. If so, set lower heaps for the MR child processes
>>> in
>>> >> >> >> hadoop-site.xml (the mapred.child.java.opts property).
>>> >> >> >>
>>> >> >> >> J-D
>>> >> >> >>
>>> >> >> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <
>>> zsongbo@gmail.com>
>>> >> >> wrote:
>>> >> >> >> > Hi Jean Daniel,
>>> >> >> >> >
>>> >> >> >> > I want your help for this issue. I attach the log files, please
>>> >> help
>>> >> >> >> analyse
>>> >> >> >> > it. Thanks.
>>> >> >> >> >
>>> >> >> >> > Test env.
>>> >> >> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info',
>>> >> VERSIONS
>>> >> >> =>
>>> >> >> >> 1,
>>> >> >> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
>>> >> >> >> >
>>> >> >> >> > All test data is generated randomly by a program.
>>> >> >> >> >
>>> >> >> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
>>> >> >> >> >  After about 8 hours running, my node-5 rangeserver failure and
>>> the
>>> >> >> >> > HRegionServer Shutdown  .
>>> >> >> >> >  It seems caused by DFSClient exceptions. (I cannot make clear
>>> what
>>> >> >> >> > happened on HDFS, but it seems the HDFS is ok.)
>>> >> >> >> >
>>> >> >> >> > Then I start HRegionServer at this node-5. (2009-03-21
>>> >> 10:53:42,747):
>>> >> >> >> >  After the HRegionServer started, regions were reassign. I can
>>> see
>>> >> the
>>> >> >> >> > reassign on WebGUI of HBase, since some regions are now on this
>>> >> node.
>>> >> >> >> >  But following things are blocked for a long time:
>>> >> >> >> >  (1) The HBase client application cannot insert data for a long
>>> >> time
>>> >> >> >> (until
>>> >> >> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
>>> >> >> >> RetriesExhaustedException
>>> >> >> >> > exception on application side (MapReduce Job).
>>> >> >> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows in
>>> >> these
>>> >> >> >> > regions.) The exception is NotServingRegionException when
>>> >> getRegion.
>>> >> >> >> >  (3) I check the history of the region of (2) from the WebGUI.
>>> I
>>> >> can
>>> >> >> see
>>> >> >> >> in
>>> >> >> >> > the history, it is assigned at 11:04:15. It is so later.
>>> >> >> >> > The history is:
>>> >> >> >> > at, 21 Mar 2009 11:10:39openRegion opened on server :
>>> >> nd1-rack0-cloud
>>> >> >> >> Sat,
>>> >> >> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd1-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd5-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd5-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd3-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd4-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd5-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on
>>> >> server
>>> >> >> :
>>> >> >> >> > nd3-rack0-cloud Sat,
>>> >> >> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
>>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split
>>> from:
>>> >> >> >> > TESTA,13576334163@2009-03-21
>>> >> >> >> > 00:35:57.526,1237569164012<
>>> >> >> >>
>>> >> >>
>>> >>
>>> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> > And following is exception when I scan a rowkey range.
>>> >> >> >> >
>>> >> >> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException:
>>> Trying to
>>> >> >> >> contact
>>> >> >> >> > region server 10.24.1.12:60020 for region
>>> >> >> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
>>> >> >> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5
>>> attempts.
>>> >> >> >> > Exceptions:
>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>>> >> >> >> >        at
>>> >> >> >> >
>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>>> >> >> >> Source)
>>> >> >> >> >        at
>>> >> >> >> >
>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>>> >> >> >> > Source)
>>> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>>> >> >> >> Source)
>>> >> >> >> >        at
>>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>>> >> >> >> > Source)
>>> >> >> >> >
>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>>> >> >> >> >        at
>>> >> >> >> >
>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>>> >> >> >> Source)
>>> >> >> >> >        at
>>> >> >> >> >
>>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>>> >> >> >> > Source)
>>> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>>> >> >> >> Source)
>>> >> >> >> >        at
>>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>>> >> >> >> > Source)
>>> >> >> >> >
>>> >> >> >> > I will send the log files to you email address.
>>> >> >> >> >
>>> >> >> >>
>>> >> >> >
>>> >> >>
>>> >> >
>>> >>
>>> >
>>>
>>
>>
>

Re: RegionServer failure and recovery take a long time

Posted by schubert zhang <zs...@gmail.com>.
Jean Daniel,
I have done following:
(1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the GC
detailedly.
(2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
want change to 200MB.
(3)  increase the lease period of master to 180 sec.

But I have not applied the patch yet.

But there are still following problems:

(1) After about one hour, one region server down by:

009-03-22 03:36:06,858 INFO org.apache.hadoop.hbase.regionserver.HRegion:
Blocking updates for 'IPC Server handler 3 on 60020' on region
CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size 128.0m
is >= than blocking 128.0m size
2009-03-22 03:36:07,114 INFO org.apache.hadoop.hbase.regionserver.HRegion:
Blocking updates for 'IPC Server handler 7 on 60020' on region
CDR,13775545206@2009-03-22 03:08:24.170,1237664116065: Memcache size 128.0m
is >= than blocking 128.0m size
2009-03-22 03:36:08,384 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer
Exception: java.io.IOException: Unable to create new block.
        at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
Source)
        at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown Source)
        at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
Source)

2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Error
Recovery for block blk_2520317265353988797_3939 bad datanode[0] nodes ==
null
2009-03-22 03:36:08,385 WARN org.apache.hadoop.hdfs.DFSClient: Could not get
block locations. Source file
"/hbase/CDR/81441797/wap/mapfiles/2481397829486034048/data" - Aborting...
2009-03-22 03:36:08,419 FATAL
org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Replay of hlog
required. Forcing server shutdown
org.apache.hadoop.hbase.DroppedSnapshotException: region:
CDR,13876322158@2009-03-22 03:08:29.315,1237664116065
        at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(Unknown
Source)
        at org.apache.hadoop.hbase.regionserver.HRegion.flushcache(Unknown
Source)
        at
org.apache.hadoop.hbase.regionserver.MemcacheFlusher.flushRegion(Unknown
Source)
        at org.apache.hadoop.hbase.regionserver.MemcacheFlusher.run(Unknown
Source)
Caused by: java.io.IOException: Could not read from stream
        at org.apache.hadoop.net.SocketInputStream.read(Unknown Source)
        at java.io.DataInputStream.readByte(DataInputStream.java:248)
        at org.apache.hadoop.io.WritableUtils.readVLong(Unknown Source)
        at org.apache.hadoop.io.WritableUtils.readVInt(Unknown Source)
        at org.apache.hadoop.io.Text.readString(Unknown Source)
        at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(Unknown
Source)
        at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(Unknown
Source)
        at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(Unknown Source)
        at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(Unknown
Source)
2009-03-22 03:36:08,422 INFO
org.apache.hadoop.hbase.regionserver.HRegionServer: Dump of metrics:
request=0.0, regions=2, stores=2, storefiles=216, storefileIndexSize=0,
memcacheSize=255, usedHeap=475, maxHeap=2991
2009-03-22 03:36:08,422 INFO
org.apache.hadoop.hbase.regionserver.MemcacheFlusher:
regionserver/0:0:0:0:0:0:0:0:60020.cacheFlusher exiting

(2) After about 6 hours, another region server down the old known issue:

2009-03-22 08:01:20,373 WARN org.apache.hadoop.hbase.util.Sleeper: We slept
2048090ms, ten times longer than scheduled: 10000
2009-03-22 08:01:20,404 WARN org.apache.hadoop.hbase.util.Sleeper: We slept
2041927ms, ten times longer than scheduled: 3000
2009-03-22 08:01:20,960 WARN
org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report to
master for 2042572 milliseconds - retrying
2009-03-22 08:01:22,469 INFO org.apache.hadoop.hdfs.DFSClient:
org.apache.hadoop.ipc.RemoteException:
org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException: No lease on
/hbase/log_10.24.1.20_1237676058534_60020/hlog.dat.1237678010758 File does
not exist. Holder DFSClient_315506831 does not have any open files.
        at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
Source)
        at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(Unknown
Source)
        at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(Unknown
Source)
        at org.apache.hadoop.hdfs.server.namenode.NameNode.addBlock(Unknown
Source)
        at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
        at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(Unknown Source)
        at org.apache.hadoop.ipc.Server$Handler.run(Unknown Source)

        at org.apache.hadoop.ipc.Client.call(Unknown Source)

2042572 ms is really too long.

Schubert

On Sun, Mar 22, 2009 at 3:01 AM, schubert zhang <zs...@gmail.com> wrote:

> Yes, I missed " ". Thank you.
>
>
> On Sun, Mar 22, 2009 at 2:17 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:
>
>> Put the options between " "
>>
>> J-D
>>
>> On Sat, Mar 21, 2009 at 2:15 PM, schubert zhang <zs...@gmail.com>
>> wrote:
>> > It's strange when I add -XX:+UseConcMarkSweepGC
>> > -XX:+CMSIncrementalMode./hbase/bin/../conf/hbase-env.sh:
>> > line 37: export: `-XX:+CMSIncrementalMode': not a valid identifer
>> >
>> > My jdk version is jdk-6u6-linux-x64, I will try the
>> > latest jdk-6u12-linux-x64 now.
>> >
>> > Schubert
>> >
>> > On Sun, Mar 22, 2009 at 1:40 AM, Jean-Daniel Cryans <
>> jdcryans@apache.org>wrote:
>> >
>> >> Schubert,
>> >>
>> >> It's based on the 0.19 branch in svn but it should patch with no
>> >> problem. In this state the patch is still just a test I'm doing but,
>> >> unless you write to thousands of regions at the same time when the
>> >> region server fails, there should be no problem. If it does cause you
>> >> trouble, please leave a comment in the jira. As you can see, it was
>> >> able to process a huge amount of logs without any problem. Also this
>> >> process is only done on the master which never receives any load so
>> >> it's even safer.
>> >>
>> >> J-D
>> >>
>> >> On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com>
>> wrote:
>> >> > Jean Daniel,
>> >> > Thanks for you kindness.
>> >> > Yes, I want more machines, and we will get them soon. :-)
>> >> > My application is write-heavy very much. Since my cluster is really
>> >> small, I
>> >> > will slow down the inserts now.
>> >> >
>> >> > One more questions about you patch HBASE-1008: It is really helpful
>> for
>> >> me.
>> >> > Does this patch take more memory? It seems not based on 0.19.1. Can
>> it be
>> >> > applied on 0.19.1?
>> >> >
>> >> > Schubert
>> >> >
>> >> > On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <
>> >> jdcryans@apache.org>wrote:
>> >> >
>> >> >> Schubert,
>> >> >>
>> >> >> I have no problem at all with your english since my first language
>> is
>> >> >> french and I must doing loads of grammatical errors too ;)
>> >> >>
>> >> >> Regards the heap, make sure that 300MB fits your need in mem or you
>> >> might
>> >> >> OOME.
>> >> >>
>> >> >> Increasing the lease period is a good idea, I have done the same.
>> Our
>> >> >> jobs take 13 hours so it avoids many restarts.
>> >> >>
>> >> >> Swappinnes at 0 => no swap at all... so if your system needs to swap
>> >> >> you might be in trouble. The advantage I see in a very low
>> swappiness
>> >> >> value (but not 0) is that it will only swap if ultimately necessary.
>> >> >>
>> >> >> On a final note, using the blocking caching feature is a bit of risk
>> >> >> in versions < 0.20. It does make random reads a lot faster (most of
>> >> >> the time) but the eviction of blocks produces a lot of garbage. The
>> >> >> guys from Stream.com are implementing something better at this very
>> >> >> moment.
>> >> >>
>> >> >> You may also want more machines :P. 6 is a very small number, we
>> >> >> usually see a lot more stability passed 10. Or instead you might
>> want
>> >> >> to slow down the inserts... It's good to be realist regards what
>> >> >> stress you put on the cluster VS the actual resources.
>> >> >>
>> >> >> J-D
>> >> >>
>> >> >> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <zsongbo@gmail.com
>> >
>> >> >> wrote:
>> >> >> > Hi Jean Daniel,
>> >> >> > Your help is so great. Thank you very much.
>> >> >> >
>> >> >> > After reading of the HBase Troubleshooting:
>> >> >> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
>> >> >> > about garbage collector and have added the -XX:+UseConcMarkSweepGC
>> >> option
>> >> >> 4
>> >> >> > hours ago. I checked the regionserves just now, one was shutdown
>> as
>> >> the
>> >> >> same
>> >> >> > cause. But its better than before.
>> >> >> >
>> >> >> > Now, I will do following turning according your guide:
>> >> >> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and
>> study
>> >> the
>> >> >> GC
>> >> >> > detailedly.
>> >> >> > (2)  decrease the heap size of mapreduce child, now I am using
>> 1024MB.
>> >> I
>> >> >> > want change to 300MB.
>> >> >> > (3)  increase the lease period of master to 180 sec.
>> >> >> > (4)  apply the you great patch.
>> >> >> >
>> >> >> > By the way, to avoid swap, I had changed the vm.swappiness = 0 now
>> >> (you
>> >> >> had
>> >> >> > tell me 20 in another email), do think it is ok?
>> >> >> >
>> >> >> > Thank you again. My english is not good, please bear with me.
>> >> >> >
>> >> >> > Schubert
>> >> >> >
>> >> >> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
>> >> >> jdcryans@apache.org>wrote:
>> >> >> >
>> >> >> >> Schubert,
>> >> >> >>
>> >> >> >> Yeah that's the good old problem with the garbage collector. In
>> your
>> >> >> >> logs I see a lot of :
>> >> >> >>
>> >> >> >> 2009-03-21 05:59:06,498 WARN
>> org.apache.hadoop.hbase.util.Sleeper: We
>> >> >> >> slept 144233ms, ten times longer than scheduled: 3000
>> >> >> >> 2009-03-21 05:59:06,600 WARN
>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to
>> report
>> >> >> >> to master for 144335 milliseconds - retrying
>> >> >> >> 2009-03-21 05:59:06,512 WARN
>> org.apache.hadoop.hbase.util.Sleeper: We
>> >> >> >> slept 143279ms, ten times longer than scheduled: 10000
>> >> >> >> 2009-03-21 05:59:06,701 INFO
>> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
>> >> >> >> MSG_CALL_SERVER_STARTUP: safeMode=false
>> >> >> >>
>> >> >> >> That usually means that the garbage collector blocked all threads
>> to
>> >> >> >> do it's stuff. But, when it happens, it takes more time than the
>> >> lease
>> >> >> >> the master maintains on the region servers (120 sec) so the
>> master
>> >> >> >> considers this region server as dead. Then the log splitting
>> takes
>> >> >> >> over on the master which is a very very long process. During that
>> >> >> >> time, sometimes more than 10 minutes, the regions from that
>> region
>> >> >> >> server are unavailable. If the cluster is small, that make things
>> >> even
>> >> >> >> far worse.
>> >> >> >>
>> >> >> >> We had these kinds of error on our cluster during the last weeks
>> and
>> >> >> >> here is how I solved it:
>> >> >> >>
>> >> >> >> - Regards the log splitting, I suggest you take a look at this
>> issue
>> >> >> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a
>> patch I
>> >> >> >> made to speed up the process. See if it helps you.
>> >> >> >>
>> >> >> >> - Regards the garbage collector, I found that the options
>> >> >> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really
>> really
>> >> >> >> helpful. See
>> >> >> >>
>> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
>> >> >> >> for more information. Set this in the hbase-env.sh file on the
>> line
>> >> >> >> export HBASE_OPTS=...
>> >> >> >>
>> >> >> >> - Finally, to make sure that the garbage collection is fast,
>> check if
>> >> >> >> there is swap. If so, set lower heaps for the MR child processes
>> in
>> >> >> >> hadoop-site.xml (the mapred.child.java.opts property).
>> >> >> >>
>> >> >> >> J-D
>> >> >> >>
>> >> >> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <
>> zsongbo@gmail.com>
>> >> >> wrote:
>> >> >> >> > Hi Jean Daniel,
>> >> >> >> >
>> >> >> >> > I want your help for this issue. I attach the log files, please
>> >> help
>> >> >> >> analyse
>> >> >> >> > it. Thanks.
>> >> >> >> >
>> >> >> >> > Test env.
>> >> >> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info',
>> >> VERSIONS
>> >> >> =>
>> >> >> >> 1,
>> >> >> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
>> >> >> >> >
>> >> >> >> > All test data is generated randomly by a program.
>> >> >> >> >
>> >> >> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
>> >> >> >> >  After about 8 hours running, my node-5 rangeserver failure and
>> the
>> >> >> >> > HRegionServer Shutdown  .
>> >> >> >> >  It seems caused by DFSClient exceptions. (I cannot make clear
>> what
>> >> >> >> > happened on HDFS, but it seems the HDFS is ok.)
>> >> >> >> >
>> >> >> >> > Then I start HRegionServer at this node-5. (2009-03-21
>> >> 10:53:42,747):
>> >> >> >> >  After the HRegionServer started, regions were reassign. I can
>> see
>> >> the
>> >> >> >> > reassign on WebGUI of HBase, since some regions are now on this
>> >> node.
>> >> >> >> >  But following things are blocked for a long time:
>> >> >> >> >  (1) The HBase client application cannot insert data for a long
>> >> time
>> >> >> >> (until
>> >> >> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
>> >> >> >> RetriesExhaustedException
>> >> >> >> > exception on application side (MapReduce Job).
>> >> >> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows in
>> >> these
>> >> >> >> > regions.) The exception is NotServingRegionException when
>> >> getRegion.
>> >> >> >> >  (3) I check the history of the region of (2) from the WebGUI.
>> I
>> >> can
>> >> >> see
>> >> >> >> in
>> >> >> >> > the history, it is assigned at 11:04:15. It is so later.
>> >> >> >> > The history is:
>> >> >> >> > at, 21 Mar 2009 11:10:39openRegion opened on server :
>> >> nd1-rack0-cloud
>> >> >> >> Sat,
>> >> >> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on
>> >> server
>> >> >> :
>> >> >> >> > nd1-rack0-cloud Sat,
>> >> >> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
>> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on
>> >> server
>> >> >> :
>> >> >> >> > nd5-rack0-cloud Sat,
>> >> >> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on
>> >> server
>> >> >> :
>> >> >> >> > nd5-rack0-cloud Sat,
>> >> >> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on
>> >> server
>> >> >> :
>> >> >> >> > nd3-rack0-cloud Sat,
>> >> >> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on
>> >> server
>> >> >> :
>> >> >> >> > nd4-rack0-cloud Sat,
>> >> >> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
>> >> >> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on
>> >> server
>> >> >> :
>> >> >> >> > nd5-rack0-cloud Sat,
>> >> >> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
>> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on
>> >> server
>> >> >> :
>> >> >> >> > nd3-rack0-cloud Sat,
>> >> >> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
>> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split
>> from:
>> >> >> >> > TESTA,13576334163@2009-03-21
>> >> >> >> > 00:35:57.526,1237569164012<
>> >> >> >>
>> >> >>
>> >>
>> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
>> >> >> >> >
>> >> >> >> >
>> >> >> >> >
>> >> >> >> > And following is exception when I scan a rowkey range.
>> >> >> >> >
>> >> >> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException:
>> Trying to
>> >> >> >> contact
>> >> >> >> > region server 10.24.1.12:60020 for region
>> >> >> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
>> >> >> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5
>> attempts.
>> >> >> >> > Exceptions:
>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>> >> >> >> >        at
>> >> >> >> >
>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>> >> >> >> Source)
>> >> >> >> >        at
>> >> >> >> >
>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>> >> >> >> > Source)
>> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>> >> >> >> Source)
>> >> >> >> >        at
>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>> >> >> >> > Source)
>> >> >> >> >
>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>> >> >> >> >        at
>> >> >> >> >
>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>> >> >> >> Source)
>> >> >> >> >        at
>> >> >> >> >
>> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>> >> >> >> > Source)
>> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>> >> >> >> Source)
>> >> >> >> >        at
>> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>> >> >> >> > Source)
>> >> >> >> >
>> >> >> >> > I will send the log files to you email address.
>> >> >> >> >
>> >> >> >>
>> >> >> >
>> >> >>
>> >> >
>> >>
>> >
>>
>
>

Re: RegionServer failure and recovery take a long time

Posted by schubert zhang <zs...@gmail.com>.
Yes, I missed " ". Thank you.

On Sun, Mar 22, 2009 at 2:17 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:

> Put the options between " "
>
> J-D
>
> On Sat, Mar 21, 2009 at 2:15 PM, schubert zhang <zs...@gmail.com> wrote:
> > It's strange when I add -XX:+UseConcMarkSweepGC
> > -XX:+CMSIncrementalMode./hbase/bin/../conf/hbase-env.sh:
> > line 37: export: `-XX:+CMSIncrementalMode': not a valid identifer
> >
> > My jdk version is jdk-6u6-linux-x64, I will try the
> > latest jdk-6u12-linux-x64 now.
> >
> > Schubert
> >
> > On Sun, Mar 22, 2009 at 1:40 AM, Jean-Daniel Cryans <jdcryans@apache.org
> >wrote:
> >
> >> Schubert,
> >>
> >> It's based on the 0.19 branch in svn but it should patch with no
> >> problem. In this state the patch is still just a test I'm doing but,
> >> unless you write to thousands of regions at the same time when the
> >> region server fails, there should be no problem. If it does cause you
> >> trouble, please leave a comment in the jira. As you can see, it was
> >> able to process a huge amount of logs without any problem. Also this
> >> process is only done on the master which never receives any load so
> >> it's even safer.
> >>
> >> J-D
> >>
> >> On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com>
> wrote:
> >> > Jean Daniel,
> >> > Thanks for you kindness.
> >> > Yes, I want more machines, and we will get them soon. :-)
> >> > My application is write-heavy very much. Since my cluster is really
> >> small, I
> >> > will slow down the inserts now.
> >> >
> >> > One more questions about you patch HBASE-1008: It is really helpful
> for
> >> me.
> >> > Does this patch take more memory? It seems not based on 0.19.1. Can it
> be
> >> > applied on 0.19.1?
> >> >
> >> > Schubert
> >> >
> >> > On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <
> >> jdcryans@apache.org>wrote:
> >> >
> >> >> Schubert,
> >> >>
> >> >> I have no problem at all with your english since my first language is
> >> >> french and I must doing loads of grammatical errors too ;)
> >> >>
> >> >> Regards the heap, make sure that 300MB fits your need in mem or you
> >> might
> >> >> OOME.
> >> >>
> >> >> Increasing the lease period is a good idea, I have done the same. Our
> >> >> jobs take 13 hours so it avoids many restarts.
> >> >>
> >> >> Swappinnes at 0 => no swap at all... so if your system needs to swap
> >> >> you might be in trouble. The advantage I see in a very low swappiness
> >> >> value (but not 0) is that it will only swap if ultimately necessary.
> >> >>
> >> >> On a final note, using the blocking caching feature is a bit of risk
> >> >> in versions < 0.20. It does make random reads a lot faster (most of
> >> >> the time) but the eviction of blocks produces a lot of garbage. The
> >> >> guys from Stream.com are implementing something better at this very
> >> >> moment.
> >> >>
> >> >> You may also want more machines :P. 6 is a very small number, we
> >> >> usually see a lot more stability passed 10. Or instead you might want
> >> >> to slow down the inserts... It's good to be realist regards what
> >> >> stress you put on the cluster VS the actual resources.
> >> >>
> >> >> J-D
> >> >>
> >> >> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <zs...@gmail.com>
> >> >> wrote:
> >> >> > Hi Jean Daniel,
> >> >> > Your help is so great. Thank you very much.
> >> >> >
> >> >> > After reading of the HBase Troubleshooting:
> >> >> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
> >> >> > about garbage collector and have added the -XX:+UseConcMarkSweepGC
> >> option
> >> >> 4
> >> >> > hours ago. I checked the regionserves just now, one was shutdown as
> >> the
> >> >> same
> >> >> > cause. But its better than before.
> >> >> >
> >> >> > Now, I will do following turning according your guide:
> >> >> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study
> >> the
> >> >> GC
> >> >> > detailedly.
> >> >> > (2)  decrease the heap size of mapreduce child, now I am using
> 1024MB.
> >> I
> >> >> > want change to 300MB.
> >> >> > (3)  increase the lease period of master to 180 sec.
> >> >> > (4)  apply the you great patch.
> >> >> >
> >> >> > By the way, to avoid swap, I had changed the vm.swappiness = 0 now
> >> (you
> >> >> had
> >> >> > tell me 20 in another email), do think it is ok?
> >> >> >
> >> >> > Thank you again. My english is not good, please bear with me.
> >> >> >
> >> >> > Schubert
> >> >> >
> >> >> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
> >> >> jdcryans@apache.org>wrote:
> >> >> >
> >> >> >> Schubert,
> >> >> >>
> >> >> >> Yeah that's the good old problem with the garbage collector. In
> your
> >> >> >> logs I see a lot of :
> >> >> >>
> >> >> >> 2009-03-21 05:59:06,498 WARN org.apache.hadoop.hbase.util.Sleeper:
> We
> >> >> >> slept 144233ms, ten times longer than scheduled: 3000
> >> >> >> 2009-03-21 05:59:06,600 WARN
> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to
> report
> >> >> >> to master for 144335 milliseconds - retrying
> >> >> >> 2009-03-21 05:59:06,512 WARN org.apache.hadoop.hbase.util.Sleeper:
> We
> >> >> >> slept 143279ms, ten times longer than scheduled: 10000
> >> >> >> 2009-03-21 05:59:06,701 INFO
> >> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
> >> >> >> MSG_CALL_SERVER_STARTUP: safeMode=false
> >> >> >>
> >> >> >> That usually means that the garbage collector blocked all threads
> to
> >> >> >> do it's stuff. But, when it happens, it takes more time than the
> >> lease
> >> >> >> the master maintains on the region servers (120 sec) so the master
> >> >> >> considers this region server as dead. Then the log splitting takes
> >> >> >> over on the master which is a very very long process. During that
> >> >> >> time, sometimes more than 10 minutes, the regions from that region
> >> >> >> server are unavailable. If the cluster is small, that make things
> >> even
> >> >> >> far worse.
> >> >> >>
> >> >> >> We had these kinds of error on our cluster during the last weeks
> and
> >> >> >> here is how I solved it:
> >> >> >>
> >> >> >> - Regards the log splitting, I suggest you take a look at this
> issue
> >> >> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a
> patch I
> >> >> >> made to speed up the process. See if it helps you.
> >> >> >>
> >> >> >> - Regards the garbage collector, I found that the options
> >> >> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really
> really
> >> >> >> helpful. See
> >> >> >>
> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
> >> >> >> for more information. Set this in the hbase-env.sh file on the
> line
> >> >> >> export HBASE_OPTS=...
> >> >> >>
> >> >> >> - Finally, to make sure that the garbage collection is fast, check
> if
> >> >> >> there is swap. If so, set lower heaps for the MR child processes
> in
> >> >> >> hadoop-site.xml (the mapred.child.java.opts property).
> >> >> >>
> >> >> >> J-D
> >> >> >>
> >> >> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <
> zsongbo@gmail.com>
> >> >> wrote:
> >> >> >> > Hi Jean Daniel,
> >> >> >> >
> >> >> >> > I want your help for this issue. I attach the log files, please
> >> help
> >> >> >> analyse
> >> >> >> > it. Thanks.
> >> >> >> >
> >> >> >> > Test env.
> >> >> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info',
> >> VERSIONS
> >> >> =>
> >> >> >> 1,
> >> >> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
> >> >> >> >
> >> >> >> > All test data is generated randomly by a program.
> >> >> >> >
> >> >> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
> >> >> >> >  After about 8 hours running, my node-5 rangeserver failure and
> the
> >> >> >> > HRegionServer Shutdown  .
> >> >> >> >  It seems caused by DFSClient exceptions. (I cannot make clear
> what
> >> >> >> > happened on HDFS, but it seems the HDFS is ok.)
> >> >> >> >
> >> >> >> > Then I start HRegionServer at this node-5. (2009-03-21
> >> 10:53:42,747):
> >> >> >> >  After the HRegionServer started, regions were reassign. I can
> see
> >> the
> >> >> >> > reassign on WebGUI of HBase, since some regions are now on this
> >> node.
> >> >> >> >  But following things are blocked for a long time:
> >> >> >> >  (1) The HBase client application cannot insert data for a long
> >> time
> >> >> >> (until
> >> >> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
> >> >> >> RetriesExhaustedException
> >> >> >> > exception on application side (MapReduce Job).
> >> >> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows in
> >> these
> >> >> >> > regions.) The exception is NotServingRegionException when
> >> getRegion.
> >> >> >> >  (3) I check the history of the region of (2) from the WebGUI. I
> >> can
> >> >> see
> >> >> >> in
> >> >> >> > the history, it is assigned at 11:04:15. It is so later.
> >> >> >> > The history is:
> >> >> >> > at, 21 Mar 2009 11:10:39openRegion opened on server :
> >> nd1-rack0-cloud
> >> >> >> Sat,
> >> >> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on
> >> server
> >> >> :
> >> >> >> > nd1-rack0-cloud Sat,
> >> >> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
> >> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on
> >> server
> >> >> :
> >> >> >> > nd5-rack0-cloud Sat,
> >> >> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on
> >> server
> >> >> :
> >> >> >> > nd5-rack0-cloud Sat,
> >> >> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on
> >> server
> >> >> :
> >> >> >> > nd3-rack0-cloud Sat,
> >> >> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on
> >> server
> >> >> :
> >> >> >> > nd4-rack0-cloud Sat,
> >> >> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
> >> >> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on
> >> server
> >> >> :
> >> >> >> > nd5-rack0-cloud Sat,
> >> >> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
> >> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on
> >> server
> >> >> :
> >> >> >> > nd3-rack0-cloud Sat,
> >> >> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
> >> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split
> from:
> >> >> >> > TESTA,13576334163@2009-03-21
> >> >> >> > 00:35:57.526,1237569164012<
> >> >> >>
> >> >>
> >>
> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
> >> >> >> >
> >> >> >> >
> >> >> >> >
> >> >> >> > And following is exception when I scan a rowkey range.
> >> >> >> >
> >> >> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying
> to
> >> >> >> contact
> >> >> >> > region server 10.24.1.12:60020 for region
> >> >> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
> >> >> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5
> attempts.
> >> >> >> > Exceptions:
> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >> >> >> >        at
> >> >> >> >
> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> >> >> >> Source)
> >> >> >> >        at
> >> >> >> >
> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> >> >> >> > Source)
> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
> >> >> >> Source)
> >> >> >> >        at
> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> >> >> >> > Source)
> >> >> >> >
> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >> >> >> >        at
> >> >> >> >
> >> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> >> >> >> Source)
> >> >> >> >        at
> >> >> >> >
> >> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> >> >> >> > Source)
> >> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
> >> >> >> Source)
> >> >> >> >        at
> >> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> >> >> >> > Source)
> >> >> >> >
> >> >> >> > I will send the log files to you email address.
> >> >> >> >
> >> >> >>
> >> >> >
> >> >>
> >> >
> >>
> >
>

Re: RegionServer failure and recovery take a long time

Posted by Jean-Daniel Cryans <jd...@apache.org>.
Put the options between " "

J-D

On Sat, Mar 21, 2009 at 2:15 PM, schubert zhang <zs...@gmail.com> wrote:
> It's strange when I add -XX:+UseConcMarkSweepGC
> -XX:+CMSIncrementalMode./hbase/bin/../conf/hbase-env.sh:
> line 37: export: `-XX:+CMSIncrementalMode': not a valid identifer
>
> My jdk version is jdk-6u6-linux-x64, I will try the
> latest jdk-6u12-linux-x64 now.
>
> Schubert
>
> On Sun, Mar 22, 2009 at 1:40 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:
>
>> Schubert,
>>
>> It's based on the 0.19 branch in svn but it should patch with no
>> problem. In this state the patch is still just a test I'm doing but,
>> unless you write to thousands of regions at the same time when the
>> region server fails, there should be no problem. If it does cause you
>> trouble, please leave a comment in the jira. As you can see, it was
>> able to process a huge amount of logs without any problem. Also this
>> process is only done on the master which never receives any load so
>> it's even safer.
>>
>> J-D
>>
>> On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com> wrote:
>> > Jean Daniel,
>> > Thanks for you kindness.
>> > Yes, I want more machines, and we will get them soon. :-)
>> > My application is write-heavy very much. Since my cluster is really
>> small, I
>> > will slow down the inserts now.
>> >
>> > One more questions about you patch HBASE-1008: It is really helpful for
>> me.
>> > Does this patch take more memory? It seems not based on 0.19.1. Can it be
>> > applied on 0.19.1?
>> >
>> > Schubert
>> >
>> > On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <
>> jdcryans@apache.org>wrote:
>> >
>> >> Schubert,
>> >>
>> >> I have no problem at all with your english since my first language is
>> >> french and I must doing loads of grammatical errors too ;)
>> >>
>> >> Regards the heap, make sure that 300MB fits your need in mem or you
>> might
>> >> OOME.
>> >>
>> >> Increasing the lease period is a good idea, I have done the same. Our
>> >> jobs take 13 hours so it avoids many restarts.
>> >>
>> >> Swappinnes at 0 => no swap at all... so if your system needs to swap
>> >> you might be in trouble. The advantage I see in a very low swappiness
>> >> value (but not 0) is that it will only swap if ultimately necessary.
>> >>
>> >> On a final note, using the blocking caching feature is a bit of risk
>> >> in versions < 0.20. It does make random reads a lot faster (most of
>> >> the time) but the eviction of blocks produces a lot of garbage. The
>> >> guys from Stream.com are implementing something better at this very
>> >> moment.
>> >>
>> >> You may also want more machines :P. 6 is a very small number, we
>> >> usually see a lot more stability passed 10. Or instead you might want
>> >> to slow down the inserts... It's good to be realist regards what
>> >> stress you put on the cluster VS the actual resources.
>> >>
>> >> J-D
>> >>
>> >> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <zs...@gmail.com>
>> >> wrote:
>> >> > Hi Jean Daniel,
>> >> > Your help is so great. Thank you very much.
>> >> >
>> >> > After reading of the HBase Troubleshooting:
>> >> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
>> >> > about garbage collector and have added the -XX:+UseConcMarkSweepGC
>> option
>> >> 4
>> >> > hours ago. I checked the regionserves just now, one was shutdown as
>> the
>> >> same
>> >> > cause. But its better than before.
>> >> >
>> >> > Now, I will do following turning according your guide:
>> >> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study
>> the
>> >> GC
>> >> > detailedly.
>> >> > (2)  decrease the heap size of mapreduce child, now I am using 1024MB.
>> I
>> >> > want change to 300MB.
>> >> > (3)  increase the lease period of master to 180 sec.
>> >> > (4)  apply the you great patch.
>> >> >
>> >> > By the way, to avoid swap, I had changed the vm.swappiness = 0 now
>> (you
>> >> had
>> >> > tell me 20 in another email), do think it is ok?
>> >> >
>> >> > Thank you again. My english is not good, please bear with me.
>> >> >
>> >> > Schubert
>> >> >
>> >> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
>> >> jdcryans@apache.org>wrote:
>> >> >
>> >> >> Schubert,
>> >> >>
>> >> >> Yeah that's the good old problem with the garbage collector. In your
>> >> >> logs I see a lot of :
>> >> >>
>> >> >> 2009-03-21 05:59:06,498 WARN org.apache.hadoop.hbase.util.Sleeper: We
>> >> >> slept 144233ms, ten times longer than scheduled: 3000
>> >> >> 2009-03-21 05:59:06,600 WARN
>> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
>> >> >> to master for 144335 milliseconds - retrying
>> >> >> 2009-03-21 05:59:06,512 WARN org.apache.hadoop.hbase.util.Sleeper: We
>> >> >> slept 143279ms, ten times longer than scheduled: 10000
>> >> >> 2009-03-21 05:59:06,701 INFO
>> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
>> >> >> MSG_CALL_SERVER_STARTUP: safeMode=false
>> >> >>
>> >> >> That usually means that the garbage collector blocked all threads to
>> >> >> do it's stuff. But, when it happens, it takes more time than the
>> lease
>> >> >> the master maintains on the region servers (120 sec) so the master
>> >> >> considers this region server as dead. Then the log splitting takes
>> >> >> over on the master which is a very very long process. During that
>> >> >> time, sometimes more than 10 minutes, the regions from that region
>> >> >> server are unavailable. If the cluster is small, that make things
>> even
>> >> >> far worse.
>> >> >>
>> >> >> We had these kinds of error on our cluster during the last weeks and
>> >> >> here is how I solved it:
>> >> >>
>> >> >> - Regards the log splitting, I suggest you take a look at this issue
>> >> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a patch I
>> >> >> made to speed up the process. See if it helps you.
>> >> >>
>> >> >> - Regards the garbage collector, I found that the options
>> >> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really really
>> >> >> helpful. See
>> >> >> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
>> >> >> for more information. Set this in the hbase-env.sh file on the line
>> >> >> export HBASE_OPTS=...
>> >> >>
>> >> >> - Finally, to make sure that the garbage collection is fast, check if
>> >> >> there is swap. If so, set lower heaps for the MR child processes in
>> >> >> hadoop-site.xml (the mapred.child.java.opts property).
>> >> >>
>> >> >> J-D
>> >> >>
>> >> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <zs...@gmail.com>
>> >> wrote:
>> >> >> > Hi Jean Daniel,
>> >> >> >
>> >> >> > I want your help for this issue. I attach the log files, please
>> help
>> >> >> analyse
>> >> >> > it. Thanks.
>> >> >> >
>> >> >> > Test env.
>> >> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info',
>> VERSIONS
>> >> =>
>> >> >> 1,
>> >> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
>> >> >> >
>> >> >> > All test data is generated randomly by a program.
>> >> >> >
>> >> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
>> >> >> >  After about 8 hours running, my node-5 rangeserver failure and the
>> >> >> > HRegionServer Shutdown  .
>> >> >> >  It seems caused by DFSClient exceptions. (I cannot make clear what
>> >> >> > happened on HDFS, but it seems the HDFS is ok.)
>> >> >> >
>> >> >> > Then I start HRegionServer at this node-5. (2009-03-21
>> 10:53:42,747):
>> >> >> >  After the HRegionServer started, regions were reassign. I can see
>> the
>> >> >> > reassign on WebGUI of HBase, since some regions are now on this
>> node.
>> >> >> >  But following things are blocked for a long time:
>> >> >> >  (1) The HBase client application cannot insert data for a long
>> time
>> >> >> (until
>> >> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
>> >> >> RetriesExhaustedException
>> >> >> > exception on application side (MapReduce Job).
>> >> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows in
>> these
>> >> >> > regions.) The exception is NotServingRegionException when
>> getRegion.
>> >> >> >  (3) I check the history of the region of (2) from the WebGUI. I
>> can
>> >> see
>> >> >> in
>> >> >> > the history, it is assigned at 11:04:15. It is so later.
>> >> >> > The history is:
>> >> >> > at, 21 Mar 2009 11:10:39openRegion opened on server :
>> nd1-rack0-cloud
>> >> >> Sat,
>> >> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
>> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on
>> server
>> >> :
>> >> >> > nd1-rack0-cloud Sat,
>> >> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
>> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on
>> server
>> >> :
>> >> >> > nd5-rack0-cloud Sat,
>> >> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
>> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on
>> server
>> >> :
>> >> >> > nd5-rack0-cloud Sat,
>> >> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
>> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on
>> server
>> >> :
>> >> >> > nd3-rack0-cloud Sat,
>> >> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
>> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on
>> server
>> >> :
>> >> >> > nd4-rack0-cloud Sat,
>> >> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
>> >> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on
>> server
>> >> :
>> >> >> > nd5-rack0-cloud Sat,
>> >> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
>> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on
>> server
>> >> :
>> >> >> > nd3-rack0-cloud Sat,
>> >> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
>> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split from:
>> >> >> > TESTA,13576334163@2009-03-21
>> >> >> > 00:35:57.526,1237569164012<
>> >> >>
>> >>
>> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
>> >> >> >
>> >> >> >
>> >> >> >
>> >> >> > And following is exception when I scan a rowkey range.
>> >> >> >
>> >> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
>> >> >> contact
>> >> >> > region server 10.24.1.12:60020 for region
>> >> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
>> >> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5 attempts.
>> >> >> > Exceptions:
>> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>> >> >> >        at
>> >> >> >
>> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>> >> >> Source)
>> >> >> >        at
>> >> >> >
>> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>> >> >> > Source)
>> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>> >> >> Source)
>> >> >> >        at
>> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>> >> >> > Source)
>> >> >> >
>> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>> >> >> >        at
>> >> >> >
>> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>> >> >> Source)
>> >> >> >        at
>> >> >> >
>> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>> >> >> > Source)
>> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>> >> >> Source)
>> >> >> >        at
>> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>> >> >> > Source)
>> >> >> >
>> >> >> > I will send the log files to you email address.
>> >> >> >
>> >> >>
>> >> >
>> >>
>> >
>>
>

Re: RegionServer failure and recovery take a long time

Posted by schubert zhang <zs...@gmail.com>.
It's strange when I add -XX:+UseConcMarkSweepGC
-XX:+CMSIncrementalMode./hbase/bin/../conf/hbase-env.sh:
line 37: export: `-XX:+CMSIncrementalMode': not a valid identifer

My jdk version is jdk-6u6-linux-x64, I will try the
latest jdk-6u12-linux-x64 now.

Schubert

On Sun, Mar 22, 2009 at 1:40 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:

> Schubert,
>
> It's based on the 0.19 branch in svn but it should patch with no
> problem. In this state the patch is still just a test I'm doing but,
> unless you write to thousands of regions at the same time when the
> region server fails, there should be no problem. If it does cause you
> trouble, please leave a comment in the jira. As you can see, it was
> able to process a huge amount of logs without any problem. Also this
> process is only done on the master which never receives any load so
> it's even safer.
>
> J-D
>
> On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com> wrote:
> > Jean Daniel,
> > Thanks for you kindness.
> > Yes, I want more machines, and we will get them soon. :-)
> > My application is write-heavy very much. Since my cluster is really
> small, I
> > will slow down the inserts now.
> >
> > One more questions about you patch HBASE-1008: It is really helpful for
> me.
> > Does this patch take more memory? It seems not based on 0.19.1. Can it be
> > applied on 0.19.1?
> >
> > Schubert
> >
> > On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <
> jdcryans@apache.org>wrote:
> >
> >> Schubert,
> >>
> >> I have no problem at all with your english since my first language is
> >> french and I must doing loads of grammatical errors too ;)
> >>
> >> Regards the heap, make sure that 300MB fits your need in mem or you
> might
> >> OOME.
> >>
> >> Increasing the lease period is a good idea, I have done the same. Our
> >> jobs take 13 hours so it avoids many restarts.
> >>
> >> Swappinnes at 0 => no swap at all... so if your system needs to swap
> >> you might be in trouble. The advantage I see in a very low swappiness
> >> value (but not 0) is that it will only swap if ultimately necessary.
> >>
> >> On a final note, using the blocking caching feature is a bit of risk
> >> in versions < 0.20. It does make random reads a lot faster (most of
> >> the time) but the eviction of blocks produces a lot of garbage. The
> >> guys from Stream.com are implementing something better at this very
> >> moment.
> >>
> >> You may also want more machines :P. 6 is a very small number, we
> >> usually see a lot more stability passed 10. Or instead you might want
> >> to slow down the inserts... It's good to be realist regards what
> >> stress you put on the cluster VS the actual resources.
> >>
> >> J-D
> >>
> >> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <zs...@gmail.com>
> >> wrote:
> >> > Hi Jean Daniel,
> >> > Your help is so great. Thank you very much.
> >> >
> >> > After reading of the HBase Troubleshooting:
> >> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
> >> > about garbage collector and have added the -XX:+UseConcMarkSweepGC
> option
> >> 4
> >> > hours ago. I checked the regionserves just now, one was shutdown as
> the
> >> same
> >> > cause. But its better than before.
> >> >
> >> > Now, I will do following turning according your guide:
> >> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study
> the
> >> GC
> >> > detailedly.
> >> > (2)  decrease the heap size of mapreduce child, now I am using 1024MB.
> I
> >> > want change to 300MB.
> >> > (3)  increase the lease period of master to 180 sec.
> >> > (4)  apply the you great patch.
> >> >
> >> > By the way, to avoid swap, I had changed the vm.swappiness = 0 now
> (you
> >> had
> >> > tell me 20 in another email), do think it is ok?
> >> >
> >> > Thank you again. My english is not good, please bear with me.
> >> >
> >> > Schubert
> >> >
> >> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
> >> jdcryans@apache.org>wrote:
> >> >
> >> >> Schubert,
> >> >>
> >> >> Yeah that's the good old problem with the garbage collector. In your
> >> >> logs I see a lot of :
> >> >>
> >> >> 2009-03-21 05:59:06,498 WARN org.apache.hadoop.hbase.util.Sleeper: We
> >> >> slept 144233ms, ten times longer than scheduled: 3000
> >> >> 2009-03-21 05:59:06,600 WARN
> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
> >> >> to master for 144335 milliseconds - retrying
> >> >> 2009-03-21 05:59:06,512 WARN org.apache.hadoop.hbase.util.Sleeper: We
> >> >> slept 143279ms, ten times longer than scheduled: 10000
> >> >> 2009-03-21 05:59:06,701 INFO
> >> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
> >> >> MSG_CALL_SERVER_STARTUP: safeMode=false
> >> >>
> >> >> That usually means that the garbage collector blocked all threads to
> >> >> do it's stuff. But, when it happens, it takes more time than the
> lease
> >> >> the master maintains on the region servers (120 sec) so the master
> >> >> considers this region server as dead. Then the log splitting takes
> >> >> over on the master which is a very very long process. During that
> >> >> time, sometimes more than 10 minutes, the regions from that region
> >> >> server are unavailable. If the cluster is small, that make things
> even
> >> >> far worse.
> >> >>
> >> >> We had these kinds of error on our cluster during the last weeks and
> >> >> here is how I solved it:
> >> >>
> >> >> - Regards the log splitting, I suggest you take a look at this issue
> >> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a patch I
> >> >> made to speed up the process. See if it helps you.
> >> >>
> >> >> - Regards the garbage collector, I found that the options
> >> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really really
> >> >> helpful. See
> >> >> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
> >> >> for more information. Set this in the hbase-env.sh file on the line
> >> >> export HBASE_OPTS=...
> >> >>
> >> >> - Finally, to make sure that the garbage collection is fast, check if
> >> >> there is swap. If so, set lower heaps for the MR child processes in
> >> >> hadoop-site.xml (the mapred.child.java.opts property).
> >> >>
> >> >> J-D
> >> >>
> >> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <zs...@gmail.com>
> >> wrote:
> >> >> > Hi Jean Daniel,
> >> >> >
> >> >> > I want your help for this issue. I attach the log files, please
> help
> >> >> analyse
> >> >> > it. Thanks.
> >> >> >
> >> >> > Test env.
> >> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info',
> VERSIONS
> >> =>
> >> >> 1,
> >> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
> >> >> >
> >> >> > All test data is generated randomly by a program.
> >> >> >
> >> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
> >> >> >  After about 8 hours running, my node-5 rangeserver failure and the
> >> >> > HRegionServer Shutdown  .
> >> >> >  It seems caused by DFSClient exceptions. (I cannot make clear what
> >> >> > happened on HDFS, but it seems the HDFS is ok.)
> >> >> >
> >> >> > Then I start HRegionServer at this node-5. (2009-03-21
> 10:53:42,747):
> >> >> >  After the HRegionServer started, regions were reassign. I can see
> the
> >> >> > reassign on WebGUI of HBase, since some regions are now on this
> node.
> >> >> >  But following things are blocked for a long time:
> >> >> >  (1) The HBase client application cannot insert data for a long
> time
> >> >> (until
> >> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
> >> >> RetriesExhaustedException
> >> >> > exception on application side (MapReduce Job).
> >> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows in
> these
> >> >> > regions.) The exception is NotServingRegionException when
> getRegion.
> >> >> >  (3) I check the history of the region of (2) from the WebGUI. I
> can
> >> see
> >> >> in
> >> >> > the history, it is assigned at 11:04:15. It is so later.
> >> >> > The history is:
> >> >> > at, 21 Mar 2009 11:10:39openRegion opened on server :
> nd1-rack0-cloud
> >> >> Sat,
> >> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on
> server
> >> :
> >> >> > nd1-rack0-cloud Sat,
> >> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
> >> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on
> server
> >> :
> >> >> > nd5-rack0-cloud Sat,
> >> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on
> server
> >> :
> >> >> > nd5-rack0-cloud Sat,
> >> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on
> server
> >> :
> >> >> > nd3-rack0-cloud Sat,
> >> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on
> server
> >> :
> >> >> > nd4-rack0-cloud Sat,
> >> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
> >> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on
> server
> >> :
> >> >> > nd5-rack0-cloud Sat,
> >> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
> >> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on
> server
> >> :
> >> >> > nd3-rack0-cloud Sat,
> >> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
> >> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split from:
> >> >> > TESTA,13576334163@2009-03-21
> >> >> > 00:35:57.526,1237569164012<
> >> >>
> >>
> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
> >> >> >
> >> >> >
> >> >> >
> >> >> > And following is exception when I scan a rowkey range.
> >> >> >
> >> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
> >> >> contact
> >> >> > region server 10.24.1.12:60020 for region
> >> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
> >> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5 attempts.
> >> >> > Exceptions:
> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >> >> >        at
> >> >> >
> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> >> >> Source)
> >> >> >        at
> >> >> >
> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> >> >> > Source)
> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
> >> >> Source)
> >> >> >        at
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> >> >> > Source)
> >> >> >
> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >> >> >        at
> >> >> >
> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> >> >> Source)
> >> >> >        at
> >> >> >
> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> >> >> > Source)
> >> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
> >> >> Source)
> >> >> >        at
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> >> >> > Source)
> >> >> >
> >> >> > I will send the log files to you email address.
> >> >> >
> >> >>
> >> >
> >>
> >
>

Re: RegionServer failure and recovery take a long time

Posted by Jean-Daniel Cryans <jd...@apache.org>.
Schubert,

It's based on the 0.19 branch in svn but it should patch with no
problem. In this state the patch is still just a test I'm doing but,
unless you write to thousands of regions at the same time when the
region server fails, there should be no problem. If it does cause you
trouble, please leave a comment in the jira. As you can see, it was
able to process a huge amount of logs without any problem. Also this
process is only done on the master which never receives any load so
it's even safer.

J-D

On Sat, Mar 21, 2009 at 1:34 PM, schubert zhang <zs...@gmail.com> wrote:
> Jean Daniel,
> Thanks for you kindness.
> Yes, I want more machines, and we will get them soon. :-)
> My application is write-heavy very much. Since my cluster is really small, I
> will slow down the inserts now.
>
> One more questions about you patch HBASE-1008: It is really helpful for me.
> Does this patch take more memory? It seems not based on 0.19.1. Can it be
> applied on 0.19.1?
>
> Schubert
>
> On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:
>
>> Schubert,
>>
>> I have no problem at all with your english since my first language is
>> french and I must doing loads of grammatical errors too ;)
>>
>> Regards the heap, make sure that 300MB fits your need in mem or you might
>> OOME.
>>
>> Increasing the lease period is a good idea, I have done the same. Our
>> jobs take 13 hours so it avoids many restarts.
>>
>> Swappinnes at 0 => no swap at all... so if your system needs to swap
>> you might be in trouble. The advantage I see in a very low swappiness
>> value (but not 0) is that it will only swap if ultimately necessary.
>>
>> On a final note, using the blocking caching feature is a bit of risk
>> in versions < 0.20. It does make random reads a lot faster (most of
>> the time) but the eviction of blocks produces a lot of garbage. The
>> guys from Stream.com are implementing something better at this very
>> moment.
>>
>> You may also want more machines :P. 6 is a very small number, we
>> usually see a lot more stability passed 10. Or instead you might want
>> to slow down the inserts... It's good to be realist regards what
>> stress you put on the cluster VS the actual resources.
>>
>> J-D
>>
>> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <zs...@gmail.com>
>> wrote:
>> > Hi Jean Daniel,
>> > Your help is so great. Thank you very much.
>> >
>> > After reading of the HBase Troubleshooting:
>> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
>> > about garbage collector and have added the -XX:+UseConcMarkSweepGC option
>> 4
>> > hours ago. I checked the regionserves just now, one was shutdown as the
>> same
>> > cause. But its better than before.
>> >
>> > Now, I will do following turning according your guide:
>> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the
>> GC
>> > detailedly.
>> > (2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
>> > want change to 300MB.
>> > (3)  increase the lease period of master to 180 sec.
>> > (4)  apply the you great patch.
>> >
>> > By the way, to avoid swap, I had changed the vm.swappiness = 0 now (you
>> had
>> > tell me 20 in another email), do think it is ok?
>> >
>> > Thank you again. My english is not good, please bear with me.
>> >
>> > Schubert
>> >
>> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
>> jdcryans@apache.org>wrote:
>> >
>> >> Schubert,
>> >>
>> >> Yeah that's the good old problem with the garbage collector. In your
>> >> logs I see a lot of :
>> >>
>> >> 2009-03-21 05:59:06,498 WARN org.apache.hadoop.hbase.util.Sleeper: We
>> >> slept 144233ms, ten times longer than scheduled: 3000
>> >> 2009-03-21 05:59:06,600 WARN
>> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
>> >> to master for 144335 milliseconds - retrying
>> >> 2009-03-21 05:59:06,512 WARN org.apache.hadoop.hbase.util.Sleeper: We
>> >> slept 143279ms, ten times longer than scheduled: 10000
>> >> 2009-03-21 05:59:06,701 INFO
>> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
>> >> MSG_CALL_SERVER_STARTUP: safeMode=false
>> >>
>> >> That usually means that the garbage collector blocked all threads to
>> >> do it's stuff. But, when it happens, it takes more time than the lease
>> >> the master maintains on the region servers (120 sec) so the master
>> >> considers this region server as dead. Then the log splitting takes
>> >> over on the master which is a very very long process. During that
>> >> time, sometimes more than 10 minutes, the regions from that region
>> >> server are unavailable. If the cluster is small, that make things even
>> >> far worse.
>> >>
>> >> We had these kinds of error on our cluster during the last weeks and
>> >> here is how I solved it:
>> >>
>> >> - Regards the log splitting, I suggest you take a look at this issue
>> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a patch I
>> >> made to speed up the process. See if it helps you.
>> >>
>> >> - Regards the garbage collector, I found that the options
>> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really really
>> >> helpful. See
>> >> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
>> >> for more information. Set this in the hbase-env.sh file on the line
>> >> export HBASE_OPTS=...
>> >>
>> >> - Finally, to make sure that the garbage collection is fast, check if
>> >> there is swap. If so, set lower heaps for the MR child processes in
>> >> hadoop-site.xml (the mapred.child.java.opts property).
>> >>
>> >> J-D
>> >>
>> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <zs...@gmail.com>
>> wrote:
>> >> > Hi Jean Daniel,
>> >> >
>> >> > I want your help for this issue. I attach the log files, please help
>> >> analyse
>> >> > it. Thanks.
>> >> >
>> >> > Test env.
>> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info', VERSIONS
>> =>
>> >> 1,
>> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
>> >> >
>> >> > All test data is generated randomly by a program.
>> >> >
>> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
>> >> >  After about 8 hours running, my node-5 rangeserver failure and the
>> >> > HRegionServer Shutdown  .
>> >> >  It seems caused by DFSClient exceptions. (I cannot make clear what
>> >> > happened on HDFS, but it seems the HDFS is ok.)
>> >> >
>> >> > Then I start HRegionServer at this node-5. (2009-03-21 10:53:42,747):
>> >> >  After the HRegionServer started, regions were reassign. I can see the
>> >> > reassign on WebGUI of HBase, since some regions are now on this node.
>> >> >  But following things are blocked for a long time:
>> >> >  (1) The HBase client application cannot insert data for a long time
>> >> (until
>> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
>> >> RetriesExhaustedException
>> >> > exception on application side (MapReduce Job).
>> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows in these
>> >> > regions.) The exception is NotServingRegionException when getRegion.
>> >> >  (3) I check the history of the region of (2) from the WebGUI. I can
>> see
>> >> in
>> >> > the history, it is assigned at 11:04:15. It is so later.
>> >> > The history is:
>> >> > at, 21 Mar 2009 11:10:39openRegion opened on server : nd1-rack0-cloud
>> >> Sat,
>> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
>> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on server
>> :
>> >> > nd1-rack0-cloud Sat,
>> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
>> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on server
>> :
>> >> > nd5-rack0-cloud Sat,
>> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
>> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on server
>> :
>> >> > nd5-rack0-cloud Sat,
>> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
>> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on server
>> :
>> >> > nd3-rack0-cloud Sat,
>> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
>> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on server
>> :
>> >> > nd4-rack0-cloud Sat,
>> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
>> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on server
>> :
>> >> > nd5-rack0-cloud Sat,
>> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
>> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on server
>> :
>> >> > nd3-rack0-cloud Sat,
>> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
>> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split from:
>> >> > TESTA,13576334163@2009-03-21
>> >> > 00:35:57.526,1237569164012<
>> >>
>> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
>> >> >
>> >> >
>> >> >
>> >> > And following is exception when I scan a rowkey range.
>> >> >
>> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
>> >> contact
>> >> > region server 10.24.1.12:60020 for region
>> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
>> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5 attempts.
>> >> > Exceptions:
>> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>> >> >        at
>> >> > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>> >> Source)
>> >> >        at
>> >> > org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>> >> > Source)
>> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>> >> Source)
>> >> >        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>> >> > Source)
>> >> >
>> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> > org.apache.hadoop.hbase.NotServingRegionException:
>> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>> >> >        at
>> >> > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>> >> Source)
>> >> >        at
>> >> > org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>> >> > Source)
>> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>> >> Source)
>> >> >        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>> >> > Source)
>> >> >
>> >> > I will send the log files to you email address.
>> >> >
>> >>
>> >
>>
>

Re: RegionServer failure and recovery take a long time

Posted by schubert zhang <zs...@gmail.com>.
Jean Daniel,
Thanks for you kindness.
Yes, I want more machines, and we will get them soon. :-)
My application is write-heavy very much. Since my cluster is really small, I
will slow down the inserts now.

One more questions about you patch HBASE-1008: It is really helpful for me.
Does this patch take more memory? It seems not based on 0.19.1. Can it be
applied on 0.19.1?

Schubert

On Sun, Mar 22, 2009 at 12:47 AM, Jean-Daniel Cryans <jd...@apache.org>wrote:

> Schubert,
>
> I have no problem at all with your english since my first language is
> french and I must doing loads of grammatical errors too ;)
>
> Regards the heap, make sure that 300MB fits your need in mem or you might
> OOME.
>
> Increasing the lease period is a good idea, I have done the same. Our
> jobs take 13 hours so it avoids many restarts.
>
> Swappinnes at 0 => no swap at all... so if your system needs to swap
> you might be in trouble. The advantage I see in a very low swappiness
> value (but not 0) is that it will only swap if ultimately necessary.
>
> On a final note, using the blocking caching feature is a bit of risk
> in versions < 0.20. It does make random reads a lot faster (most of
> the time) but the eviction of blocks produces a lot of garbage. The
> guys from Stream.com are implementing something better at this very
> moment.
>
> You may also want more machines :P. 6 is a very small number, we
> usually see a lot more stability passed 10. Or instead you might want
> to slow down the inserts... It's good to be realist regards what
> stress you put on the cluster VS the actual resources.
>
> J-D
>
> On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <zs...@gmail.com>
> wrote:
> > Hi Jean Daniel,
> > Your help is so great. Thank you very much.
> >
> > After reading of the HBase Troubleshooting:
> > http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
> > about garbage collector and have added the -XX:+UseConcMarkSweepGC option
> 4
> > hours ago. I checked the regionserves just now, one was shutdown as the
> same
> > cause. But its better than before.
> >
> > Now, I will do following turning according your guide:
> > (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the
> GC
> > detailedly.
> > (2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
> > want change to 300MB.
> > (3)  increase the lease period of master to 180 sec.
> > (4)  apply the you great patch.
> >
> > By the way, to avoid swap, I had changed the vm.swappiness = 0 now (you
> had
> > tell me 20 in another email), do think it is ok?
> >
> > Thank you again. My english is not good, please bear with me.
> >
> > Schubert
> >
> > On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <
> jdcryans@apache.org>wrote:
> >
> >> Schubert,
> >>
> >> Yeah that's the good old problem with the garbage collector. In your
> >> logs I see a lot of :
> >>
> >> 2009-03-21 05:59:06,498 WARN org.apache.hadoop.hbase.util.Sleeper: We
> >> slept 144233ms, ten times longer than scheduled: 3000
> >> 2009-03-21 05:59:06,600 WARN
> >> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
> >> to master for 144335 milliseconds - retrying
> >> 2009-03-21 05:59:06,512 WARN org.apache.hadoop.hbase.util.Sleeper: We
> >> slept 143279ms, ten times longer than scheduled: 10000
> >> 2009-03-21 05:59:06,701 INFO
> >> org.apache.hadoop.hbase.regionserver.HRegionServer:
> >> MSG_CALL_SERVER_STARTUP: safeMode=false
> >>
> >> That usually means that the garbage collector blocked all threads to
> >> do it's stuff. But, when it happens, it takes more time than the lease
> >> the master maintains on the region servers (120 sec) so the master
> >> considers this region server as dead. Then the log splitting takes
> >> over on the master which is a very very long process. During that
> >> time, sometimes more than 10 minutes, the regions from that region
> >> server are unavailable. If the cluster is small, that make things even
> >> far worse.
> >>
> >> We had these kinds of error on our cluster during the last weeks and
> >> here is how I solved it:
> >>
> >> - Regards the log splitting, I suggest you take a look at this issue
> >> https://issues.apache.org/jira/browse/HBASE-1008 as it has a patch I
> >> made to speed up the process. See if it helps you.
> >>
> >> - Regards the garbage collector, I found that the options
> >> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really really
> >> helpful. See
> >> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
> >> for more information. Set this in the hbase-env.sh file on the line
> >> export HBASE_OPTS=...
> >>
> >> - Finally, to make sure that the garbage collection is fast, check if
> >> there is swap. If so, set lower heaps for the MR child processes in
> >> hadoop-site.xml (the mapred.child.java.opts property).
> >>
> >> J-D
> >>
> >> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <zs...@gmail.com>
> wrote:
> >> > Hi Jean Daniel,
> >> >
> >> > I want your help for this issue. I attach the log files, please help
> >> analyse
> >> > it. Thanks.
> >> >
> >> > Test env.
> >> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info', VERSIONS
> =>
> >> 1,
> >> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
> >> >
> >> > All test data is generated randomly by a program.
> >> >
> >> > HRegionServer Failure (2009-03-21 08:27:41,090):
> >> >  After about 8 hours running, my node-5 rangeserver failure and the
> >> > HRegionServer Shutdown  .
> >> >  It seems caused by DFSClient exceptions. (I cannot make clear what
> >> > happened on HDFS, but it seems the HDFS is ok.)
> >> >
> >> > Then I start HRegionServer at this node-5. (2009-03-21 10:53:42,747):
> >> >  After the HRegionServer started, regions were reassign. I can see the
> >> > reassign on WebGUI of HBase, since some regions are now on this node.
> >> >  But following things are blocked for a long time:
> >> >  (1) The HBase client application cannot insert data for a long time
> >> (until
> >> > 2009/03/21 11:11:27, its about 18 minutes).  It is
> >> RetriesExhaustedException
> >> > exception on application side (MapReduce Job).
> >> >  (2) Some regions cannot be accessed (I cannot scan/get rows in these
> >> > regions.) The exception is NotServingRegionException when getRegion.
> >> >  (3) I check the history of the region of (2) from the WebGUI. I can
> see
> >> in
> >> > the history, it is assigned at 11:04:15. It is so later.
> >> > The history is:
> >> > at, 21 Mar 2009 11:10:39openRegion opened on server : nd1-rack0-cloud
> >> Sat,
> >> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on server
> :
> >> > nd1-rack0-cloud Sat,
> >> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
> >> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on server
> :
> >> > nd5-rack0-cloud Sat,
> >> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on server
> :
> >> > nd5-rack0-cloud Sat,
> >> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
> >> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on server
> :
> >> > nd3-rack0-cloud Sat,
> >> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
> >> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on server
> :
> >> > nd4-rack0-cloud Sat,
> >> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
> >> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on server
> :
> >> > nd5-rack0-cloud Sat,
> >> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
> >> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on server
> :
> >> > nd3-rack0-cloud Sat,
> >> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
> >> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split from:
> >> > TESTA,13576334163@2009-03-21
> >> > 00:35:57.526,1237569164012<
> >>
> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
> >> >
> >> >
> >> >
> >> > And following is exception when I scan a rowkey range.
> >> >
> >> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
> >> contact
> >> > region server 10.24.1.12:60020 for region
> >> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
> >> > '13576334163@2009-03-2100:35:57.526', but failed after 5 attempts.
> >> > Exceptions:
> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >> >        at
> >> > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> >> Source)
> >> >        at
> >> > org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> >> > Source)
> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
> >> Source)
> >> >        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> >> > Source)
> >> >
> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> > org.apache.hadoop.hbase.NotServingRegionException:
> >> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >> >        at
> >> > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> >> Source)
> >> >        at
> >> > org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> >> > Source)
> >> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
> >> Source)
> >> >        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> >> > Source)
> >> >
> >> > I will send the log files to you email address.
> >> >
> >>
> >
>

Re: RegionServer failure and recovery take a long time

Posted by Jean-Daniel Cryans <jd...@apache.org>.
Schubert,

I have no problem at all with your english since my first language is
french and I must doing loads of grammatical errors too ;)

Regards the heap, make sure that 300MB fits your need in mem or you might OOME.

Increasing the lease period is a good idea, I have done the same. Our
jobs take 13 hours so it avoids many restarts.

Swappinnes at 0 => no swap at all... so if your system needs to swap
you might be in trouble. The advantage I see in a very low swappiness
value (but not 0) is that it will only swap if ultimately necessary.

On a final note, using the blocking caching feature is a bit of risk
in versions < 0.20. It does make random reads a lot faster (most of
the time) but the eviction of blocks produces a lot of garbage. The
guys from Stream.com are implementing something better at this very
moment.

You may also want more machines :P. 6 is a very small number, we
usually see a lot more stability passed 10. Or instead you might want
to slow down the inserts... It's good to be realist regards what
stress you put on the cluster VS the actual resources.

J-D

On Sat, Mar 21, 2009 at 12:24 PM, schubert zhang <zs...@gmail.com> wrote:
> Hi Jean Daniel,
> Your help is so great. Thank you very much.
>
> After reading of the HBase Troubleshooting:
> http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
> about garbage collector and have added the -XX:+UseConcMarkSweepGC option 4
> hours ago. I checked the regionserves just now, one was shutdown as the same
> cause. But its better than before.
>
> Now, I will do following turning according your guide:
> (1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the GC
> detailedly.
> (2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
> want change to 300MB.
> (3)  increase the lease period of master to 180 sec.
> (4)  apply the you great patch.
>
> By the way, to avoid swap, I had changed the vm.swappiness = 0 now (you had
> tell me 20 in another email), do think it is ok?
>
> Thank you again. My english is not good, please bear with me.
>
> Schubert
>
> On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <jd...@apache.org>wrote:
>
>> Schubert,
>>
>> Yeah that's the good old problem with the garbage collector. In your
>> logs I see a lot of :
>>
>> 2009-03-21 05:59:06,498 WARN org.apache.hadoop.hbase.util.Sleeper: We
>> slept 144233ms, ten times longer than scheduled: 3000
>> 2009-03-21 05:59:06,600 WARN
>> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
>> to master for 144335 milliseconds - retrying
>> 2009-03-21 05:59:06,512 WARN org.apache.hadoop.hbase.util.Sleeper: We
>> slept 143279ms, ten times longer than scheduled: 10000
>> 2009-03-21 05:59:06,701 INFO
>> org.apache.hadoop.hbase.regionserver.HRegionServer:
>> MSG_CALL_SERVER_STARTUP: safeMode=false
>>
>> That usually means that the garbage collector blocked all threads to
>> do it's stuff. But, when it happens, it takes more time than the lease
>> the master maintains on the region servers (120 sec) so the master
>> considers this region server as dead. Then the log splitting takes
>> over on the master which is a very very long process. During that
>> time, sometimes more than 10 minutes, the regions from that region
>> server are unavailable. If the cluster is small, that make things even
>> far worse.
>>
>> We had these kinds of error on our cluster during the last weeks and
>> here is how I solved it:
>>
>> - Regards the log splitting, I suggest you take a look at this issue
>> https://issues.apache.org/jira/browse/HBASE-1008 as it has a patch I
>> made to speed up the process. See if it helps you.
>>
>> - Regards the garbage collector, I found that the options
>> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really really
>> helpful. See
>> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
>> for more information. Set this in the hbase-env.sh file on the line
>> export HBASE_OPTS=...
>>
>> - Finally, to make sure that the garbage collection is fast, check if
>> there is swap. If so, set lower heaps for the MR child processes in
>> hadoop-site.xml (the mapred.child.java.opts property).
>>
>> J-D
>>
>> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <zs...@gmail.com> wrote:
>> > Hi Jean Daniel,
>> >
>> > I want your help for this issue. I attach the log files, please help
>> analyse
>> > it. Thanks.
>> >
>> > Test env.
>> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info', VERSIONS =>
>> 1,
>> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
>> >
>> > All test data is generated randomly by a program.
>> >
>> > HRegionServer Failure (2009-03-21 08:27:41,090):
>> >  After about 8 hours running, my node-5 rangeserver failure and the
>> > HRegionServer Shutdown  .
>> >  It seems caused by DFSClient exceptions. (I cannot make clear what
>> > happened on HDFS, but it seems the HDFS is ok.)
>> >
>> > Then I start HRegionServer at this node-5. (2009-03-21 10:53:42,747):
>> >  After the HRegionServer started, regions were reassign. I can see the
>> > reassign on WebGUI of HBase, since some regions are now on this node.
>> >  But following things are blocked for a long time:
>> >  (1) The HBase client application cannot insert data for a long time
>> (until
>> > 2009/03/21 11:11:27, its about 18 minutes).  It is
>> RetriesExhaustedException
>> > exception on application side (MapReduce Job).
>> >  (2) Some regions cannot be accessed (I cannot scan/get rows in these
>> > regions.) The exception is NotServingRegionException when getRegion.
>> >  (3) I check the history of the region of (2) from the WebGUI. I can see
>> in
>> > the history, it is assigned at 11:04:15. It is so later.
>> > The history is:
>> > at, 21 Mar 2009 11:10:39openRegion opened on server : nd1-rack0-cloud
>> Sat,
>> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
>> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on server :
>> > nd1-rack0-cloud Sat,
>> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
>> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on server :
>> > nd5-rack0-cloud Sat,
>> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
>> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on server :
>> > nd5-rack0-cloud Sat,
>> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
>> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on server :
>> > nd3-rack0-cloud Sat,
>> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
>> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on server :
>> > nd4-rack0-cloud Sat,
>> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
>> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on server :
>> > nd5-rack0-cloud Sat,
>> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
>> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on server :
>> > nd3-rack0-cloud Sat,
>> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
>> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split from:
>> > TESTA,13576334163@2009-03-21
>> > 00:35:57.526,1237569164012<
>> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
>> >
>> >
>> >
>> > And following is exception when I scan a rowkey range.
>> >
>> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
>> contact
>> > region server 10.24.1.12:60020 for region
>> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
>> > '13576334163@2009-03-2100:35:57.526', but failed after 5 attempts.
>> > Exceptions:
>> > org.apache.hadoop.hbase.NotServingRegionException:
>> > org.apache.hadoop.hbase.NotServingRegionException:
>> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>> >        at
>> > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>> Source)
>> >        at
>> > org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>> > Source)
>> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>> Source)
>> >        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>> > Source)
>> >
>> > org.apache.hadoop.hbase.NotServingRegionException:
>> > org.apache.hadoop.hbase.NotServingRegionException:
>> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>> >        at
>> > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
>> Source)
>> >        at
>> > org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
>> > Source)
>> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
>> Source)
>> >        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
>> > Source)
>> >
>> > I will send the log files to you email address.
>> >
>>
>

Re: RegionServer failure and recovery take a long time

Posted by schubert zhang <zs...@gmail.com>.
Hi Jean Daniel,
Your help is so great. Thank you very much.

After reading of the HBase Troubleshooting:
http://wiki.apache.org/hadoop/Hbase/Troubleshooting, I also doubt
about garbage collector and have added the -XX:+UseConcMarkSweepGC option 4
hours ago. I checked the regionserves just now, one was shutdown as the same
cause. But its better than before.

Now, I will do following turning according your guide:
(1)  and -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode, and study the GC
detailedly.
(2)  decrease the heap size of mapreduce child, now I am using 1024MB. I
want change to 300MB.
(3)  increase the lease period of master to 180 sec.
(4)  apply the you great patch.

By the way, to avoid swap, I had changed the vm.swappiness = 0 now (you had
tell me 20 in another email), do think it is ok?

Thank you again. My english is not good, please bear with me.

Schubert

On Sat, Mar 21, 2009 at 11:39 PM, Jean-Daniel Cryans <jd...@apache.org>wrote:

> Schubert,
>
> Yeah that's the good old problem with the garbage collector. In your
> logs I see a lot of :
>
> 2009-03-21 05:59:06,498 WARN org.apache.hadoop.hbase.util.Sleeper: We
> slept 144233ms, ten times longer than scheduled: 3000
> 2009-03-21 05:59:06,600 WARN
> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
> to master for 144335 milliseconds - retrying
> 2009-03-21 05:59:06,512 WARN org.apache.hadoop.hbase.util.Sleeper: We
> slept 143279ms, ten times longer than scheduled: 10000
> 2009-03-21 05:59:06,701 INFO
> org.apache.hadoop.hbase.regionserver.HRegionServer:
> MSG_CALL_SERVER_STARTUP: safeMode=false
>
> That usually means that the garbage collector blocked all threads to
> do it's stuff. But, when it happens, it takes more time than the lease
> the master maintains on the region servers (120 sec) so the master
> considers this region server as dead. Then the log splitting takes
> over on the master which is a very very long process. During that
> time, sometimes more than 10 minutes, the regions from that region
> server are unavailable. If the cluster is small, that make things even
> far worse.
>
> We had these kinds of error on our cluster during the last weeks and
> here is how I solved it:
>
> - Regards the log splitting, I suggest you take a look at this issue
> https://issues.apache.org/jira/browse/HBASE-1008 as it has a patch I
> made to speed up the process. See if it helps you.
>
> - Regards the garbage collector, I found that the options
> "-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really really
> helpful. See
> http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
> for more information. Set this in the hbase-env.sh file on the line
> export HBASE_OPTS=...
>
> - Finally, to make sure that the garbage collection is fast, check if
> there is swap. If so, set lower heaps for the MR child processes in
> hadoop-site.xml (the mapred.child.java.opts property).
>
> J-D
>
> On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <zs...@gmail.com> wrote:
> > Hi Jean Daniel,
> >
> > I want your help for this issue. I attach the log files, please help
> analyse
> > it. Thanks.
> >
> > Test env.
> >  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info', VERSIONS =>
> 1,
> > COMPRESSION => 'BLOCK', BLOCKCACHE => true}
> >
> > All test data is generated randomly by a program.
> >
> > HRegionServer Failure (2009-03-21 08:27:41,090):
> >  After about 8 hours running, my node-5 rangeserver failure and the
> > HRegionServer Shutdown  .
> >  It seems caused by DFSClient exceptions. (I cannot make clear what
> > happened on HDFS, but it seems the HDFS is ok.)
> >
> > Then I start HRegionServer at this node-5. (2009-03-21 10:53:42,747):
> >  After the HRegionServer started, regions were reassign. I can see the
> > reassign on WebGUI of HBase, since some regions are now on this node.
> >  But following things are blocked for a long time:
> >  (1) The HBase client application cannot insert data for a long time
> (until
> > 2009/03/21 11:11:27, its about 18 minutes).  It is
> RetriesExhaustedException
> > exception on application side (MapReduce Job).
> >  (2) Some regions cannot be accessed (I cannot scan/get rows in these
> > regions.) The exception is NotServingRegionException when getRegion.
> >  (3) I check the history of the region of (2) from the WebGUI. I can see
> in
> > the history, it is assigned at 11:04:15. It is so later.
> > The history is:
> > at, 21 Mar 2009 11:10:39openRegion opened on server : nd1-rack0-cloud
> Sat,
> > 21 Mar 2009 11:04:15assignmentRegion assigned to server
> > 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on server :
> > nd1-rack0-cloud Sat,
> > 21 Mar 2009 06:47:57assignmentRegion assigned to server
> > 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on server :
> > nd5-rack0-cloud Sat,
> > 21 Mar 2009 06:27:21assignmentRegion assigned to server
> > 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on server :
> > nd5-rack0-cloud Sat,
> > 21 Mar 2009 06:24:53assignmentRegion assigned to server
> > 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on server :
> > nd3-rack0-cloud Sat,
> > 21 Mar 2009 06:24:13assignmentRegion assigned to server
> > 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on server :
> > nd4-rack0-cloud Sat,
> > 21 Mar 2009 06:19:02assignmentRegion assigned to server
> > 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on server :
> > nd5-rack0-cloud Sat,
> > 21 Mar 2009 05:59:36assignmentRegion assigned to server
> > 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on server :
> > nd3-rack0-cloud Sat,
> > 21 Mar 2009 03:50:12assignmentRegion assigned to server
> > 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split from:
> > TESTA,13576334163@2009-03-21
> > 00:35:57.526,1237569164012<
> http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012
> >
> >
> >
> > And following is exception when I scan a rowkey range.
> >
> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
> contact
> > region server 10.24.1.12:60020 for region
> > TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
> > '13576334163@2009-03-2100:35:57.526', but failed after 5 attempts.
> > Exceptions:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >        at
> > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> Source)
> >        at
> > org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> > Source)
> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
> Source)
> >        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> > Source)
> >
> > org.apache.hadoop.hbase.NotServingRegionException:
> > org.apache.hadoop.hbase.NotServingRegionException:
> > TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
> >        at
> > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown
> Source)
> >        at
> > org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> > Source)
> >        at sun.reflect.GeneratedMethodAccessor6.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(Unknown
> Source)
> >        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> > Source)
> >
> > I will send the log files to you email address.
> >
>

Re: RegionServer failure and recovery take a long time

Posted by Jean-Daniel Cryans <jd...@apache.org>.
Schubert,

Yeah that's the good old problem with the garbage collector. In your
logs I see a lot of :

2009-03-21 05:59:06,498 WARN org.apache.hadoop.hbase.util.Sleeper: We
slept 144233ms, ten times longer than scheduled: 3000
2009-03-21 05:59:06,600 WARN
org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report
to master for 144335 milliseconds - retrying
2009-03-21 05:59:06,512 WARN org.apache.hadoop.hbase.util.Sleeper: We
slept 143279ms, ten times longer than scheduled: 10000
2009-03-21 05:59:06,701 INFO
org.apache.hadoop.hbase.regionserver.HRegionServer:
MSG_CALL_SERVER_STARTUP: safeMode=false

That usually means that the garbage collector blocked all threads to
do it's stuff. But, when it happens, it takes more time than the lease
the master maintains on the region servers (120 sec) so the master
considers this region server as dead. Then the log splitting takes
over on the master which is a very very long process. During that
time, sometimes more than 10 minutes, the regions from that region
server are unavailable. If the cluster is small, that make things even
far worse.

We had these kinds of error on our cluster during the last weeks and
here is how I solved it:

- Regards the log splitting, I suggest you take a look at this issue
https://issues.apache.org/jira/browse/HBASE-1008 as it has a patch I
made to speed up the process. See if it helps you.

- Regards the garbage collector, I found that the options
"-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode" were really really
helpful. See http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
for more information. Set this in the hbase-env.sh file on the line
export HBASE_OPTS=...

- Finally, to make sure that the garbage collection is fast, check if
there is swap. If so, set lower heaps for the MR child processes in
hadoop-site.xml (the mapred.child.java.opts property).

J-D

On Sat, Mar 21, 2009 at 2:31 AM, schubert zhang <zs...@gmail.com> wrote:
> Hi Jean Daniel,
>
> I want your help for this issue. I attach the log files, please help analyse
> it. Thanks.
>
> Test env.
>  5+1 nodes cluster.  table: create 'TESTA', {NAME => 'info', VERSIONS => 1,
> COMPRESSION => 'BLOCK', BLOCKCACHE => true}
>
> All test data is generated randomly by a program.
>
> HRegionServer Failure (2009-03-21 08:27:41,090):
>  After about 8 hours running, my node-5 rangeserver failure and the
> HRegionServer Shutdown  .
>  It seems caused by DFSClient exceptions. (I cannot make clear what
> happened on HDFS, but it seems the HDFS is ok.)
>
> Then I start HRegionServer at this node-5. (2009-03-21 10:53:42,747):
>  After the HRegionServer started, regions were reassign. I can see the
> reassign on WebGUI of HBase, since some regions are now on this node.
>  But following things are blocked for a long time:
>  (1) The HBase client application cannot insert data for a long time (until
> 2009/03/21 11:11:27, its about 18 minutes).  It is RetriesExhaustedException
> exception on application side (MapReduce Job).
>  (2) Some regions cannot be accessed (I cannot scan/get rows in these
> regions.) The exception is NotServingRegionException when getRegion.
>  (3) I check the history of the region of (2) from the WebGUI. I can see in
> the history, it is assigned at 11:04:15. It is so later.
> The history is:
> at, 21 Mar 2009 11:10:39openRegion opened on server : nd1-rack0-cloud Sat,
> 21 Mar 2009 11:04:15assignmentRegion assigned to server
> 10.24.1.12:60020Sat, 21 Mar 2009 06:48:03openRegion opened on server :
> nd1-rack0-cloud Sat,
> 21 Mar 2009 06:47:57assignmentRegion assigned to server
> 10.24.1.12:60020Sat, 21 Mar 2009 06:27:25openRegion opened on server :
> nd5-rack0-cloud Sat,
> 21 Mar 2009 06:27:21assignmentRegion assigned to server
> 10.24.1.20:60020Sat, 21 Mar 2009 06:26:13openRegion opened on server :
> nd5-rack0-cloud Sat,
> 21 Mar 2009 06:24:53assignmentRegion assigned to server
> 10.24.1.20:60020Sat, 21 Mar 2009 06:24:28openRegion opened on server :
> nd3-rack0-cloud Sat,
> 21 Mar 2009 06:24:13assignmentRegion assigned to server
> 10.24.1.16:60020Sat, 21 Mar 2009 06:19:08openRegion opened on server :
> nd4-rack0-cloud Sat,
> 21 Mar 2009 06:19:02assignmentRegion assigned to server
> 10.24.1.18:60020Sat, 21 Mar 2009 05:59:39openRegion opened on server :
> nd5-rack0-cloud Sat,
> 21 Mar 2009 05:59:36assignmentRegion assigned to server
> 10.24.1.20:60020Sat, 21 Mar 2009 03:50:15openRegion opened on server :
> nd3-rack0-cloud Sat,
> 21 Mar 2009 03:50:12assignmentRegion assigned to server
> 10.24.1.16:60020Sat, 21 Mar 2009 03:50:08splitRegion split from:
> TESTA,13576334163@2009-03-21
> 00:35:57.526,1237569164012<http://nd0-rack0-cloud:60010/regionhistorian.jsp?regionname=CDR,13576334163@2009-03-21%2000:35:57.526,1237569164012>
>
>
> And following is exception when I scan a rowkey range.
>
> org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to contact
> region server 10.24.1.12:60020 for region
> TESTA,13576334163@2009-03-2100:35:57.526,1237578615553, row
> '13576334163@2009-03-2100:35:57.526', but failed after 5 attempts.
> Exceptions:
> org.apache.hadoop.hbase.NotServingRegionException:
> org.apache.hadoop.hbase.NotServingRegionException:
> TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>        at
> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown Source)
>        at
> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> Source)
>        at sun.reflect.GeneratedMethodAccessor6.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(Unknown Source)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> Source)
>
> org.apache.hadoop.hbase.NotServingRegionException:
> org.apache.hadoop.hbase.NotServingRegionException:
> TESTA,13576334163@2009-03-21 00:35:57.526,1237578615553
>        at
> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(Unknown Source)
>        at
> org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(Unknown
> Source)
>        at sun.reflect.GeneratedMethodAccessor6.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(Unknown Source)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(Unknown
> Source)
>
> I will send the log files to you email address.
>