bharathv commented on a change in pull request #2071:
URL: https://github.com/apache/hbase/pull/2071#discussion_r455409660
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -493,12 +499,35 @@ private void
checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
}
}
- private void checkClusterKey(String clusterKey) throws DoNotRetryIOException
{
+ private void checkClusterKeyForHBaseInterClusterReplicationEndpoint(String
clusterKey)
+ throws DoNotRetryIOException {
try {
ZKConfig.validateClusterKey(clusterKey);
} catch (IOException e) {
throw new DoNotRetryIOException("Invalid cluster key: " + clusterKey, e);
}
+ String peerClusterId = "";
+ try {
+ Configuration otherConf = HBaseConfiguration.createClusterConf(conf,
clusterKey);
+ ZKWatcher zkWatcher =
+ new ZKWatcher(otherConf, this + "check-peer-cluster-key", new
Abortable() {
+ @Override public void abort(String why, Throwable e) {
+ }
+
+ @Override public boolean isAborted() {
+ return false;
+ }
+ });
+ peerClusterId = ZKClusterId.readClusterIdZNode(zkWatcher);
Review comment:
ReplicationPeerManager is colocated with master, right? you can use
master.getClusterId() without doing all this? (that is cached, so saves a ZK
round trip).
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -493,12 +499,35 @@ private void
checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
}
}
- private void checkClusterKey(String clusterKey) throws DoNotRetryIOException
{
+ private void checkClusterKeyForHBaseInterClusterReplicationEndpoint(String
clusterKey)
+ throws DoNotRetryIOException {
try {
ZKConfig.validateClusterKey(clusterKey);
} catch (IOException e) {
throw new DoNotRetryIOException("Invalid cluster key: " + clusterKey, e);
}
+ String peerClusterId = "";
Review comment:
You are fetching the currentClusterId right? Why name it as
peerClusterId? I think the argument to this method is peerClusterKey.
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -337,25 +347,21 @@ public void removeAllQueuesAndHFileRefs(String peerId)
throws ReplicationExcepti
private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws
DoNotRetryIOException {
String replicationEndpointImpl = peerConfig.getReplicationEndpointImpl();
- boolean checkClusterKey = true;
+ ReplicationEndpoint endpoint = null;
if (!StringUtils.isBlank(replicationEndpointImpl)) {
- // try creating a instance
- ReplicationEndpoint endpoint;
try {
+ // try creating a instance
endpoint = Class.forName(replicationEndpointImpl)
.asSubclass(ReplicationEndpoint.class).getDeclaredConstructor().newInstance();
} catch (Throwable e) {
throw new DoNotRetryIOException(
"Can not instantiate configured replication endpoint class=" +
replicationEndpointImpl,
e);
}
- // do not check cluster key if we are not
HBaseInterClusterReplicationEndpoint
- if (!(endpoint instanceof HBaseInterClusterReplicationEndpoint)) {
- checkClusterKey = false;
- }
}
- if (checkClusterKey) {
- checkClusterKey(peerConfig.getClusterKey());
+ // Default is HBaseInterClusterReplicationEndpoint and only it need to
check cluster key
+ if (endpoint == null || endpoint instanceof
HBaseInterClusterReplicationEndpoint) {
Review comment:
Replication#canReplicateToSameCluster() isn't it supposed to do this?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]