[
https://issues.apache.org/jira/browse/HBASE-15769?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15273674#comment-15273674
]
Ashish Singhi commented on HBASE-15769:
---------------------------------------
In this case, peer entry will be added into ZK but internally we fail to create
a ReplicationSource for this peer cluster, so there will be no wal entry added
into ZK and no data will be replicated to peer cluster.
Below is the exception we get when we try to create a source for this peer
cluster,
{noformat}
2016-05-06 13:48:17,367 ERROR [main-EventThread]
regionserver.ReplicationSourceManager: Error while adding a new peer
java.io.IOException: java.lang.reflect.InvocationTargetException
at
org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:240)
at
org.apache.hadoop.hbase.client.ConnectionManager.createConnection(ConnectionManager.java:442)
at
org.apache.hadoop.hbase.client.ConnectionManager.createConnectionInternal(ConnectionManager.java:351)
at
org.apache.hadoop.hbase.client.HConnectionManager.createConnection(HConnectionManager.java:144)
at
org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint.init(HBaseInterClusterReplicationEndpoint.java:116)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.getReplicationSource(ReplicationSourceManager.java:432)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.addSource(ReplicationSourceManager.java:257)
at
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.peerListChanged(ReplicationSourceManager.java:526)
at
org.apache.hadoop.hbase.replication.ReplicationTrackerZKImpl$PeersWatcher.nodeChildrenChanged(ReplicationTrackerZKImpl.java:187)
at
org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:633)
at
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:566)
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
at
org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:238)
... 11 more
Caused by: java.lang.IllegalArgumentException: Path must start with / character
at org.apache.zookeeper.common.PathUtils.validatePath(PathUtils.java:51)
at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1630)
at
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:221)
at org.apache.hadoop.hbase.zookeeper.ZKUtil.checkExists(ZKUtil.java:544)
at
org.apache.hadoop.hbase.zookeeper.ZKClusterId.readClusterIdZNode(ZKClusterId.java:65)
at
org.apache.hadoop.hbase.client.ZooKeeperRegistry.getClusterId(ZooKeeperRegistry.java:105)
at
org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.retrieveClusterId(ConnectionManager.java:901)
at
org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.<init>(ConnectionManager.java:657)
{noformat}
bq. Perform validation on cluster key for add_peer
+1
> Perform validation on cluster key for add_peer
> ----------------------------------------------
>
> Key: HBASE-15769
> URL: https://issues.apache.org/jira/browse/HBASE-15769
> Project: HBase
> Issue Type: Bug
> Reporter: Ted Yu
> Priority: Minor
>
> [~pateljaymin] reported the following.
> In hbase cluster, adding a peer for replication, incorrectly defined
> zookeeper znode does not fail new peer creation. However, replication breaks
> down.
> For example,
> below string incorrectly defines zookeeper znode as "hbase" instead of
> "/hbase"...
> {code}
> > add_peer '1', "host1.com,host2.com,host3.com:2181:hbase
> {code}
> This left replication broken, even after we dropped and added a new correct
> entry for the peer.
> It wasn't until we bounced the cluster that the issue was fixed.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)