<?xml version="1.0" encoding="UTF-8"?>
<commit>
  <added type="array"/>
  <modified type="array">
    <modified>
      <diff>@@ -137,7 +137,8 @@ public class BootStrapper implements Runnable
         if (logger_.isDebugEnabled())
           logger_.debug(&quot;Total number of new ranges &quot; + newRanges.length);
         /* Calculate the list of nodes that handle the new ranges */
-        Map&lt;Range, List&lt;EndPoint&gt;&gt; newRangeToEndPointMap = StorageService.instance().constructRangeToEndPointMap(newRanges);
+        Map&lt;Range, List&lt;EndPoint&gt;&gt; newRangeToEndPointMap = 
+            StorageService.instance().constructRangeToEndPointMap(newRanges, tokenMetadata_.cloneTokenEndPointMapIncludingBootstrapNodes());
         /* Calculate ranges that need to be sent and from whom to where */
         Map&lt;Range, List&lt;BootstrapSourceTarget&gt;&gt; rangesWithSourceTarget = LeaveJoinProtocolHelper.getRangeSourceTargetInfo(oldRangeToEndPointMap, newRangeToEndPointMap);
         return rangesWithSourceTarget;</diff>
      <filename>src/java/org/apache/cassandra/dht/BootStrapper.java</filename>
    </modified>
    <modified>
      <diff>@@ -213,6 +213,24 @@ public class TokenMetadata
         }
     }
     
+    public Map&lt;Token, EndPoint&gt; cloneTokenEndPointMapIncludingBootstrapNodes()
+    {
+        lock_.readLock().lock();
+        try
+        {            
+            Map h1 = new HashMap&lt;Token, EndPoint&gt;( tokenToEndPointMap_ );
+            for (Map.Entry&lt;Token, EndPoint&gt; e: bootstrapNodes.entrySet())
+            {
+                h1.put(e.getKey(), e.getValue());
+            }
+            return h1;
+        }
+        finally
+        {
+            lock_.readLock().unlock();
+        }
+    }
+    
     /*
      * Returns a safe clone of endPointTokenMap_.
     */</diff>
      <filename>src/java/org/apache/cassandra/locator/TokenMetadata.java</filename>
    </modified>
    <modified>
      <diff>@@ -301,7 +301,7 @@ public final class StorageService implements IEndPointStateChangeSubscriber, Sto
         if (isBootstrapMode)
         {
             logger_.info(&quot;Starting in bootstrap mode&quot;);
-            doBootstrap(StorageService.getLocalStorageEndPoint());
+            bootStrapper_.submit(new BootStrapper(new EndPoint[]{StorageService.getLocalStorageEndPoint()}, storageMetadata_.getToken()));
             Gossiper.instance().addApplicationState(BOOTSTRAP_MODE, new ApplicationState(&quot;&quot;));
         }
     }
@@ -317,9 +317,9 @@ public final class StorageService implements IEndPointStateChangeSubscriber, Sto
     }
 
     /* TODO: used for testing */
-    public void updateTokenMetadata(Token token, EndPoint endpoint)
+    public void updateTokenMetadata(Token token, EndPoint endpoint, boolean bs)
     {
-        tokenMetadata_.update(token, endpoint);
+        tokenMetadata_.update(token, endpoint, bs);
     }
 
     public IEndPointSnitch getEndPointSnitch()
@@ -593,9 +593,8 @@ public final class StorageService implements IEndPointStateChangeSubscriber, Sto
      * Starts the bootstrap operations for the specified endpoint.
      * @param endpoint
      */
-    public final void doBootstrap(EndPoint endpoint)
+    public final void doBootstrap(EndPoint endpoint, Token token)
     {
-        Token token = tokenMetadata_.getToken(endpoint);
         bootStrapper_.submit(new BootStrapper(new EndPoint[]{endpoint}, token));
     }
     </diff>
      <filename>src/java/org/apache/cassandra/service/StorageService.java</filename>
    </modified>
    <modified>
      <diff>@@ -41,7 +41,7 @@ public class BootStrapperTest {
         /* New token needs to be part of the map for the algorithm
          * to calculate the ranges correctly
          */
-        StorageService.instance().updateTokenMetadata(newToken, newEndPoint);
+        StorageService.instance().updateTokenMetadata(newToken, newEndPoint, true);
 
         BootStrapper b = new BootStrapper(new EndPoint[]{newEndPoint}, newToken );
         Map&lt;Range,List&lt;BootstrapSourceTarget&gt;&gt; res = b.getRangesWithSourceTarget();
@@ -74,7 +74,7 @@ public class BootStrapperTest {
         {
             EndPoint e  = new EndPoint(&quot;127.0.0.&quot;+i, 100);
             Token t = p.getDefaultToken();
-            StorageService.instance().updateTokenMetadata(t, e);
+            StorageService.instance().updateTokenMetadata(t, e, false);
         }
         return p;
     }</diff>
      <filename>test/unit/org/apache/cassandra/dht/BootStrapperTest.java</filename>
    </modified>
  </modified>
  <removed type="array"/>
  <parents type="array">
    <parent>
      <id>390ca5a248cc960855e748b48d7f2fc2903faa5f</id>
    </parent>
  </parents>
  <author>
    <name>Jonathan Ellis</name>
    <email>jbellis@apache.org</email>
  </author>
  <url>http://github.com/ryanking/cassandra/commit/037a04bdc764e9424ec1600d5116bf726e03e9d8</url>
  <id>037a04bdc764e9424ec1600d5116bf726e03e9d8</id>
  <committed-date>2009-11-03T12:03:32-08:00</committed-date>
  <authored-date>2009-11-03T12:03:32-08:00</authored-date>
  <message>Fixes null token getting passed to Bootstapper; Correctly includes new nodes in src-target calculation.  patch by Sandeep Tata; reviewed by jbellis for CASSANDRA-501

git-svn-id: https://svn.apache.org/repos/asf/incubator/cassandra/branches/cassandra-0.4@832531 13f79535-47bb-0310-9956-ffa450edef68</message>
  <tree>68dad27e8d47635967944a090b56d79b80fbad64</tree>
  <committer>
    <name>Jonathan Ellis</name>
    <email>jbellis@apache.org</email>
  </committer>
</commit>
