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:
parent
5486d8df70
commit
7f90ea5860
|
@ -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
|
||||
|
|
|
@ -899,8 +899,11 @@ public class ZKUtil {
|
|||
String znode)
|
||||
throws KeeperException {
|
||||
try {
|
||||
zkw.getZooKeeper().create(znode, new byte[0], Ids.OPEN_ACL_UNSAFE,
|
||||
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 {
|
||||
|
|
Loading…
Reference in New Issue