commit 00471dcd7d015ca75f4f9c05b67fb3e728b62a5b Author: Todd Lipcon Date: Mon Dec 5 11:10:04 2011 -0800 Cleanup in recoverablezk diff --git src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java index c7b62c9..15e741d 100644 --- src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java +++ src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java @@ -50,8 +50,7 @@ public class RetryCounter { public void sleepUntilNextRetry() throws InterruptedException { int attempts = getAttemptTimes(); long sleepTime = (long) (retryIntervalMillis * Math.pow(2, attempts)); - LOG.info("The " + attempts + " times to retry after sleeping " + sleepTime - + " ms"); + LOG.info("Sleeping " + sleepTime + "ms before retry #" + attempts + "..."); timeUnit.sleep(sleepTime); } @@ -66,4 +65,4 @@ public class RetryCounter { public int getAttemptTimes() { return maxRetries-retriesRemaining+1; } -} \ No newline at end of file +} diff --git src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java index a097ec0..b862aed 100644 --- src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java +++ src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java @@ -71,10 +71,18 @@ public class RecoverableZooKeeper { private final byte[] id; private int retryIntervalMillis; - private static final int ID_OFFSET = Bytes.SIZEOF_INT; + // The metadata attached to each piece of data has the + // format: + // 1-byte constant + // 4-byte big-endian integer (length of next field) + // identifier corresponding uniquely to this process + // It is prepended to the data supplied by the user. + // the magic number is to be backward compatible private static final byte MAGIC =(byte) 0XFF; - private static final int MAGIC_OFFSET = Bytes.SIZEOF_BYTE; + private static final int MAGIC_SIZE = Bytes.SIZEOF_BYTE; + private static final int ID_LENGTH_OFFSET = MAGIC_SIZE; + private static final int ID_LENGTH_SIZE = Bytes.SIZEOF_INT; public RecoverableZooKeeper(String quorumServers, int seesionTimeout, Watcher watcher, int maxRetries, int retryIntervalMillis) @@ -91,12 +99,9 @@ public class RecoverableZooKeeper { } /** - * delete is an idempotent operation. Retry before throw out exception. - * This function will not throw out NoNodeException if the path is not existed - * @param path - * @param version - * @throws InterruptedException - * @throws KeeperException + * delete is an idempotent operation. Retry before throwing exception. + * This function will not throw NoNodeException if the path does not + * exist. */ public void delete(String path, int version) throws InterruptedException, KeeperException { @@ -120,12 +125,7 @@ public class RecoverableZooKeeper { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper delete failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "delete"); break; default: @@ -139,12 +139,8 @@ public class RecoverableZooKeeper { } /** - * exists is an idempotent operation. Retry before throw out exception - * @param path - * @param watcher + * exists is an idempotent operation. Retry before throwing exception * @return A Stat instance - * @throws KeeperException - * @throws InterruptedException */ public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException { @@ -156,12 +152,7 @@ public class RecoverableZooKeeper { switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper exists failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "exists"); break; default: @@ -174,12 +165,8 @@ public class RecoverableZooKeeper { } /** - * exists is an idempotent operation. Retry before throw out exception - * @param path - * @param watch + * exists is an idempotent operation. Retry before throwing exception * @return A Stat instance - * @throws KeeperException - * @throws InterruptedException */ public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { @@ -191,12 +178,7 @@ public class RecoverableZooKeeper { switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper exists failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "exists"); break; default: @@ -208,13 +190,19 @@ public class RecoverableZooKeeper { } } + private void retryOrThrow(RetryCounter retryCounter, KeeperException e, + String opName) throws KeeperException { + LOG.warn("Possibly transient ZooKeeper exception: " + e); + if (!retryCounter.shouldRetry()) { + LOG.error("ZooKeeper " + opName + " failed after " + + retryCounter.getMaxRetries() + " retries"); + throw e; + } + } + /** - * getChildren is an idempotent operation. Retry before throw out exception - * @param path - * @param watcher + * getChildren is an idempotent operation. Retry before throwing exception * @return List of children znodes - * @throws KeeperException - * @throws InterruptedException */ public List getChildren(String path, Watcher watcher) throws KeeperException, InterruptedException { @@ -226,12 +214,7 @@ public class RecoverableZooKeeper { switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper getChildren failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "getChildren"); break; default: @@ -244,12 +227,8 @@ public class RecoverableZooKeeper { } /** - * getChildren is an idempotent operation. Retry before throw out exception - * @param path - * @param watch + * getChildren is an idempotent operation. Retry before throwing exception * @return List of children znodes - * @throws KeeperException - * @throws InterruptedException */ public List getChildren(String path, boolean watch) throws KeeperException, InterruptedException { @@ -261,12 +240,7 @@ public class RecoverableZooKeeper { switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper getChildren failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "getChildren"); break; default: @@ -279,13 +253,8 @@ public class RecoverableZooKeeper { } /** - * getData is an idempotent operation. Retry before throw out exception - * @param path - * @param watcher - * @param stat + * getData is an idempotent operation. Retry before throwing exception * @return Data - * @throws KeeperException - * @throws InterruptedException */ public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException, InterruptedException { @@ -298,12 +267,7 @@ public class RecoverableZooKeeper { switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper getData failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "getData"); break; default: @@ -316,13 +280,8 @@ public class RecoverableZooKeeper { } /** - * getData is an idemnpotent operation. Retry before throw out exception - * @param path - * @param watch - * @param stat + * getData is an idemnpotent operation. Retry before throwing exception * @return Data - * @throws KeeperException - * @throws InterruptedException */ public byte[] getData(String path, boolean watch, Stat stat) throws KeeperException, InterruptedException { @@ -335,12 +294,7 @@ public class RecoverableZooKeeper { switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper getData failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "getData"); break; default: @@ -356,12 +310,7 @@ public class RecoverableZooKeeper { * setData is NOT an idempotent operation. Retry may cause BadVersion Exception * Adding an identifier field into the data to check whether * badversion is caused by the result of previous correctly setData - * @param path - * @param data - * @param version * @return Stat instance - * @throws KeeperException - * @throws InterruptedException */ public Stat setData(String path, byte[] data, int version) throws KeeperException, InterruptedException { @@ -374,33 +323,28 @@ public class RecoverableZooKeeper { switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper setData failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "setData"); break; case BADVERSION: // try to verify whether the previous setData success or not try{ Stat stat = new Stat(); byte[] revData = zk.getData(path, false, stat); - int idLength = Bytes.toInt(revData, ID_OFFSET); - int dataLength = revData.length-ID_OFFSET-idLength; - int dataOffset = ID_OFFSET+idLength; + int idLength = Bytes.toInt(revData, ID_LENGTH_SIZE); + int dataLength = revData.length-ID_LENGTH_SIZE-idLength; + int dataOffset = ID_LENGTH_SIZE+idLength; - if(Bytes.compareTo(revData, ID_OFFSET, id.length, + if(Bytes.compareTo(revData, ID_LENGTH_SIZE, id.length, revData, dataOffset, dataLength) == 0) { // the bad version is caused by previous successful setData return stat; } } catch(KeeperException keeperException){ - // the ZK is not reliable at this moment. just throw out exception + // the ZK is not reliable at this moment. just throwing exception throw keeperException; } - // throw out other exceptions and verified bad version exceptions + // throw other exceptions and verified bad version exceptions default: throw e; } @@ -413,8 +357,8 @@ public class RecoverableZooKeeper { /** *

* NONSEQUENTIAL create is idempotent operation. - * Retry before throw out exceptions. - * But this function will not throw out the NodeExist exception back to the + * Retry before throwing exceptions. + * But this function will not throw the NodeExist exception back to the * application. *

*

@@ -423,13 +367,7 @@ public class RecoverableZooKeeper { * or not. *

* - * @param path - * @param data - * @param acl - * @param createMode * @return Path - * @throws KeeperException - * @throws InterruptedException */ public String create(String path, byte[] data, List acl, CreateMode createMode) @@ -481,12 +419,7 @@ public class RecoverableZooKeeper { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper create failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "create"); break; default: @@ -520,12 +453,7 @@ public class RecoverableZooKeeper { switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: - LOG.warn("Possibly transient ZooKeeper exception: " + e); - if (!retryCounter.shouldRetry()) { - LOG.error("ZooKeeper create failed after " - + retryCounter.getMaxRetries() + " retries"); - throw e; - } + retryOrThrow(retryCounter, e, "create"); break; default: @@ -566,9 +494,9 @@ public class RecoverableZooKeeper { return data; } - int idLength = Bytes.toInt(data, MAGIC_OFFSET); - int dataLength = data.length-MAGIC_OFFSET-ID_OFFSET-idLength; - int dataOffset = MAGIC_OFFSET+ID_OFFSET+idLength; + int idLength = Bytes.toInt(data, ID_LENGTH_OFFSET); + int dataLength = data.length-MAGIC_SIZE-ID_LENGTH_SIZE-idLength; + int dataOffset = MAGIC_SIZE+ID_LENGTH_SIZE+idLength; byte[] newData = new byte[dataLength]; System.arraycopy(data, dataOffset, newData, 0, dataLength); @@ -582,7 +510,7 @@ public class RecoverableZooKeeper { return data; } - byte[] newData = new byte[MAGIC_OFFSET+ID_OFFSET+id.length+data.length]; + byte[] newData = new byte[MAGIC_SIZE+ID_LENGTH_SIZE+id.length+data.length]; int pos = 0; pos = Bytes.putByte(newData, pos, MAGIC); pos = Bytes.putInt(newData, pos, id.length);