You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by Mickey <hu...@gmail.com> on 2013/09/02 14:16:29 UTC

Region server blocked at waitForAckedSeqno

Hi, all

I was testing HBase with HDFS QJM HA recently. Hadoop version is CDH 4.3.0
and HBase is based on 0.94 with some patches(include HBASE-8211)
In a test, I met a blocking issue in HBase.  I killed a node which is the
active namenode, also datanode, regionserver on it.

The HDFS fail over successfully. The master tried re-assign the regions
after detecting the regionserver down. But no region can be online.

>From the log I found all operations to .META. failed. Printing the jstack
of the region server who contains the .META. , I found info below:
"regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
nid=0x27ee5 in Object.wait() [0x00007f318add9000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at
org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
        - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
        at
org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
        at
org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1525)
        at
org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1510)
        at
org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116)
        at
org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:1208)
        at sun.reflect.GeneratedMethodAccessor26.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:303)
        at
org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1290)
        at
org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1247)
        at
org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1400)
        at
org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:1199)
        at java.lang.Thread.run(Thread.java:662)

The logSyncer is always waiting on waitForAckedSeqno. All the HLog
operations seems blocked. Is this a bug? Or I missed some important patches?

Hope to get your suggestions soon.

Best regards,
Mickey

Re: Region server blocked at waitForAckedSeqno

Posted by Mickey <hu...@gmail.com>.
Himanshu,
I'm also wondering there seems no log about the DFSClinet or
DFSOutputStream at first. But the code in
waitForAckedSeqno seems will loop forever until an ack or interrupted...
The other set of handler is just like the replication handler. Please just
ignore it.

Elliott Clark,
My HDFS is the version in CDH 4.3.0.  No version problem in cluster.

I was trying to test NameNode HA, so I kill the process violently.  In the
test I killed the active namenode,datanode and regionserver on the same
node. The name node failed over successfully. But then I found HBase
blocked at the logSyncer.

I will try to collect the HDFS log at that moment to see if there is
something useful.
I didn't pick the jstack for the NN and DN, so no jstack this time.





2013/9/4 Elliott Clark <ec...@apache.org>

> I have seen this before.  The last guess was that it's a bug somewhere
> in the HDFS client (one of my colleagues was looking into it at the
> time).  It's missed an ack'd seq number and probably will never
> recover without a dn and rs restart.  I'll try and dig up any of the
> pertinent info.
>
> What version of HDFS are you running ?
> Same version of client and server ?
> Anything happening with the network at the time ?
> Was there a nn fail over at the time ?
>
>
>
> On Tue, Sep 3, 2013 at 7:49 PM, Himanshu Vashishtha <hv...@gmail.com>
> wrote:
> > Looking at the jstack, log roller and log syncer, both are blocked to get
> > the sequence number:
> > {code}
> > "regionserver60020.logRoller" daemon prio=10 tid=0x00007f317007f800
> > nid=0x27ee6 in Object.wait() [0x00007f318acd8000]
> >    java.lang.Thread.State: TIMED_WAITING (on object monitor)
> >         at java.lang.Object.wait(Native Method)
> >         at
> >
> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
> >         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
> >         at
> >
> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
> >
> > .....
> > "regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
> > nid=0x27ee5 in Object.wait() [0x00007f318add9000]
> >    java.lang.Thread.State: TIMED_WAITING (on object monitor)
> >         at java.lang.Object.wait(Native Method)
> >         at
> >
> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
> >         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
> >         at
> >
> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
> >
> > {code}
> >
> >
> > This blocks other append ops.
> >
> > What do you see in the NN and DN logs, which has this log file. Can you
> > pastebin NN, DN logs along with their jstack.
> >
> > On another note, I don't see the above exception in the log you attached.
> > Is that really the meta regionserver log? All I could see for meta table
> is
> > that it is calling MetaEditor to update meta, like an ordinary client.
> You
> > seem to have your own set of Handlers?
> > blah... "COP IPC Server handler 87 on 60020:" blah....
> >
> >
> > Thanks,
> > Himanshu
> >
> >
> > On Mon, Sep 2, 2013 at 8:30 PM, Mickey <hu...@gmail.com> wrote:
> >
> >> Hi Himanshu,
> >> It lasted for more than one hour. At last I tried to stop the region
> >> server in and failed. From the jstack it was still blocked by
> >>
> >> the HLog syncer. So I kill the process with "kill -9" and then the HBase
> >> got well.
> >>
> >> hbase.regionserver.logroll.errors.tolerated is the default value 0.
> >>
> >> My HBase cluster is mainly based on 0.94.1.
> >>
> >> Attachment is the region server which contains the .META. and the jstack
> >> when it is blocked.
> >>
> >> Thanks,
> >> Mickey
> >>
> >>
> >>
> >> 2013/9/2 Himanshu Vashishtha <hv...@gmail.com>
> >>
> >>> Hey Mickey,
> >>>
> >>> I have few followup questions:
> >>>
> >>> For how long these threads blocked? What happens afterwards,
> regionserver
> >>> resumes, or aborts?
> >>> And, could you pastebin the logs after the above exception?
> >>> Sync failure causes a log roll, which is retried based on value of
> >>> hbase.regionserver.logroll.errors.tolerated
> >>> Which 0.94 version you are using?
> >>>
> >>> Thanks,
> >>> Himanshu
> >>>
> >>>
> >>>
> >>> On Mon, Sep 2, 2013 at 5:16 AM, Mickey <hu...@gmail.com> wrote:
> >>>
> >>> > Hi, all
> >>> >
> >>> > I was testing HBase with HDFS QJM HA recently. Hadoop version is CDH
> >>> 4.3.0
> >>> > and HBase is based on 0.94 with some patches(include HBASE-8211)
> >>> > In a test, I met a blocking issue in HBase.  I killed a node which is
> >>> the
> >>> > active namenode, also datanode, regionserver on it.
> >>> >
> >>> > The HDFS fail over successfully. The master tried re-assign the
> regions
> >>> > after detecting the regionserver down. But no region can be online.
> >>> >
> >>> > From the log I found all operations to .META. failed. Printing the
> >>> jstack
> >>> > of the region server who contains the .META. , I found info below:
> >>> > "regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
> >>> > nid=0x27ee5 in Object.wait() [0x00007f318add9000]
> >>> >    java.lang.Thread.State: TIMED_WAITING (on object monitor)
> >>> >         at java.lang.Object.wait(Native Method)
> >>> >         at
> >>> >
> >>> >
> >>>
> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
> >>> >         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
> >>> >         at
> >>> >
> >>> >
> >>>
> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
> >>> >         at
> >>> >
> org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1525)
> >>> >         at
> >>> >
> org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1510)
> >>> >         at
> >>> >
> >>>
> org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116)
> >>> >         at
> >>> >
> org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:1208)
> >>> >         at sun.reflect.GeneratedMethodAccessor26.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:303)
> >>> >         at
> >>> > org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1290)
> >>> >         at
> >>> > org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1247)
> >>> >         at
> >>> > org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1400)
> >>> >         at
> >>> >
> >>>
> org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:1199)
> >>> >         at java.lang.Thread.run(Thread.java:662)
> >>> >
> >>> > The logSyncer is always waiting on waitForAckedSeqno. All the HLog
> >>> > operations seems blocked. Is this a bug? Or I missed some important
> >>> > patches?
> >>> >
> >>> > Hope to get your suggestions soon.
> >>> >
> >>> > Best regards,
> >>> > Mickey
> >>> >
> >>>
> >>
> >>
>

Re: Region server blocked at waitForAckedSeqno

Posted by Elliott Clark <ec...@apache.org>.
I have seen this before.  The last guess was that it's a bug somewhere
in the HDFS client (one of my colleagues was looking into it at the
time).  It's missed an ack'd seq number and probably will never
recover without a dn and rs restart.  I'll try and dig up any of the
pertinent info.

What version of HDFS are you running ?
Same version of client and server ?
Anything happening with the network at the time ?
Was there a nn fail over at the time ?



On Tue, Sep 3, 2013 at 7:49 PM, Himanshu Vashishtha <hv...@gmail.com> wrote:
> Looking at the jstack, log roller and log syncer, both are blocked to get
> the sequence number:
> {code}
> "regionserver60020.logRoller" daemon prio=10 tid=0x00007f317007f800
> nid=0x27ee6 in Object.wait() [0x00007f318acd8000]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at
> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
>         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
>         at
> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
>
> .....
> "regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
> nid=0x27ee5 in Object.wait() [0x00007f318add9000]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at
> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
>         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
>         at
> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
>
> {code}
>
>
> This blocks other append ops.
>
> What do you see in the NN and DN logs, which has this log file. Can you
> pastebin NN, DN logs along with their jstack.
>
> On another note, I don't see the above exception in the log you attached.
> Is that really the meta regionserver log? All I could see for meta table is
> that it is calling MetaEditor to update meta, like an ordinary client. You
> seem to have your own set of Handlers?
> blah... "COP IPC Server handler 87 on 60020:" blah....
>
>
> Thanks,
> Himanshu
>
>
> On Mon, Sep 2, 2013 at 8:30 PM, Mickey <hu...@gmail.com> wrote:
>
>> Hi Himanshu,
>> It lasted for more than one hour. At last I tried to stop the region
>> server in and failed. From the jstack it was still blocked by
>>
>> the HLog syncer. So I kill the process with "kill -9" and then the HBase
>> got well.
>>
>> hbase.regionserver.logroll.errors.tolerated is the default value 0.
>>
>> My HBase cluster is mainly based on 0.94.1.
>>
>> Attachment is the region server which contains the .META. and the jstack
>> when it is blocked.
>>
>> Thanks,
>> Mickey
>>
>>
>>
>> 2013/9/2 Himanshu Vashishtha <hv...@gmail.com>
>>
>>> Hey Mickey,
>>>
>>> I have few followup questions:
>>>
>>> For how long these threads blocked? What happens afterwards, regionserver
>>> resumes, or aborts?
>>> And, could you pastebin the logs after the above exception?
>>> Sync failure causes a log roll, which is retried based on value of
>>> hbase.regionserver.logroll.errors.tolerated
>>> Which 0.94 version you are using?
>>>
>>> Thanks,
>>> Himanshu
>>>
>>>
>>>
>>> On Mon, Sep 2, 2013 at 5:16 AM, Mickey <hu...@gmail.com> wrote:
>>>
>>> > Hi, all
>>> >
>>> > I was testing HBase with HDFS QJM HA recently. Hadoop version is CDH
>>> 4.3.0
>>> > and HBase is based on 0.94 with some patches(include HBASE-8211)
>>> > In a test, I met a blocking issue in HBase.  I killed a node which is
>>> the
>>> > active namenode, also datanode, regionserver on it.
>>> >
>>> > The HDFS fail over successfully. The master tried re-assign the regions
>>> > after detecting the regionserver down. But no region can be online.
>>> >
>>> > From the log I found all operations to .META. failed. Printing the
>>> jstack
>>> > of the region server who contains the .META. , I found info below:
>>> > "regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
>>> > nid=0x27ee5 in Object.wait() [0x00007f318add9000]
>>> >    java.lang.Thread.State: TIMED_WAITING (on object monitor)
>>> >         at java.lang.Object.wait(Native Method)
>>> >         at
>>> >
>>> >
>>> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
>>> >         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
>>> >         at
>>> >
>>> >
>>> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
>>> >         at
>>> > org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1525)
>>> >         at
>>> > org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1510)
>>> >         at
>>> >
>>> org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116)
>>> >         at
>>> > org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:1208)
>>> >         at sun.reflect.GeneratedMethodAccessor26.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:303)
>>> >         at
>>> > org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1290)
>>> >         at
>>> > org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1247)
>>> >         at
>>> > org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1400)
>>> >         at
>>> >
>>> org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:1199)
>>> >         at java.lang.Thread.run(Thread.java:662)
>>> >
>>> > The logSyncer is always waiting on waitForAckedSeqno. All the HLog
>>> > operations seems blocked. Is this a bug? Or I missed some important
>>> > patches?
>>> >
>>> > Hope to get your suggestions soon.
>>> >
>>> > Best regards,
>>> > Mickey
>>> >
>>>
>>
>>

Re: Region server blocked at waitForAckedSeqno

Posted by Himanshu Vashishtha <hv...@gmail.com>.
Looking at the jstack, log roller and log syncer, both are blocked to get
the sequence number:
{code}
"regionserver60020.logRoller" daemon prio=10 tid=0x00007f317007f800
nid=0x27ee6 in Object.wait() [0x00007f318acd8000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at
org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
        - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
        at
org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)

.....
"regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
nid=0x27ee5 in Object.wait() [0x00007f318add9000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at
org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
        - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
        at
org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)

{code}


This blocks other append ops.

What do you see in the NN and DN logs, which has this log file. Can you
pastebin NN, DN logs along with their jstack.

On another note, I don't see the above exception in the log you attached.
Is that really the meta regionserver log? All I could see for meta table is
that it is calling MetaEditor to update meta, like an ordinary client. You
seem to have your own set of Handlers?
blah... "COP IPC Server handler 87 on 60020:" blah....


Thanks,
Himanshu


On Mon, Sep 2, 2013 at 8:30 PM, Mickey <hu...@gmail.com> wrote:

> Hi Himanshu,
> It lasted for more than one hour. At last I tried to stop the region
> server in and failed. From the jstack it was still blocked by
>
> the HLog syncer. So I kill the process with "kill -9" and then the HBase
> got well.
>
> hbase.regionserver.logroll.errors.tolerated is the default value 0.
>
> My HBase cluster is mainly based on 0.94.1.
>
> Attachment is the region server which contains the .META. and the jstack
> when it is blocked.
>
> Thanks,
> Mickey
>
>
>
> 2013/9/2 Himanshu Vashishtha <hv...@gmail.com>
>
>> Hey Mickey,
>>
>> I have few followup questions:
>>
>> For how long these threads blocked? What happens afterwards, regionserver
>> resumes, or aborts?
>> And, could you pastebin the logs after the above exception?
>> Sync failure causes a log roll, which is retried based on value of
>> hbase.regionserver.logroll.errors.tolerated
>> Which 0.94 version you are using?
>>
>> Thanks,
>> Himanshu
>>
>>
>>
>> On Mon, Sep 2, 2013 at 5:16 AM, Mickey <hu...@gmail.com> wrote:
>>
>> > Hi, all
>> >
>> > I was testing HBase with HDFS QJM HA recently. Hadoop version is CDH
>> 4.3.0
>> > and HBase is based on 0.94 with some patches(include HBASE-8211)
>> > In a test, I met a blocking issue in HBase.  I killed a node which is
>> the
>> > active namenode, also datanode, regionserver on it.
>> >
>> > The HDFS fail over successfully. The master tried re-assign the regions
>> > after detecting the regionserver down. But no region can be online.
>> >
>> > From the log I found all operations to .META. failed. Printing the
>> jstack
>> > of the region server who contains the .META. , I found info below:
>> > "regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
>> > nid=0x27ee5 in Object.wait() [0x00007f318add9000]
>> >    java.lang.Thread.State: TIMED_WAITING (on object monitor)
>> >         at java.lang.Object.wait(Native Method)
>> >         at
>> >
>> >
>> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
>> >         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
>> >         at
>> >
>> >
>> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
>> >         at
>> > org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1525)
>> >         at
>> > org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1510)
>> >         at
>> >
>> org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116)
>> >         at
>> > org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:1208)
>> >         at sun.reflect.GeneratedMethodAccessor26.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:303)
>> >         at
>> > org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1290)
>> >         at
>> > org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1247)
>> >         at
>> > org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1400)
>> >         at
>> >
>> org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:1199)
>> >         at java.lang.Thread.run(Thread.java:662)
>> >
>> > The logSyncer is always waiting on waitForAckedSeqno. All the HLog
>> > operations seems blocked. Is this a bug? Or I missed some important
>> > patches?
>> >
>> > Hope to get your suggestions soon.
>> >
>> > Best regards,
>> > Mickey
>> >
>>
>
>

Re: Region server blocked at waitForAckedSeqno

Posted by Mickey <hu...@gmail.com>.
Hi Himanshu,
It lasted for more than one hour. At last I tried to stop the region server
in and failed. From the jstack it was still blocked by

the HLog syncer. So I kill the process with "kill -9" and then the HBase
got well.

hbase.regionserver.logroll.errors.tolerated is the default value 0.

My HBase cluster is mainly based on 0.94.1.

Attachment is the region server which contains the .META. and the jstack
when it is blocked.

Thanks,
Mickey



2013/9/2 Himanshu Vashishtha <hv...@gmail.com>

> Hey Mickey,
>
> I have few followup questions:
>
> For how long these threads blocked? What happens afterwards, regionserver
> resumes, or aborts?
> And, could you pastebin the logs after the above exception?
> Sync failure causes a log roll, which is retried based on value of
> hbase.regionserver.logroll.errors.tolerated
> Which 0.94 version you are using?
>
> Thanks,
> Himanshu
>
>
>
> On Mon, Sep 2, 2013 at 5:16 AM, Mickey <hu...@gmail.com> wrote:
>
> > Hi, all
> >
> > I was testing HBase with HDFS QJM HA recently. Hadoop version is CDH
> 4.3.0
> > and HBase is based on 0.94 with some patches(include HBASE-8211)
> > In a test, I met a blocking issue in HBase.  I killed a node which is the
> > active namenode, also datanode, regionserver on it.
> >
> > The HDFS fail over successfully. The master tried re-assign the regions
> > after detecting the regionserver down. But no region can be online.
> >
> > From the log I found all operations to .META. failed. Printing the jstack
> > of the region server who contains the .META. , I found info below:
> > "regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
> > nid=0x27ee5 in Object.wait() [0x00007f318add9000]
> >    java.lang.Thread.State: TIMED_WAITING (on object monitor)
> >         at java.lang.Object.wait(Native Method)
> >         at
> >
> >
> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
> >         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
> >         at
> >
> >
> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
> >         at
> > org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1525)
> >         at
> > org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1510)
> >         at
> > org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116)
> >         at
> > org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:1208)
> >         at sun.reflect.GeneratedMethodAccessor26.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:303)
> >         at
> > org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1290)
> >         at
> > org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1247)
> >         at
> > org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1400)
> >         at
> >
> org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:1199)
> >         at java.lang.Thread.run(Thread.java:662)
> >
> > The logSyncer is always waiting on waitForAckedSeqno. All the HLog
> > operations seems blocked. Is this a bug? Or I missed some important
> > patches?
> >
> > Hope to get your suggestions soon.
> >
> > Best regards,
> > Mickey
> >
>

Re: Region server blocked at waitForAckedSeqno

Posted by Himanshu Vashishtha <hv...@gmail.com>.
Hey Mickey,

I have few followup questions:

For how long these threads blocked? What happens afterwards, regionserver
resumes, or aborts?
And, could you pastebin the logs after the above exception?
Sync failure causes a log roll, which is retried based on value of
hbase.regionserver.logroll.errors.tolerated
Which 0.94 version you are using?

Thanks,
Himanshu



On Mon, Sep 2, 2013 at 5:16 AM, Mickey <hu...@gmail.com> wrote:

> Hi, all
>
> I was testing HBase with HDFS QJM HA recently. Hadoop version is CDH 4.3.0
> and HBase is based on 0.94 with some patches(include HBASE-8211)
> In a test, I met a blocking issue in HBase.  I killed a node which is the
> active namenode, also datanode, regionserver on it.
>
> The HDFS fail over successfully. The master tried re-assign the regions
> after detecting the regionserver down. But no region can be online.
>
> From the log I found all operations to .META. failed. Printing the jstack
> of the region server who contains the .META. , I found info below:
> "regionserver60020.logSyncer" daemon prio=10 tid=0x00007f317007e800
> nid=0x27ee5 in Object.wait() [0x00007f318add9000]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at
>
> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1708)
>         - locked <0x00007f34ae7b3638> (a java.util.LinkedList)
>         at
>
> org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1609)
>         at
> org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1525)
>         at
> org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1510)
>         at
> org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116)
>         at
> org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:1208)
>         at sun.reflect.GeneratedMethodAccessor26.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:303)
>         at
> org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1290)
>         at
> org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1247)
>         at
> org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1400)
>         at
> org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:1199)
>         at java.lang.Thread.run(Thread.java:662)
>
> The logSyncer is always waiting on waitForAckedSeqno. All the HLog
> operations seems blocked. Is this a bug? Or I missed some important
> patches?
>
> Hope to get your suggestions soon.
>
> Best regards,
> Mickey
>