Fix CS warnings in sal-remoterpc-connector and enable enforcement
[controller.git] / opendaylight / md-sal / sal-remoterpc-connector / src / main / java / org / opendaylight / controller / remote / rpc / registry / gossip / Gossiper.java
index f6ce5e55f3ee63602fc92529e92d6e93d0ff9bb3..9230591d46b4d80e244d3d81c653dfebc559a4cf 100644 (file)
@@ -12,19 +12,14 @@ import akka.actor.ActorRefProvider;
 import akka.actor.ActorSelection;
 import akka.actor.Address;
 import akka.actor.Cancellable;
-import akka.actor.UntypedActor;
 import akka.cluster.Cluster;
 import akka.cluster.ClusterActorRefProvider;
 import akka.cluster.ClusterEvent;
 import akka.cluster.Member;
 import akka.dispatch.Mapper;
-import akka.event.Logging;
-import akka.event.LoggingAdapter;
 import akka.pattern.Patterns;
-import org.opendaylight.controller.remote.rpc.utils.ActorUtil;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -32,15 +27,20 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
-
-import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.GetBucketVersions;
-import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.GetBucketVersionsReply;
-import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.GetBucketsByMembers;
-import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.GetBucketsByMembersReply;
-import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.UpdateRemoteBuckets;
-import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipEnvelope;
-import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipStatus;
-import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipTick;
+import org.opendaylight.controller.cluster.common.actor.AbstractUntypedActorWithMetering;
+import org.opendaylight.controller.remote.rpc.RemoteRpcProviderConfig;
+import org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.GetBucketVersions;
+import org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.GetBucketVersionsReply;
+import org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.GetBucketsByMembers;
+import org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.GetBucketsByMembersReply;
+import org.opendaylight.controller.remote.rpc.registry.gossip.Messages.BucketStoreMessages.UpdateRemoteBuckets;
+import org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipEnvelope;
+import org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipStatus;
+import org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipTick;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 /**
  * Gossiper that syncs bucket store across nodes in the cluster.
@@ -58,10 +58,9 @@ import static org.opendaylight.controller.remote.rpc.registry.gossip.Messages.Go
  * for update.
  *
  */
+public class Gossiper extends AbstractUntypedActorWithMetering {
 
-public class Gossiper extends UntypedActor {
-
-    final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
+    private final Logger log = LoggerFactory.getLogger(getClass());
 
     private Cluster cluster;
 
@@ -71,7 +70,7 @@ public class Gossiper extends UntypedActor {
     private Address selfAddress;
 
     /**
-     * All known cluster members
+     * All known cluster members.
      */
     private List<Address> clusterMembers = new ArrayList<>();
 
@@ -79,19 +78,26 @@ public class Gossiper extends UntypedActor {
 
     private Boolean autoStartGossipTicks = true;
 
-    public Gossiper(){}
+    private final RemoteRpcProviderConfig config;
+
+    public Gossiper(RemoteRpcProviderConfig config) {
+        this.config = Preconditions.checkNotNull(config);
+    }
 
     /**
-     * Helpful for testing
+     * Constructor for testing.
+     *
      * @param autoStartGossipTicks used for turning off gossip ticks during testing.
      *                             Gossip tick can be manually sent.
      */
-    public Gossiper(Boolean autoStartGossipTicks){
+    @VisibleForTesting
+    public Gossiper(Boolean autoStartGossipTicks, RemoteRpcProviderConfig config) {
+        this(config);
         this.autoStartGossipTicks = autoStartGossipTicks;
     }
 
     @Override
-    public void preStart(){
+    public void preStart() {
         ActorRefProvider provider = getContext().provider();
         selfAddress = provider.getDefaultAddress();
 
@@ -106,7 +112,7 @@ public class Gossiper extends UntypedActor {
         if (autoStartGossipTicks) {
             gossipTask = getContext().system().scheduler().schedule(
                     new FiniteDuration(1, TimeUnit.SECONDS),        //initial delay
-                    ActorUtil.GOSSIP_TICK_INTERVAL,                 //interval
+                    config.getGossipTickInterval(),                 //interval
                     getSelf(),                                       //target
                     new Messages.GossiperMessages.GossipTick(),      //message
                     getContext().dispatcher(),                       //execution context
@@ -116,44 +122,42 @@ public class Gossiper extends UntypedActor {
     }
 
     @Override
-    public void postStop(){
-        if (cluster != null)
+    public void postStop() {
+        if (cluster != null) {
             cluster.unsubscribe(getSelf());
-        if (gossipTask != null)
+        }
+        if (gossipTask != null) {
             gossipTask.cancel();
+        }
     }
 
+    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
-    public void onReceive(Object message) throws Exception {
-
-        log.debug("Received message: node[{}], message[{}]", selfAddress, message);
-
+    protected void handleReceive(Object message) throws Exception {
         //Usually sent by self via gossip task defined above. But its not enforced.
         //These ticks can be sent by another actor as well which is esp. useful while testing
-        if (message instanceof GossipTick)
+        if (message instanceof GossipTick) {
             receiveGossipTick();
-
-        //Message from remote gossiper with its bucket versions
-        else if (message instanceof GossipStatus)
+        } else if (message instanceof GossipStatus) {
+            // Message from remote gossiper with its bucket versions
             receiveGossipStatus((GossipStatus) message);
-
-        //Message from remote gossiper with buckets. This is usually in response to GossipStatus message
-        //The contained buckets are newer as determined by the remote gossiper by comparing the GossipStatus
-        //message with its local versions
-        else if (message instanceof GossipEnvelope)
+        } else if (message instanceof GossipEnvelope) {
+            // Message from remote gossiper with buckets. This is usually in response to GossipStatus
+            // message. The contained buckets are newer as determined by the remote gossiper by
+            // comparing the GossipStatus message with its local versions.
             receiveGossip((GossipEnvelope) message);
-
-        else if (message instanceof ClusterEvent.MemberUp) {
+        } else if (message instanceof ClusterEvent.MemberUp) {
             receiveMemberUp(((ClusterEvent.MemberUp) message).member());
 
         } else if (message instanceof ClusterEvent.MemberRemoved) {
             receiveMemberRemoveOrUnreachable(((ClusterEvent.MemberRemoved) message).member());
 
-        } else if ( message instanceof ClusterEvent.UnreachableMember){
+        } else if ( message instanceof ClusterEvent.UnreachableMember) {
             receiveMemberRemoveOrUnreachable(((ClusterEvent.UnreachableMember) message).member());
 
-        } else
+        } else {
             unhandled(message);
+        }
     }
 
     /**
@@ -163,7 +167,7 @@ public class Gossiper extends UntypedActor {
      */
     void receiveMemberRemoveOrUnreachable(Member member) {
         //if its self, then stop itself
-        if (selfAddress.equals(member.address())){
+        if (selfAddress.equals(member.address())) {
             getContext().stop(getSelf());
             return;
         }
@@ -173,46 +177,52 @@ public class Gossiper extends UntypedActor {
     }
 
     /**
-     * Add member to the local copy of member list if it doesnt already
-     * @param member
+     * Add member to the local copy of member list if it doesn't already.
+     *
+     * @param member the member to add
      */
     void receiveMemberUp(Member member) {
 
-        if (selfAddress.equals(member.address()))
+        if (selfAddress.equals(member.address())) {
             return; //ignore up notification for self
+        }
 
-        if (!clusterMembers.contains(member.address()))
+        if (!clusterMembers.contains(member.address())) {
             clusterMembers.add(member.address());
+        }
 
         log.debug("Added member [{}], Active member list [{}]", member.address(), clusterMembers);
     }
 
     /**
-     * Sends Gossip status to other members in the cluster. <br/>
-     * 1. If there are no member, ignore the tick. </br>
-     * 2. If there's only 1 member, send gossip status (bucket versions) to it. <br/>
+     * Sends Gossip status to other members in the cluster.
+     * <br>
+     * 1. If there are no member, ignore the tick. <br>
+     * 2. If there's only 1 member, send gossip status (bucket versions) to it. <br>
      * 3. If there are more than one member, randomly pick one and send gossip status (bucket versions) to it.
      */
-    void receiveGossipTick(){
-        if (clusterMembers.size() == 0) return; //no members to send gossip status to
+    void receiveGossipTick() {
+        if (clusterMembers.size() == 0) {
+            return; //no members to send gossip status to
+        }
 
-        Address remoteMemberToGossipTo = null;
+        Address remoteMemberToGossipTo;
 
-        if (clusterMembers.size() == 1)
+        if (clusterMembers.size() == 1) {
             remoteMemberToGossipTo = clusterMembers.get(0);
-        else {
+        else {
             Integer randomIndex = ThreadLocalRandom.current().nextInt(0, clusterMembers.size());
             remoteMemberToGossipTo = clusterMembers.get(randomIndex);
         }
 
-        log.debug("Gossiping to [{}]", remoteMemberToGossipTo);
+        log.trace("Gossiping to [{}]", remoteMemberToGossipTo);
         getLocalStatusAndSendTo(remoteMemberToGossipTo);
     }
 
     /**
      * Process gossip status received from a remote gossiper. Remote versions are compared with
-     * the local copy. <p>
-     *
+     * the local copy.
+     * <p/>
      * For each bucket
      * <ul>
      *  <li>If local copy is newer, the newer buckets are sent in GossipEnvelope to remote</li>
@@ -222,14 +232,15 @@ public class Gossiper extends UntypedActor {
      *
      * @param status bucket versions from a remote member
      */
-    void receiveGossipStatus(GossipStatus status){
+    void receiveGossipStatus(GossipStatus status) {
         //Don't accept messages from non-members
-        if (!clusterMembers.contains(status.from()))
+        if (!clusterMembers.contains(status.from())) {
             return;
+        }
 
         final ActorRef sender = getSender();
         Future<Object> futureReply =
-                Patterns.ask(getContext().parent(), new GetBucketVersions(), ActorUtil.ASK_DURATION.toMillis());
+                Patterns.ask(getContext().parent(), new GetBucketVersions(), config.getAskDuration());
 
         futureReply.map(getMapperToProcessRemoteStatus(sender, status), getContext().dispatcher());
 
@@ -240,10 +251,13 @@ public class Gossiper extends UntypedActor {
      *
      * @param envelope contains buckets from a remote gossiper
      */
-    void receiveGossip(GossipEnvelope envelope){
+    <T extends Copier<T>> void receiveGossip(GossipEnvelope<T> envelope) {
         //TODO: Add more validations
         if (!selfAddress.equals(envelope.to())) {
-            log.debug("Ignoring message intended for someone else. From [{}] to [{}]", envelope.from(), envelope.to());
+            if (log.isTraceEnabled()) {
+                log.trace("Ignoring message intended for someone else. From [{}] to [{}]", envelope.from(),
+                        envelope.to());
+            }
             return;
         }
 
@@ -252,62 +266,62 @@ public class Gossiper extends UntypedActor {
     }
 
     /**
-     * Helper to send received buckets to bucket store
+     * Helper to send received buckets to bucket store.
      *
-     * @param buckets
+     * @param buckets map of Buckets to update
      */
-    void updateRemoteBuckets(Map<Address, Bucket> buckets) {
-
-        UpdateRemoteBuckets updateRemoteBuckets = new UpdateRemoteBuckets(buckets);
+    <T extends Copier<T>> void updateRemoteBuckets(Map<Address, Bucket<T>> buckets) {
+        UpdateRemoteBuckets<T> updateRemoteBuckets = new UpdateRemoteBuckets<>(buckets);
         getContext().parent().tell(updateRemoteBuckets, getSelf());
     }
 
     /**
-     * Gets the buckets from bucket store for the given node addresses and sends them to remote gossiper
+     * Gets the buckets from bucket store for the given node addresses and sends them to remote gossiper.
      *
      * @param remote     remote node to send Buckets to
      * @param addresses  node addresses whose buckets needs to be sent
      */
-    void sendGossipTo(final ActorRef remote, final Set<Address> addresses){
+    void sendGossipTo(final ActorRef remote, final Set<Address> addresses) {
 
         Future<Object> futureReply =
-                Patterns.ask(getContext().parent(), new GetBucketsByMembers(addresses), ActorUtil.ASK_DURATION.toMillis());
+                Patterns.ask(getContext().parent(), new GetBucketsByMembers(addresses), config.getAskDuration());
         futureReply.map(getMapperToSendGossip(remote), getContext().dispatcher());
     }
 
     /**
-     * Gets bucket versions from bucket store and sends to the supplied address
+     * Gets bucket versions from bucket store and sends to the supplied address.
      *
      * @param remoteActorSystemAddress remote gossiper to send to
      */
-    void getLocalStatusAndSendTo(Address remoteActorSystemAddress){
+    void getLocalStatusAndSendTo(Address remoteActorSystemAddress) {
 
         //Get local status from bucket store and send to remote
         Future<Object> futureReply =
-                Patterns.ask(getContext().parent(), new GetBucketVersions(), ActorUtil.ASK_DURATION.toMillis());
+                Patterns.ask(getContext().parent(), new GetBucketVersions(), config.getAskDuration());
 
         //Find gossiper on remote system
         ActorSelection remoteRef = getContext().system().actorSelection(
                 remoteActorSystemAddress.toString() + getSelf().path().toStringWithoutAddress());
 
-        log.debug("Sending bucket versions to [{}]", remoteRef);
+        log.trace("Sending bucket versions to [{}]", remoteRef);
 
         futureReply.map(getMapperToSendLocalStatus(remoteRef), getContext().dispatcher());
 
     }
 
     /**
-     * Helper to send bucket versions received from local store
+     * Helper to send bucket versions received from local store.
+     *
      * @param remote        remote gossiper to send versions to
      * @param localVersions bucket versions received from local store
      */
-    void sendGossipStatusTo(ActorRef remote, Map<Address, Long> localVersions){
+    void sendGossipStatusTo(ActorRef remote, Map<Address, Long> localVersions) {
 
         GossipStatus status = new GossipStatus(selfAddress, localVersions);
         remote.tell(status, getSelf());
     }
 
-    void sendGossipStatusTo(ActorSelection remote, Map<Address, Long> localVersions){
+    void sendGossipStatusTo(ActorSelection remote, Map<Address, Long> localVersions) {
 
         GossipStatus status = new GossipStatus(selfAddress, localVersions);
         remote.tell(status, getSelf());
@@ -317,7 +331,7 @@ public class Gossiper extends UntypedActor {
     /// Private factories to create mappers
     ///
 
-    private Mapper<Object, Void> getMapperToSendLocalStatus(final ActorSelection remote){
+    private Mapper<Object, Void> getMapperToSendLocalStatus(final ActorSelection remote) {
 
         return new Mapper<Object, Void>() {
             @Override
@@ -352,7 +366,7 @@ public class Gossiper extends UntypedActor {
      * @return a {@link akka.dispatch.Mapper} that gets evaluated in future
      *
      */
-    private Mapper<Object, Void> getMapperToProcessRemoteStatus(final ActorRef sender, final GossipStatus status){
+    private Mapper<Object, Void> getMapperToProcessRemoteStatus(final ActorRef sender, final GossipStatus status) {
 
         final Map<Address, Long> remoteVersions = status.getVersions();
 
@@ -374,23 +388,25 @@ public class Gossiper extends UntypedActor {
                     localIsNewer.removeAll(remoteVersions.keySet());
 
 
-                    for (Address address : remoteVersions.keySet()){
+                    for (Address address : remoteVersions.keySet()) {
 
-                        if (localVersions.get(address) == null || remoteVersions.get(address) == null)
+                        if (localVersions.get(address) == null || remoteVersions.get(address) == null) {
                             continue; //this condition is taken care of by above diffs
-                        if (localVersions.get(address) <  remoteVersions.get(address))
+                        }
+                        if (localVersions.get(address) <  remoteVersions.get(address)) {
                             localIsOlder.add(address);
-                        else if (localVersions.get(address) > remoteVersions.get(address))
+                        } else if (localVersions.get(address) > remoteVersions.get(address)) {
                             localIsNewer.add(address);
-                        else
-                            continue;
+                        }
                     }
 
-                    if (!localIsOlder.isEmpty())
+                    if (!localIsOlder.isEmpty()) {
                         sendGossipStatusTo(sender, localVersions );
+                    }
 
-                    if (!localIsNewer.isEmpty())
+                    if (!localIsNewer.isEmpty()) {
                         sendGossipTo(sender, localIsNewer);//send newer buckets to remote
+                    }
 
                 }
                 return null;
@@ -405,19 +421,20 @@ public class Gossiper extends UntypedActor {
      * {@link org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipEnvelope}
      *
      * @param sender the remote member that sent
-     *               {@link org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipStatus}
-     *               in reply to which bucket is being sent back
+     *           {@link org.opendaylight.controller.remote.rpc.registry.gossip.Messages.GossiperMessages.GossipStatus}
+     *           in reply to which bucket is being sent back
      * @return a {@link akka.dispatch.Mapper} that gets evaluated in future
      *
      */
     private Mapper<Object, Void> getMapperToSendGossip(final ActorRef sender) {
 
         return new Mapper<Object, Void>() {
+            @SuppressWarnings({ "rawtypes", "unchecked" })
             @Override
             public Void apply(Object msg) {
                 if (msg instanceof GetBucketsByMembersReply) {
-                    Map<Address, Bucket> buckets = ((GetBucketsByMembersReply) msg).getBuckets();
-                    log.debug("Buckets to send from {}: {}", selfAddress, buckets);
+                    Map<Address, Bucket<?>> buckets = ((GetBucketsByMembersReply) msg).getBuckets();
+                    log.trace("Buckets to send from {}: {}", selfAddress, buckets);
                     GossipEnvelope envelope = new GossipEnvelope(selfAddress, sender.path().address(), buckets);
                     sender.tell(envelope, getSelf());
                 }