Author: jbellis
Date: Mon Nov 16 14:31:47 2009
New Revision: 880772

URL: http://svn.apache.org/viewvc?rev=880772&view=rev
Log:
add getSortedApplicationStates to preserve state ordering when re-gossiping to 
other nodes.  patch by Jaakko Laine; reviewed by jbellis for CASSANDRA-548

Modified:
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java?rev=880772&r1=880771&r2=880772&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java
 Mon Nov 16 14:31:47 2009
@@ -37,7 +37,7 @@
  *      Gossiper.instance().addApplicationState("LOAD STATE", loadState);
  */
 
-public class ApplicationState
+public class ApplicationState implements Comparable<ApplicationState>
 {
     private static ICompactSerializer<ApplicationState> serializer_;
     static
@@ -79,6 +79,11 @@
     {
         return version_;
     }
+
+    public int compareTo(ApplicationState apState)
+    {
+        return this.version_ - apState.getStateVersion();
+    }
 }
 
 class ApplicationStateSerializer implements 
ICompactSerializer<ApplicationState>

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java?rev=880772&r1=880771&r2=880772&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java 
(original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java 
Mon Nov 16 14:31:47 2009
@@ -76,7 +76,7 @@
         return applicationState_.get(key);
     }
     
-    public Map<String, ApplicationState> getApplicationState()
+    public Map<String, ApplicationState> getApplicationStateMap()
     {
         return applicationState_;
     }
@@ -118,6 +118,22 @@
         //isAlive_ = false;
         isAGossiper_ = value;        
     }
+
+    public List<Map.Entry<String,ApplicationState>> 
getSortedApplicationStates()
+    {
+        ArrayList<Map.Entry<String, ApplicationState>> entries = new 
ArrayList<Map.Entry<String, ApplicationState>>();
+        entries.addAll(applicationState_.entrySet());
+        Collections.sort(entries, new Comparator<Map.Entry<String, 
ApplicationState>>()
+        {
+            public int compare(Map.Entry<String, ApplicationState> lhs, 
Map.Entry<String, ApplicationState> rhs)
+            {
+                return lhs.getValue().compareTo(rhs.getValue());
+            }
+        });
+
+        return entries;
+    }
+
 }
 
 class EndPointStateSerializer implements ICompactSerializer<EndPointState>

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java?rev=880772&r1=880771&r2=880772&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java 
(original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java 
Mon Nov 16 14:31:47 2009
@@ -28,7 +28,7 @@
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.LogUtil;
+
 import org.apache.log4j.Logger;
 
 /**
@@ -209,7 +209,7 @@
     {
         List<Integer> versions = new ArrayList<Integer>();
         versions.add( epState.getHeartBeatState().getHeartBeatVersion() );
-        Map<String, ApplicationState> appStateMap = 
epState.getApplicationState();
+        Map<String, ApplicationState> appStateMap = 
epState.getApplicationStateMap();
 
         Set<String> keys = appStateMap.keySet();
         for ( String key : keys )
@@ -431,7 +431,7 @@
             {
                 reqdEndPointState = new 
EndPointState(epState.getHeartBeatState());
             }
-            Map<String, ApplicationState> appStateMap = 
epState.getApplicationState();
+            Map<String, ApplicationState> appStateMap = 
epState.getApplicationStateMap();
             /* Accumulate all application states whose versions are greater 
than "version" variable */
             Set<String> keys = appStateMap.keySet();
             for ( String key : keys )
@@ -627,13 +627,12 @@
 
     void applyApplicationStateLocally(InetAddress addr, EndPointState 
localStatePtr, EndPointState remoteStatePtr)
     {
-        Map<String, ApplicationState> localAppStateMap = 
localStatePtr.getApplicationState();
-        Map<String, ApplicationState> remoteAppStateMap = 
remoteStatePtr.getApplicationState();
+        Map<String, ApplicationState> localAppStateMap = 
localStatePtr.getApplicationStateMap();
 
-        Set<String> remoteKeys = remoteAppStateMap.keySet();
-        for ( String remoteKey : remoteKeys )
+        for (Map.Entry<String,ApplicationState> remoteEntry : 
remoteStatePtr.getSortedApplicationStates())
         {
-            ApplicationState remoteAppState = remoteAppStateMap.get(remoteKey);
+            String remoteKey = remoteEntry.getKey();
+            ApplicationState remoteAppState = remoteEntry.getValue();
             ApplicationState localAppState = localAppStateMap.get(remoteKey);
 
             /* If state doesn't exist locally for this key then just apply it 
*/

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=880772&r1=880771&r2=880772&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
 Mon Nov 16 14:31:47 2009
@@ -485,14 +485,10 @@
 
     public void onJoin(InetAddress endpoint, EndPointState epState)
     {
-        ApplicationState stateNormal = 
epState.getApplicationState(StorageService.STATE_NORMAL);
-        ApplicationState stateBootstrapping = 
epState.getApplicationState(StorageService.STATE_BOOTSTRAPPING);
-
-        if (stateNormal != null)
-            onChange(endpoint, StorageService.STATE_NORMAL, stateNormal);
-
-        if (stateBootstrapping != null)
-            onChange(endpoint, StorageService.STATE_BOOTSTRAPPING, 
stateBootstrapping);
+        for (Map.Entry<String,ApplicationState> entry : 
epState.getSortedApplicationStates())
+        {
+            onChange(endpoint, entry.getKey(), entry.getValue());
+        }
     }
 
     public void onAlive(InetAddress endpoint, EndPointState state)


Reply via email to