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

Ryan Daum commented on CASSANDRA-713:
-------------------------------------

I am getting the same exception on the same configuration (RF 3, 
RandomPartitioner, 6 nodes, about 5-10gigs of data per node) again with 0.5 
final from debian packages when I do the following:

1. run 'loadbalance' on 6th node
2. see, after a reasonable amount of time: 

 INFO [STREAM-STAGE:1] 2010-01-28 18:32:26,624 BootStrapper.java (line 119) New 
token will be 62831091626283968915592956651596253668 to assume load from 
/10.252.90.224
 INFO [STREAM-STAGE:1] 2010-01-28 18:32:26,624 StorageService.java (line 1392) 
re-bootstrapping to new token 62831091626283968915592956651596253668
 INFO [STREAM-STAGE:1] 2010-01-28 18:32:26,625 StorageService.java (line 342) 
bootstrap sleeping 30000

3. wait what seems a very unreasonable amount of hours (in this case 24 hours, 
local node had 5 gigs of data in it). node has not rejoined ring with a new 
token range.  get very concerned nothing is happening.
4. try taking node offline, all other nodes in the cluster now complain 
incessantly with the same stack trace reported originally.
5. notice that all writing clients now get exceptions on _all_ writes:

Caused by: org.apache.thrift.TApplicationException: Internal error processing 
insert
        at 
org.apache.thrift.TApplicationException.read(TApplicationException.java:107)
        at 
org.apache.cassandra.service.Cassandra$Client.recv_insert(Cassandra.java:569)
        at 
org.apache.cassandra.service.Cassandra$Client.insert(Cassandra.java:547)

6. restarting the node makes the exceptions go away and reports:

 INFO [STREAM-STAGE:1] 2010-01-28 18:32:26,624 BootStrapper.java (line 119) New 
token will be 62831091626283968915592956651596253668 to assume load from 
/10.252.90.224
 INFO [STREAM-STAGE:1] 2010-01-28 18:32:26,624 StorageService.java (line 1392) 
re-bootstrapping to new token 62831091626283968915592956651596253668
 INFO [STREAM-STAGE:1] 2010-01-28 18:32:26,625 StorageService.java (line 342) 
bootstrap sleeping 30000

but if past experience is any judge, this will never lead to this node 
rejoining the ring.

> Stacktrace when node taken offline
> ----------------------------------
>
>                 Key: CASSANDRA-713
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-713
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.5
>            Reporter: Ryan Daum
>            Assignee: Jaakko Laine
>             Fix For: 0.5
>
>
> I took a node offline last week and then attempted to re-bootstrap its token 
> range with a new cassandra install on the same IP. I made gossip forget about 
> the node by restarting all other instances, then brought up the new node. It 
> said was bootstrapping, but it never finished bootstrapping after several 
> days. The node never showed up in the ring, but when I take it offline, I get 
> the following exception continually from all other nodes in the cluster:
> ERROR [pool-1-thread-8] 2010-01-18 21:01:32,405 Cassandra.java (line 1096) 
> Internal error processing batch_insert
> java.lang.NullPointerException
>         at 
> org.apache.cassandra.dht.BigIntegerToken.compareTo(BigIntegerToken.java:38)
>         at 
> org.apache.cassandra.dht.BigIntegerToken.compareTo(BigIntegerToken.java:23)
>         at java.util.Collections.indexedBinarySearch(Collections.java:215)
>         at java.util.Collections.binarySearch(Collections.java:201)
>         at 
> org.apache.cassandra.locator.AbstractReplicationStrategy.getHintedMapForEndpoints(AbstractReplicationStrategy.java:130)
>         at 
> org.apache.cassandra.locator.AbstractReplicationStrategy.getHintedEndpoints(AbstractReplicationStrategy.java:76)
>         at 
> org.apache.cassandra.service.StorageService.getHintedEndpointMap(StorageService.java:1183)
>         at 
> org.apache.cassandra.service.StorageProxy.insertBlocking(StorageProxy.java:169)
>         at 
> org.apache.cassandra.service.CassandraServer.doInsert(CassandraServer.java:466)
>         at 
> org.apache.cassandra.service.CassandraServer.batch_insert(CassandraServer.java:445)
>         at 
> org.apache.cassandra.service.Cassandra$Processor$batch_insert.process(Cassandra.java:1088)
>         at 
> org.apache.cassandra.service.Cassandra$Processor.process(Cassandra.java:817)
>         at 
> org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:253)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>         at java.lang.Thread.run(Thread.java:619)
> In addition, I get frequent UnavailableExceptions on the other nodes.
> I cannot remove the token range for this node because it never officially 
> joined the ring.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to