[
https://issues.apache.org/jira/browse/HBASE-26042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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://XXXX: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 300000 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)
> - com.lmax.disruptor.MultiProducerSequencer.next() @bci=2, line=105
> (Interpreted frame)
> - com.lmax.disruptor.RingBuffer.next() @bci=4, line=263 (Compiled frame)
> -
> 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)
> - org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL$$Lambda$270.run()
> @bci=8 (Compiled frame)
> -
> org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.begin(java.lang.Runnable)
> @bci=36, line=140 (Interpreted frame)
> -
> 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.lmax.disruptor.RingBuffer) @bci=62, line=1030 (Compiled frame)
> -
> 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)
> -
> org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.appendMarker(org.apache.hadoop.hbase.client.RegionInfo,
> org.apache.hadoop.hbase.wal.WALKeyImpl, org.apache.hadoop.hbase.wal.WALEdit)
> @bci=5, line=1081 (Compiled frame)
> -
> org.apache.hadoop.hbase.regionserver.wal.WALUtil.doFullMarkerAppendTransaction(org.apache.hadoop.hbase.wal.WAL,
> java.util.NavigableMap, org.apache.hadoop.hbase.client.RegionInfo,
> org.apache.hadoop.hbase.wal.WALEdit,
> org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl,
> java.util.Map, boolean) @bci=39, line=161 (Compiled frame)
> -
> org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeFlushMarker(org.apache.hadoop.hbase.wal.WAL,
> java.util.NavigableMap, org.apache.hadoop.hbase.client.RegionInfo,
> org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos$FlushDescriptor,
> boolean, org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl)
> @bci=13, line=89 (Compiled frame)
> {code}
>
> Interesting is how more than one thread is able to be inside the synchronize
> block in mvcc#begin seemingly....
> In
--
This message was sent by Atlassian Jira
(v8.3.4#803005)