[ 
https://issues.apache.org/jira/browse/CASSANDRA-14952?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

mck updated CASSANDRA-14952:
----------------------------
    Authors: Jaydeepkumar Chovatia, mck  (was: Jaydeepkumar Chovatia)

> NPE when using allocate_tokens_for_keyspace and add new DC
> ----------------------------------------------------------
>
>                 Key: CASSANDRA-14952
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14952
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Cluster/Gossip
>            Reporter: Jaydeepkumar Chovatia
>            Assignee: Jaydeepkumar Chovatia
>            Priority: Low
>             Fix For: 3.0.x
>
>
> Received following NPE while bootstrapping very first node in the new 
> datacenter with {{allocate_tokens_for_keyspace}} yaml option
> {code:java}
> INFO  21:44:13 JOINING: getting bootstrap token
> Exception (java.lang.NullPointerException) encountered during startup: null
> java.lang.NullPointerException
>       at 
> org.apache.cassandra.dht.tokenallocator.TokenAllocation.getStrategy(TokenAllocation.java:208)
>       at 
> org.apache.cassandra.dht.tokenallocator.TokenAllocation.getStrategy(TokenAllocation.java:170)
>       at 
> org.apache.cassandra.dht.tokenallocator.TokenAllocation.allocateTokens(TokenAllocation.java:55)
>       at 
> org.apache.cassandra.dht.BootStrapper.allocateTokens(BootStrapper.java:206)
>       at 
> org.apache.cassandra.dht.BootStrapper.getBootstrapTokens(BootStrapper.java:173)
>       at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:854)
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:666)
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:579)
>       at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:351)
>       at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:586)
>       at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:714)
> {code}
> Please find reproducible steps here:
>  1. Set {{allocate_tokens_for_keyspace}} property with 
> {{Networktopologystrategy}} say {{{{Networktopologystrategy, 'dc1' : 1, 'dc2' 
> : 1}}}}
>  2. Start first node in {{dc1}}
>  3. Now bootstrap second node in {{dc2,}} it will throw above exception.
> RCA:
>  
> [doAddEndpoint|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/locator/TokenMetadata.java#L1325]
>  is invoked from the 
> [bootstrap|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/service/StorageService.java#L1254]
>  and at this time [local node's rack 
> information|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/locator/TokenMetadata.java#L1276]
>  is available
> However with have {{allocate_tokens_for_keyspace}} option, daemon tries to 
> access rack information even before calling 
> [bootstrap|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/service/StorageService.java#L1241]
>  function, at [this 
> place|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/service/StorageService.java#L878]
>  which results in NPE
> Fix:
>  Since this is applicable to only very first node for new dc, we can check 
> for {{null}} as:
> {code:java}
> diff --git 
> a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java 
> b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
> index 8d8a6ffeca..e162757d95 100644
> --- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
> +++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
> @@ -205,7 +205,11 @@ public class TokenAllocation
>          final int replicas = rs.getReplicationFactor(dc);
>  
>          Topology topology = tokenMetadata.getTopology();
> -        int racks = topology.getDatacenterRacks().get(dc).asMap().size();
> +        int racks = 1;
> +        if (topology.getDatacenterRacks().get(dc) != null)
> +        {
> +            racks = topology.getDatacenterRacks().get(dc).asMap().size();
> +        }
>  
>          if (racks >= replicas)
>          {
> {code}
> Let me know your comments.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to