HBASE-4957 Clean up some log messages, code in RecoverableZooKeeper (Todd)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1304940 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
larsh 2012-03-24 22:02:41 +00:00
parent 9cfe7cfa3c
commit 82670f81b8
2 changed files with 53 additions and 126 deletions

View File

@ -52,8 +52,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);
}

View File

@ -77,10 +77,18 @@ public class RecoverableZooKeeper {
private int sessionTimeout;
private String quorumServers;
private static final int ID_OFFSET = Bytes.SIZEOF_INT;
// The metadata attached to each piece of data has the
// format:
// <magic> 1-byte constant
// <id length> 4-byte big-endian integer (length of next field)
// <id> 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 sessionTimeout,
Watcher watcher, int maxRetries, int retryIntervalMillis)
@ -111,12 +119,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 {
@ -141,12 +146,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -160,12 +160,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 {
@ -178,12 +174,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -196,12 +187,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 {
@ -214,12 +201,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -231,13 +213,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<String> getChildren(String path, Watcher watcher)
throws KeeperException, InterruptedException {
@ -250,12 +238,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -268,12 +251,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<String> getChildren(String path, boolean watch)
throws KeeperException, InterruptedException {
@ -286,12 +265,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -304,13 +278,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 {
@ -324,12 +293,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -342,13 +306,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 {
@ -362,12 +321,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -383,12 +337,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 {
@ -402,33 +351,28 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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;
}
@ -441,8 +385,8 @@ public class RecoverableZooKeeper {
/**
* <p>
* 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.
* </p>
* <p>
@ -451,13 +395,7 @@ public class RecoverableZooKeeper {
* or not.
* </p>
*
* @param path
* @param data
* @param acl
* @param createMode
* @return Path
* @throws KeeperException
* @throws InterruptedException
*/
public String create(String path, byte[] data, List<ACL> acl,
CreateMode createMode)
@ -510,12 +448,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -550,12 +483,7 @@ public class RecoverableZooKeeper {
case CONNECTIONLOSS:
case SESSIONEXPIRED:
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:
@ -596,9 +524,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);
@ -612,7 +540,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);