hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jdcry...@apache.org
Subject svn commit: r1379288 - in /hbase/branches/0.92: CHANGES.txt src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
Date Fri, 31 Aug 2012 04:56:28 GMT
Author: jdcryans
Date: Fri Aug 31 04:56:28 2012
New Revision: 1379288

URL: http://svn.apache.org/viewvc?rev=1379288&view=rev
Log:
HBASE-6321  ReplicationSource dies reading the peer's id

Modified:
    hbase/branches/0.92/CHANGES.txt
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java

Modified: hbase/branches/0.92/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/CHANGES.txt?rev=1379288&r1=1379287&r2=1379288&view=diff
==============================================================================
--- hbase/branches/0.92/CHANGES.txt (original)
+++ hbase/branches/0.92/CHANGES.txt Fri Aug 31 04:56:28 2012
@@ -99,11 +99,12 @@ Release 0.92.2 - Unreleased
    HBASE-6334  TestImprovement for TestHRegion.testWritesWhileGetting (Gregory Chanan)
    HBASE-4470  ServerNotRunningException coming out of assignRootAndMeta kills the Master
(Gregory Chanan) 
    HBASE-6460  hbck "-repairHoles" usage inconsistent with "-fixHdfsOrphans" (Jie Huang)
-   HBASE-6507 [hbck] TestHBaseFsck ran into TableNotEnabledException
-   HBASE-6552 TestAcidGuarantees system test should flush more aggresively (Gregory Chanan)
-   HBASE-6512 Incorrect OfflineMetaRepair log class name (Liang Xie)
-   HBASE-6565 Coprocessor exec result Map is not thread safe (Yuan Kang)
-   HBASE-6686 HFile Quarantine fails with missing dirs in hadoop 2.0
+   HBASE-6507  [hbck] TestHBaseFsck ran into TableNotEnabledException
+   HBASE-6552  TestAcidGuarantees system test should flush more aggresively (Gregory Chanan)
+   HBASE-6512  Incorrect OfflineMetaRepair log class name (Liang Xie)
+   HBASE-6565  Coprocessor exec result Map is not thread safe (Yuan Kang)
+   HBASE-6686  HFile Quarantine fails with missing dirs in hadoop 2.0
+   HBASE-6321  ReplicationSource dies reading the peer's id
 
   IMPROVEMENTS
    HBASE-5592  Make it easier to get a table from shell (Ben West)

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java?rev=1379288&r1=1379287&r2=1379288&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
(original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
Fri Aug 31 04:56:28 2012
@@ -29,6 +29,7 @@ import java.util.SortedMap;
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
@@ -38,7 +39,9 @@ import org.apache.hadoop.hbase.Abortable
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
@@ -226,19 +229,7 @@ public class ReplicationZookeeper {
     try {
       addresses = fetchSlavesAddresses(peer.getZkw());
     } catch (KeeperException ke) {
-      if (ke instanceof ConnectionLossException
-          || ke instanceof SessionExpiredException) {
-        LOG.warn(
-            "Lost the ZooKeeper connection for peer " + peer.getClusterKey(),
-            ke);
-        try {
-          peer.reloadZkWatcher();
-        } catch(IOException io) {
-          LOG.warn(
-              "Creation of ZookeeperWatcher failed for peer "
-                  + peer.getClusterKey(), io);
-        }
-      }
+      reconnectPeer(ke, peer);
       addresses = Collections.emptyList();
     }
     peer.setRegionServers(addresses);
@@ -705,6 +696,50 @@ public class ReplicationZookeeper {
     return data == null || data.length() == 0 ? 0 : Long.parseLong(data);
   }
 
+  /**
+   * Returns the UUID of the provided peer id. Should a connection loss or session
+   * expiration happen, the ZK handler will be reopened once and if it still doesn't
+   * work then it will bail and return null.
+   * @param peerId the peer's ID that will be converted into a UUID
+   * @return a UUID or null if there's a ZK connection issue
+   */
+  public UUID getPeerUUID(String peerId) {
+    ReplicationPeer peer = getPeerClusters().get(peerId);
+    UUID peerUUID = null;
+    try {
+      peerUUID = getUUIDForCluster(peer.getZkw());
+    } catch (KeeperException ke) {
+      reconnectPeer(ke, peer);
+    }
+    return peerUUID;
+  }
+
+  /**
+   * Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions
+   * @param zkw watcher connected to an ensemble
+   * @return the UUID read from zookeeper
+   * @throws KeeperException
+   */
+  public UUID getUUIDForCluster(ZooKeeperWatcher zkw) throws KeeperException {
+    return UUID.fromString(ClusterId.readClusterIdZNode(zkw));
+  }
+
+  private void reconnectPeer(KeeperException ke, ReplicationPeer peer) {
+    if (ke instanceof ConnectionLossException
+      || ke instanceof SessionExpiredException) {
+      LOG.warn(
+        "Lost the ZooKeeper connection for peer " + peer.getClusterKey(),
+        ke);
+      try {
+        peer.reloadZkWatcher();
+      } catch(IOException io) {
+        LOG.warn(
+          "Creation of ZookeeperWatcher failed for peer "
+            + peer.getClusterKey(), io);
+      }
+    }
+  }
+
   public void registerRegionServerListener(ZooKeeperListener listener) {
     this.zookeeper.registerListener(listener);
   }

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java?rev=1379288&r1=1379287&r2=1379288&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
(original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
Fri Aug 31 04:56:28 2012
@@ -188,8 +188,7 @@ public class ReplicationSource extends T
     this.metrics = new ReplicationSourceMetrics(peerClusterZnode);
 
     try {
-      this.clusterId = UUID.fromString(ClusterId.readClusterIdZNode(zkHelper
-          .getZookeeperWatcher()));
+      this.clusterId = zkHelper.getUUIDForCluster(zkHelper.getZookeeperWatcher());
     } catch (KeeperException ke) {
       throw new IOException("Could not read cluster id", ke);
     }
@@ -248,13 +247,19 @@ public class ReplicationSource extends T
     if (!this.isActive()) {
       return;
     }
+    int sleepMultiplier = 1;
     // delay this until we are in an asynchronous thread
-    try {
-      this.peerClusterId = UUID.fromString(ClusterId
-          .readClusterIdZNode(zkHelper.getPeerClusters().get(peerId).getZkw()));
-    } catch (KeeperException ke) {
-      this.terminate("Could not read peer's cluster id", ke);
+    while (this.peerClusterId == null) {
+      this.peerClusterId = zkHelper.getPeerUUID(this.peerId);
+      if (this.peerClusterId == null) {
+        if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
+          sleepMultiplier++;
+        }
+      }
     }
+    // resetting to 1 to reuse later
+    sleepMultiplier = 1;
+
     LOG.info("Replicating "+clusterId + " -> " + peerClusterId);
 
     // If this is recovered, the queue is already full and the first log
@@ -268,7 +273,6 @@ public class ReplicationSource extends T
             peerClusterZnode, e);
       }
     }
-    int sleepMultiplier = 1;
     // Loop until we close down
     while (isActive()) {
       // Sleep until replication is enabled again



Mime
View raw message