[ 
https://issues.apache.org/jira/browse/HBASE-23008?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zheng Wang updated HBASE-23008:
-------------------------------
    Description: 
My live cluster config master-master replication,and only one is used to put 
data,as active cluster.
 Recently ,i find there are a great many znode in 
/hbase/replication/rs/#server/#peer in backup cluster,at least 10000+.
  
 I think the reason is , the wal entry in backup cluster are filtered by 
ClusterMarkingEntryFilter totaly, so ReplicationSourceWALReader will not put 
any data to entryBatchQueue,and ReplicationSourceShipper always blocked at 
entryReader.take(),it has no chance to delete hlog znode.

The thread stack of walReader and walShiper is below:
{code:java}
"main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2.replicationSource.wal-reader.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
 #157238 daemon prio=5 os_prio=0 tid=0x00007f7634be8800 nid=0x377ef waiting on 
condition 
[0x00007f6114c0e000]"main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2.replicationSource.wal-reader.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
 #157238 daemon prio=5 os_prio=0 tid=0x00007f7634be8800 nid=0x377ef waiting on 
condition [0x00007f6114c0e000]   java.lang.Thread.State: TIMED_WAITING 
(sleeping) at java.lang.Thread.sleep(Native Method) at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.handleEmptyWALEntryBatch(ReplicationSourceWALReader.java:192)
 at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:142)
"main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
 #157237 daemon prio=5 os_prio=0 tid=0x00007f76350b0000 nid=0x377ee waiting on 
condition [0x00007f6108173000]   java.lang.Thread.State: WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for  <0x00007f6f99bb6718> 
(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 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.take(ReplicationSourceWALReader.java:248)
 at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.run(ReplicationSourceShipper.java:108)
{code}

  was:
My live cluster config master-master replication,and only one is used to put 
data,as active cluster.
Recently ,i find there are a great many znode in 
/hbase/replication/rs/#host/#peer in backup cluster,at least 10000+.
 
I think the reason is , the wal entry in backup cluster are filtered by 
ClusterMarkingEntryFilter totaly, so ReplicationSourceWALReader will not put 
any data to entryBatchQueue,and ReplicationSourceShipper always blocked at 
entryReader.take(),it has no chance to delete hlog znode.


The thread stack of walReader and walShiper is below:
{code:java}
"main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2.replicationSource.wal-reader.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
 #157238 daemon prio=5 os_prio=0 tid=0x00007f7634be8800 nid=0x377ef waiting on 
condition 
[0x00007f6114c0e000]"main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2.replicationSource.wal-reader.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
 #157238 daemon prio=5 os_prio=0 tid=0x00007f7634be8800 nid=0x377ef waiting on 
condition [0x00007f6114c0e000]   java.lang.Thread.State: TIMED_WAITING 
(sleeping) at java.lang.Thread.sleep(Native Method) at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.handleEmptyWALEntryBatch(ReplicationSourceWALReader.java:192)
 at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:142)
"main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
 #157237 daemon prio=5 os_prio=0 tid=0x00007f76350b0000 nid=0x377ee waiting on 
condition [0x00007f6108173000]   java.lang.Thread.State: WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for  <0x00007f6f99bb6718> 
(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 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.take(ReplicationSourceWALReader.java:248)
 at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.run(ReplicationSourceShipper.java:108)
{code}


> ReplicationSourceShipper has no chance to delete hlog znode when the wal 
> entry batch always empty
> -------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-23008
>                 URL: https://issues.apache.org/jira/browse/HBASE-23008
>             Project: HBase
>          Issue Type: Bug
>          Components: Replication
>    Affects Versions: 2.0.0
>            Reporter: Zheng Wang
>            Priority: Major
>
> My live cluster config master-master replication,and only one is used to put 
> data,as active cluster.
>  Recently ,i find there are a great many znode in 
> /hbase/replication/rs/#server/#peer in backup cluster,at least 10000+.
>   
>  I think the reason is , the wal entry in backup cluster are filtered by 
> ClusterMarkingEntryFilter totaly, so ReplicationSourceWALReader will not put 
> any data to entryBatchQueue,and ReplicationSourceShipper always blocked at 
> entryReader.take(),it has no chance to delete hlog znode.
> The thread stack of walReader and walShiper is below:
> {code:java}
> "main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2.replicationSource.wal-reader.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
>  #157238 daemon prio=5 os_prio=0 tid=0x00007f7634be8800 nid=0x377ef waiting 
> on condition 
> [0x00007f6114c0e000]"main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2.replicationSource.wal-reader.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
>  #157238 daemon prio=5 os_prio=0 tid=0x00007f7634be8800 nid=0x377ef waiting 
> on condition [0x00007f6114c0e000]   java.lang.Thread.State: TIMED_WAITING 
> (sleeping) at java.lang.Thread.sleep(Native Method) at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.handleEmptyWALEntryBatch(ReplicationSourceWALReader.java:192)
>  at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:142)
> "main-EventThread.replicationSource,2.replicationSource.bj1-203-centos17%2C16020%2C1567586932902.bj1-203-centos17%2C16020%2C1567586932902.regiongroup-0,2"
>  #157237 daemon prio=5 os_prio=0 tid=0x00007f76350b0000 nid=0x377ee waiting 
> on condition [0x00007f6108173000]   java.lang.Thread.State: WAITING (parking) 
> at sun.misc.Unsafe.park(Native Method) - parking to wait for  
> <0x00007f6f99bb6718> (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 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.take(ReplicationSourceWALReader.java:248)
>  at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceShipper.run(ReplicationSourceShipper.java:108)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to