[jira] [Commented] (CASSANDRA-14952) NPE when using allocate_tokens_for_keyspace and add new DC

2019-08-08 Thread Jaydeepkumar Chovatia (JIRA)


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

Jaydeepkumar Chovatia commented on CASSANDRA-14952:
---

[~michaelsembwever] Looks good to me. Sorry for the late response.

> 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



[jira] [Commented] (CASSANDRA-14952) NPE when using allocate_tokens_for_keyspace and add new DC

2019-08-04 Thread mck (JIRA)


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

mck commented on CASSANDRA-14952:
-

A slightly modified version of your fix [~chovatia.jayd...@gmail.com]


|| branch || circleci || asf jenkins testall || asf jenkins dtests ||
| 
[CASSANDRA-14952|https://github.com/thelastpickle/cassandra/commit/3a72a51f9cb06ac85a4c78f3719a598a3a754909]
  | 
[circleci|https://circleci.com/workflow-run/b1f8b919-f889-47c5-9019-22a3468a428d]
 | 
[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-testall/40//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-testall/40/]
 | 
[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/675//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/675/]
 | |

> 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
>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}

[jira] [Commented] (CASSANDRA-14952) NPE when using allocate_tokens_for_keyspace and add new DC

2019-08-01 Thread mck (JIRA)


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

mck commented on CASSANDRA-14952:
-

There's a few peculiarities in how {{allocate_tokens_for_keyspace}} bootstraps 
in new datacenters.

For example subsequent nodes in a new datacenter will also fail, unless RF=2, 
until there is at least one node in each rack up until RF number of racks. That 
failure is a {{ConfigurationException}} with the message {code}"Token 
allocation failed: the number of racks %d in datacenter %s is lower than its 
replication factor %d."{code}

It is an undocumented requirement that one node in each rack, up until RF 
number of racks, are bootstrapped with manually calculated tokens, when adding 
a new datacenter and using {{allocate_tokens_for_keyspace}}.

Do we want to treat the first node added in a new datacenter as a unique unit, 
which is what we get with {{rack = 1}}?
[~chovatia.jayd...@gmail.com], unless any speaks up, let me do some testing on 
it and get back to you…

> 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
>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)
> +

[jira] [Commented] (CASSANDRA-14952) NPE when using allocate_tokens_for_keyspace and add new DC

2019-01-10 Thread Jaydeepkumar Chovatia (JIRA)


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

Jaydeepkumar Chovatia commented on CASSANDRA-14952:
---

 

[~blambov] Could you please check this bug and provide your opinion?

> 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
>Priority: Minor
> 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.3#76005)

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