[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2022-03-08 Thread Benoit Sigoure (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17502810#comment-17502810
 ] 

Benoit Sigoure commented on HBASE-26042:


We've run into this issue on a test cluster with HBase 2.4.8.

Let me know if I can collect anything else to help you, as things are still 
stuck right now and we can probably keep it untouched for another day or two as 
it's a test cluster.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5, 2.4.8
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, debug-dump.txt, 
> hbase-cvp-regionserver-cvp328.sjc.aristanetworks.com.log, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-08-20 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17402421#comment-17402421
 ] 

Michael Stack commented on HBASE-26042:
---

Update. We cured the provocation that was causing lots of IOEs during WAL 
rolling so we don't see this issue anymore.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, line=338 
> (Compiled 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-15 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17381464#comment-17381464
 ] 

Michael Stack commented on HBASE-26042:
---

{quote}bq. if you have a heap dump from that state? Poke around the call back 
its channel ID vs closed channel and the AsyncFSOutput instance state of the 
new WAL?
{quote}
Dang. The heaps here are too big But let me try (Might be a while given 
we've done some work to undo the provocation – the DN NPE'ing and even crashing 
on 'java.lang.NullPointerException at 
sun.nio.ch.EPollArrayWrapper.isEventsHighKilled(EPollArrayWrapper.java:174)' – 
a JDK/not-enough-fds issue). Thanks B.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-15 Thread Bharath Vissapragada (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17381451#comment-17381451
 ] 

Bharath Vissapragada commented on HBASE-26042:
--

Ya scratch that. Now that I look closely, all the flushes are single threaded. 
Think a concurrent flush is not possible in this specific thread stack at least 
because this is done _before_ swapping the WAL writer reference (consume path 
can be ignored). I'm still staring at the code (from the perspective of a 
single flush not being ack-ed + channel close scenario and race between 
completed on one channel and failed on another) but don't have any theory yet. 

[~stack] Its probably easy to prove/disprove this theory/direction if you have 
a heap dump from that state? Poke around the call back its channel ID vs closed 
channel and the AsyncFSOutput instance state of the new WAL?


> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-15 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17381416#comment-17381416
 ] 

Michael Stack commented on HBASE-26042:
---

Thanks for taking a look.

I don't have a test. I have just production logs/hread dumps (cited above)
{quote}So theoretically it should have no problem to call it outside the 
consume thread, as they should be no overlap.
{quote}
Agree. The hung WALRoller thread looks like the UT test that (in violation) has 
two threads trying to do flush concurrently. Let me see if it can happen in 
practice...

 

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-15 Thread Duo Zhang (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17381146#comment-17381146
 ] 

Duo Zhang commented on HBASE-26042:
---

The UT itself breaks the rule. It runs flush in a background thread which will 
cause problems, as two flushes could happen at the same time, and also a write 
and a flush could run at the same time. This will definately cause problems.

If this is the case, then we need to find out how could this happen in real 
production.

So [~stack] could you please provide your test? The writeMagicAndWALHeader is 
called when we create a WAL writer, typically we will call it in the rollWriter 
thread, and this is a sync call, which means only after writeMagicAndWALHeader 
is succeeded, we will move on and start to use it in AsyncFSWAL, i.e, actually 
write wal entries to it. So theoretically it should have no problem to call it 
outside the consume thread, as they should be no overlap.

Thanks.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-15 Thread Duo Zhang (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17381122#comment-17381122
 ] 

Duo Zhang commented on HBASE-26042:
---

Let me take a look. I guess the intention here is that writeMagicAndWALHeader 
is only called when initializing so it is not a problem, but maybe this is not 
the case.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-15 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17381058#comment-17381058
 ] 

Michael Stack commented on HBASE-26042:
---

Played w/ [~bharathv] PR.  I can manufacture one of these w/ this PR:
{code:java}
"ForkJoinPool.commonPool-worker-19" #219 daemon prio=5 os_prio=31 cpu=20.49ms 
elapsed=48.71s tid=0x7f8bf9ced800 nid=0x21a03 waiting on condition  
[0x700013869000]
   java.lang.Thread.State: WAITING (parking)
  at jdk.internal.misc.Unsafe.park(java.base@11.0.11/Native Method)
  - parking to wait for  <0x00078c0d0498> (a 
java.util.concurrent.CompletableFuture$Signaller)
  at 
java.util.concurrent.locks.LockSupport.park(java.base@11.0.11/LockSupport.java:194)
  at 
java.util.concurrent.CompletableFuture$Signaller.block(java.base@11.0.11/CompletableFuture.java:1796)
  at 
java.util.concurrent.ForkJoinPool.managedBlock(java.base@11.0.11/ForkJoinPool.java:3118)
  at 
java.util.concurrent.CompletableFuture.waitingGet(java.base@11.0.11/CompletableFuture.java:1823)
  at 
java.util.concurrent.CompletableFuture.get(java.base@11.0.11/CompletableFuture.java:1998)
  at 
org.apache.hadoop.hbase.io.asyncfs.TestFanOutOneBlockAsyncDFSOutput.lambda$testRecover$0(TestFanOutOneBlockAsyncDFSOutput.java:155)
  at 
org.apache.hadoop.hbase.io.asyncfs.TestFanOutOneBlockAsyncDFSOutput$$Lambda$142/0x000800454c40.run(Unknown
 Source)
  at 
java.util.concurrent.CompletableFuture$AsyncRun.run$$$capture(java.base@11.0.11/CompletableFuture.java:1736)
  at 
java.util.concurrent.CompletableFuture$AsyncRun.run(java.base@11.0.11/CompletableFuture.java)
  at 
java.util.concurrent.CompletableFuture$AsyncRun.exec(java.base@11.0.11/CompletableFuture.java:1728)
  at 
java.util.concurrent.ForkJoinTask.doExec$$$capture(java.base@11.0.11/ForkJoinTask.java:290)
  at 
java.util.concurrent.ForkJoinTask.doExec(java.base@11.0.11/ForkJoinTask.java)
  at 
java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(java.base@11.0.11/ForkJoinPool.java:1020)
  at 
java.util.concurrent.ForkJoinPool.scan(java.base@11.0.11/ForkJoinPool.java:1656)
  at 
java.util.concurrent.ForkJoinPool.runWorker(java.base@11.0.11/ForkJoinPool.java:1594)
  at 
java.util.concurrent.ForkJoinWorkerThread.run(java.base@11.0.11/ForkJoinWorkerThread.java:183)
 {code}
This looks like:
{code:java}
 "regionserver/ps1532:16020.logRoller" #395 daemon prio=5 os_prio=0 
tid=0x7f7c4403b800 nid=0xa2a7 waiting on condition [0x7f51f3c4c000]
   java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <0x7f682625cc00> (a 
java.util.concurrent.CompletableFuture$Signaller)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at 
java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1707)
at 
java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
at 
java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1742)
at 
java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(AsyncProtobufLogWriter.java:189)
at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(AsyncProtobufLogWriter.java:202)
at 
org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(AbstractProtobufLogWriter.java:170)
at 
org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(AsyncFSWALProvider.java:113)
at 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:651)
at 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:128)
at 
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:797)
at 
org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(AbstractWALRoller.java:263)
at 
org.apache.hadoop.hbase.wal.AbstractWALRoller.run(AbstractWALRoller.java:179){code}
In the test, there are two threads calling flush. This is in violation of 
FanOutOneBlockAsyncDFSOutput class comment as it states it is not thread-safe – 
for use by the single thread consume executor – so the two threads mess up each 
others' state (if they are sequenced they both fail properly w/ broken stream 
exceptions) but the lockup looks similar. There is a flush called in 
AsyncProtobufLogWriter#writeMagicAndWALHeader; i.e. not by the consume thread.

Still digging.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
>   

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-07 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17376820#comment-17376820
 ] 

Michael Stack commented on HBASE-26042:
---

[~bharathv] let me try. Sweet.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, line=338 
> (Compiled frame)
>  - com.lmax.disruptor.MultiProducerSequencer.next(int) @bci=82, line=136 
> 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-07 Thread Bharath Vissapragada (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17376818#comment-17376818
 ] 

Bharath Vissapragada commented on HBASE-26042:
--

bq.  I think there is some racy code in FanOutOneBlockAsyncDFSOutput especially 
around when a new writer is marked 'BROKEN' and a flush is called resulting in 
some waitingAckQueue members not being cleaned up correctly. Just a theory at 
this point, but probably easy to poke around with a heap dump or some related 
logging around these code paths.

[~stack] Attached a unit test repro patch (which includes comments that explain 
the race). I followed the theory of broken cb cleanup in 
FanOutOneBlockAsyncDFSOutput that I alluded to above. With the patch, you can 
run TestFanOutOneBlockAsyncDFSOutput#testRecover under a debugger and set break 
point at latch#await() to inspect the state. WDYT.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: HBASE-26042-test-repro.patch, js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-07 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17376817#comment-17376817
 ] 

Michael Stack commented on HBASE-26042:
---

Tried w/ 2.3.5 and 2.4.3. The wal roll 'fixes' the pipeline. I can't repro the 
abort/hang.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, line=338 
> (Compiled frame)
>  - 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-07 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17376749#comment-17376749
 ] 

Michael Stack commented on HBASE-26042:
---

Reproduced by killing non-local DN:
{code:java}
2021-07-07 16:50:41,361 WARN 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL: sync failed
org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
readAddress(..) failed: Connection reset by peer {code}
Recovers fine though. Sequence to lock up WAL must be more involved. Need more 
info.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-07 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17376684#comment-17376684
 ] 

Michael Stack commented on HBASE-26042:
---

Tried reproducing
{code:java}
2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
wal.AsyncFSWAL: sync 
failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
 readAddress(..) failed: Connection reset by peer  {code}
... by killing DNs. There is complaint about broken stream but then all gets 
neatly closed up and after log roll, all is good again.

Pausing the DN process got me these:
{code:java}
2021-07-07 14:44:38,471 WARN 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL: sync failed
java.io.IOException: Timeout(6ms) waiting for response
at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.lambda$userEventTriggered$4(FanOutOneBlockAsyncDFSOutput.java:300)
at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.failed(FanOutOneBlockAsyncDFSOutput.java:233)
at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.access$300(FanOutOneBlockAsyncDFSOutput.java:98)
at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.userEventTriggered(FanOutOneBlockAsyncDFSOutput.java:299)
at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:346)
at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:332)
at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:324)
at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:117)
at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.userEventTriggered(ByteToMessageDecoder.java:365)
at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:346)
at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:332)
at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:324)
at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelIdle(IdleStateHandler.java:371)
at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$ReaderIdleTimeoutTask.run(IdleStateHandler.java:504)
at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$AbstractIdleTask.run(IdleStateHandler.java:476)
at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.PromiseTask.runTask(PromiseTask.java:98)
at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.ScheduledFutureTask.run(ScheduledFutureTask.java:170)
at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)
at 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
at 
org.apache.hbase.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 {code}
...not the same as what I want. Here the exception has a nice stack trace and 
is being 'handled'.

Related, on another cluster, where fds were 64k and I'd loaded up a single RS 
w/ hundreds of Regions, I saw this:
{code:java}
2021-07-07 00:59:27,372 WARN 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline: An 
exceptionCaught() event was fired, and it reached at the tail of the pipeline. 
It usually means the last handler in the pipeline did not handle the 
exception.org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
 accept(..) failed: Too many open files {code}
Indeed lsof confirmed fds were at limit. This WARN looks a bit like the one I'm 
trying to deal w/ here w/ some nice extra info out of netty.

[~huaxiangsun] found this CASSANDRA-13649 which has similar looking WARNs from 
netty and accredits lack of an exception handler on the netty pipeline as 
reason for the WARNs showing in logs.

We don't have an exception handler on our server-side pipeline. Adding one that 
does cleanup might prevent 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-01 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17373103#comment-17373103
 ] 

Michael Stack commented on HBASE-26042:
---

Some background notes:

On this cluster, the hang is unusual. What we see more often is 'clean' RS 
aborts like below:
{code:java}
2021-06-26 12:18:11,725 ERROR [regionserver/XYZ:16020.logRoller] 
regionserver.HRegionServer: * ABORTING region server 
XYZ,16020,1622749552385: IOE in log roller *
org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
readAddress(..) failed: Connection reset by peer {code}
Looking in DN around this time, its this:
{code:java}
2021-06-26 12:18:18,210 ERROR [DataXceiver for client 
DFSClient_NONMAPREDUCE_588272444_1 at /A.B.C.D:28343 [Receiving block 
BP-1200783253-A.B.C.D1581116871410:blk_3413027507_2339403003]] 
datanode.DataNode: A.B.C.D:9866:DataXceiver error processing WRITE_BLOCK 
operation  src: /A.B.C.D:28343 dst: /A.B.C.D:9866
java.lang.NullPointerException
at 
sun.nio.ch.EPollArrayWrapper.isEventsHighKilled(EPollArrayWrapper.java:174) 
{code}
(In this particular case the DN restarted: i.e. 'Connection reset by peer' but 
it doesn't always cause a DN restart)

Poking around, it seems NPE when isEventsHighKilled means the DN doesn't have 
enough fds (and/or JDK bug). It has 16k. Its running about 2k fds when idle. 
Will try upping the fd count.

The DN NPE (or DN abort) looks to cause the RS to abort if it comes up in the 
RS around log roll. Elsewhere, perhaps, it causes the hang – hard to tell for 
sure.

Here is another example of an abort w/ corresponding DN NPE on 
isEventsHighKilled complaint.
{code:java}
2021-06-25 19:21:12,921 INFO  [DataStreamer for file 
/hbase/data/default/xxx/3ab32c85a5c46f466dc21ecbcff53f6f/.tmp/f/81484a889b0e4c11bfcc71736a023d29]
 hdfs.DataStreamer: Exception in createBlockOutputStream 
blk_3406122337_2332497742
java.io.EOFException: Unexpected EOF while trying to read response from server
at 
org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed(PBHelperClient.java:539)
at 
org.apache.hadoop.hdfs.DataStreamer.createBlockOutputStream(DataStreamer.java:1762)
at 
org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1679)
at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:716)
2021-06-25 19:21:12,922 WARN  [DataStreamer for file 
/hbase/data/default/xxx/3ab32c85a5c46f466dc21ecbcff53f6f/.tmp/f/81484a889b0e4c11bfcc71736a023d29]
 hdfs.DataStreamer: Abandoning 
BP-1200783253-A.B.C.240-1581116871410:blk_3406122337_2332497742
2021-06-25 19:21:12,924 WARN  [DataStreamer for file 
/hbase/data/default/xxx/3ab32c85a5c46f466dc21ecbcff53f6f/.tmp/f/81484a889b0e4c11bfcc71736a023d29]
 hdfs.DataStreamer: Excluding datanode 
DatanodeInfoWithStorage[17.58.122.105:9866,DS-1015be7f-b986-4853-b56f-280ac2e8db4a,DISK]
2021-06-25 19:21:12,924 INFO  [DataStreamer for file 
/hbase/data/default/xxx/3ab32c85a5c46f466dc21ecbcff53f6f/.tmp/f/81484a889b0e4c11bfcc71736a023d29]
 hdfs.DataStreamer: Removing node 
DatanodeInfoWithStorage[17.58.122.105:9866,DS-1015be7f-b986-4853-b56f-280ac2e8db4a,DISK]
 from the excluded nodes list
2021-06-25 19:21:13,493 INFO  [regionserver/ps1586:16020.leaseChecker] 
regionserver.RSRpcServices: Scanner -2032258505885651566 lease expired on 
region 
xxx,\x9D\xDC\x00\x06Nl\xC5\xDEH\xCB\xF2\xE9\xC9J\x05-,1615374707984.0857141d2bb68aa1acb5f543f2bb78bd.
2021-06-25 19:21:14,189 ERROR [regionserver/ps1586:16020.logRoller] 
regionserver.HRegionServer: * ABORTING region server 
ps1586.a.b.c.d,16020,1622767149688: IOE in log roller *
java.io.IOException: Connection to 17.58.122.105/17.58.122.105:9866 closed
at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.lambda$channelInactive$2(FanOutOneBlockAsyncDFSOutput.java:286)
at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.failed(FanOutOneBlockAsyncDFSOutput.java:233)
at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.access$300(FanOutOneBlockAsyncDFSOutput.java:98)
at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.channelInactive(FanOutOneBlockAsyncDFSOutput.java:285)
 {code}

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-01 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17373028#comment-17373028
 ] 

Michael Stack commented on HBASE-26042:
---

[~bharathv] thanks for taking a look.

I looked at that stack. I couldn't find in the thread dump what was going to 
unblock the above park.

The last thing I was looking at here was in AsyncFSWAL#syncFailed... I wonder 
if this a correct reset on faile:

{color:#660e7a}highestUnsyncedTxid {color}= 
{color:#660e7a}highestSyncedTxid{color}.get();

I should probably just work on trying to repro in the small. My attempt above 
used TestAsyncFSWAL but it uses mocks.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-07-01 Thread Bharath Vissapragada (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372926#comment-17372926
 ] 

Bharath Vissapragada commented on HBASE-26042:
--

Thanks for the jstacks, I think consume is not scheduled further because the 
sync is broken and that is not clearing up the ring buffer. To me the most 
suspicious stack is the following hung flush thread.

{noformat}
Thread 9464: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=175 (Compiled frame)
 - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, line=1707 
(Compiled frame)
 - 
java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
 @bci=119, line=3323 (Compiled frame)
 - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
line=1742 (Compiled frame)
 - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
frame)
 - 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
 @bci=16, line=189 (Compiled frame)
 - 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
 org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) @bci=9, 
line=202 (Compiled frame)
{noformat}

Given the state of this thread, I think the new writer instance (from the roll) 
is also broken for some reason and some of the callbacks (one of which is the 
future from the above thread) are not cleaned up correctly. I think there is 
some racy code in FanOutOneBlockAsyncDFSOutput especially around when a new 
writer is marked 'BROKEN' and a flush is called resulting in some 
waitingAckQueue members not being cleaned up correctly. Just a theory at this 
point, but probably easy to poke around with a heap dump or some related 
logging around these code paths.


> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-30 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372361#comment-17372361
 ] 

Michael Stack commented on HBASE-26042:
---

{quote}What is the ring buffer consume thread doing? If it could write out 
entries, finally we could release the slots of the ring buffer and move forward?
{quote}
I didn't take the thread dumps. They are w/o thread names. There doesn't seem 
to be any 'consume' going on. We skipped out scheduling a consume on 'sync 
failed'? Trying to figure it...

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-30 Thread Duo Zhang (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372298#comment-17372298
 ] 

Duo Zhang commented on HBASE-26042:
---

What is the ring buffer consume thread doing? If it could write out entries, 
finally we could release the slots of the ring buffer and move forward?

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, line=338 
> (Compiled frame)
>  - 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-30 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372295#comment-17372295
 ] 

Michael Stack commented on HBASE-26042:
---

Deadlock in coarse form is 116 of 200 handlers are in mvcc#begin of which 70 
are down on the ringbuffer waiting on minimum sequence to catch up.  86 
handlers are trying to do mvcc#complete but can't get in because more are in 
mvcc#begins holding the synchronize lock.

The log roll wouldn't fix this issue; even if it managed to write the WAL 
header (where it is currently stuck) it would then go into a wait on safe point 
before progressing w/ replacement..

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-30 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372272#comment-17372272
 ] 

Michael Stack commented on HBASE-26042:
---

Looking at thread dumps we seem to be stuck on the ring buffer. There are 415 
threads total, 200 of which are handlers. ~70 are stuck here:

 
{code:java}
6037 Thread 8667: (state = BLOCKED)
6038  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
may be imprecise)
6039  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, 
line=338 (Compiled frame)
6040  - com.lmax.disruptor.MultiProducerSequencer.next(int) @bci=82, line=136 
(Compiled frame)
6041  - com.lmax.disruptor.MultiProducerSequencer.next() @bci=2, line=105 
(Interpreted frame)
6042  - com.lmax.disruptor.RingBuffer.next() @bci=4, line=263 (Compiled frame)
6043  - 
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.lambda$stampSequenceIdAndPublishToRingBuffer$1(org.apache.commons.lang3.mutable.MutableLong,
 com.lmax.disruptor.RingBuffer) @bci=2, line=1031 (Compiled frame)
6044  - 
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL$$Lambda$270.run() @bci=8 
(Compiled frame)
6045  - 
org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.begin(java.lang.Runnable)
 @bci=36, line=140 (Interpreted frame)
6046  - 
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.stampSequenceIdAndPublishToRingBuffer(org.apache.hadoop.hbase.client.RegionInfo,
 org.apache.hadoop.hbase.wal.WALKeyImpl, org.apache.hadoop.hbase.wal.WALEdit, 
boolean, com.lma x.disruptor.RingBuffer) @bci=62, line=1030 (Compiled frame)
6047  - 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.append(org.apache.hadoop.hbase.client.RegionInfo,
 org.apache.hadoop.hbase.wal.WALKeyImpl, org.apache.hadoop.hbase.wal.WALEdit, 
boolean) @bci=10, line=589 (Compiled frame) {code}
 

waiting for a minimum sequence number to catch up 
[https://github.com/LMAX-Exchange/disruptor/blob/master/src/main/java/com/lmax/disruptor/MultiProducerSequencer.java#L111]

 

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
> Attachments: js1, js2
>
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-30 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372109#comment-17372109
 ] 

Michael Stack commented on HBASE-26042:
---

[~apurtell] My attempt at repro did not pan out unfortunately.

[~bharathv] coming...

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, line=338 
> (Compiled frame)
>  - com.lmax.disruptor.MultiProducerSequencer.next(int) @bci=82, line=136 
> (Compiled frame)

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-30 Thread Andrew Kyle Purtell (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372108#comment-17372108
 ] 

Andrew Kyle Purtell commented on HBASE-26042:
-

{quote}2021-06-29 20:33:35,183 WARN  
[AsyncFSWAL-0-hdfs://localhost.localdomain:37680/user/mstack/test-data/329a3111-d35d-18ae-9107-065eee2a4e62]
 wal.AsyncFSWAL(299): sync 
failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
 Injected(..) failed: Connection reset by peer{quote}

So this can be simulated by injecting an IOE? Is it possible to provide a unit 
test that repros the issue? Then we can investigate and fix in any dev 
environment...

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-30 Thread Bharath Vissapragada (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372080#comment-17372080
 ] 

Bharath Vissapragada commented on HBASE-26042:
--

Any chance you can attach a full jstack? Thanks.

[~vjasani] [~dmanning] [~apurtell] FYI.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, line=338 
> (Compiled frame)
>  - com.lmax.disruptor.MultiProducerSequencer.next(int) @bci=82, line=136 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-30 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17372049#comment-17372049
 ] 

Michael Stack commented on HBASE-26042:
---

[~zhangduo] thanks for taking  look and 'different regions' is probably why. 
Thanks.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, line=338 
> (Compiled frame)
>  - com.lmax.disruptor.MultiProducerSequencer.next(int) @bci=82, line=136 
> (Compiled frame)

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-29 Thread Duo Zhang (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17371814#comment-17371814
 ] 

Duo Zhang commented on HBASE-26042:
---

{quote}
Interesting is how more than one thread is able to be inside the synchronize 
block in mvcc#begin seemingly
{quote}

Different regions?

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result after 30 ms for txid=552128301, WAL system stuck? {code}
> Whats supposed to happen when other side goes away like this is that we will 
> roll the WAL – go set up a new one. You can see it happening if you run
> {code:java}
> mvn test 
> -Dtest=org.apache.hadoop.hbase.regionserver.wal.TestAsyncFSWAL#testBrokenWriter
>  {code}
> I tried hacking the test to repro the above hang by throwing same exception 
> in above test (on linux because need epoll to repro) but all just worked.
> Thread dumps of the hungup WAL subsystem are a little odd. The log roller is 
> stuck w/o timeout trying to write a long on the WAL header:
>  
> {code:java}
> Thread 9464: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - java.util.concurrent.CompletableFuture$Signaller.block() @bci=19, 
> line=1707 (Compiled frame)
>  - 
> java.util.concurrent.ForkJoinPool.managedBlock(java.util.concurrent.ForkJoinPool$ManagedBlocker)
>  @bci=119, line=3323 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.waitingGet(boolean) @bci=115, 
> line=1742 (Compiled frame)
>  - java.util.concurrent.CompletableFuture.get() @bci=11, line=1908 (Compiled 
> frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(java.util.function.Consumer)
>  @bci=16, line=189 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(byte[],
>  org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$WALHeader) 
> @bci=9, line=202 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(org.apache.hadoop.fs.FileSystem,
>  org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, boolean, 
> long) @bci=107, line=170 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(org.apache.hadoop.conf.Configuration,
>  org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, boolean, long, 
> org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup, java.lang.Class) 
> @bci=61, line=113 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=22, line=651 (Compiled frame)
>  - 
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(org.apache.hadoop.fs.Path)
>  @bci=2, line=128 (Compiled frame)
>  - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(boolean) 
> @bci=101, line=797 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(long) 
> @bci=18, line=263 (Compiled frame)
>  - org.apache.hadoop.hbase.wal.AbstractWALRoller.run() @bci=198, line=179 
> (Compiled frame) {code}
>  
> Other threads are BLOCKED trying to append the WAL w/ flush markers etc. 
> unable to add the ringbuffer:
>  
> {code:java}
> Thread 9465: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.parkNanos(long) @bci=11, line=338 
> (Compiled frame)
>  - 

[jira] [Commented] (HBASE-26042) WAL lockup on 'sync failed' org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: readAddress(..) failed: Connection reset by peer

2021-06-29 Thread Michael Stack (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17371796#comment-17371796
 ] 

Michael Stack commented on HBASE-26042:
---

Here is what it looked like when I tried to repro the hang-up in test env:
{code:java}
2021-06-29 20:33:35,183 WARN  
[AsyncFSWAL-0-hdfs://localhost.localdomain:37680/user/mstack/test-data/329a3111-d35d-18ae-9107-065eee2a4e62]
 wal.AsyncFSWAL(299): sync failed
org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
Injected(..) failed: Connection reset by peer
2021-06-29 20:33:35,184 DEBUG [LogRoller] wal.AbstractWALRoller(170): WAL  
wal:(num 1625024015181) roll requested {code}
See how we immediately follow the error with 'WAL wal:(num 1625024015181) roll 
requested '.

Looking at the thread dump, four threads are trying to write a flush marker to 
the WAL... three have made it inside mvcc#begin and are trying to stamp 
sequenceid (see thread dump above) while one other is here:
{code:java}
Thread 9466: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=175 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt() 
@bci=1, line=836 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(java.util.concurrent.locks.AbstractQueuedSynchronizer$Node,
 int) @bci=67, line=870 (Compiled frame)
 - java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(int) @bci=17, 
line=1199 (Compiled frame)
 - java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock() @bci=5, 
line=943 (Compiled frame)
 - 
org.apache.hadoop.hbase.regionserver.HRegion.internalPrepareFlushCache(org.apache.hadoop.hbase.wal.WAL,
 long, java.util.Collection, org.apache.hadoop.hbase.monitoring.MonitoredTask, 
boolean, org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker) @bci=306, 
line=2641 (Compiled frame)
 - 
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(org.apache.hadoop.hbase.wal.WAL,
 long, java.util.Collection, org.apache.hadoop.hbase.monitoring.MonitoredTask, 
boolean, org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker) @bci=11, 
line=2573 (Compiled frame)
 - 
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(java.util.Collection,
 org.apache.hadoop.hbase.monitoring.MonitoredTask, boolean, 
org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker) @bci=13, line=2547 
(Compiled frame)
 - org.apache.hadoop.hbase.regionserver.HRegion.flushcache(boolean, boolean, 
org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker) @bci=558, line=2436 
(Compiled frame)
 - 
org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(org.apache.hadoop.hbase.regionserver.HRegion,
 boolean, boolean, org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker) 
@bci=86, line=610 (Compiled frame)
 - 
org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushOneForGlobalPressure(org.apache.hadoop.hbase.regionserver.FlushType)
 @bci=738, line=288 (Compiled frame)
 - 
org.apache.hadoop.hbase.regionserver.MemStoreFlusher.access$700(org.apache.hadoop.hbase.regionserver.MemStoreFlusher,
 org.apache.hadoop.hbase.regionserver.FlushType) @bci=2, line=67 (Compiled 
frame)
 - org.apache.hadoop.hbase.regionserver.MemStoreFlusher$FlushHandler.run() 
@bci=128, line=344 (Compiled frame) {code}
Then there is the logroller BLOCKED as per above. Looks like a deadlock but I 
can't see it – yet anyways.

> WAL lockup on 'sync failed' 
> org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException: 
> readAddress(..) failed: Connection reset by peer
> 
>
> Key: HBASE-26042
> URL: https://issues.apache.org/jira/browse/HBASE-26042
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.3.5
>Reporter: Michael Stack
>Priority: Major
>
> Making note of issue seen in production cluster.
> Node had been struggling under load for a few days with slow syncs up to 10 
> seconds, a few STUCK MVCCs from which it recovered and some java pauses up to 
> three seconds in length.
> Then the below happened:
> {code:java}
> 2021-06-27 13:41:27,604 WARN  [AsyncFSWAL-0-hdfs://:8020/hbase] 
> wal.AsyncFSWAL: sync 
> failedorg.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeIoException:
>  readAddress(..) failed: Connection reset by peer {code}
> ... and WAL turned dead in the water. Scanners start expiring. RPC prints 
> text versions of requests complaining requestsTooSlow. Then we start to see 
> these:
> {code:java}
> org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to get sync 
> result