HBASE-6858 Fix the incorrect BADVERSION checking in the recoverable zookeeper

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1398920 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
jxiang 2012-10-16 18:46:54 +00:00
parent 700ff4d0b9
commit bb563c3603
1 changed files with 26 additions and 25 deletions

View File

@ -20,8 +20,10 @@ package org.apache.hadoop.hbase.zookeeper;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -75,6 +77,7 @@ public class RecoverableZooKeeper {
private Watcher watcher;
private int sessionTimeout;
private String quorumServers;
private final Random salter;
// The metadata attached to each piece of data has the
// format:
@ -104,6 +107,7 @@ public class RecoverableZooKeeper {
this.watcher = watcher;
this.sessionTimeout = sessionTimeout;
this.quorumServers = quorumServers;
salter = new SecureRandom();
}
public void reconnectAfterExpiration()
@ -341,6 +345,7 @@ public class RecoverableZooKeeper {
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
byte[] newData = appendMetaData(data);
boolean isRetry = false;
while (true) {
try {
return zk.setData(path, newData, version);
@ -352,24 +357,20 @@ public class RecoverableZooKeeper {
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_LENGTH_SIZE);
int dataLength = revData.length-ID_LENGTH_SIZE-idLength;
int dataOffset = ID_LENGTH_SIZE+idLength;
if(Bytes.compareTo(revData, ID_LENGTH_SIZE, id.length,
revData, dataOffset, dataLength) == 0) {
// the bad version is caused by previous successful setData
return stat;
if (isRetry) {
// try to verify whether the previous setData success or not
try{
Stat stat = new Stat();
byte[] revData = zk.getData(path, false, stat);
if(Bytes.compareTo(revData, newData) == 0) {
// the bad version is caused by previous successful setData
return stat;
}
} catch(KeeperException keeperException){
// the ZK is not reliable at this moment. just throwing exception
throw keeperException;
}
} catch(KeeperException keeperException){
// the ZK is not reliable at this moment. just throwing exception
throw keeperException;
}
}
// throw other exceptions and verified bad version exceptions
default:
throw e;
@ -377,6 +378,7 @@ public class RecoverableZooKeeper {
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
isRetry = true;
}
}
@ -521,30 +523,29 @@ public class RecoverableZooKeeper {
if(magic != MAGIC) {
return data;
}
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);
return newData;
}
private byte[] appendMetaData(byte[] data) {
if(data == null || data.length == 0){
return data;
}
byte[] newData = new byte[MAGIC_SIZE+ID_LENGTH_SIZE+id.length+data.length];
byte[] salt = Bytes.toBytes(salter.nextLong());
int idLength = id.length + salt.length;
byte[] newData = new byte[MAGIC_SIZE+ID_LENGTH_SIZE+idLength+data.length];
int pos = 0;
pos = Bytes.putByte(newData, pos, MAGIC);
pos = Bytes.putInt(newData, pos, id.length);
pos = Bytes.putInt(newData, pos, idLength);
pos = Bytes.putBytes(newData, pos, id, 0, id.length);
pos = Bytes.putBytes(newData, pos, salt, 0, salt.length);
pos = Bytes.putBytes(newData, pos, data, 0, data.length);
return newData;
}