hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From la...@apache.org
Subject [3/3] git commit: HBASE-9746 RegionServer can't start when replication tries to replicate to an unknown host.
Date Mon, 25 Aug 2014 23:37:12 GMT
HBASE-9746 RegionServer can't start when replication tries to replicate to an unknown host.


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

Branch: refs/heads/0.98
Commit: ccbad5dba20835e8584ae037fc7586e45deaa427
Parents: 7b04466
Author: Lars Hofhansl <larsh@apache.org>
Authored: Mon Aug 25 16:32:01 2014 -0700
Committer: Lars Hofhansl <larsh@apache.org>
Committed: Mon Aug 25 16:34:45 2014 -0700

----------------------------------------------------------------------
 .../hbase/zookeeper/RecoverableZooKeeper.java   | 75 ++++++++++++--------
 .../apache/hadoop/hbase/zookeeper/ZKConfig.java |  8 +--
 .../hbase/zookeeper/ZooKeeperWatcher.java       |  4 +-
 3 files changed, 50 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ccbad5db/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
index 565ca3b..103d4b8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -109,7 +109,6 @@ public class RecoverableZooKeeper {
       Watcher watcher, int maxRetries, int retryIntervalMillis, String identifier)
   throws IOException {
     // TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as
we should.
-    this.zk = new ZooKeeper(quorumServers, sessionTimeout, watcher);
     this.retryCounterFactory =
       new RetryCounterFactory(maxRetries+1, retryIntervalMillis);
 
@@ -125,16 +124,36 @@ public class RecoverableZooKeeper {
     this.watcher = watcher;
     this.sessionTimeout = sessionTimeout;
     this.quorumServers = quorumServers;
+    try {checkZk();} catch (Exception x) {/* ignore */}
     salter = new SecureRandom();
   }
 
+  /**
+   * Try to create a Zookeeper connection. Turns any exception encountered into a
+   * {@link KeeperException.OperationTimeoutException} so it can retried.
+   * @return The created Zookeeper connection object
+   * @throws KeeperException
+   */
+  protected ZooKeeper checkZk() throws KeeperException {
+    if (this.zk == null) {
+      try {
+        this.zk = new ZooKeeper(quorumServers, sessionTimeout, watcher);
+      } catch (Exception uhe) {
+        LOG.warn("Unable to create ZooKeeper Connection", uhe);
+        throw new KeeperException.OperationTimeoutException();
+      }
+    }
+    return zk;
+  }
+
   public void reconnectAfterExpiration()
-        throws IOException, InterruptedException {
-    LOG.info("Closing dead ZooKeeper connection, session" +
-      " was: 0x"+Long.toHexString(zk.getSessionId()));
-    zk.close();
-    this.zk = new ZooKeeper(this.quorumServers,
-      this.sessionTimeout, this.watcher);
+        throws IOException, KeeperException, InterruptedException {
+    if (zk != null) {
+      LOG.info("Closing dead ZooKeeper connection, session" +
+        " was: 0x"+Long.toHexString(zk.getSessionId()));
+      zk.close();
+    }
+    checkZk();
     LOG.info("Recreated a ZooKeeper, session" +
       " is: 0x"+Long.toHexString(zk.getSessionId()));
   }
@@ -153,7 +172,7 @@ public class RecoverableZooKeeper {
       boolean isRetry = false; // False for first attempt, true for all retries.
       while (true) {
         try {
-          zk.delete(path, version);
+          checkZk().delete(path, version);
           return;
         } catch (KeeperException e) {
           switch (e.code()) {
@@ -196,7 +215,7 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return zk.exists(path, watcher);
+          return checkZk().exists(path, watcher);
         } catch (KeeperException e) {
           switch (e.code()) {
             case CONNECTIONLOSS:
@@ -228,7 +247,7 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return zk.exists(path, watch);
+          return checkZk().exists(path, watch);
         } catch (KeeperException e) {
           switch (e.code()) {
             case CONNECTIONLOSS:
@@ -270,7 +289,7 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return zk.getChildren(path, watcher);
+          return checkZk().getChildren(path, watcher);
         } catch (KeeperException e) {
           switch (e.code()) {
             case CONNECTIONLOSS:
@@ -302,7 +321,7 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return zk.getChildren(path, watch);
+          return checkZk().getChildren(path, watch);
         } catch (KeeperException e) {
           switch (e.code()) {
             case CONNECTIONLOSS:
@@ -334,7 +353,7 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          byte[] revData = zk.getData(path, watcher, stat);
+          byte[] revData = checkZk().getData(path, watcher, stat);
           return this.removeMetaData(revData);
         } catch (KeeperException e) {
           switch (e.code()) {
@@ -367,7 +386,7 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          byte[] revData = zk.getData(path, watch, stat);
+          byte[] revData = checkZk().getData(path, watch, stat);
           return this.removeMetaData(revData);
         } catch (KeeperException e) {
           switch (e.code()) {
@@ -404,7 +423,7 @@ public class RecoverableZooKeeper {
       boolean isRetry = false;
       while (true) {
         try {
-          return zk.setData(path, newData, version);
+          return checkZk().setData(path, newData, version);
         } catch (KeeperException e) {
           switch (e.code()) {
             case CONNECTIONLOSS:
@@ -417,7 +436,7 @@ public class RecoverableZooKeeper {
                 // try to verify whether the previous setData success or not
                 try{
                   Stat stat = new Stat();
-                  byte[] revData = zk.getData(path, false, stat);
+                  byte[] revData = checkZk().getData(path, false, stat);
                   if(Bytes.compareTo(revData, newData) == 0) {
                     // the bad version is caused by previous successful setData
                     return stat;
@@ -486,7 +505,7 @@ public class RecoverableZooKeeper {
     boolean isRetry = false; // False for first attempt, true for all retries.
     while (true) {
       try {
-        return zk.create(path, data, acl, createMode);
+        return checkZk().create(path, data, acl, createMode);
       } catch (KeeperException e) {
         switch (e.code()) {
           case NODEEXISTS:
@@ -494,7 +513,7 @@ public class RecoverableZooKeeper {
               // If the connection was lost, there is still a possibility that
               // we have successfully created the node at our previous attempt,
               // so we read the node and compare.
-              byte[] currentData = zk.getData(path, false, null);
+              byte[] currentData = checkZk().getData(path, false, null);
               if (currentData != null &&
                   Bytes.compareTo(currentData, data) == 0) {
                 // We successfully created a non-sequential node
@@ -540,7 +559,7 @@ public class RecoverableZooKeeper {
           }
         }
         first = false;
-        return zk.create(newPath, data, acl, createMode);
+        return checkZk().create(newPath, data, acl, createMode);
       } catch (KeeperException e) {
         switch (e.code()) {
           case CONNECTIONLOSS:
@@ -596,7 +615,7 @@ public class RecoverableZooKeeper {
       Iterable<Op> multiOps = prepareZKMulti(ops);
       while (true) {
         try {
-          return zk.multi(multiOps);
+          return checkZk().multi(multiOps);
         } catch (KeeperException e) {
           switch (e.code()) {
             case CONNECTIONLOSS:
@@ -623,11 +642,11 @@ public class RecoverableZooKeeper {
     String parent = path.substring(0, lastSlashIdx);
     String nodePrefix = path.substring(lastSlashIdx+1);
 
-    List<String> nodes = zk.getChildren(parent, false);
+    List<String> nodes = checkZk().getChildren(parent, false);
     List<String> matching = filterByPrefix(nodes, nodePrefix);
     for (String node : matching) {
       String nodePath = parent + "/" + node;
-      Stat stat = zk.exists(nodePath, false);
+      Stat stat = checkZk().exists(nodePath, false);
       if (stat != null) {
         return nodePath;
       }
@@ -671,15 +690,15 @@ public class RecoverableZooKeeper {
   }
 
   public long getSessionId() {
-    return zk.getSessionId();
+    return zk == null ? null : zk.getSessionId();
   }
 
   public void close() throws InterruptedException {
-    zk.close();
+    if (zk != null) zk.close();
   }
 
   public States getState() {
-    return zk.getState();
+    return zk == null ? null : zk.getState();
   }
 
   public ZooKeeper getZooKeeper() {
@@ -687,11 +706,11 @@ public class RecoverableZooKeeper {
   }
 
   public byte[] getSessionPasswd() {
-    return zk.getSessionPasswd();
+    return zk == null ? null : zk.getSessionPasswd();
   }
 
-  public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) {
-    this.zk.sync(path, null, null);
+  public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException
{
+    checkZk().sync(path, null, null);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ccbad5db/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
index 5f5502e..23438c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
@@ -214,13 +214,7 @@ public class ZKConfig {
       else if (key.startsWith("server.")) {
         String host = value.substring(0, value.indexOf(':'));
         servers.add(host);
-        try {
-          //noinspection ResultOfMethodCallIgnored
-          InetAddress.getByName(host);
-          anyValid = true;
-        } catch (UnknownHostException e) {
-          LOG.warn(StringUtils.stringifyException(e));
-        }
+        anyValid = true;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ccbad5db/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index fcdf1a6..2c88a3c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -287,7 +287,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable
{
     return recoverableZooKeeper;
   }
 
-  public void reconnectAfterExpiration() throws IOException, InterruptedException {
+  public void reconnectAfterExpiration() throws IOException, KeeperException, InterruptedException
{
     recoverableZooKeeper.reconnectAfterExpiration();
   }
 
@@ -424,7 +424,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable
{
    * previously read version and data.  We want to ensure that the version read
    * is up-to-date from when we begin the operation.
    */
-  public void sync(String path) {
+  public void sync(String path) throws KeeperException {
     this.recoverableZooKeeper.sync(path, null, null);
   }
 


Mime
View raw message