Merged in new registry code
diff --git a/mastership-test.sh b/mastership-test.sh
index 06907f0..4fa47d5 100755
--- a/mastership-test.sh
+++ b/mastership-test.sh
@@ -1 +1 @@
-java -Dlogback.configurationFile=logback.xml -cp target/floodlight-only.jar:lib/*:lib/titan/* net.floodlightcontroller.mastership.MastershipRunner $1
+java -Dlogback.configurationFile=logback.xml -cp target/floodlight-only.jar:lib/*:lib/titan/* net.onrc.onos.registry.controller.RegistryRunner $1
diff --git a/src/main/java/net/floodlightcontroller/core/FloodlightProvider.java b/src/main/java/net/floodlightcontroller/core/FloodlightProvider.java
index 5c847b7..cd15927 100644
--- a/src/main/java/net/floodlightcontroller/core/FloodlightProvider.java
+++ b/src/main/java/net/floodlightcontroller/core/FloodlightProvider.java
@@ -13,11 +13,11 @@
 import net.floodlightcontroller.core.INetMapTopologyService.ITopoRouteService;
 import net.floodlightcontroller.counter.ICounterStoreService;
 import net.floodlightcontroller.flowcache.IFlowService;
-import net.floodlightcontroller.mastership.IMastershipService;
 import net.floodlightcontroller.perfmon.IPktInProcessingTimeService;
 import net.floodlightcontroller.restserver.IRestApiService;
 import net.floodlightcontroller.storage.IStorageSourceService;
 import net.floodlightcontroller.threadpool.IThreadPoolService;
+import net.onrc.onos.registry.controller.IControllerRegistryService;
 
 public class FloodlightProvider implements IFloodlightModule {
     Controller controller;
@@ -52,9 +52,10 @@
         dependencies.add(IRestApiService.class);
         dependencies.add(ICounterStoreService.class);
         dependencies.add(IThreadPoolService.class);
-        dependencies.add(IMastershipService.class);
         dependencies.add(IFlowService.class);
         dependencies.add(ITopoRouteService.class);
+        dependencies.add(IControllerRegistryService.class);
+
         return dependencies;
     }
 
@@ -70,9 +71,11 @@
            context.getServiceImpl(IRestApiService.class));
        controller.setThreadPoolService(
            context.getServiceImpl(IThreadPoolService.class));
-       controller.setMastershipService(context.getServiceImpl(IMastershipService.class));
        controller.setFlowService(context.getServiceImpl(IFlowService.class));
        controller.setTopoRouteService(context.getServiceImpl(ITopoRouteService.class));
+       controller.setMastershipService(
+    		   context.getServiceImpl(IControllerRegistryService.class));
+
        controller.init(context.getConfigParams(this));
     }
 
diff --git a/src/main/java/net/floodlightcontroller/core/internal/Controller.java b/src/main/java/net/floodlightcontroller/core/internal/Controller.java
index cbcd9a0..78b93ed 100644
--- a/src/main/java/net/floodlightcontroller/core/internal/Controller.java
+++ b/src/main/java/net/floodlightcontroller/core/internal/Controller.java
@@ -67,7 +67,6 @@
 import net.floodlightcontroller.core.web.CoreWebRoutable;
 import net.floodlightcontroller.counter.ICounterStoreService;
 import net.floodlightcontroller.flowcache.IFlowService;
-import net.floodlightcontroller.mastership.IMastershipService;
 import net.floodlightcontroller.packet.Ethernet;
 import net.floodlightcontroller.perfmon.IPktInProcessingTimeService;
 import net.floodlightcontroller.restserver.IRestApiService;
@@ -77,6 +76,7 @@
 import net.floodlightcontroller.storage.OperatorPredicate;
 import net.floodlightcontroller.storage.StorageException;
 import net.floodlightcontroller.threadpool.IThreadPoolService;
+import net.onrc.onos.registry.controller.IControllerRegistryService;
 
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.buffer.ChannelBuffer;
@@ -191,9 +191,9 @@
     protected IStorageSourceService storageSource;
     protected IPktInProcessingTimeService pktinProcTime;
     protected IThreadPoolService threadPool;
-    protected IMastershipService masterHelper;
     protected IFlowService flowService;
     protected ITopoRouteService topoRouteService;
+    protected IControllerRegistryService masterHelper;
     
     // Configuration options
     protected int openFlowPort = 6633;
@@ -396,10 +396,6 @@
         this.threadPool = tp;
     }
 
-    public void setMastershipService(IMastershipService serviceImpl) {
-	this.masterHelper = serviceImpl;		
-    }
-
     public void setFlowService(IFlowService serviceImpl) {
 	this.flowService = serviceImpl;		
     }
@@ -407,6 +403,10 @@
     public void setTopoRouteService(ITopoRouteService serviceImpl) {
 	this.topoRouteService = serviceImpl;		
     }
+
+	public void setMastershipService(IControllerRegistryService serviceImpl) {
+		this.masterHelper = serviceImpl;		
+	}
 	
     @Override
     public Role getRole() {
diff --git a/src/main/java/net/floodlightcontroller/mastership/IMastershipService.java b/src/main/java/net/floodlightcontroller/mastership/IMastershipService.java
deleted file mode 100644
index cc0ddf4..0000000
--- a/src/main/java/net/floodlightcontroller/mastership/IMastershipService.java
+++ /dev/null
@@ -1,47 +0,0 @@
-package net.floodlightcontroller.mastership;
-
-import java.util.Collection;
-
-import net.floodlightcontroller.core.module.IFloodlightService;
-
-//Will change to something like IRegistryService
-public interface IMastershipService extends IFloodlightService {
-	
-	// Callback for all mastership changes. 
-	// Change callback is called when mastership is acquired or released
-	public interface MastershipCallback {
-		public void changeCallback(long dpid, boolean isMaster);
-	}
-	
-	// Acquire mastership for a switch. 
-	public void acquireMastership(long dpid, MastershipCallback cb) throws Exception;
-	
-	// Release mastership for a switch
-	public void releaseMastership(long dpid);
-	
-	// Check if I am the master of a switch. This is a nonblocking call that checks if the caller is a 
-	public boolean amMaster(long dpid);
-	
-	// Set/Get mastership identifier.
-	// This is typically a unique identifier of the controller that does not change across restarts
-	public void setMastershipId (String id);
-	public String getMastershipId ();
-	
-	/**
-	 * Register a controller to the ONOS cluster
-	 * @param controller A string identifying the controller and (possibly) how to talk to it.
-	 * (We will have to develop a convention for this - most likely hostname:port)
-	 */
-	public void registerController(String controllerId) throws Exception;
-	
-	/**
-	 * Get all controllers in the cluster
-	 * @return
-	 */
-	public Collection<String> getAllControllers() throws Exception;
-	
-	
-	public String getControllerForSwitch(long dpid) throws Exception;
-	
-	public Collection<Long> getSwitchesControlledByController(String controllerId);
-}
diff --git a/src/main/java/net/floodlightcontroller/mastership/MastershipManager.java b/src/main/java/net/floodlightcontroller/mastership/MastershipManager.java
deleted file mode 100644
index bff588b..0000000
--- a/src/main/java/net/floodlightcontroller/mastership/MastershipManager.java
+++ /dev/null
@@ -1,328 +0,0 @@
-package net.floodlightcontroller.mastership;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import net.floodlightcontroller.core.module.FloodlightModuleContext;
-import net.floodlightcontroller.core.module.FloodlightModuleException;
-import net.floodlightcontroller.core.module.IFloodlightModule;
-import net.floodlightcontroller.core.module.IFloodlightService;
-
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.openflow.util.HexString;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.netflix.curator.RetryPolicy;
-import com.netflix.curator.framework.CuratorFramework;
-import com.netflix.curator.framework.CuratorFrameworkFactory;
-import com.netflix.curator.framework.api.CuratorWatcher;
-import com.netflix.curator.framework.recipes.cache.ChildData;
-import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
-import com.netflix.curator.framework.recipes.cache.PathChildrenCache.StartMode;
-import com.netflix.curator.framework.recipes.leader.LeaderLatch;
-import com.netflix.curator.framework.recipes.leader.Participant;
-import com.netflix.curator.retry.ExponentialBackoffRetry;
-
-public class MastershipManager implements IFloodlightModule, IMastershipService {
-
-	protected static Logger log = LoggerFactory.getLogger(MastershipManager.class);
-	protected String mastershipId = null;
-	
-	//TODO read this from configuration
-	protected String connectionString = "localhost:2181";
-	private final String namespace = "onos";
-	private final String switchLatchesPath = "/switches";
-	
-	protected CuratorFramework client;
-	
-	private final String controllerPath = "/controllers";
-	protected PathChildrenCache controllerCache;
-
-	protected Map<String, LeaderLatch> switchLatches;
-	protected Map<String, MastershipCallback> switchCallbacks;
-	
-	protected class ParamaterizedCuratorWatcher implements CuratorWatcher {
-		private String dpid;
-		private boolean isLeader = false;
-		private String latchPath;
-		
-		public ParamaterizedCuratorWatcher(String dpid, String latchPath){
-			this.dpid = dpid;
-			this.latchPath = latchPath;
-		}
-		
-		@Override
-		public synchronized void process(WatchedEvent event) throws Exception {
-			log.debug("Watch Event: {}", event);
-
-			LeaderLatch latch = switchLatches.get(dpid);
-			
-			if (event.getState() == KeeperState.Disconnected){
-				if (isLeader) {
-					log.debug("Disconnected while leader - lost leadership for {}", dpid);
-					
-					isLeader = false;
-					switchCallbacks.get(dpid).changeCallback(HexString.toLong(dpid), false);
-				}
-				return;
-			}
-			
-			try {
-				
-				Participant leader = latch.getLeader();
-
-				if (leader.getId().equals(mastershipId) && !isLeader){
-					log.debug("Became leader for {}", dpid);
-					
-					isLeader = true;
-					switchCallbacks.get(dpid).changeCallback(HexString.toLong(dpid), true);
-				}
-				else if (!leader.getId().equals(mastershipId) && isLeader){
-					log.debug("Lost leadership for {}", dpid);
-					
-					isLeader = false;
-					switchCallbacks.get(dpid).changeCallback(HexString.toLong(dpid), false);
-				}
-			} catch (Exception e){
-				if (isLeader){
-					log.debug("Exception checking leadership status. Assume leadship lost for {}",
-							dpid);
-					
-					isLeader = false;
-					switchCallbacks.get(dpid).changeCallback(HexString.toLong(dpid), false);
-				}
-			}
-			
-			client.getChildren().usingWatcher(this).inBackground().forPath(latchPath);
-			//client.getChildren().usingWatcher(this).forPath(latchPath);
-		}
-	}
-
-	
-	@Override
-	public void acquireMastership(long dpid, MastershipCallback cb) throws Exception {
-		
-		if (mastershipId == null){
-			throw new RuntimeException("Must set mastershipId before calling aquireMastership");
-		}
-		
-		String dpidStr = HexString.toHexString(dpid);
-		String latchPath = switchLatchesPath + "/" + dpidStr;
-		
-		if (switchLatches.get(dpidStr) != null){
-			throw new RuntimeException("Leader election for switch " + dpidStr +
-					"is already running");
-		}
-		
-		LeaderLatch latch = new LeaderLatch(client, latchPath, mastershipId);
-		switchLatches.put(dpidStr, latch);
-		switchCallbacks.put(dpidStr, cb);
-		
-		try {
-			//client.getChildren().usingWatcher(watcher).inBackground().forPath(singleLatchPath);
-			client.getChildren().usingWatcher(
-					new ParamaterizedCuratorWatcher(dpidStr, latchPath))
-					.inBackground().forPath(latchPath);
-			latch.start();
-		} catch (Exception e) {
-			log.warn("Error starting leader latch: {}", e.getMessage());
-			throw e;
-		}
-		
-	}
-
-	@Override
-	public void releaseMastership(long dpid) {
-		String dpidStr = HexString.toHexString(dpid);
-		
-		LeaderLatch latch = switchLatches.get(dpidStr);
-		if (latch == null) {
-			log.debug("Trying to release mastership for switch we are not contesting");
-			return;
-		}
-		
-		try {
-			latch.close();
-		} catch (IOException e) {
-			//I think it's OK not to do anything here. Either the node got deleted correctly,
-			//or the connection went down and the node got deleted.
-		} finally {
-			switchLatches.remove(dpidStr);
-			switchCallbacks.remove(dpidStr);
-		}
-	}
-
-	@Override
-	public boolean amMaster(long dpid) {
-		LeaderLatch latch = switchLatches.get(HexString.toHexString(dpid));
-		
-		if (latch == null) {
-			log.warn("No leader latch for dpid {}", HexString.toHexString(dpid));
-			return false;
-		}
-		
-		try {
-			return latch.getLeader().getId().equals(mastershipId);
-		} catch (Exception e) {
-			//TODO swallow exception?
-			return false;
-		}
-	}
-
-	@Override
-	public void setMastershipId(String id) {
-		mastershipId = id;
-	}
-
-	@Override
-	public String getMastershipId() {
-		return mastershipId;
-	}
-	
-	@Override
-	public Collection<String> getAllControllers() throws Exception {
-		log.debug("Getting all controllers");
-		
-		List<String> controllers = new ArrayList<String>();
-		for (ChildData data : controllerCache.getCurrentData()){
-
-			String d = null;
-			try {
-				d = new String(data.getData(), "UTF-8");
-			} catch (UnsupportedEncodingException e) {
-				throw new Exception("Error encoding string", e);
-			}
-
-			controllers.add(d);
-		}
-		return controllers;
-	}
-
-	@Override
-	public void registerController(String id) throws Exception {
-		byte bytes[] = null;
-		try {
-			bytes = id.getBytes("UTF-8");
-		} catch (UnsupportedEncodingException e1) {
-			throw new Exception("Error encoding string", e1);
-		}
-		
-		String path = controllerPath + "/" + id;
-		
-		log.info("Registering controller with id {}", id);
-		
-		//Create ephemeral node with my id
-		try {
-			client.create().withProtection().withMode(CreateMode.EPHEMERAL)
-					.forPath(path, bytes);
-		} catch (Exception e) {
-			throw new Exception("Error contacting the Zookeeper service", e);
-		}
-	}
-	
-	@Override
-	public String getControllerForSwitch(long dpid) throws Exception {
-		// TODO Work out how we should store this controller/switch data.
-		// The leader latch might be a index to the /controllers collections
-		// which holds more info on the controller (how to talk to it for example).
-		
-		
-		String strDpid = HexString.toHexString(dpid);
-		LeaderLatch latch = switchLatches.get(strDpid);
-		
-		if (latch == null){
-			log.warn("Tried to get controller for non-existent switch");
-			return null;
-		}
-		
-		Participant leader = null;
-		try {
-			leader = latch.getLeader();
-		} catch (Exception e) {
-			throw new Exception("Error contacting the Zookeeper service", e);
-		}
-		
-		return leader.getId();
-	}
-	
-	@Override
-	public Collection<Long> getSwitchesControlledByController(String controllerId) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-	
-	/*
-	 * IFloodlightModule
-	 */
-	
-	@Override
-	public Collection<Class<? extends IFloodlightService>> getModuleServices() {
-		Collection<Class<? extends IFloodlightService>> l = 
-				new ArrayList<Class<? extends IFloodlightService>>();
-		l.add(IMastershipService.class);
-		return l;
-	}
-	
-	@Override
-	public Map<Class<? extends IFloodlightService>, IFloodlightService> getServiceImpls() {
-		Map<Class<? extends IFloodlightService>, IFloodlightService> m = 
-				new HashMap<Class<? extends IFloodlightService>, IFloodlightService>();
-		m.put(IMastershipService.class,  this);
-		return m;
-	}
-	
-	@Override
-	public Collection<Class<? extends IFloodlightService>> getModuleDependencies() {
-		// no module dependencies
-		return null;
-	}
-	
-	@Override
-	public void init (FloodlightModuleContext context) throws FloodlightModuleException {
-		/*
-		try {
-			String localHostname = java.net.InetAddress.getLocalHost().getHostName();
-			mastershipId = localHostname;
-			log.debug("Setting mastership id to {}", mastershipId);
-		} catch (UnknownHostException e) {
-			// TODO Handle this exception
-			e.printStackTrace();
-		}
-
-		switchLatches = new HashMap<String, LeaderLatch>();
-		switchCallbacks = new HashMap<String, MastershipCallback>();
-		
-		RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
-		client = CuratorFrameworkFactory.newClient(connectionString, retryPolicy);
-		
-		client.start();
-		
-		client = client.usingNamespace(namespace);
-		
-		controllerCache = new PathChildrenCache(client, controllerPath, true);
-		
-		try {
-			controllerCache.start(StartMode.BUILD_INITIAL_CACHE);
-			
-			
-		} catch (Exception e) {
-			// TODO Auto-generated catch block
-			e.printStackTrace();
-		}
-	*/
-	}
-	
-	@Override
-	public void startUp (FloodlightModuleContext context) {
-		// Nothing to be done on startup
-	}
-}
diff --git a/src/main/java/net/floodlightcontroller/mastership/MastershipRouteResource.java b/src/main/java/net/floodlightcontroller/mastership/MastershipRouteResource.java
deleted file mode 100644
index f75c518..0000000
--- a/src/main/java/net/floodlightcontroller/mastership/MastershipRouteResource.java
+++ /dev/null
@@ -1,32 +0,0 @@
-package net.floodlightcontroller.mastership;
-
-import org.restlet.resource.ServerResource;
-import org.restlet.resource.Get;
-import org.restlet.resource.Post;
-import org.restlet.resource.Delete;
-
-import org.slf4j.LoggerFactory;
-import org.slf4j.Logger;
-
-public class MastershipRouteResource extends ServerResource {
-
-	protected static Logger log = LoggerFactory.getLogger(MastershipRouteResource.class);
-
-	@Get
-	public String get(String fmJson) {
-		// TODO
-		return null;
-	}
-	
-	@Post
-	public String store (String fmJson) {
-		//TODO
-		return null;
-	}
-	
-	@Delete
-	public String delete (String fmJson) {
-		//TODO
-		return null;
-	}
-}
diff --git a/src/main/java/net/floodlightcontroller/mastership/MastershipRunner.java b/src/main/java/net/floodlightcontroller/mastership/MastershipRunner.java
deleted file mode 100644
index a28a9c5..0000000
--- a/src/main/java/net/floodlightcontroller/mastership/MastershipRunner.java
+++ /dev/null
@@ -1,70 +0,0 @@
-package net.floodlightcontroller.mastership;
-
-import net.floodlightcontroller.core.module.FloodlightModuleContext;
-import net.floodlightcontroller.core.module.FloodlightModuleException;
-import net.floodlightcontroller.mastership.IMastershipService.MastershipCallback;
-
-import org.openflow.util.HexString;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Used for lightweight testing of the mastership module without having
- * to load up the entire ONOS.
- * @author jono
- *
- */
-public class MastershipRunner {
-	protected static Logger log = LoggerFactory.getLogger(MastershipRunner.class);
-
-	public static void main(String args[]){
-		FloodlightModuleContext fmc = new FloodlightModuleContext();
-		MastershipManager mm = new MastershipManager();
-		
-		String id = null;
-		if (args.length > 0){
-			id = args[0];
-			log.info("Using unique id: {}", id);
-		}
-		
-		try {
-			mm.init(fmc);
-			mm.startUp(fmc);
-			
-			if (id != null){
-				mm.setMastershipId(id);
-			}
-				
-			mm.acquireMastership(1L, 
-				new MastershipCallback(){
-					@Override
-					public void changeCallback(long dpid, boolean isMaster) {
-						if (isMaster){
-							log.debug("Callback for becoming master for {}", HexString.toHexString(dpid));
-						}
-						else {
-							log.debug("Callback for losing mastership for {}", HexString.toHexString(dpid));
-						}
-					}
-				});
-			
-			mm.registerController(id);
-			
-			Thread.sleep(5000);
-			
-			//"Server" loop
-			while (true) {
-				Thread.sleep(60000);
-			}
-			
-		} catch (FloodlightModuleException e) {
-			e.printStackTrace();
-		} catch (InterruptedException e) {
-			e.printStackTrace();
-		} catch (Exception e) {
-			e.printStackTrace();
-		}
-		
-		log.debug("is master: {}", mm.amMaster(1L));
-	}
-}
diff --git a/src/main/java/net/floodlightcontroller/mastership/MastershipWebRoutable.java b/src/main/java/net/floodlightcontroller/mastership/MastershipWebRoutable.java
deleted file mode 100644
index 56d5d66..0000000
--- a/src/main/java/net/floodlightcontroller/mastership/MastershipWebRoutable.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package net.floodlightcontroller.mastership;
-
-import org.restlet.Context;
-import org.restlet.Restlet;
-import org.restlet.routing.Router;
-
-import net.floodlightcontroller.restserver.RestletRoutable;
-
-public class MastershipWebRoutable implements RestletRoutable {
-
-	@Override
-	public Restlet getRestlet(Context context) {
-		Router router = new Router(context);
-		router.attach("/json", MastershipRouteResource.class);
-		return router;
-	}
-
-	@Override
-	public String basePath() {
-		return "/wm/mastership";
-	}
-
-}
diff --git a/src/main/java/net/onrc/onos/registry/controller/ControllerRegistryEntry.java b/src/main/java/net/onrc/onos/registry/controller/ControllerRegistryEntry.java
new file mode 100644
index 0000000..69fdf0b
--- /dev/null
+++ b/src/main/java/net/onrc/onos/registry/controller/ControllerRegistryEntry.java
@@ -0,0 +1,27 @@
+package net.onrc.onos.registry.controller;
+
+import org.codehaus.jackson.annotate.JsonProperty;
+
+
+public class ControllerRegistryEntry implements Comparable<ControllerRegistryEntry> {
+
+	private String controllerId;
+	private int sequenceNumber;
+	
+	public ControllerRegistryEntry(String controllerId, int sequenceNumber) {
+		this.controllerId = controllerId;
+		this.sequenceNumber = sequenceNumber;
+	}
+	
+	@JsonProperty("controllerId")
+	public String getControllerId(){
+		return controllerId;
+	}
+
+	@Override
+	public int compareTo(ControllerRegistryEntry o) {
+		return sequenceNumber - o.sequenceNumber;
+		//return 0;
+	}
+
+}
diff --git a/src/main/java/net/onrc/onos/registry/controller/ControllerRegistryResource.java b/src/main/java/net/onrc/onos/registry/controller/ControllerRegistryResource.java
new file mode 100644
index 0000000..51f1d32
--- /dev/null
+++ b/src/main/java/net/onrc/onos/registry/controller/ControllerRegistryResource.java
@@ -0,0 +1,35 @@
+package net.onrc.onos.registry.controller;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.restlet.resource.Get;
+import org.restlet.resource.ServerResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ControllerRegistryResource extends ServerResource {
+
+	protected static Logger log = LoggerFactory.getLogger(ControllerRegistryResource.class);
+
+	@Get("json")
+	public Collection<String> getControllers() {
+		IControllerRegistryService registry = 
+				(IControllerRegistryService) getContext().getAttributes().
+				get(IControllerRegistryService.class.getCanonicalName());
+		
+		Collection<String> controllers = null;
+		try {
+			controllers = registry.getAllControllers();
+		} catch (RegistryException e) {
+			log.warn("Error retrieving controller list: {}", e.getMessage());
+		}
+		
+		if (controllers == null){
+			controllers = new ArrayList<String>();
+		}
+		
+		return controllers;
+	}
+	
+}
diff --git a/src/main/java/net/onrc/onos/registry/controller/IControllerRegistryService.java b/src/main/java/net/onrc/onos/registry/controller/IControllerRegistryService.java
new file mode 100644
index 0000000..e924f6a
--- /dev/null
+++ b/src/main/java/net/onrc/onos/registry/controller/IControllerRegistryService.java
@@ -0,0 +1,49 @@
+package net.onrc.onos.registry.controller;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import net.floodlightcontroller.core.module.IFloodlightService;
+
+public interface IControllerRegistryService extends IFloodlightService {
+	
+	// Callback for all mastership changes. 
+	// Change callback is called when mastership is acquired or released
+	public interface ControlChangeCallback {
+		public void controlChanged(long dpid, boolean hasControl);
+	}
+	
+	// Acquire mastership for a switch. 
+	public void requestControl(long dpid, ControlChangeCallback cb) throws RegistryException;
+	
+	// Release mastership for a switch
+	public void releaseControl(long dpid);
+	
+	// Check if I am the master of a switch. This is a nonblocking call that checks if the caller is a 
+	public boolean hasControl(long dpid);
+	
+	// Set/Get mastership identifier.
+	// This is typically a unique identifier of the controller that does not change across restarts
+	public void setMastershipId (String id);
+	public String getMastershipId ();
+	
+	/**
+	 * Register a controller to the ONOS cluster
+	 * @param controller A string identifying the controller
+	 */
+	public void registerController(String controllerId) throws RegistryException;
+	
+	/**
+	 * Get all controllers in the cluster
+	 * @return
+	 */
+	public Collection<String> getAllControllers() throws RegistryException;
+	
+	
+	public String getControllerForSwitch(long dpid) throws RegistryException;
+	
+	public Map<String, List<ControllerRegistryEntry>> getAllSwitches();
+	
+	public Collection<Long> getSwitchesControlledByController(String controllerId);
+}
diff --git a/src/main/java/net/floodlightcontroller/mastership/IMastershipHelper.java b/src/main/java/net/onrc/onos/registry/controller/IMastershipHelper.java
similarity index 96%
rename from src/main/java/net/floodlightcontroller/mastership/IMastershipHelper.java
rename to src/main/java/net/onrc/onos/registry/controller/IMastershipHelper.java
index b4becfa..9e07d46 100644
--- a/src/main/java/net/floodlightcontroller/mastership/IMastershipHelper.java
+++ b/src/main/java/net/onrc/onos/registry/controller/IMastershipHelper.java
@@ -1,4 +1,4 @@
-package net.floodlightcontroller.mastership;
+package net.onrc.onos.registry.controller;
 
 public interface IMastershipHelper {
 	
diff --git a/src/main/java/net/floodlightcontroller/mastership/MastershipHelper.java b/src/main/java/net/onrc/onos/registry/controller/MastershipHelper.java
similarity index 92%
rename from src/main/java/net/floodlightcontroller/mastership/MastershipHelper.java
rename to src/main/java/net/onrc/onos/registry/controller/MastershipHelper.java
index fdac8ca..2fcd002 100644
--- a/src/main/java/net/floodlightcontroller/mastership/MastershipHelper.java
+++ b/src/main/java/net/onrc/onos/registry/controller/MastershipHelper.java
@@ -1,4 +1,4 @@
-package net.floodlightcontroller.mastership;
+package net.onrc.onos.registry.controller;
 
 public class MastershipHelper implements IMastershipHelper {
 
diff --git a/src/main/java/net/floodlightcontroller/mastership/RegistryException.java b/src/main/java/net/onrc/onos/registry/controller/RegistryException.java
similarity index 61%
rename from src/main/java/net/floodlightcontroller/mastership/RegistryException.java
rename to src/main/java/net/onrc/onos/registry/controller/RegistryException.java
index 134117c..3b237c2 100644
--- a/src/main/java/net/floodlightcontroller/mastership/RegistryException.java
+++ b/src/main/java/net/onrc/onos/registry/controller/RegistryException.java
@@ -1,14 +1,15 @@
-package net.floodlightcontroller.mastership;
+package net.onrc.onos.registry.controller;
 
 public class RegistryException extends Exception {
 
 	private static final long serialVersionUID = -8276300722010217913L;
-
+	
 	/*
 	public RegistryException() {
 		// TODO Auto-generated constructor stub
 	}
 
+	
 	public RegistryException(String message) {
 		super(message);
 		// TODO Auto-generated constructor stub
@@ -18,17 +19,10 @@
 		super(cause);
 		// TODO Auto-generated constructor stub
 	}
-
+	*/
+	
 	public RegistryException(String message, Throwable cause) {
 		super(message, cause);
-		// TODO Auto-generated constructor stub
 	}
 
-	public RegistryException(String message, Throwable cause,
-			boolean enableSuppression, boolean writableStackTrace) {
-		super(message, cause, enableSuppression, writableStackTrace);
-		// TODO Auto-generated constructor stub
-	}
-	*/
-
 }
diff --git a/src/main/java/net/onrc/onos/registry/controller/RegistryRunner.java b/src/main/java/net/onrc/onos/registry/controller/RegistryRunner.java
new file mode 100644
index 0000000..164f328
--- /dev/null
+++ b/src/main/java/net/onrc/onos/registry/controller/RegistryRunner.java
@@ -0,0 +1,82 @@
+package net.onrc.onos.registry.controller;
+
+import net.floodlightcontroller.core.module.FloodlightModuleContext;
+import net.floodlightcontroller.core.module.FloodlightModuleException;
+import net.onrc.onos.registry.controller.IControllerRegistryService.ControlChangeCallback;
+
+import org.openflow.util.HexString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Used for lightweight testing of the mastership module without having
+ * to load up the entire ONOS.
+ * @author jono
+ *
+ */
+public class RegistryRunner {
+	protected static Logger log = LoggerFactory.getLogger(RegistryRunner.class);
+
+	public static void main(String args[]){
+		FloodlightModuleContext fmc = new FloodlightModuleContext();
+		ZookeeperRegistry rm = new ZookeeperRegistry();
+		
+		fmc.addConfigParam(rm, "enableZookeeper", "true");
+		
+		String id = null;
+		if (args.length > 0){
+			id = args[0];
+			log.info("Using unique id: {}", id);
+		}
+		
+		try {
+			rm.init(fmc);
+			rm.startUp(fmc);
+			
+			if (id == null){
+				log.error("No unique ID supplied");
+				System.exit(1);
+			}
+			
+			rm.registerController(id);
+			//rm.setMastershipId(id);
+				
+			rm.requestControl(1L, 
+				new ControlChangeCallback(){
+					@Override
+					public void controlChanged(long dpid, boolean isMaster) {
+						if (isMaster){
+							log.debug("Callback for becoming master for {}", HexString.toHexString(dpid));
+						}
+						else {
+							log.debug("Callback for losing mastership for {}", HexString.toHexString(dpid));
+						}
+					}
+				});
+			
+			Thread.sleep(1000);
+			
+			/*
+			Map<String, List<ControllerRegistryEntry>> switches = rm.getAllSwitches();
+			for (List<ControllerRegistryEntry> ls : switches.values()){
+				for (ControllerRegistryEntry cre : ls){
+					log.debug("ctrlr: {}", cre.getControllerId());
+				}
+			}
+			*/
+			//"Server" loop
+			while (true) {
+				Thread.sleep(60000);
+			}
+			
+		} catch (FloodlightModuleException e) {
+			e.printStackTrace();
+		} catch (InterruptedException e) {
+			e.printStackTrace();
+		} catch (Exception e) {
+			e.printStackTrace();
+		}
+		
+		log.debug("is master: {}", rm.hasControl(1L));
+	}
+}
diff --git a/src/main/java/net/onrc/onos/registry/controller/RegistryWebRoutable.java b/src/main/java/net/onrc/onos/registry/controller/RegistryWebRoutable.java
new file mode 100644
index 0000000..74dede4
--- /dev/null
+++ b/src/main/java/net/onrc/onos/registry/controller/RegistryWebRoutable.java
@@ -0,0 +1,24 @@
+package net.onrc.onos.registry.controller;
+
+import net.floodlightcontroller.restserver.RestletRoutable;
+
+import org.restlet.Context;
+import org.restlet.Restlet;
+import org.restlet.routing.Router;
+
+public class RegistryWebRoutable implements RestletRoutable {
+
+	@Override
+	public Restlet getRestlet(Context context) {
+		Router router = new Router(context);
+		router.attach("/controllers/json", ControllerRegistryResource.class);
+		router.attach("/switches/json", SwitchRegistryResource.class);
+		return router;
+	}
+
+	@Override
+	public String basePath() {
+		return "/wm/registry";
+	}
+
+}
diff --git a/src/main/java/net/onrc/onos/registry/controller/StandaloneRegistry.java b/src/main/java/net/onrc/onos/registry/controller/StandaloneRegistry.java
new file mode 100644
index 0000000..e3181fe
--- /dev/null
+++ b/src/main/java/net/onrc/onos/registry/controller/StandaloneRegistry.java
@@ -0,0 +1,162 @@
+package net.onrc.onos.registry.controller;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import net.floodlightcontroller.core.module.FloodlightModuleContext;
+import net.floodlightcontroller.core.module.FloodlightModuleException;
+import net.floodlightcontroller.core.module.IFloodlightModule;
+import net.floodlightcontroller.core.module.IFloodlightService;
+import net.floodlightcontroller.restserver.IRestApiService;
+
+import org.openflow.util.HexString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StandaloneRegistry implements IFloodlightModule,
+		IControllerRegistryService {
+	protected static Logger log = LoggerFactory.getLogger(StandaloneRegistry.class);
+	
+	protected IRestApiService restApi;
+	
+	//protected String controllerId;
+	
+	protected String controllerId = null;
+	protected Map<String, ControlChangeCallback> switchCallbacks;
+	
+
+	@Override
+	public void requestControl(long dpid, ControlChangeCallback cb)
+			throws RegistryException {
+		if (controllerId == null) {
+			throw new RuntimeException("Must register a controller before calling requestControl");
+		}
+		
+		switchCallbacks.put(HexString.toHexString(dpid), cb);
+		
+		log.debug("Control granted for {}", HexString.toHexString(dpid));
+		
+		//Immediately grant request for control
+		if (cb != null) {
+			cb.controlChanged(dpid, true);
+		}
+	}
+
+	@Override
+	public void releaseControl(long dpid) {
+		ControlChangeCallback cb = switchCallbacks.remove(HexString.toHexString(dpid));
+		
+		log.debug("Control released for {}", HexString.toHexString(dpid));
+		
+		if (cb != null){
+			cb.controlChanged(dpid, false);
+		}
+	}
+
+	@Override
+	public boolean hasControl(long dpid) {
+		return switchCallbacks.containsKey(HexString.toHexString(dpid));
+	}
+
+	@Override
+	public void setMastershipId(String id) {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Override
+	public String getMastershipId() {
+		return controllerId;
+	}
+
+	@Override
+	public void registerController(String controllerId)
+			throws RegistryException {
+		this.controllerId = controllerId;
+	}
+
+	@Override
+	public Collection<String> getAllControllers() throws RegistryException {
+		List<String> l = new ArrayList<String>();
+		l.add(controllerId);
+		return l;
+	}
+
+	@Override
+	public String getControllerForSwitch(long dpid) throws RegistryException {
+		return controllerId;
+	}
+
+	@Override
+	public Map<String, List<ControllerRegistryEntry>> getAllSwitches() {
+		Map<String, List<ControllerRegistryEntry>> switches = 
+				new HashMap<String, List<ControllerRegistryEntry>>();
+		
+		for (String strSwitch : switchCallbacks.keySet()){
+			log.debug("Swtich _{}", strSwitch);
+			List<ControllerRegistryEntry> list = new ArrayList<ControllerRegistryEntry>();
+			list.add(new ControllerRegistryEntry(controllerId, 0));
+			
+			switches.put(strSwitch, list);
+		}
+		
+		return switches;
+	}
+
+	@Override
+	public Collection<Long> getSwitchesControlledByController(
+			String controllerId) {
+		throw new RuntimeException("Not yet implemented");
+	}
+
+	@Override
+	public Collection<Class<? extends IFloodlightService>> getModuleServices() {
+		Collection<Class<? extends IFloodlightService>> l = 
+				new ArrayList<Class<? extends IFloodlightService>>();
+		l.add(IControllerRegistryService.class);
+		return l;
+	}
+
+	@Override
+	public Map<Class<? extends IFloodlightService>, IFloodlightService> getServiceImpls() {
+		Map<Class<? extends IFloodlightService>, IFloodlightService> m = 
+				new HashMap<Class<? extends IFloodlightService>, IFloodlightService>();
+		m.put(IControllerRegistryService.class,  this);
+		return m;
+	}
+
+	@Override
+	public Collection<Class<? extends IFloodlightService>> getModuleDependencies() {
+		Collection<Class<? extends IFloodlightService>> l =
+                new ArrayList<Class<? extends IFloodlightService>>();
+        l.add(IRestApiService.class);
+		return l;
+	}
+
+	@Override
+	public void init(FloodlightModuleContext context)
+			throws FloodlightModuleException {
+		restApi = context.getServiceImpl(IRestApiService.class);
+		
+		switchCallbacks = new HashMap<String, ControlChangeCallback>();
+		
+		//Put some data in for testing
+		/*
+		try {
+			registerController("hurro");
+			requestControl(2L, null);
+		} catch (RegistryException e1) {
+			// TODO Auto-generated catch block
+			e1.printStackTrace();
+		}*/
+	}
+
+	@Override
+	public void startUp(FloodlightModuleContext context) {
+		restApi.addRestletRoutable(new RegistryWebRoutable());
+	}
+
+}
diff --git a/src/main/java/net/onrc/onos/registry/controller/SwitchRegistryResource.java b/src/main/java/net/onrc/onos/registry/controller/SwitchRegistryResource.java
new file mode 100644
index 0000000..d6f3dee
--- /dev/null
+++ b/src/main/java/net/onrc/onos/registry/controller/SwitchRegistryResource.java
@@ -0,0 +1,37 @@
+package net.onrc.onos.registry.controller;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.restlet.resource.Get;
+import org.restlet.resource.ServerResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SwitchRegistryResource extends ServerResource {
+
+	protected static Logger log = LoggerFactory.getLogger(SwitchRegistryResource.class);
+	
+	@Get("json")
+	public Map<String, List<ControllerRegistryEntry>> getAllControllers(){
+		IControllerRegistryService registry = 
+				(IControllerRegistryService) getContext().getAttributes().
+				get(IControllerRegistryService.class.getCanonicalName());
+		
+		Map<String, List<ControllerRegistryEntry>> switches = null;
+		switches = registry.getAllSwitches();
+		
+		if (switches == null){
+			switches = new HashMap<String, List<ControllerRegistryEntry>>();
+		}
+		
+		for (List<ControllerRegistryEntry> list: switches.values()){
+			for (ControllerRegistryEntry en : list) {
+				log.debug("Controller id {}", en.getControllerId());
+			}
+		}
+		
+		return switches;
+	}
+}
diff --git a/src/main/java/net/onrc/onos/registry/controller/ZookeeperRegistry.java b/src/main/java/net/onrc/onos/registry/controller/ZookeeperRegistry.java
new file mode 100644
index 0000000..58b8bc7
--- /dev/null
+++ b/src/main/java/net/onrc/onos/registry/controller/ZookeeperRegistry.java
@@ -0,0 +1,487 @@
+package net.onrc.onos.registry.controller;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import net.floodlightcontroller.core.module.FloodlightModuleContext;
+import net.floodlightcontroller.core.module.FloodlightModuleException;
+import net.floodlightcontroller.core.module.IFloodlightModule;
+import net.floodlightcontroller.core.module.IFloodlightService;
+import net.floodlightcontroller.restserver.IRestApiService;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.openflow.util.HexString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.netflix.curator.RetryPolicy;
+import com.netflix.curator.framework.CuratorFramework;
+import com.netflix.curator.framework.CuratorFrameworkFactory;
+import com.netflix.curator.framework.api.CuratorWatcher;
+import com.netflix.curator.framework.recipes.cache.ChildData;
+import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
+import com.netflix.curator.framework.recipes.cache.PathChildrenCache.StartMode;
+import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
+import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
+import com.netflix.curator.framework.recipes.leader.LeaderLatch;
+import com.netflix.curator.framework.recipes.leader.Participant;
+import com.netflix.curator.retry.ExponentialBackoffRetry;
+
+public class ZookeeperRegistry implements IFloodlightModule, IControllerRegistryService {
+
+	protected static Logger log = LoggerFactory.getLogger(ZookeeperRegistry.class);
+	protected String controllerId = null;
+	
+	protected IRestApiService restApi;
+	
+	//TODO read this from configuration
+	protected String connectionString = "localhost:2181";
+	
+	
+	private final String namespace = "onos";
+	private final String switchLatchesPath = "/switches";
+	private final String controllerPath = "/controllers";
+	
+	protected CuratorFramework client;
+	
+	protected PathChildrenCache controllerCache;
+	protected PathChildrenCache switchCache;
+
+	protected Map<String, LeaderLatch> switchLatches;
+	protected Map<String, ControlChangeCallback> switchCallbacks;
+	protected Map<String, PathChildrenCache> switchPathCaches;
+	
+	//protected boolean zookeeperEnabled = false;
+	
+	protected class ParamaterizedCuratorWatcher implements CuratorWatcher {
+		private String dpid;
+		private boolean isLeader = false;
+		private String latchPath;
+		
+		public ParamaterizedCuratorWatcher(String dpid, String latchPath){
+			this.dpid = dpid;
+			this.latchPath = latchPath;
+		}
+		
+		@Override
+		public synchronized void process(WatchedEvent event) throws Exception {
+			log.debug("Watch Event: {}", event);
+
+			LeaderLatch latch = switchLatches.get(dpid);
+			
+			if (event.getState() == KeeperState.Disconnected){
+				if (isLeader) {
+					log.debug("Disconnected while leader - lost leadership for {}", dpid);
+					
+					isLeader = false;
+					ControlChangeCallback cb = switchCallbacks.get(dpid);
+					if (cb != null) {
+						//Allow callback to be null if the requester doesn't want a callback
+						cb.controlChanged(HexString.toLong(dpid), false);
+					}
+				}
+				return;
+			}
+			
+			try {
+				
+				Participant leader = latch.getLeader();
+
+				if (leader.getId().equals(controllerId) && !isLeader){
+					log.debug("Became leader for {}", dpid);
+					
+					isLeader = true;
+					switchCallbacks.get(dpid).controlChanged(HexString.toLong(dpid), true);
+				}
+				else if (!leader.getId().equals(controllerId) && isLeader){
+					log.debug("Lost leadership for {}", dpid);
+					
+					isLeader = false;
+					switchCallbacks.get(dpid).controlChanged(HexString.toLong(dpid), false);
+				}
+			} catch (Exception e){
+				if (isLeader){
+					log.debug("Exception checking leadership status. Assume leadship lost for {}",
+							dpid);
+					
+					isLeader = false;
+					switchCallbacks.get(dpid).controlChanged(HexString.toLong(dpid), false);
+				}
+			}
+			
+			client.getChildren().usingWatcher(this).inBackground().forPath(latchPath);
+			//client.getChildren().usingWatcher(this).forPath(latchPath);
+		}
+	}
+	
+	
+	/*
+	 * Listens for changes to the switch znodes in Zookeeper. This maintains the second level of
+	 * PathChildrenCaches that hold the controllers contending for each switch - there's one for
+	 * each switch.
+	 */
+	PathChildrenCacheListener switchPathCacheListener = new PathChildrenCacheListener() {
+		@Override
+		public void childEvent(CuratorFramework client,
+				PathChildrenCacheEvent event) throws Exception {
+			// TODO Auto-generated method stub
+			log.debug("Root switch path cache got {} event", event.getType());
+			
+			String strSwitch = null;
+			if (event.getData() != null){
+				log.debug("Event path {}", event.getData().getPath());
+				String[] splitted = event.getData().getPath().split("/");
+				strSwitch = splitted[splitted.length - 1];
+				log.debug("Switch name is {}", strSwitch);
+			}
+			
+			switch (event.getType()){
+			case CHILD_ADDED:
+			case CHILD_UPDATED:
+				//Check we have a PathChildrenCache for this child, add one if not
+				if (switchPathCaches.get(strSwitch) == null){
+					PathChildrenCache pc = new PathChildrenCache(client, 
+							event.getData().getPath(), true);
+					pc.start(StartMode.NORMAL);
+					switchPathCaches.put(strSwitch, pc);
+				}
+				break;
+			case CHILD_REMOVED:
+				//Remove our PathChildrenCache for this child
+				PathChildrenCache pc = switchPathCaches.remove(strSwitch);
+				pc.close();
+				break;
+			default:
+				//All other events are connection status events. We need to do anything
+				//as the path cache handles these on its own.
+				break;
+			}
+			
+		}
+	};
+
+	
+	@Override
+	public void requestControl(long dpid, ControlChangeCallback cb) throws RegistryException {
+		/*
+		if (!zookeeperEnabled) {
+			//If zookeeper connection is disabled all control requests succeed immediately
+			if (cb != null){
+				cb.controlChanged(dpid, true);
+			}
+			return;
+		}*/
+		
+		if (controllerId == null){
+			throw new RuntimeException("Must register a controller before calling requestControl");
+		}
+		
+		String dpidStr = HexString.toHexString(dpid);
+		String latchPath = switchLatchesPath + "/" + dpidStr;
+		
+		if (switchLatches.get(dpidStr) != null){
+			throw new RuntimeException("Leader election for switch " + dpidStr +
+					"is already running");
+		}
+		
+		LeaderLatch latch = new LeaderLatch(client, latchPath, controllerId);
+		switchLatches.put(dpidStr, latch);
+		switchCallbacks.put(dpidStr, cb);
+		
+		try {
+			//client.getChildren().usingWatcher(watcher).inBackground().forPath(singleLatchPath);
+			client.getChildren().usingWatcher(
+					new ParamaterizedCuratorWatcher(dpidStr, latchPath))
+					.inBackground().forPath(latchPath);
+			latch.start();
+		} catch (Exception e) {
+			log.warn("Error starting leader latch: {}", e.getMessage());
+			throw new RegistryException("Error starting leader latch for " + dpidStr, e);
+		}
+		
+	}
+
+	@Override
+	public void releaseControl(long dpid) {
+		//if (!zookeeperEnabled) return;
+		
+		String dpidStr = HexString.toHexString(dpid);
+		
+		LeaderLatch latch = switchLatches.get(dpidStr);
+		if (latch == null) {
+			log.debug("Trying to release mastership for switch we are not contesting");
+			return;
+		}
+		
+		try {
+			latch.close();
+		} catch (IOException e) {
+			//I think it's OK not to do anything here. Either the node got deleted correctly,
+			//or the connection went down and the node got deleted.
+		} finally {
+			switchLatches.remove(dpidStr);
+			switchCallbacks.remove(dpidStr);
+		}
+	}
+
+	@Override
+	public boolean hasControl(long dpid) {
+		//if (!zookeeperEnabled) return false;
+		
+		LeaderLatch latch = switchLatches.get(HexString.toHexString(dpid));
+		
+		if (latch == null) {
+			log.warn("No leader latch for dpid {}", HexString.toHexString(dpid));
+			return false;
+		}
+		
+		try {
+			return latch.getLeader().getId().equals(controllerId);
+		} catch (Exception e) {
+			//TODO swallow exception?
+			return false;
+		}
+	}
+
+	@Override
+	public void setMastershipId(String id) {
+		//TODO remove this method if not needed
+		//controllerId = id;
+	}
+
+	@Override
+	public String getMastershipId() {
+		return controllerId;
+	}
+	
+	@Override
+	public Collection<String> getAllControllers() throws RegistryException {
+		//if (!zookeeperEnabled) return null;
+		
+		log.debug("Getting all controllers");
+		
+		List<String> controllers = new ArrayList<String>();
+		for (ChildData data : controllerCache.getCurrentData()){
+
+			String d = null;
+			try {
+				d = new String(data.getData(), "UTF-8");
+			} catch (UnsupportedEncodingException e) {
+				throw new RegistryException("Error encoding string", e);
+			}
+
+			controllers.add(d);
+		}
+		return controllers;
+	}
+
+	@Override
+	public void registerController(String id) throws RegistryException {
+		//if (!zookeeperEnabled) return;
+		
+		controllerId = id;
+		
+		byte bytes[] = null;
+		try {
+			bytes = id.getBytes("UTF-8");
+		} catch (UnsupportedEncodingException e1) {
+			throw new RegistryException("Error encoding string", e1);
+		}
+		
+		String path = controllerPath + "/" + id;
+		
+		log.info("Registering controller with id {}", id);
+		
+		//Create ephemeral node in controller registry
+		try {
+			client.create().withProtection().withMode(CreateMode.EPHEMERAL)
+					.forPath(path, bytes);
+		} catch (Exception e) {
+			throw new RegistryException("Error contacting the Zookeeper service", e);
+		}
+	}
+	
+	@Override
+	public String getControllerForSwitch(long dpid) throws RegistryException {
+		//if (!zookeeperEnabled) return null;
+		// TODO Work out how we should store this controller/switch data.
+		// The leader latch might be a index to the /controllers collections
+		// which holds more info on the controller (how to talk to it for example).
+		
+		
+		String strDpid = HexString.toHexString(dpid);
+		LeaderLatch latch = switchLatches.get(strDpid);
+		
+		if (latch == null){
+			log.warn("Tried to get controller for non-existent switch");
+			return null;
+		}
+		
+		Participant leader = null;
+		try {
+			leader = latch.getLeader();
+		} catch (Exception e) {
+			throw new RegistryException("Error contacting the Zookeeper service", e);
+		}
+		
+		return leader.getId();
+	}
+	
+	@Override
+	public Collection<Long> getSwitchesControlledByController(String controllerId) {
+		//TODO remove this if not needed
+		throw new RuntimeException("Not yet implemented");
+	}
+	
+
+	@Override
+	public Map<String, List<ControllerRegistryEntry>> getAllSwitches() {
+		Map<String, List<ControllerRegistryEntry>> data = 
+				new HashMap<String, List<ControllerRegistryEntry>>();
+		
+		for (Map.Entry<String, PathChildrenCache> entry : switchPathCaches.entrySet()){
+			List<ControllerRegistryEntry> contendingControllers =
+					 new ArrayList<ControllerRegistryEntry>(); 
+			
+			if (entry.getValue().getCurrentData().size() < 1){
+				log.info("Switch entry with no leader elections: {}", entry.getKey());
+				continue;
+			}
+			
+			for (ChildData d : entry.getValue().getCurrentData()) {
+				/*
+				if (d.getPath().length() < 1){
+					log.info("Switch entry with no leader elections: {}", d.getPath());
+					continue;
+				}
+				*/
+				
+				String controllerId = null;
+				try {
+					controllerId = new String(d.getData(), "UTF-8");
+				} catch (UnsupportedEncodingException e) {
+					log.warn("Encoding exception: {}", e.getMessage());
+				}
+				
+				String[] splitted = d.getPath().split("-");
+				int sequenceNumber = Integer.parseInt(splitted[splitted.length - 1]);
+				
+				contendingControllers.add(new ControllerRegistryEntry(controllerId, sequenceNumber));
+			 }
+			
+			Collections.sort(contendingControllers);
+			data.put(entry.getKey(), contendingControllers);
+		}
+		return data;
+	}
+	
+	/*
+	 * IFloodlightModule
+	 */
+	
+	@Override
+	public Collection<Class<? extends IFloodlightService>> getModuleServices() {
+		Collection<Class<? extends IFloodlightService>> l = 
+				new ArrayList<Class<? extends IFloodlightService>>();
+		l.add(IControllerRegistryService.class);
+		return l;
+	}
+	
+	@Override
+	public Map<Class<? extends IFloodlightService>, IFloodlightService> getServiceImpls() {
+		Map<Class<? extends IFloodlightService>, IFloodlightService> m = 
+				new HashMap<Class<? extends IFloodlightService>, IFloodlightService>();
+		m.put(IControllerRegistryService.class,  this);
+		return m;
+	}
+	
+	@Override
+	public Collection<Class<? extends IFloodlightService>> getModuleDependencies() {
+		Collection<Class<? extends IFloodlightService>> l =
+                new ArrayList<Class<? extends IFloodlightService>>();
+        l.add(IRestApiService.class);
+		return l;
+	}
+	
+	@Override
+	public void init (FloodlightModuleContext context) throws FloodlightModuleException {
+		
+		log.info("Initialising the Zookeeper Registry - Zookeeper connection required");
+		
+		restApi = context.getServiceImpl(IRestApiService.class);
+		
+		//We have a config option that determines whether we try and connect to 
+		//zookeeper or not. By default zookeeper connection is disabled. When we don't
+		//have a zookeeper connection we act as though we are the only server in the 
+		//cluster, i.e. all control requests will succeed.
+		/*Map<String, String> configOptions = context.getConfigParams(this);
+		String enableZookeeper = configOptions.get("enableZookeeper");
+		if (enableZookeeper != null) {
+			log.info("Enabling Zookeeper connection");
+			zookeeperEnabled = true;
+		}
+		else {
+			log.info("Zookeeper connectivity is disabled - running in standalone mode");
+			return;
+		}*/
+		
+		/*
+		try {
+			String localHostname = java.net.InetAddress.getLocalHost().getHostName();
+			controllerId = localHostname;
+			log.debug("Setting controller id to {}", controllerId);
+		} catch (UnknownHostException e) {
+			// TODO Handle this exception
+			e.printStackTrace();
+		}*/
+
+		switchLatches = new HashMap<String, LeaderLatch>();
+		switchCallbacks = new HashMap<String, ControlChangeCallback>();
+		switchPathCaches = new HashMap<String, PathChildrenCache>();
+		
+		RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+		client = CuratorFrameworkFactory.newClient(connectionString, retryPolicy);
+		
+		client.start();
+		
+		client = client.usingNamespace(namespace);
+		
+		//Put some data in for testing
+		/*
+		try {
+			registerController("zookeeperController");
+			requestControl(2L, null);
+		} catch (RegistryException e1) {
+			// TODO Auto-generated catch block
+			e1.printStackTrace();
+		}*/
+		
+		controllerCache = new PathChildrenCache(client, controllerPath, true);
+		switchCache = new PathChildrenCache(client, switchLatchesPath, true);
+		switchCache.getListenable().addListener(switchPathCacheListener);
+		
+		try {
+			controllerCache.start(StartMode.BUILD_INITIAL_CACHE);
+			
+			//Don't prime the cache, we want a notification for each child node in the path
+			switchCache.start(StartMode.NORMAL);
+		} catch (Exception e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		}
+	}
+	
+	@Override
+	public void startUp (FloodlightModuleContext context) {
+		restApi.addRestletRoutable(new RegistryWebRoutable());
+	}
+
+}
diff --git a/src/main/resources/META-INF/services/net.floodlightcontroller.core.module.IFloodlightModule b/src/main/resources/META-INF/services/net.floodlightcontroller.core.module.IFloodlightModule
index e683c79..99ca4c8 100644
--- a/src/main/resources/META-INF/services/net.floodlightcontroller.core.module.IFloodlightModule
+++ b/src/main/resources/META-INF/services/net.floodlightcontroller.core.module.IFloodlightModule
@@ -23,7 +23,9 @@
 net.floodlightcontroller.core.test.MockFloodlightProvider
 net.floodlightcontroller.core.test.MockThreadPoolService
 net.floodlightcontroller.firewall.Firewall
-net.floodlightcontroller.mastership.MastershipManager
 net.floodlightcontroller.onoslistener.OnosPublisher
 net.floodlightcontroller.flowcache.FlowManager
 net.floodlightcontroller.routing.TopoRouteService
+net.onrc.onos.registry.controller.ZookeeperRegistry
+net.onrc.onos.registry.controller.StandaloneRegistry
+
diff --git a/src/main/resources/floodlightdefault.properties b/src/main/resources/floodlightdefault.properties
index b3d2de6..9ea7a92 100644
--- a/src/main/resources/floodlightdefault.properties
+++ b/src/main/resources/floodlightdefault.properties
@@ -8,8 +8,9 @@
 net.floodlightcontroller.jython.JythonDebugInterface,\
 net.floodlightcontroller.counter.CounterStore,\
 net.floodlightcontroller.perfmon.PktInProcessingTime,\
-net.floodlightcontroller.ui.web.StaticWebRoutable, \
-net.floodlightcontroller.onoslistener.OnosPublisher
+net.floodlightcontroller.ui.web.StaticWebRoutable,\
+net.floodlightcontroller.onoslistener.OnosPublisher, \
+net.onrc.onos.registry.controller.StandaloneRegistry
 net.floodlightcontroller.restserver.RestApiServer.port = 8080
 net.floodlightcontroller.core.FloodlightProvider.openflowport = 6633
 net.floodlightcontroller.jython.JythonDebugInterface.port = 6655