Bug 2410: Use generated serialVersionUID for messages 36/14836/2
authortpantelis <tpanteli@brocade.com>
Mon, 2 Feb 2015 03:10:19 +0000 (22:10 -0500)
committertpantelis <tpanteli@brocade.com>
Mon, 2 Feb 2015 21:22:17 +0000 (16:22 -0500)
Changed the remaining message classes whose serialVersionUID had
previously been set to 1 to the generated value to maintain
backwards compatibility with Helium.

It turns out CommitEntry, PersistEntry, and SaveSnapShot are no longer
used so I removed them.

Change-Id: Ib3c251b7eaf8b8ce5aa4edf693aca23bf67d0402
Signed-off-by: tpantelis <tpanteli@brocade.com>
15 files changed:
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/Snapshot.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/CommitEntry.java [deleted file]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/PersistEntry.java [deleted file]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SaveSnapshot.java [deleted file]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/ExecuteRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/InvokeRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/RpcResponse.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/registry/RoutingTable.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/registry/gossip/BucketImpl.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/registry/gossip/Messages.java

index 77bf10370184894d6a510990e08fb1a8bc785a84..feccea7edba37b5faff9782cf986146f0a784af9 100644 (file)
@@ -12,7 +12,8 @@ import java.util.List;
 
 
 public class Snapshot implements Serializable {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = -8298574936724056236L;
+
     private final byte[] state;
     private final List<ReplicatedLogEntry> unAppliedEntries;
     private final long lastIndex;
diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/CommitEntry.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/CommitEntry.java
deleted file mode 100644 (file)
index 6335e32..0000000
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.cluster.raft.base.messages;
-
-import java.io.Serializable;
-
-/**
- * Message sent to commit an entry to the log
- */
-public class CommitEntry implements Serializable {
-    private static final long serialVersionUID = 1L;
-}
diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/PersistEntry.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/PersistEntry.java
deleted file mode 100644 (file)
index 68ecc12..0000000
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.cluster.raft.base.messages;
-
-import java.io.Serializable;
-
-/**
- * Message sent to Persist an entry into the transaction journal
- */
-public class PersistEntry implements Serializable {
-    private static final long serialVersionUID = 1L;
-}
diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SaveSnapshot.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SaveSnapshot.java
deleted file mode 100644 (file)
index 7b7f085..0000000
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.cluster.raft.base.messages;
-
-import java.io.Serializable;
-
-/**
- * This message is sent by a RaftActor to itself so that a subclass can process
- * it and use it to save it's state
- */
-public class SaveSnapshot implements Serializable {
-    private static final long serialVersionUID = 1L;
-}
index a782eda5659d9389606ea4e7606a5f61dd949c07..32ed85b281fee0feb4b0533c0b72cb5ade40093a 100644 (file)
@@ -12,7 +12,7 @@ package org.opendaylight.controller.cluster.raft.messages;
  * Reply for the AppendEntriesRpc message
  */
 public class AppendEntriesReply extends AbstractRaftRPC {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = -7487547356392536683L;
 
     // true if follower contained entry matching
     // prevLogIndex and prevLogTerm
@@ -38,6 +38,7 @@ public class AppendEntriesReply extends AbstractRaftRPC {
         this.logLastTerm = logLastTerm;
     }
 
+    @Override
     public long getTerm() {
         return term;
     }
index 71e7ecc189fc882eaa0917b19af99376ed05f1b7..15621bf894b14f75ec845821654202877f4b4382 100644 (file)
@@ -9,13 +9,13 @@
 package org.opendaylight.controller.cluster.raft.messages;
 
 public class InstallSnapshotReply extends AbstractRaftRPC {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 642227896390779503L;
 
     // The followerId - this will be used to figure out which follower is
     // responding
     private final String followerId;
     private final int chunkIndex;
-    private boolean success;
+    private final boolean success;
 
     public InstallSnapshotReply(long term, String followerId, int chunkIndex,
         boolean success) {
index 8321d0c25bcecce32617c55fe652c2b7be189541..9ba5acb664700f4873d51fdb9f2637ef882c360e 100644 (file)
@@ -12,7 +12,7 @@ package org.opendaylight.controller.cluster.raft.messages;
  * Invoked by candidates to gather votes (ยง5.2).
  */
 public class RequestVote extends AbstractRaftRPC {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = -6967509186297108657L;
 
     // candidate requesting vote
     private String candidateId;
@@ -35,6 +35,7 @@ public class RequestVote extends AbstractRaftRPC {
     public RequestVote() {
     }
 
+    @Override
     public long getTerm() {
         return term;
     }
index da3ba5c39f6989b86c1f76b7830d2c0426302ddb..b3c95d6ecaab2b19524fb4ae06c3c8f6491658c0 100644 (file)
@@ -9,7 +9,7 @@
 package org.opendaylight.controller.cluster.raft.messages;
 
 public class RequestVoteReply extends AbstractRaftRPC {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 8427899326488775660L;
 
     // true means candidate received vot
     private final boolean voteGranted;
@@ -19,6 +19,7 @@ public class RequestVoteReply extends AbstractRaftRPC {
         this.voteGranted = voteGranted;
     }
 
+    @Override
     public long getTerm() {
         return term;
     }
index 10876045ae272c436e54143c8a0da8bf1c2e41e7..22e2dbd47d4d7148c8e41cfe05ce8532df527466 100644 (file)
@@ -27,6 +27,15 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import org.opendaylight.controller.cluster.DataPersistenceProvider;
 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
@@ -46,15 +55,6 @@ import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContex
 import org.opendaylight.yangtools.yang.model.api.ModuleIdentifier;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import scala.concurrent.duration.Duration;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 /**
  * The ShardManager has the following jobs,
@@ -555,7 +555,8 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     }
 
     static class SchemaContextModules implements Serializable {
-        private static final long serialVersionUID = 1L;
+        private static final long serialVersionUID = -8884620101025936590L;
+
         private final Set<String> modules;
 
         SchemaContextModules(Set<String> modules){
index 4bb5258b4005f21eabc62c2707f2d3edca59b229..5d780be641e2cce6d2b48a2628cd675f9a24c8ee 100644 (file)
@@ -9,29 +9,28 @@ package org.opendaylight.controller.remote.rpc.messages;
 
 
 import com.google.common.base.Preconditions;
-import org.opendaylight.yangtools.yang.common.QName;
-
 import java.io.Serializable;
+import org.opendaylight.yangtools.yang.common.QName;
 
 public class ExecuteRpc implements Serializable {
-  private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1128904894827335676L;
 
-  private final String inputCompositeNode;
-  private final QName rpc;
+    private final String inputCompositeNode;
+    private final QName rpc;
 
-  public ExecuteRpc(final String inputCompositeNode, final QName rpc) {
-    Preconditions.checkNotNull(inputCompositeNode, "Composite Node input string should be present");
-    Preconditions.checkNotNull(rpc, "rpc Qname should not be null");
+    public ExecuteRpc(final String inputCompositeNode, final QName rpc) {
+        Preconditions.checkNotNull(inputCompositeNode, "Composite Node input string should be present");
+        Preconditions.checkNotNull(rpc, "rpc Qname should not be null");
 
-    this.inputCompositeNode = inputCompositeNode;
-    this.rpc = rpc;
-  }
+        this.inputCompositeNode = inputCompositeNode;
+        this.rpc = rpc;
+    }
 
-  public String getInputCompositeNode() {
-    return inputCompositeNode;
-  }
+    public String getInputCompositeNode() {
+        return inputCompositeNode;
+    }
 
-  public QName getRpc() {
-    return rpc;
-  }
+    public QName getRpc() {
+        return rpc;
+    }
 }
index 652569b7baf705ea0efa5c4d35f530edf85942f0..9c40dbfc58a556bc28f337c9ee683b947846c045 100644 (file)
@@ -8,37 +8,36 @@
 package org.opendaylight.controller.remote.rpc.messages;
 
 import com.google.common.base.Preconditions;
+import java.io.Serializable;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.CompositeNode;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 
-import java.io.Serializable;
-
 public class InvokeRpc implements Serializable {
-  private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = -2813459607858108953L;
 
-  private final QName rpc;
-  private final YangInstanceIdentifier identifier;
-  private final CompositeNode input;
+    private final QName rpc;
+    private final YangInstanceIdentifier identifier;
+    private final CompositeNode input;
 
-  public InvokeRpc(final QName rpc, final YangInstanceIdentifier identifier, final CompositeNode input) {
-    Preconditions.checkNotNull(rpc, "rpc qname should not be null");
-    Preconditions.checkNotNull(input, "rpc input should not be null");
+    public InvokeRpc(final QName rpc, final YangInstanceIdentifier identifier, final CompositeNode input) {
+        Preconditions.checkNotNull(rpc, "rpc qname should not be null");
+        Preconditions.checkNotNull(input, "rpc input should not be null");
 
-    this.rpc = rpc;
-    this.identifier = identifier;
-    this.input = input;
-  }
+        this.rpc = rpc;
+        this.identifier = identifier;
+        this.input = input;
+    }
 
-  public QName getRpc() {
-    return rpc;
-  }
+    public QName getRpc() {
+        return rpc;
+    }
 
-  public YangInstanceIdentifier getIdentifier() {
-    return identifier;
-  }
+    public YangInstanceIdentifier getIdentifier() {
+        return identifier;
+    }
 
-  public CompositeNode getInput() {
-    return input;
-  }
+    public CompositeNode getInput() {
+        return input;
+    }
 }
index 387cb90112dad6ffbab6fb7cce7bf7c1d92442f1..e6b208cb6fe63d64eca185b9d90d2d19144ed440 100644 (file)
@@ -10,14 +10,15 @@ package org.opendaylight.controller.remote.rpc.messages;
 import java.io.Serializable;
 
 public class RpcResponse implements Serializable {
-  private static final long serialVersionUID = 1L;
-  private final String resultCompositeNode;
+    private static final long serialVersionUID = -4211279498688989245L;
 
-  public RpcResponse(final String resultCompositeNode) {
-    this.resultCompositeNode = resultCompositeNode;
-  }
+    private final String resultCompositeNode;
 
-  public String getResultCompositeNode() {
-    return resultCompositeNode;
-  }
+    public RpcResponse(final String resultCompositeNode) {
+        this.resultCompositeNode = resultCompositeNode;
+    }
+
+    public String getResultCompositeNode() {
+        return resultCompositeNode;
+    }
 }
index 52b1106c873872609e4300abe52d558ee89a6011..f67657f6927801931fae2fd0434481169f2f3de8 100644 (file)
@@ -17,7 +17,7 @@ import org.opendaylight.controller.remote.rpc.registry.gossip.Copier;
 import org.opendaylight.controller.sal.connector.api.RpcRouter;
 
 public class RoutingTable implements Copier<RoutingTable>, Serializable {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 5592610415175278760L;
 
     private final Map<RpcRouter.RouteIdentifier<?, ?, ?>, Long> table = new HashMap<>();
     private ActorRef router;
index b81175e9a253176870ddf92901d8d2debaa4d698..4c4573d909491f0414d6bd00ebac5eb35a2c7b94 100644 (file)
@@ -10,7 +10,7 @@ package org.opendaylight.controller.remote.rpc.registry.gossip;
 import java.io.Serializable;
 
 public class BucketImpl<T extends Copier<T>> implements Bucket<T>, Serializable {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 294779770032719196L;
 
     private Long version = System.currentTimeMillis();
 
index b05bd7d0f6f314c990e4a1d7909115b6c1ad1348..00437e7e5639fc382e8b1434d4b02495ef1ac56e 100644 (file)
@@ -46,7 +46,8 @@ public class Messages {
         }
 
         public static class ContainsBuckets implements Serializable{
-            private static final long serialVersionUID = 1L;
+            private static final long serialVersionUID = -4940160367495308286L;
+
             private final Map<Address, Bucket> buckets;
 
             public ContainsBuckets(Map<Address, Bucket> buckets){
@@ -87,7 +88,8 @@ public class Messages {
         }
 
         public static class ContainsBucketVersions implements Serializable{
-            private static final long serialVersionUID = 1L;
+            private static final long serialVersionUID = -8172148925383801613L;
+
             Map<Address, Long> versions;
 
             public ContainsBucketVersions(Map<Address, Long> versions) {
@@ -119,15 +121,16 @@ public class Messages {
 
     public static class GossiperMessages{
         public static class Tick implements Serializable {
-            private static final long serialVersionUID = 1L;
+            private static final long serialVersionUID = -4770935099506366773L;
         }
 
         public static final class GossipTick extends Tick {
-            private static final long serialVersionUID = 1L;
+            private static final long serialVersionUID = 5803354404380026143L;
         }
 
         public static final class GossipStatus extends ContainsBucketVersions implements Serializable{
-            private static final long serialVersionUID = 1L;
+            private static final long serialVersionUID = -593037395143883265L;
+
             private final Address from;
 
             public GossipStatus(Address from, Map<Address, Long> versions) {
@@ -141,7 +144,8 @@ public class Messages {
         }
 
         public static final class GossipEnvelope extends ContainsBuckets implements Serializable {
-            private static final long serialVersionUID = 1L;
+            private static final long serialVersionUID = 8346634072582438818L;
+
             private final Address from;
             private final Address to;