[
https://issues.apache.org/jira/browse/HBASE-29320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18009130#comment-18009130
]
haosen chen commented on HBASE-29320:
-------------------------------------
I think the problem is here. When the region completes initialization, a new
seqId is generated and set to the current writePoint
(this.mvcc.advanceTo(openSeqNum). It is also planned to use this seqId as the
seqId of the REGION_OPEN event. However, when the REGION_OPEN event is written
to the WAL (writeRegionOpenMarker), writePoint+1 is used as the final seqId,
which means that a seqId will be skipped.
{code:java}
this.openSeqNum = initialize(reporter);
this.mvcc.advanceTo(openSeqNum);
// The openSeqNum must be increased every time when a region is assigned, as we
rely on it to
// determine whether a region has been successfully reopened. So here we always
write open
// marker, even if the table is read only.
if (wal != null && getRegionServerServices() != null &&
RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
writeRegionOpenMarker(wal, openSeqNum);
} {code}
{code:java}
public WriteEntry begin(Runnable action) {
synchronized (writeQueue) {
long nextWriteNumber = writePoint.incrementAndGet();
WriteEntry e = new WriteEntry(nextWriteNumber);
writeQueue.add(e);
action.run();
return e;
}
} {code}
> Serial replication blocking in SerialReplicationChecker#waitUntilCanPush
> ------------------------------------------------------------------------
>
> Key: HBASE-29320
> URL: https://issues.apache.org/jira/browse/HBASE-29320
> Project: HBase
> Issue Type: Bug
> Components: Replication
> Affects Versions: 2.6.2
> Environment: Cluster A version 2.2.7, Cluster B version 2.6.1
> B -> A 's replication is enabled
> Reporter: Longping Jie
> Priority: Major
> Attachments: 9398.jstack, image-2025-05-16-09-50-32-605.png,
> image-2025-05-16-09-52-25-251.png
>
>
> We have two hbase clusters that enable replciation and set up serial
> replication.
> Cluster A version 2.2.7, Cluster B version 2.6.1
> It is normal for a long time after replication is enabled, After that, there
> is no sign, B -> A's replication source queue blocked, As shown
> !image-2025-05-16-09-50-32-605.png|width=986,height=454!
> I randomly select a node to see the delay of replication,As show
> !image-2025-05-16-09-52-25-251.png|width=993,height=190!
> Blocking code location, As follows the stack:
> {code:java}
> // code placeholder
> "regionserver/hbase-10:16020.replicationSource.shipperhbase-10%2C16020%2C1747312708324,peerId"
> #627 daemon prio=5 os_prio=0 tid=0x00007f9400b8a800 nid=0x2cf8 waiting on
> condition [0x00007f44470d9000]
> java.lang.Thread.State: TIMED_WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00007f89fff3e820> (a
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
> at
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
> at
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.poll(ReplicationSourceWALReader.java:313)
> at
> org.apache.hadoop.hbase.replication.regionserver.SerialReplicationSourceWALReader.poll(SerialReplicationSourceWALReader.java:35)
> at
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.run(ReplicationSourceShipper.java:109)
> "regionserver/hbase-10:16020.replicationSource.wal-reader.hbase-10%2C16020%2C1747312708324,newHbase227"
> #628 daemon prio=5 os_prio=0 tid=0x00007f9400b8c800 nid=0x2cf6 waiting on
> condition [0x00007f44473da000]
> java.lang.Thread.State: TIMED_WAITING (sleeping)
> at java.lang.Thread.sleep(Native Method)
> at
> org.apache.hadoop.hbase.replication.regionserver.SerialReplicationChecker.waitUntilCanPush(SerialReplicationChecker.java:270)
> at
> org.apache.hadoop.hbase.replication.regionserver.SerialReplicationSourceWALReader.readWALEntries(SerialReplicationSourceWALReader.java:89)
> at
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:177)
> at
> org.apache.hadoop.hbase.replication.regionserver.SerialReplicationSourceWALReader.run(SerialReplicationSourceWALReader.java:35)
> "AsyncFSWAL-0-hdfs://coreHBaseProdHa/hbase-prefix:hbase-10,16020,1747312708324"
> #626 daemon prio=5 os_prio=0 tid=0x00007f9400b87000 nid=0x2cf5 waiting on
> condition [0x00007f44474db000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00007f89ffe20038> (a
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
> at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
> at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748){code}
>
> Open the debug log for SerialReplicationChecker class,The logs are as follows:
> 2025-05-15T21:53:29,978 DEBUG
> [regionserver/hbase-10:16020.replicationSource.wal-reader.hbase-10%2C16020%2C1747312708324,peerId]
> regionserver.SerialReplicationChecker: Replication barrier for
> ad-instation/e3fdadb00255826881021b3baf97e976/1590=[#edits: 0 = <>]:
> ReplicationBarrierResult [barriers=[1585, 1589], state=OPEN,
> parentRegionNames=]
> 2025-05-15T21:53:29,979 DEBUG
> [regionserver/hbase-10:16020.replicationSource.wal-reader.hbase-10%2C16020%2C1747312708324,peerId]
> regionserver.SerialReplicationChecker: Previous range for
> ad-instation/e3fdadb00255826881021b3baf97e976/1590=[#edits: 0 = <>] has not
> been finished yet, give up
> 2025-05-15T21:53:29,979 DEBUG
> [regionserver/hbase-10:16020.replicationSource.wal-reader.hbase-10%2C16020%2C1747312708324,peerId]
> regionserver.SerialReplicationChecker: Can not push
> ad-instation/e3fdadb00255826881021b3baf97e976/1590=[#edits: 0 = <>], wait
>
> I don't know why, replication keeps getting stuck inside
> SerialReplicationChecker#waitUntilCanPush
> {code:java}
> // code placeholder
> public void waitUntilCanPush(Entry entry, Cell firstCellInEdit)
> throws IOException, InterruptedException {
> byte[] row = CellUtil.cloneRow(firstCellInEdit);
> while (!canPush(entry, row)) {
> LOG.debug("Can not push {}, wait", entry);
> Thread.sleep(waitTimeMs);
> }
> } {code}
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)