[
https://issues.apache.org/jira/browse/HBASE-24716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sandeep Pal updated HBASE-24716:
--------------------------------
Description:
[listPeerConfigs()|https://github.com/apache/hbase/blob/branch-1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java#L295]
for getting the list of peers along with their configuration is not a reliable
API.
It is not very robust to errors, logs FATAL and swallows the
[exceptions|[https://github.com/apache/hbase/blob/branch-1/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java#L254]
Snippet:
catch (KeeperException e) {
this.abortable.abort("Cannot get the list of peers ", e);
} catch (ReplicationException e) {
this.abortable.abort("Cannot get the list of peers ", e);
}
return peers;
The abortable (connection in this case) also doesn't abort the region server
and just logs. This makes upstream believe that there is nothing wrong and
proceed without any action which is not good.
{code:java}
2020-07-07 23:11:37,857 FATAL [14774961,peer_id]
client.ConnectionManager$HConnectionImplementation - Cannot get the list of
peersorg.apache.zookeeper.KeeperException$SessionExpiredException:
KeeperErrorCode = Session expired for /hbase/replication/peersat
org.apache.zookeeper.KeeperException.create(KeeperException.java:130)at
org.apache.zookeeper.KeeperException.create(KeeperException.java:54)at
org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1549)at
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.getChildren(RecoverableZooKeeper.java:312)at
org.apache.hadoop.hbase.zookeeper.ZKUtil.listChildrenNoWatch(ZKUtil.java:513)at
org.apache.hadoop.hbase.replication.ReplicationPeersZKImpl.getAllPeerConfigs(ReplicationPeersZKImpl.java:249)at
org.apache.hadoop.hbase.client.replication.ReplicationAdmin.listPeerConfigs(ReplicationAdmin.java:332)
{code}
was:
[listPeerConfigs()|[https://git.soma.salesforce.com/bigdata-packaging/hbase/blob/1.6.0-sfdc-1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java#L295]]
for getting the list of peers along with their configuration is not a reliable
API.
It is not very robust to errors, logs FATAL and swallows the
[exceptions|[https://github.com/apache/hbase/blob/branch-1/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java#L254]]
Snippet:
catch (KeeperException e) {
this.abortable.abort("Cannot get the list of peers ", e);
} catch (ReplicationException e) {
this.abortable.abort("Cannot get the list of peers ", e);
}
return peers;
The abortable (connection in this case) also doesn't abort the region server
and just logs. This makes upstream believe that there is nothing wrong and
proceed without any action which is not good.
{code:java}
2020-07-07 23:11:37,857 FATAL [14774961,peer_id]
client.ConnectionManager$HConnectionImplementation - Cannot get the list of
peersorg.apache.zookeeper.KeeperException$SessionExpiredException:
KeeperErrorCode = Session expired for /hbase/replication/peersat
org.apache.zookeeper.KeeperException.create(KeeperException.java:130)at
org.apache.zookeeper.KeeperException.create(KeeperException.java:54)at
org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1549)at
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.getChildren(RecoverableZooKeeper.java:312)at
org.apache.hadoop.hbase.zookeeper.ZKUtil.listChildrenNoWatch(ZKUtil.java:513)at
org.apache.hadoop.hbase.replication.ReplicationPeersZKImpl.getAllPeerConfigs(ReplicationPeersZKImpl.java:249)at
org.apache.hadoop.hbase.client.replication.ReplicationAdmin.listPeerConfigs(ReplicationAdmin.java:332)
{code}
> Do the error handling for replication admin failures
> ----------------------------------------------------
>
> Key: HBASE-24716
> URL: https://issues.apache.org/jira/browse/HBASE-24716
> Project: HBase
> Issue Type: Improvement
> Components: Replication
> Reporter: Sandeep Pal
> Assignee: Sandeep Pal
> Priority: Major
>
> [listPeerConfigs()|https://github.com/apache/hbase/blob/branch-1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java#L295]
> for getting the list of peers along with their configuration is not a
> reliable API.
> It is not very robust to errors, logs FATAL and swallows the
> [exceptions|[https://github.com/apache/hbase/blob/branch-1/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java#L254]
>
> Snippet:
> catch (KeeperException e) {
> this.abortable.abort("Cannot get the list of peers ", e);
> } catch (ReplicationException e) {
> this.abortable.abort("Cannot get the list of peers ", e);
> }
> return peers;
>
> The abortable (connection in this case) also doesn't abort the region server
> and just logs. This makes upstream believe that there is nothing wrong and
> proceed without any action which is not good.
>
>
> {code:java}
> 2020-07-07 23:11:37,857 FATAL [14774961,peer_id]
> client.ConnectionManager$HConnectionImplementation - Cannot get the list of
> peersorg.apache.zookeeper.KeeperException$SessionExpiredException:
> KeeperErrorCode = Session expired for /hbase/replication/peersat
> org.apache.zookeeper.KeeperException.create(KeeperException.java:130)at
> org.apache.zookeeper.KeeperException.create(KeeperException.java:54)at
> org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1549)at
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.getChildren(RecoverableZooKeeper.java:312)at
>
> org.apache.hadoop.hbase.zookeeper.ZKUtil.listChildrenNoWatch(ZKUtil.java:513)at
>
> org.apache.hadoop.hbase.replication.ReplicationPeersZKImpl.getAllPeerConfigs(ReplicationPeersZKImpl.java:249)at
>
> org.apache.hadoop.hbase.client.replication.ReplicationAdmin.listPeerConfigs(ReplicationAdmin.java:332)
> {code}
>
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)