LeadershipStore updates:
 - Now tracking leader and candidates for a topic using a single map.
 - Using term numbers that are incremented by one every time a new leader is elected.
 - Introduced a separate LeadershipStore to conform to the  manager-store pattern

Change-Id: I1d03a6c5e8ff0e68ef0c1e3a6c2d425c4856e470
diff --git a/core/api/src/main/java/org/onosproject/cluster/Leader.java b/core/api/src/main/java/org/onosproject/cluster/Leader.java
new file mode 100644
index 0000000..8e5528d
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/cluster/Leader.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.cluster;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Objects;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * Topic leader.
+ * <p>
+ * Identified by the {@link NodeId node identifier} and a monotonically increasing term number.
+ * The term number is incremented by one every time a new node is elected as leader.
+ * Also available is the system clock time at the instant when this node was elected as leader.
+ * Keep in mind though that as with any system clock based time stamps this particular information
+ * susceptible to clock skew and should only be relied on for simple diagnostic purposes.
+ */
+public class Leader {
+    private final NodeId nodeId;
+    private final long term;
+    private final long termStartTime;
+
+    public Leader(NodeId nodeId, long term, long termStartTime) {
+        this.nodeId = checkNotNull(nodeId);
+        checkArgument(term >= 0, "term must be non-negative");
+        this.term = term;
+        checkArgument(termStartTime >= 0, "termStartTime must be non-negative");
+        this.termStartTime = termStartTime;
+    }
+
+    /**
+     * Returns the identifier for of leader.
+     * @return node identifier
+     */
+    public NodeId nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * Returns the leader's term.
+     * @return leader term
+     */
+    public long term() {
+        return term;
+    }
+
+    /**
+     * Returns the system time when the current leadership term started.
+     * @return current leader term start time
+     */
+    public long termStartTime() {
+        return termStartTime;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (other != null && other instanceof Leader) {
+            Leader that = (Leader) other;
+            return Objects.equal(this.nodeId, that.nodeId) &&
+                    this.term ==  that.term &&
+                    this.termStartTime == that.termStartTime;
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(nodeId, term, termStartTime);
+    }
+
+    @Override
+    public String toString() {
+        return MoreObjects.toStringHelper(getClass())
+            .add("nodeId", nodeId)
+            .add("term", term)
+            .add("termStartTime", termStartTime)
+            .toString();
+    }
+}
diff --git a/core/api/src/main/java/org/onosproject/cluster/Leadership.java b/core/api/src/main/java/org/onosproject/cluster/Leadership.java
index 113e19c..b31c03f 100644
--- a/core/api/src/main/java/org/onosproject/cluster/Leadership.java
+++ b/core/api/src/main/java/org/onosproject/cluster/Leadership.java
@@ -17,63 +17,31 @@
 
 import java.util.Objects;
 import java.util.List;
-import java.util.Optional;
-
-import org.joda.time.DateTime;
 
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 
 /**
- * Abstract leadership concept. The information carried by this construct
- * include the topic of contention, the {@link NodeId}s of Nodes that could
- * become leader for the topic, the epoch when the term for a given leader
- * began, and the system time when the term began. Note:
- * <ul>
- * <li>The list of NodeIds may include the current leader at index 0, and the
- * rest in decreasing preference order.</li>
- * <li>The epoch is the logical age of a Leadership construct, and should be
- * used for comparing two Leaderships, but only of the same topic.</li>
- * <li>The leader may be null if its accuracy can't be guaranteed. This applies
- * to CANDIDATES_CHANGED events and candidate board contents.</li>
- * </ul>
+ * State of leadership for topic.
+ * <p>
+ * Provided by this construct is the current {@link Leader leader} and the list of
+ * {@link NodeId nodeId}s currently registered as candidates for election for the topic.
+ * Keep in mind that only registered candidates can become leaders.
  */
 public class Leadership {
 
     private final String topic;
-    private final Optional<NodeId> leader;
+    private final Leader leader;
     private final List<NodeId> candidates;
-    private final long epoch;
-    private final long electedTime;
 
-    public Leadership(String topic, NodeId leader, long epoch, long electedTime) {
+    public Leadership(String topic, Leader leader, List<NodeId> candidates) {
         this.topic = topic;
-        this.leader = Optional.of(leader);
-        this.candidates = ImmutableList.of(leader);
-        this.epoch = epoch;
-        this.electedTime = electedTime;
-    }
-
-    public Leadership(String topic, NodeId leader, List<NodeId> candidates,
-            long epoch, long electedTime) {
-        this.topic = topic;
-        this.leader = Optional.of(leader);
+        this.leader = leader;
         this.candidates = ImmutableList.copyOf(candidates);
-        this.epoch = epoch;
-        this.electedTime = electedTime;
-    }
-
-    public Leadership(String topic, List<NodeId> candidates,
-            long epoch, long electedTime) {
-        this.topic = topic;
-        this.leader = Optional.empty();
-        this.candidates = ImmutableList.copyOf(candidates);
-        this.epoch = epoch;
-        this.electedTime = electedTime;
     }
 
     /**
-     * The topic for which this leadership applies.
+     * Returns the leadership topic.
      *
      * @return leadership topic.
      */
@@ -82,13 +50,21 @@
     }
 
     /**
-     * The nodeId of leader for this topic.
+     * Returns the {@link NodeId nodeId} of the leader.
      *
-     * @return leader node.
+     * @return leader node identifier; will be null if there is no leader
      */
-    // This will return Optional<NodeId> in the future.
-    public NodeId leader() {
-        return leader.orElse(null);
+    public NodeId leaderNodeId() {
+        return leader == null ? null : leader.nodeId();
+    }
+
+    /**
+     * Returns the leader for this topic.
+     *
+     * @return leader; will be null if there is no leader for topic
+     */
+    public Leader leader() {
+        return leader;
     }
 
     /**
@@ -101,38 +77,9 @@
         return candidates;
     }
 
-    /**
-     * The epoch when the leadership was assumed.
-     * <p>
-     * Comparing epochs is only appropriate for leadership events for the same
-     * topic. The system guarantees that for any given topic the epoch for a new
-     * term is higher (not necessarily by 1) than the epoch for any previous
-     * term.
-     *
-     * @return leadership epoch
-     */
-    public long epoch() {
-        return epoch;
-    }
-
-    /**
-     * The system time when the term started.
-     * <p>
-     * The elected time is initially set on the node coordinating
-     * the leader election using its local system time. Due to possible
-     * clock skew, relying on this value for determining event ordering
-     * is discouraged. Epoch is more appropriate for determining
-     * event ordering.
-     *
-     * @return elected time.
-     */
-    public long electedTime() {
-        return electedTime;
-    }
-
     @Override
     public int hashCode() {
-        return Objects.hash(topic, leader, candidates, epoch, electedTime);
+        return Objects.hash(topic, leader, candidates);
     }
 
     @Override
@@ -144,9 +91,7 @@
             final Leadership other = (Leadership) obj;
             return Objects.equals(this.topic, other.topic) &&
                     Objects.equals(this.leader, other.leader) &&
-                    Objects.equals(this.candidates, other.candidates) &&
-                    Objects.equals(this.epoch, other.epoch) &&
-                    Objects.equals(this.electedTime, other.electedTime);
+                    Objects.equals(this.candidates, other.candidates);
         }
         return false;
     }
@@ -157,8 +102,6 @@
             .add("topic", topic)
             .add("leader", leader)
             .add("candidates", candidates)
-            .add("epoch", epoch)
-            .add("electedTime", new DateTime(electedTime))
             .toString();
     }
 }
diff --git a/core/api/src/main/java/org/onosproject/cluster/LeadershipAdminService.java b/core/api/src/main/java/org/onosproject/cluster/LeadershipAdminService.java
new file mode 100644
index 0000000..c58ca71
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/cluster/LeadershipAdminService.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.cluster;
+
+/**
+ * Interface for administratively manipulating leadership assignments.
+ */
+public interface LeadershipAdminService {
+
+    /**
+     * Attempts to assign leadership for a topic to a specified node.
+     * @param topic leadership topic
+     * @param nodeId identifier of the node to be made leader
+     * @return true is the transfer was successfully executed. This method returns {@code false}
+     * if {@code nodeId} is not one of the candidates for for the topic.
+     */
+    boolean transferLeadership(String topic, NodeId nodeId);
+
+    /**
+     * Make a node to be the next leader by promoting it to top of candidate list.
+     * @param topic leadership topic
+     * @param nodeId identifier of node to be next leader
+     * @return {@code true} if nodeId is now the top candidate. This method returns {@code false}
+     * if {@code nodeId} is not one of the candidates for for the topic.
+     */
+    boolean promoteToTopOfCandidateList(String topic, NodeId nodeId);
+
+    /**
+     * Removes all active leadership registrations for a given node.
+     * <p>
+     * This method will also evict the node from leaderships that it currently owns.
+     * @param nodeId node identifier
+     */
+    void unregister(NodeId nodeId);
+}
diff --git a/core/api/src/main/java/org/onosproject/cluster/LeadershipEvent.java b/core/api/src/main/java/org/onosproject/cluster/LeadershipEvent.java
index d1fe670..6fb310d 100644
--- a/core/api/src/main/java/org/onosproject/cluster/LeadershipEvent.java
+++ b/core/api/src/main/java/org/onosproject/cluster/LeadershipEvent.java
@@ -27,33 +27,25 @@
 public class LeadershipEvent extends AbstractEvent<LeadershipEvent.Type, Leadership> {
 
     /**
-     * Type of leadership-related events.
+     * Type of leadership events.
      */
     public enum Type {
         /**
-         * Signifies that the leader has been elected.
-         * The event subject is the new leader.
-         * This event does not guarantee accurate candidate information.
+         * Signifies a change in both the leader as well as change to the list of candidates. Keep in mind though that
+         * the first node entering the race will trigger this event as it will become a candidate and automatically get
+         * promoted to become leader.
          */
-        LEADER_ELECTED,
+        LEADER_AND_CANDIDATES_CHANGED,
 
         /**
-         * Signifies that the leader has been re-elected.
-         * The event subject is the leader.
-         * This event does not guarantee accurate candidate information.
+         * Signifies that the leader for a topic has changed.
          */
-        LEADER_REELECTED,
+        // TODO: We may not need this. We currently do not support a way for a current leader to step down
+        // while still reamining a candidate
+        LEADER_CHANGED,
 
         /**
-         * Signifies that the leader has been booted and lost leadership.
-         * The event subject is the former leader.
-         * This event does not guarantee accurate candidate information.
-         */
-        LEADER_BOOTED,
-
-        /**
-         * Signifies that the list of candidates for leadership for a topic has
-         * changed. This event does not guarantee accurate leader information.
+         * Signifies a change in the list of candidates for a topic.
          */
         CANDIDATES_CHANGED
     }
diff --git a/core/api/src/main/java/org/onosproject/cluster/LeadershipService.java b/core/api/src/main/java/org/onosproject/cluster/LeadershipService.java
index 7d1f607..b0f7a8d 100644
--- a/core/api/src/main/java/org/onosproject/cluster/LeadershipService.java
+++ b/core/api/src/main/java/org/onosproject/cluster/LeadershipService.java
@@ -17,87 +17,73 @@
 
 import org.onosproject.event.ListenerService;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.CompletableFuture;
 
 /**
  * Service for leader election.
+ * <p>
  * Leadership contests are organized around topics. A instance can join the
  * leadership race for a topic or withdraw from a race it has previously joined.
+ * <p>
  * Listeners can be added to receive notifications asynchronously for various
  * leadership contests.
+ * <p>
+ * When a node gets elected as a leader for a topic, all nodes receive notifications
+ * indicating a change in leadership.
  */
 public interface LeadershipService
     extends ListenerService<LeadershipEvent, LeadershipEventListener> {
 
     /**
-     * Returns the current leader for the topic.
+     * Returns the {@link NodeId node identifier} that is the current leader for a topic.
      *
-     * @param path topic
-     * @return nodeId of the leader, null if so such topic exists.
+     * @param topic leadership topic
+     * @return node identifier of the current leader; {@code null} if there is no leader for the topic
      */
-    NodeId getLeader(String path);
+    default NodeId getLeader(String topic) {
+        Leadership leadership = getLeadership(topic);
+        return leadership == null ? null : leadership.leaderNodeId();
+    }
 
     /**
-     * Returns the current leadership info for the topic.
+     * Returns the current {@link Leadership leadership} for a topic.
      *
-     * @param path topic
-     * @return leadership info or null if so such topic exists.
+     * @param topic leadership topic
+     * @return leadership or {@code null} if no such topic exists
      */
-    Leadership getLeadership(String path);
+    Leadership getLeadership(String topic);
 
     /**
-     * Returns the set of topics owned by the specified node.
+     * Returns the set of topics owned by the specified {@link NodeId node}.
      *
-     * @param nodeId node Id.
+     * @param nodeId node identifier.
      * @return set of topics for which this node is the current leader.
      */
-    Set<String> ownedTopics(NodeId nodeId);
+    default Set<String> ownedTopics(NodeId nodeId) {
+        return Maps.filterValues(getLeaderBoard(), v -> Objects.equal(nodeId, v.leaderNodeId())).keySet();
+    }
 
     /**
-     * Joins the leadership contest.
+     * Enters a leadership contest.
      *
-     * @param path topic for which this controller node wishes to be a leader
+     * @param topic leadership topic
      * @return {@code Leadership} future
      */
-    CompletableFuture<Leadership> runForLeadership(String path);
+    Leadership runForLeadership(String topic);
 
     /**
      * Withdraws from a leadership contest.
      *
-     * @param path topic for which this controller node no longer wishes to be a leader
-     * @return future that is successfully completed when withdraw is done
+     * @param topic leadership topic
      */
-    CompletableFuture<Void> withdraw(String path);
-
-    /**
-     * If the local nodeId is the leader for specified topic, this method causes it to
-     * step down temporarily from leadership.
-     * <p>
-     * The node will continue to be in contention for leadership and can
-     * potentially become the leader again if and when it becomes the highest
-     * priority candidate
-     * <p>
-     * If the local nodeId is not the leader, this method will make no changes and
-     * simply return false.
-     *
-     * @param path topic for which this controller node should give up leadership
-     * @return true if this node stepped down from leadership, false otherwise
-     */
-    boolean stepdown(String path);
-
-    /**
-     * Moves the specified nodeId to the top of the candidates list for the topic.
-     * <p>
-     * If the node is not a candidate for this topic, this method will be a noop.
-     *
-     * @param path leadership topic
-     * @param nodeId nodeId to make the top candidate
-     * @return true if nodeId is now the top candidate, false otherwise
-     */
-    boolean makeTopCandidate(String path, NodeId nodeId);
+    void withdraw(String topic);
 
     /**
      * Returns the current leader board.
@@ -107,18 +93,22 @@
     Map<String, Leadership> getLeaderBoard();
 
     /**
-     * Returns the candidates for all known topics.
+     * Returns the candidate nodes for each topic.
      *
      * @return A mapping from topics to corresponding list of candidates.
      */
-    Map<String, List<NodeId>> getCandidates();
+    default Map<String, List<NodeId>> getCandidates() {
+        return ImmutableMap.copyOf(Maps.transformValues(getLeaderBoard(), v -> ImmutableList.copyOf(v.candidates())));
+    }
 
     /**
-     * Returns the candidates for a given topic.
+     * Returns the candidate nodes for a given topic.
      *
-     * @param path topic
-     * @return A lists of NodeIds, which may be empty.
+     * @param topic leadership topic
+     * @return A lists of {@link NodeId nodeIds}, which may be empty.
      */
-    List<NodeId> getCandidates(String path);
-
-}
+    default List<NodeId> getCandidates(String topic) {
+        Leadership leadership = getLeadership(topic);
+        return leadership == null ? ImmutableList.of() : ImmutableList.copyOf(leadership.candidates());
+    }
+}
\ No newline at end of file
diff --git a/core/api/src/main/java/org/onosproject/cluster/LeadershipStore.java b/core/api/src/main/java/org/onosproject/cluster/LeadershipStore.java
new file mode 100644
index 0000000..e3a0503
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/cluster/LeadershipStore.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.cluster;
+
+import java.util.Map;
+import org.onosproject.store.Store;
+
+/**
+ * Store interface for managing {@link LeadershipService} state.
+ */
+public interface LeadershipStore extends Store<LeadershipEvent, LeadershipStoreDelegate> {
+
+    /**
+     * Adds registration for the local instance to be leader for topic.
+     *
+     * @param topic leadership topic
+     * @return Updated leadership after operation is completed
+     */
+    Leadership addRegistration(String topic);
+
+    /**
+     * Unregisters the local instance from leadership contest for topic.
+     *
+     * @param topic leadership topic
+     */
+    void removeRegistration(String topic);
+
+    /**
+     * Unregisters an instance from all leadership contests.
+     *
+     * @param nodeId node identifier
+     */
+    void removeRegistration(NodeId nodeId);
+
+    /**
+     * Updates state so that given node is leader for a topic.
+     *
+     * @param topic leadership topic
+     * @param toNodeId identifier of the desired leader
+     * @return {@code true} if the transfer succeeded; {@code false} otherwise.
+     * This method can return {@code false} if the node is not registered for the topic
+     */
+    boolean moveLeadership(String topic, NodeId toNodeId);
+
+    /**
+     * Attempts to make a node the top candidate.
+     *
+     * @param topic leadership topic
+     * @param nodeId node identifier
+     * @return {@code true} if the specified node is now the top candidate.
+     * This method will return {@code false} if the node is not registered for the topic
+     */
+    boolean makeTopCandidate(String topic, NodeId nodeId);
+
+    /**
+     * Returns the current leadership for topic.
+     *
+     * @param topic leadership topic
+     * @return current leadership
+     */
+    Leadership getLeadership(String topic);
+
+    /**
+     * Return current leadership for all topics.
+     *
+     * @return topic to leadership mapping
+     */
+    Map<String, Leadership> getLeaderships();
+}
\ No newline at end of file
diff --git a/core/api/src/main/java/org/onosproject/cluster/LeadershipStoreDelegate.java b/core/api/src/main/java/org/onosproject/cluster/LeadershipStoreDelegate.java
new file mode 100644
index 0000000..894fdc3
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/cluster/LeadershipStoreDelegate.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2015 Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.cluster;
+
+import org.onosproject.store.StoreDelegate;
+
+/**
+ * {@link LeadershipStore} delegate abstraction.
+ */
+public interface LeadershipStoreDelegate extends StoreDelegate<LeadershipEvent> {
+}