HBASE-15393 Enable table replication command will fail when parent znode is not default in peer cluster (Ashish Singhi)
This commit is contained in:
parent
27cf0c8c36
commit
d083e4f29f
|
@ -18,6 +18,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client.replication;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -54,11 +57,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
|
@ -595,7 +593,7 @@ public class ReplicationAdmin implements Closeable {
|
|||
*/
|
||||
private void checkAndSyncTableDescToPeers(final TableName tableName, final byte[][] splits)
|
||||
throws IOException {
|
||||
List<ReplicationPeer> repPeers = listValidReplicationPeers();
|
||||
List<ReplicationPeer> repPeers = listReplicationPeers();
|
||||
if (repPeers == null || repPeers.size() <= 0) {
|
||||
throw new IllegalArgumentException("Found no peer cluster for replication.");
|
||||
}
|
||||
|
@ -636,46 +634,29 @@ public class ReplicationAdmin implements Closeable {
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
List<ReplicationPeer> listValidReplicationPeers() {
|
||||
List<ReplicationPeer> listReplicationPeers() {
|
||||
Map<String, ReplicationPeerConfig> peers = listPeerConfigs();
|
||||
if (peers == null || peers.size() <= 0) {
|
||||
return null;
|
||||
}
|
||||
List<ReplicationPeer> validPeers = new ArrayList<ReplicationPeer>(peers.size());
|
||||
List<ReplicationPeer> listOfPeers = new ArrayList<ReplicationPeer>(peers.size());
|
||||
for (Entry<String, ReplicationPeerConfig> peerEntry : peers.entrySet()) {
|
||||
String peerId = peerEntry.getKey();
|
||||
Stat s = null;
|
||||
try {
|
||||
Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
|
||||
Configuration peerConf = pair.getSecond();
|
||||
ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst(),
|
||||
parseTableCFsFromConfig(this.getPeerTableCFs(peerId)));
|
||||
s =
|
||||
zkw.getRecoverableZooKeeper().exists(peerConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT),
|
||||
null);
|
||||
if (null == s) {
|
||||
LOG.info(peerId + ' ' + pair.getFirst().getClusterKey() + " is invalid now.");
|
||||
continue;
|
||||
}
|
||||
validPeers.add(peer);
|
||||
listOfPeers.add(peer);
|
||||
} catch (ReplicationException e) {
|
||||
LOG.warn("Failed to get valid replication peers. "
|
||||
+ "Error connecting to peer cluster with peerId=" + peerId);
|
||||
+ "Error connecting to peer cluster with peerId=" + peerId + ". Error message="
|
||||
+ e.getMessage());
|
||||
LOG.debug("Failure details to get valid replication peers.", e);
|
||||
continue;
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("Failed to get valid replication peers. KeeperException code="
|
||||
+ e.code().intValue());
|
||||
LOG.debug("Failure details to get valid replication peers.", e);
|
||||
continue;
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("Failed to get valid replication peers due to InterruptedException.");
|
||||
LOG.debug("Failure details to get valid replication peers.", e);
|
||||
Thread.currentThread().interrupt();
|
||||
continue;
|
||||
}
|
||||
}
|
||||
return validPeers;
|
||||
return listOfPeers;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -136,7 +136,7 @@ public class TestReplicationAdmin {
|
|||
config.getConfiguration().put("key2", "value2");
|
||||
admin.addPeer(ID_ONE, config, null);
|
||||
|
||||
List<ReplicationPeer> peers = admin.listValidReplicationPeers();
|
||||
List<ReplicationPeer> peers = admin.listReplicationPeers();
|
||||
assertEquals(1, peers.size());
|
||||
ReplicationPeer peerOne = peers.get(0);
|
||||
assertNotNull(peerOne);
|
||||
|
|
Loading…
Reference in New Issue