HBASE-3708 createAndFailSilent is not so silent; leaves lots of logging in ensemble logs

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1092001 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-04-14 04:39:48 +00:00
parent 5486d8df70
commit 7f90ea5860
2 changed files with 9 additions and 4 deletions

View File

@ -206,6 +206,8 @@ Release 0.90.3 - Unreleased
HBASE-3771 All jsp pages don't clean their HBA
HBASE-3685 when multiple columns are combined with TimestampFilter, only
one column is returned (Jerry Chen)
HBASE-3708 createAndFailSilent is not so silent; leaves lots of logging
in ensemble logs (Dmitriy Ryaboy)
IMPROVEMENTS
HBASE-3747 ReplicationSource should differanciate remote and local exceptions

View File

@ -320,7 +320,7 @@ public class ZKUtil {
* @return list of znode names, null if the node doesn't exist
* @throws KeeperException
*/
public static List<String> listChildrenAndWatchThem(ZooKeeperWatcher zkw,
public static List<String> listChildrenAndWatchThem(ZooKeeperWatcher zkw,
String znode) throws KeeperException {
List<String> children = listChildrenAndWatchForNewChildren(zkw, znode);
if (children == null) {
@ -718,7 +718,7 @@ public class ZKUtil {
* @param zkw zk reference
* @param znode path of node
* @param data data to set for node
* @throws KeeperException
* @throws KeeperException
*/
public static void createSetData(final ZooKeeperWatcher zkw, final String znode,
final byte [] data)
@ -899,8 +899,11 @@ public class ZKUtil {
String znode)
throws KeeperException {
try {
zkw.getZooKeeper().create(znode, new byte[0], Ids.OPEN_ACL_UNSAFE,
CreateMode.PERSISTENT);
ZooKeeper zk = zkw.getZooKeeper();
if (zk.exists(znode, false) != null) {
zk.create(znode, new byte[0], Ids.OPEN_ACL_UNSAFE,
CreateMode.PERSISTENT);
}
} catch(KeeperException.NodeExistsException nee) {
} catch(KeeperException.NoAuthException nee){
try {