Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (revision 1028504) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (working copy) @@ -21,8 +21,9 @@ import java.io.IOException; import java.util.HashSet; +import java.util.List; import java.util.Set; -import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.CopyOnWriteArrayList; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -64,8 +65,8 @@ private Abortable abortable; // listeners to be notified - private final Set listeners = - new CopyOnWriteArraySet(); + private final List listeners = + new CopyOnWriteArrayList(); // set of unassigned nodes watched private Set unassignedNodes = new HashSet(); @@ -201,6 +202,15 @@ } /** + * Register the specified listener to receive ZooKeeper events and add it as + * the first in the list of current listeners. + * @param listener + */ + public void registerListenerFirst(ZooKeeperListener listener) { + listeners.add(0, listener); + } + + /** * Get the connection to ZooKeeper. * @return connection reference to zookeeper */ Index: src/main/java/org/apache/hadoop/hbase/master/HMaster.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/HMaster.java (revision 1028504) +++ src/main/java/org/apache/hadoop/hbase/master/HMaster.java (working copy) @@ -337,7 +337,7 @@ this.assignmentManager = new AssignmentManager(this, serverManager, this.catalogTracker, this.executorService); - zooKeeper.registerListener(assignmentManager); + zooKeeper.registerListenerFirst(assignmentManager); this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);