cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r769015 - in /incubator/cassandra/trunk/src/org/apache/cassandra/locator: RackAwareStrategy.java RackUnawareStrategy.java
Date Mon, 27 Apr 2009 15:01:17 GMT
Author: jbellis
Date: Mon Apr 27 15:01:17 2009
New Revision: 769015

URL: http://svn.apache.org/viewvc?rev=769015&view=rev
Log:
clean up partition strategies; in particular, implement getStorageEndPoints(String[] keys)
in terms of getStorageEndPoints(BigInteger token), instead of copy/pasting code
patch by jbellis; reviewed by Jun Rao for CASSANDRA-65

Modified:
    incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackAwareStrategy.java
    incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackUnawareStrategy.java

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackAwareStrategy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackAwareStrategy.java?rev=769015&r1=769014&r2=769015&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackAwareStrategy.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackAwareStrategy.java Mon
Apr 27 15:01:17 2009
@@ -30,7 +30,7 @@
     
     public EndPoint[] getStorageEndPoints(BigInteger token)
     {
-        int startIndex = 0 ;
+        int startIndex;
         List<EndPoint> list = new ArrayList<EndPoint>();
         boolean bDataCenter = false;
         boolean bOtherRack = false;
@@ -52,7 +52,7 @@
         foundCount++;
         if( N == 1 )
         {
-            return list.toArray(new EndPoint[0]);
+            return list.toArray(new EndPoint[list.size()]);
         }
         startIndex = (index + 1)%totalNodes;
         IEndPointSnitch endPointSnitch = StorageService.instance().getEndPointSnitch();
@@ -84,7 +84,6 @@
                         bOtherRack = true;
                         foundCount++;
                     }
-                    continue;
                 }
             }
             catch (UnknownHostException e)
@@ -101,98 +100,20 @@
             {
                 list.add(tokenToEndPointMap.get(tokens.get(i)));
                 foundCount++;
-                continue;
             }
         }
         retrofitPorts(list);
-        return list.toArray(new EndPoint[0]);
+        return list.toArray(new EndPoint[list.size()]);
     }
     
     public Map<String, EndPoint[]> getStorageEndPoints(String[] keys)
     {
     	Map<String, EndPoint[]> results = new HashMap<String, EndPoint[]>();
-    	List<EndPoint> list = new ArrayList<EndPoint>();
-    	int startIndex = 0 ;
-    	int foundCount = 0;
-    	boolean bDataCenter = false;
-        boolean bOtherRack = false;
-    	
-    	Map<BigInteger, EndPoint> tokenToEndPointMap = tokenMetadata_.cloneTokenEndPointMap();
-    	int N = DatabaseDescriptor.getReplicationFactor();
-        List<BigInteger> tokens = new ArrayList<BigInteger>(tokenToEndPointMap.keySet());
-        Collections.sort(tokens);
-        
+
         for ( String key : keys )
         {
         	BigInteger token = StorageService.hash(key);
-        	int index = Collections.binarySearch(tokens, token);
-            if(index < 0)
-            {
-                index = (index + 1) * (-1);
-                if (index >= tokens.size())
-                    index = 0;
-            }
-            int totalNodes = tokens.size();
-            // Add the node at the index by default
-            list.add(tokenToEndPointMap.get(tokens.get(index)));
-            foundCount++;
-            if( N == 1 )
-            {
-            	results.put( key, list.toArray(new EndPoint[0]) );
-                return results;
-            }
-            startIndex = (index + 1)%totalNodes;
-            IEndPointSnitch endPointSnitch = StorageService.instance().getEndPointSnitch();
-            
-            for (int i = startIndex, count = 1; count < totalNodes && foundCount
< N; ++count, i = (i+1)%totalNodes)
-            {
-                try
-                {
-                    // First try to find one in a different data center
-                    if(!endPointSnitch.isInSameDataCenter(tokenToEndPointMap.get(tokens.get(index)),
tokenToEndPointMap.get(tokens.get(i))))
-                    {
-                        // If we have already found something in a diff datacenter no need
to find another
-                        if( !bDataCenter )
-                        {
-                            list.add(tokenToEndPointMap.get(tokens.get(i)));
-                            bDataCenter = true;
-                            foundCount++;
-                        }
-                        continue;
-                    }
-                    // Now  try to find one on a different rack
-                    if(!endPointSnitch.isOnSameRack(tokenToEndPointMap.get(tokens.get(index)),
tokenToEndPointMap.get(tokens.get(i))) &&
-                            endPointSnitch.isInSameDataCenter(tokenToEndPointMap.get(tokens.get(index)),
tokenToEndPointMap.get(tokens.get(i))))
-                    {
-                        // If we have already found something in a diff rack no need to find
another
-                        if( !bOtherRack )
-                        {
-                            list.add(tokenToEndPointMap.get(tokens.get(i)));
-                            bOtherRack = true;
-                            foundCount++;
-                        }
-                        continue;
-                    }
-                }
-                catch (UnknownHostException e)
-                {
-                    logger_.debug(LogUtil.throwableToString(e));
-                }
-
-            }
-            // If we found N number of nodes we are good. This loop wil just exit. Otherwise
just
-            // loop through the list and add until we have N nodes.
-            for (int i = startIndex, count = 1; count < totalNodes && foundCount
< N; ++count, i = (i+1)%totalNodes)
-            {
-                if( ! list.contains(tokenToEndPointMap.get(tokens.get(i))))
-                {
-                    list.add(tokenToEndPointMap.get(tokens.get(i)));
-                    foundCount++;
-                    continue;
-                }
-            }
-            retrofitPorts(list);
-            results.put(key, list.toArray(new EndPoint[0]));
+            results.put(key, getStorageEndPoints(token));
         }
         
         return results;

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackUnawareStrategy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackUnawareStrategy.java?rev=769015&r1=769014&r2=769015&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackUnawareStrategy.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/locator/RackUnawareStrategy.java Mon
Apr 27 15:01:17 2009
@@ -39,7 +39,7 @@
     
     public EndPoint[] getStorageEndPoints(BigInteger token, Map<BigInteger, EndPoint>
tokenToEndPointMap)
     {
-        int startIndex = 0 ;
+        int startIndex;
         List<EndPoint> list = new ArrayList<EndPoint>();
         int foundCount = 0;
         int N = DatabaseDescriptor.getReplicationFactor();
@@ -65,57 +65,22 @@
             {
                 list.add(tokenToEndPointMap.get(tokens.get(i)));
                 foundCount++;
-                continue;
             }
         }
         retrofitPorts(list);
-        return list.toArray(new EndPoint[0]);
+        return list.toArray(new EndPoint[list.size()]);
     }
             
     public Map<String, EndPoint[]> getStorageEndPoints(String[] keys)
-    {              
-        Arrays.sort(keys);
-        Range[] ranges = StorageService.instance().getAllRanges();
-        
+    {
     	Map<String, EndPoint[]> results = new HashMap<String, EndPoint[]>();
-    	List<EndPoint> list = new ArrayList<EndPoint>();
-    	int startIndex = 0 ;
-    	int foundCount = 0;
-    	
-    	Map<BigInteger, EndPoint> tokenToEndPointMap = tokenMetadata_.cloneTokenEndPointMap();
-    	int N = DatabaseDescriptor.getReplicationFactor();
-        List<BigInteger> tokens = new ArrayList<BigInteger>(tokenToEndPointMap.keySet());
-        Collections.sort(tokens);
+
         for ( String key : keys )
         {
         	BigInteger token = StorageService.hash(key);
-        	int index = Collections.binarySearch(tokens, token);
-            if(index < 0)
-            {
-                index = (index + 1) * (-1);
-                if (index >= tokens.size())
-                    index = 0;
-            }
-            int totalNodes = tokens.size();
-            // Add the node at the index by default
-            list.add(tokenToEndPointMap.get(tokens.get(index)));
-            foundCount++;
-            startIndex = (index + 1)%totalNodes;
-            // If we found N number of nodes we are good. This loop will just exit. Otherwise
just
-            // loop through the list and add until we have N nodes.
-            for (int i = startIndex, count = 1; count < totalNodes && foundCount
< N; ++count, i = (i+1)%totalNodes)
-            {
-                if( ! list.contains(tokenToEndPointMap.get(tokens.get(i))))
-                {
-                    list.add(tokenToEndPointMap.get(tokens.get(i)));
-                    foundCount++;
-                    continue;
-                }
-            }
-            retrofitPorts(list);
-            results.put(key, list.toArray(new EndPoint[0]));
+            results.put(key, getStorageEndPoints(token));
         }
-        
+
         return results;
     }
 }



Mime
View raw message