You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/03/05 15:28:20 UTC

hbase git commit: HBASE-15393 Enable table replication command will fail when parent znode is not default in peer cluster (Ashish Singhi)

Repository: hbase
Updated Branches:
  refs/heads/master 27cf0c8c3 -> d083e4f29


HBASE-15393 Enable table replication command will fail when parent znode is not default in peer cluster (Ashish Singhi)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d083e4f2
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d083e4f2
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d083e4f2

Branch: refs/heads/master
Commit: d083e4f29f5ae5698be7cce8414a2c97b6800af4
Parents: 27cf0c8
Author: tedyu <yu...@gmail.com>
Authored: Sat Mar 5 06:28:09 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Sat Mar 5 06:28:09 2016 -0800

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java    | 39 +++++---------------
 .../replication/TestReplicationAdmin.java       |  2 +-
 2 files changed, 11 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d083e4f2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index c2e7489..dcf1957 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -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);
-        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.");
+            + "Error connecting to peer cluster with peerId=" + peerId + ". Error message="
+            + e.getMessage());
         LOG.debug("Failure details to get valid replication peers.", e);
-        Thread.currentThread().interrupt();
         continue;
       }
     }
-    return validPeers;
+    return listOfPeers;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d083e4f2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index e18220d..7e1b3aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -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);