Index: src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (revision 1338978) +++ src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (working copy) @@ -1816,7 +1816,7 @@ } state.update(RegionState.State.OFFLINE); try { - ZKAssign.asyncCreateNodeOffline(master.getZooKeeper(), state.getRegion(), + ZKAssign.asyncCreateOrForceNodeOffline(master.getZooKeeper(), state.getRegion(), this.master.getServerName(), cb, ctx); } catch (KeeperException e) { if (e instanceof NodeExistsException) { Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java (revision 1338978) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java (working copy) @@ -183,6 +183,45 @@ } /** + * Creates or force updates an unassigned node in the OFFLINE state for the + * specified region. + *
+ * Runs asynchronously. Depends on no pre-existing znode. + * + *
+ * Sets a watcher on the unassigned region node. + * + * @param zkw zk reference + * @param region region to be created as offline + * @param serverName server event originates from + * @param cb + * @param ctx + * @throws KeeperException if unexpected zookeeper exception + */ + public static void asyncCreateOrForceNodeOffline(ZooKeeperWatcher zkw, + HRegionInfo region, ServerName serverName, + final AsyncCallback.StringCallback cb, final Object ctx) + throws KeeperException { + LOG.debug(zkw.prefix("Async create (or updating) of unassigned node for " + + region.getEncodedName() + " with OFFLINE state")); + RegionTransition rt = RegionTransition.createRegionTransition( + EventType.M_ZK_REGION_OFFLINE, region.getRegionName(), serverName); + String node = getNodeName(zkw, region.getEncodedName()); + int version = ZKUtil.checkExists(zkw, node); + if (version == -1) { + ZKUtil.asyncCreate(zkw, node, rt.toByteArray(), cb, ctx); + } else { + ZKUtil.asyncSetData(zkw, node, rt.toByteArray(), + new AsyncCallback.StatCallback() { + @Override + public void processResult(int rc, String path, Object ctx, Stat stat) { + cb.processResult(rc, path, ctx, path); + } + }, ctx); + } + } + + /** * Forces an existing unassigned node to the OFFLINE state for the specified * region. * Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (revision 1338978) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (working copy) @@ -886,6 +886,29 @@ } /** + * Async sets the data of the existing znode to be the specified data. The + * node must exist but no checks are done on the existing data or version. + *
+ * Throws an exception if the node does not exist. + * + * @param zkw zk reference + * @param znode path of node + * @param data data to set for node + * @param cb + * @param ctx + */ + public static void asyncSetData(ZooKeeperWatcher zkw, String znode, + byte[] data, final AsyncCallback.StatCallback cb, final Object ctx) { + try { + waitForZKConnectionIfAuthenticating(zkw); + zkw.getRecoverableZooKeeper().getZooKeeper() + .setData(znode, data, -1, cb, ctx); + } catch (InterruptedException e) { + zkw.interruptedException(e); + } + } + + /** * Creates the specified node, if the node does not exist. Does not set a * watch and fails silently if the node already exists. *