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

Russell Alexander Spitzer commented on CASSANDRA-6493:
------------------------------------------------------

Correct I didn't see this over several runs over the weekend testing on the 
pre-6488 build. Head of the git log from that build

{code}
commit c133ff88982948fdb12669bf766e9848102a3496
Author: Russell Spitzer <russell.spit...@gmail.com>
Date:   Fri Dec 13 12:00:53 2013 -0800

    Patch to fix NPE ( this is patch a3d91dc9d67572e16d9ad92f22b89eb969373899)

commit 11455738fa61c6eb02895a5a8d3fbbe4d8cb24b4
Author: Brandon Williams <brandonwilli...@apache.org>
Date:   Fri Dec 13 12:10:47 2013 -0600

    Pig: don't assume all DataBags are DefaultDataBags
    Patch by Mike Spertus, reviewed by brandonwilliams for CASSANDRA-6420
{code}

> Exceptions when a second Datacenter is Added
> --------------------------------------------
>
>                 Key: CASSANDRA-6493
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6493
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Ubuntu, EC2 M1.large
>            Reporter: Russell Alexander Spitzer
>
> On adding a second datacenter several exceptions were raised.
> Test outline:
> Start 25 Node DC1
> Keyspace Setup Replication 3
> Begin insert against DC1 Using Stress
> While the inserts are occuring
> Start up 25 Node DC2
> Alter Keyspace to include Replication in 2nd DC
> Run rebuild on DC2
> Wait for stress to finish
> Run repair on Cluster
> ... Some other operations
> At the point when the second datacenter is added several warnings go off 
> because nodetool status is not functioning, and a few moments later the start 
> operation reports a failure because a node has not successfully turned on. 
> The first start attempt yielded the following exception on a node in the 
> second DC.
> {code}
> CassandraDaemon.java (line 464) Exception encountered during startup
> java.lang.AssertionError: -7560216458456714666 not found in 
> -9222060278673125462, -9220751250790085193, ..... ALL THE TOKENS ...,  
> 9218575851928340117, 9219681798686280387
> at 
> org.apache.cassandra.locator.TokenMetadata.getPredecessor(TokenMetadata.java:752)
> at 
> org.apache.cassandra.locator.TokenMetadata.getPrimaryRangesFor(TokenMetadata.java:696)
> at 
> org.apache.cassandra.locator.TokenMetadata.getPrimaryRangeFor(TokenMetadata.java:703)
> at 
> org.apache.cassandra.locator.AbstractReplicationStrategy.getRangeAddresses(AbstractReplicationStrategy.java:187)
> at 
> org.apache.cassandra.dht.RangeStreamer.getAllRangesWithSourcesFor(RangeStreamer.java:147)
> at org.apache.cassandra.dht.RangeStreamer.addRanges(RangeStreamer.java:121)
> at org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:81)
> at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:979)
> at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:745)
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:586)
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:483)
>       at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:348)
>       at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:447)
>       at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:490)
> {code}
> The test automatically tries to restart nodes if they fail during startup, 
> The second attempt for this node succeeded but a 'nodetool status' still 
> failed and a different node in the second DC logged the following and failed 
> to start up.
> {code}
> ERROR [main] 2013-12-16 18:02:04,869 CassandraDaemon.java (line 464) 
> Exception encountered during startup
> java.util.ConcurrentModificationException
>       at java.util.TreeMap$PrivateEntryIterator.nextEntry(TreeMap.java:1115)
>       at java.util.TreeMap$KeyIterator.next(TreeMap.java:1169)
>       at org.apache.commons.lang.StringUtils.join(StringUtils.java:3382)
>       at org.apache.commons.lang.StringUtils.join(StringUtils.java:3444)
>       at 
> org.apache.cassandra.locator.TokenMetadata.getPredecessor(TokenMetadata.java:752)
>       at 
> org.apache.cassandra.locator.TokenMetadata.getPrimaryRangesFor(TokenMetadata.java:696)
>       at 
> org.apache.cassandra.locator.TokenMetadata.getPrimaryRangeFor(TokenMetadata.java:703)
>       at 
> org.apache.cassandra.locator.AbstractReplicationStrategy.getRangeAddresses(AbstractReplicationStrategy.java:187)
>       at 
> org.apache.cassandra.dht.RangeStreamer.getAllRangesWithSourcesFor(RangeStreamer.java:147)
>       at 
> org.apache.cassandra.dht.RangeStreamer.addRanges(RangeStreamer.java:121)
>       at org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:81)
>       at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:979)
>       at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:745)
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:586)
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:483)
>       at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:348)
>       at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:447)
>       at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:490)
> ERROR [StorageServiceShutdownHook] 2013-12-16 18:02:04,876 
> CassandraDaemon.java (line 191) Exception in thread 
> Thread[StorageServiceShutdownHook,5,main]
> java.lang.NullPointerException
>       at 
> org.apache.cassandra.service.StorageService.stopNativeTransport(StorageService.java:358)
>       at 
> org.apache.cassandra.service.StorageService.shutdownClientServers(StorageService.java:373)
>       at 
> org.apache.cassandra.service.StorageService.access$000(StorageService.java:89)
>       at 
> org.apache.cassandra.service.StorageService$1.runMayThrow(StorageService.java:551)
>       at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>       at java.lang.Thread.run(Thread.java:724)
> {code}



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)

Reply via email to