[
https://issues.apache.org/jira/browse/HBASE-28184?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17784216#comment-17784216
]
Rushabh Shah commented on HBASE-28184:
--------------------------------------
There is one test failure
TestBasicWALEntryStreamFSHLog#testCleanClosedWALs in branch-2.5 (and in
branch-2.4 also I assume). Don' know why I didn't see in the jenkins build
report for branch-2.5 PR.
It fails with this error:
{noformat}
java.lang.AssertionError:
Expected :0
Actual :1
<Click to see difference>
{noformat}
I can see the following exception:
{noformat}
2023-11-08T13:50:40,578 DEBUG [Listener at localhost/50204]
wal.ProtobufLogReader(442): EOF at position 210
2023-11-08T13:50:40,791 DEBUG [Listener at localhost/50204]
wal.ProtobufLogReader(447): Encountered a malformed edit, seeking back to last
good position in file, from 218 to 210
java.io.EOFException: Invalid PB, EOF? Ignoring; originalPosition=210,
currentPosition=218
at
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.readNext(ProtobufLogReader.java:376)
~[classes/:?]
at
org.apache.hadoop.hbase.regionserver.wal.ReaderBase.next(ReaderBase.java:104)
~[classes/:?]
at
org.apache.hadoop.hbase.regionserver.wal.ReaderBase.next(ReaderBase.java:92)
~[classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.readNextEntryAndRecordReaderPosition(WALEntryStream.java:259)
~[classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:181)
~[classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:102)
~[classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.peek(WALEntryStream.java:111)
~[classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.next(WALEntryStream.java:118)
~[classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.WALEntryStreamTestBase$WALEntryStreamWithRetries.access$001(WALEntryStreamTestBase.java:82)
~[test-classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.WALEntryStreamTestBase$WALEntryStreamWithRetries.lambda$next$0(WALEntryStreamTestBase.java:95)
~[test-classes/:?]
at org.apache.hadoop.hbase.Waiter.waitFor(Waiter.java:184)
~[test-classes/:?]
at org.apache.hadoop.hbase.Waiter.waitFor(Waiter.java:135)
~[test-classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.WALEntryStreamTestBase$WALEntryStreamWithRetries.next(WALEntryStreamTestBase.java:94)
~[test-classes/:?]
at
org.apache.hadoop.hbase.replication.regionserver.TestBasicWALEntryStream.testCleanClosedWALs(TestBasicWALEntryStream.java:726)
~[test-classes/:?]
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
~[?:1.8.0_292]
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
~[?:1.8.0_292]
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
~[?:1.8.0_292]
at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_292]
at
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
~[junit-4.13.2.jar:4.13.2]
at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
~[junit-4.13.2.jar:4.13.2]
at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
~[junit-4.13.2.jar:4.13.2]
at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
~[junit-4.13.2.jar:4.13.2]
at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
~[junit-4.13.2.jar:4.13.2]
at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
~[junit-4.13.2.jar:4.13.2]
at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
~[junit-4.13.2.jar:4.13.2]
at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
~[junit-4.13.2.jar:4.13.2]
at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299)
~[junit-4.13.2.jar:4.13.2]
at
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293)
~[junit-4.13.2.jar:4.13.2]
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
~[?:1.8.0_292]
at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_292]
Caused by:
org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException:
Message missing required fields: encoded_region_name, table_name,
log_sequence_number, write_time
at
org.apache.hbase.thirdparty.com.google.protobuf.UninitializedMessageException.asInvalidProtocolBufferException(UninitializedMessageException.java:79)
~[hbase-shaded-protobuf-4.1.5.jar:4.1.5]
at
org.apache.hbase.thirdparty.com.google.protobuf.AbstractParser.checkMessageInitialized(AbstractParser.java:68)
~[hbase-shaded-protobuf-4.1.5.jar:4.1.5]
at
org.apache.hbase.thirdparty.com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:232)
~[hbase-shaded-protobuf-4.1.5.jar:4.1.5]
at
org.apache.hbase.thirdparty.com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:237)
~[hbase-shaded-protobuf-4.1.5.jar:4.1.5]
at
org.apache.hbase.thirdparty.com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:48)
~[hbase-shaded-protobuf-4.1.5.jar:4.1.5]
at
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseDelimitedFrom(ProtobufUtil.java:3610)
~[classes/:?]
at
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.readNext(ProtobufLogReader.java:370)
~[classes/:?]
... 40 more
{noformat}
I am running hbase-2.5 branch with hadoop-3.0.
FYI [~zhangduo] [~haxiaolin]
> Tailing the WAL is very slow if there are multiple peers.
> ---------------------------------------------------------
>
> Key: HBASE-28184
> URL: https://issues.apache.org/jira/browse/HBASE-28184
> Project: HBase
> Issue Type: Bug
> Components: Replication
> Affects Versions: 2.0.0
> Reporter: Rushabh Shah
> Assignee: Rushabh Shah
> Priority: Major
> Fix For: 2.6.0, 2.4.18, 3.0.0-beta-1, 2.5.7
>
>
> Noticed in one of our production clusters which has 4 peers.
> Due to sudden ingestion of data, the size of log queue increased to a peak of
> 506. We have configured log roll size to 256 MB. Most of the edits in the WAL
> were from a table for which replication is disabled.
> So all ReplicationSourceWALReader thread had to do was to replay the WAL and
> NOT replicate them. Still it took 12 hours to drain the queue.
> Took few jstacks and found that ReplicationSourceWALReader was waiting to
> acquire rollWriterLock
> [here|https://github.com/apache/hbase/blob/branch-2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java#L1231]
> {noformat}
> "regionserver/<rs>,1" #1036 daemon prio=5 os_prio=0 tid=0x00007f44b374e800
> nid=0xbd7f waiting on condition [0x00007f37b4d19000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00007f3897a3e150> (a
> java.util.concurrent.locks.ReentrantLock$FairSync)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:837)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:872)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1202)
> at
> java.util.concurrent.locks.ReentrantLock$FairSync.lock(ReentrantLock.java:228)
> at
> java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290)
> at
> org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.getLogFileSizeIfBeingWritten(AbstractFSWAL.java:1102)
> at
> org.apache.hadoop.hbase.wal.WALProvider.lambda$null$0(WALProvider.java:128)
> at
> org.apache.hadoop.hbase.wal.WALProvider$$Lambda$177/1119730685.apply(Unknown
> Source)
> at
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
> at
> java.util.ArrayList$ArrayListSpliterator.tryAdvance(ArrayList.java:1361)
> at
> java.util.stream.ReferencePipeline.forEachWithCancel(ReferencePipeline.java:126)
> at
> java.util.stream.AbstractPipeline.copyIntoWithCancel(AbstractPipeline.java:499)
> at
> java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:486)
> at
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
> at
> java.util.stream.FindOps$FindOp.evaluateSequential(FindOps.java:152)
> at
> java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
> at
> java.util.stream.ReferencePipeline.findAny(ReferencePipeline.java:536)
> at
> org.apache.hadoop.hbase.wal.WALProvider.lambda$getWALFileLengthProvider$2(WALProvider.java:129)
> at
> org.apache.hadoop.hbase.wal.WALProvider$$Lambda$140/1246380717.getLogFileSizeIfBeingWritten(Unknown
> Source)
> at
> org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.readNextEntryAndRecordReaderPosition(WALEntryStream.java:260)
> at
> org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:172)
> at
> org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:101)
> at
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:222)
> at
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:157)
> {noformat}
> All the peers will contend for this lock during every batch read.
> Look at the code snippet below. We are guarding this section with
> rollWriterLock if we are replicating the active WAL file. But in our case we
> are NOT replicating active WAL file but still we acquire this lock only to
> return OptionalLong.empty();
> {noformat}
> /**
> * if the given {@code path} is being written currently, then return its
> length.
> * <p>
> * This is used by replication to prevent replicating unacked log entries.
> See
> * https://issues.apache.org/jira/browse/HBASE-14004 for more details.
> */
> @Override
> public OptionalLong getLogFileSizeIfBeingWritten(Path path) {
> rollWriterLock.lock();
> try {
> ...
> ...
> } finally {
> rollWriterLock.unlock();
> }
> {noformat}
> We can check the size of log queue and if it is greater than 1 then we can
> return early without acquiring the lock.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)