You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Alain RODRIGUEZ <ar...@gmail.com> on 2012/08/06 10:30:53 UTC

Re: Dead node still being pinged

Hi sorry about reopening this old thread but I think that this
function (unsafeAssassinateEndpoint) can help me to resolve a problem
I have in production for a while :

http://grokbase.com/t/cassandra/user/127knx7nn0/unreachable-node-not-in-nodetool-ring

I have no Idea about how to use it, Can you enlighten me about this point ?

Do you think it will resolve my issue ?

What are the risks of using this "unsafe" "assassinate" function ?
(It's quite frightening... :p)

Alain

2012/6/14 Nicolas Lalevée <ni...@hibnet.org>:
>
> Le 13 juin 2012 à 20:52, aaron morton a écrit :
>
>>> You meant -Dcassandra.load_ring_state=false right ?
>> yes, sorry.
>>
>>> Maybe I could open a jira about my issue ? Maybe there was a config mess on my part at some point, ie the unsynchronized date on my machines, but I think it would be nice if cassandra could resolve itself of that inconsistent state.
>> The old nodes are not listed in the ring are they ?
>>
>> You can try calling unsafeAssassinateEndpoint() on the Gossip MBean.
>
> unsafe, assassinate, hum :)
> I had read the source code of that function to reassure myself, but I did called it.
> And it worked, I don't see any packet from new nodes to the old nodes anymore.
> The gossip info changed. I have now some 'LEFT' statuses instead of 'removed' ones:
> /10.10.0.24
>   REMOVAL_COORDINATOR:REMOVER,0
>   STATUS:LEFT,141713094015402114482637574571109123934,1339920978684
> /10.10.0.22
>   REMOVAL_COORDINATOR:REMOVER,113427455640312814857969558651062452224
>   STATUS:LEFT,141713094015402114482637574571109123934,1339920834956
>
> Thanks you very much Aaron.
>
> Nicolas
>
>
>>
>> Cheers
>>
>> -----------------
>> Aaron Morton
>> Freelance Developer
>> @aaronmorton
>> http://www.thelastpickle.com
>>
>> On 14/06/2012, at 12:06 AM, Nicolas Lalevée wrote:
>>
>>>
>>> Le 13 juin 2012 à 10:30, aaron morton a écrit :
>>>
>>>> Here is what I *think* is going on, if Brandon is around he may be able to help out.
>>>>
>>>>
>>>> The old nodes are being included in the Gossip rounds, because Gossiper.doGossipToUnreachableMember() just looks at the nodes that are unreachable. It does not check if they have been removed from the cluster.
>>>>
>>>> Information about the removed nodes is kept by gossip so that if a node is removed while it is down it will shut down when restarted. This information *should* stay in gossip for 3 days.
>>>>
>>>> In your gossip info, the last long on the STATUS lines is the expiry time for this info…
>>>>
>>>> /10.10.0.24
>>>> STATUS:removed,127605887595351923798765477786913079296,1336530323263
>>>> REMOVAL_COORDINATOR:REMOVER,0
>>>> /10.10.0.22
>>>> STATUS:removed,42535295865117307932921825928971026432,1336529659203
>>>> REMOVAL_COORDINATOR:REMOVER,113427455640312814857969558651062452224
>>>>
>>>> For the first line it's
>>>> In [48]: datetime.datetime.fromtimestamp(1336530323263/1000)
>>>> Out[48]: datetime.datetime(2012, 5, 9, 14, 25, 23)
>>>>
>>>> So that's good.
>>>>
>>>> The Gossip round will remove the 0.24 and 0.22 nodes from the local state if the expiry time has passed, and the node is marked as dead and it's not in the token ring.
>>>>
>>>> You can see if the node thinks 0.24 and 0.22 are up by looking getSimpleStates() on the FailureDetectorMBean. (I use jmxterm to do this sort of thing)
>>>
>>> The two old nodes are still seen as down:
>>> SimpleStates:[/10.10.0.22:DOWN, /10.10.0.24:DOWN, /10.10.0.26:UP, /10.10.0.25:UP, /10.10.0.27:UP]
>>>
>>>>
>>>> The other thing that can confuse things is the gossip generation. If your old nodes were started with a datetime in the future that can muck things up.
>>>
>>> I have just checked, my old nodes machines are nicely synchronized. My new nodes have some lag of few seconds, some in the future, some in the past. I definitively need to fix that.
>>>
>>>> The simple to try is starting the server with the -Dcassandra.join_ring=false JVM option. This will force the node to get the ring info from othernodes. Check things with nodetool gossip info to see if the other nodes tell it about the old ones again.
>>>
>>> You meant -Dcassandra.load_ring_state=false right ?
>>>
>>> Then nothing changed.
>>>
>>>> Sorry, gossip can be tricky to diagnose over email.
>>>
>>> No worry, I really appreciate that you take time looking into my issues.
>>>
>>> Maybe I could open a jira about my issue ? Maybe there was a config mess on my part at some point, ie the unsynchronized date on my machines, but I think it would be nice if cassandra could resolve itself of that inconsistent state.
>>>
>>> Nicolas
>>>
>>>>
>>>>
>>>>
>>>>
>>>> -----------------
>>>> Aaron Morton
>>>> Freelance Developer
>>>> @aaronmorton
>>>> http://www.thelastpickle.com
>>>>
>>>> On 12/06/2012, at 10:33 PM, Nicolas Lalevée wrote:
>>>>
>>>>> I have one dirty solution to try: bring data-2 and data-4 back up and down again. Is there any way I can tell cassandra to not get any data, so when I would get my old node up, no streaming would start ?
>>>>>
>>>>> cheers,
>>>>> Nicolas
>>>>>
>>>>> Le 12 juin 2012 à 12:25, Nicolas Lalevée a écrit :
>>>>>
>>>>>> Le 12 juin 2012 à 11:03, aaron morton a écrit :
>>>>>>
>>>>>>> Try purging the hints for 10.10.0.24 using the HintedHandOffManager MBean.
>>>>>>
>>>>>> As far as I could tell, there were no hinted hand off to be delivered. Nevertheless I have called "deleteHintsForEndpoint" on every node for the two expected to be out nodes.
>>>>>> Nothing changed, I still see packet being send to these old nodes.
>>>>>>
>>>>>> I looked closer to ResponsePendingTasks of MessagingService. Actually the numbers change, between 0 and about 4. So tasks are ending but new ones come just after.
>>>>>>
>>>>>> Nicolas
>>>>>>
>>>>>>>
>>>>>>> Cheers
>>>>>>>
>>>>>>> -----------------
>>>>>>> Aaron Morton
>>>>>>> Freelance Developer
>>>>>>> @aaronmorton
>>>>>>> http://www.thelastpickle.com
>>>>>>>
>>>>>>> On 12/06/2012, at 3:33 AM, Nicolas Lalevée wrote:
>>>>>>>
>>>>>>>> finally, thanks to the groovy jmx builder, it was not that hard.
>>>>>>>>
>>>>>>>>
>>>>>>>> Le 11 juin 2012 à 12:12, Samuel CARRIERE a écrit :
>>>>>>>>
>>>>>>>>> If I were you, I would connect (through JMX, with jconsole) to one of the nodes that is sending messages to an old node, and would have a look at these MBean :
>>>>>>>>> - org.apache.net.FailureDetector : does SimpleStates looks good ? (or do you see an IP of an old node)
>>>>>>>>
>>>>>>>> SimpleStates:[/10.10.0.22:DOWN, /10.10.0.24:DOWN, /10.10.0.26:UP, /10.10.0.25:UP, /10.10.0.27:UP]
>>>>>>>>
>>>>>>>>> - org.apache.net.MessagingService : do you see one of the old IP in one of the attributes ?
>>>>>>>>
>>>>>>>> data-5:
>>>>>>>> CommandCompletedTasks:
>>>>>>>> [10.10.0.22:2, 10.10.0.26:6147307, 10.10.0.27:6084684, 10.10.0.24:2]
>>>>>>>> CommandPendingTasks:
>>>>>>>> [10.10.0.22:0, 10.10.0.26:0, 10.10.0.27:0, 10.10.0.24:0]
>>>>>>>> ResponseCompletedTasks:
>>>>>>>> [10.10.0.22:1487, 10.10.0.26:6187204, 10.10.0.27:6062890, 10.10.0.24:1495]
>>>>>>>> ResponsePendingTasks:
>>>>>>>> [10.10.0.22:0, 10.10.0.26:0, 10.10.0.27:0, 10.10.0.24:0]
>>>>>>>>
>>>>>>>> data-6:
>>>>>>>> CommandCompletedTasks:
>>>>>>>> [10.10.0.22:2, 10.10.0.27:6064992, 10.10.0.24:2, 10.10.0.25:6308102]
>>>>>>>> CommandPendingTasks:
>>>>>>>> [10.10.0.22:0, 10.10.0.27:0, 10.10.0.24:0, 10.10.0.25:0]
>>>>>>>> ResponseCompletedTasks:
>>>>>>>> [10.10.0.22:1463, 10.10.0.27:6067943, 10.10.0.24:1474, 10.10.0.25:6367692]
>>>>>>>> ResponsePendingTasks:
>>>>>>>> [10.10.0.22:0, 10.10.0.27:0, 10.10.0.24:2, 10.10.0.25:0]
>>>>>>>>
>>>>>>>> data-7:
>>>>>>>> CommandCompletedTasks:
>>>>>>>> [10.10.0.22:2, 10.10.0.26:6043653, 10.10.0.24:2, 10.10.0.25:5964168]
>>>>>>>> CommandPendingTasks:
>>>>>>>> [10.10.0.22:0, 10.10.0.26:0, 10.10.0.24:0, 10.10.0.25:0]
>>>>>>>> ResponseCompletedTasks:
>>>>>>>> [10.10.0.22:1424, 10.10.0.26:6090251, 10.10.0.24:1431, 10.10.0.25:6094954]
>>>>>>>> ResponsePendingTasks:
>>>>>>>> [10.10.0.22:4, 10.10.0.26:0, 10.10.0.24:1, 10.10.0.25:0]
>>>>>>>>
>>>>>>>>> - org.apache.net.StreamingService : do you see an old IP in StreamSources or StreamDestinations ?
>>>>>>>>
>>>>>>>> nothing streaming on the 3 nodes.
>>>>>>>> nodetool netstats confirmed that.
>>>>>>>>
>>>>>>>>> - org.apache.internal.HintedHandoff : are there non-zero ActiveCount, CurrentlyBlockedTasks, PendingTasks, TotalBlockedTask ?
>>>>>>>>
>>>>>>>> On the 3 nodes, all at 0.
>>>>>>>>
>>>>>>>> I don't know much what I'm looking at, but it seems that some ResponsePendingTasks needs to end.
>>>>>>>>
>>>>>>>> Nicolas
>>>>>>>>
>>>>>>>>>
>>>>>>>>> Samuel
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Nicolas Lalevée <ni...@hibnet.org>
>>>>>>>>> 08/06/2012 21:03
>>>>>>>>> Veuillez répondre à
>>>>>>>>> user@cassandra.apache.org
>>>>>>>>>
>>>>>>>>> A
>>>>>>>>> user@cassandra.apache.org
>>>>>>>>> cc
>>>>>>>>> Objet
>>>>>>>>> Re: Dead node still being pinged
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Le 8 juin 2012 à 20:02, Samuel CARRIERE a écrit :
>>>>>>>>>
>>>>>>>>>> I'm in the train but just a guess : maybe it's hinted handoff. A look in the logs of the new nodes could confirm that : look for the IP of an old node and maybe you'll find hinted handoff related messages.
>>>>>>>>>
>>>>>>>>> I grepped on every node about every old node, I got nothing since the "crash".
>>>>>>>>>
>>>>>>>>> If it can be of some help, here is some grepped log of the crash:
>>>>>>>>>
>>>>>>>>> system.log.1: WARN [RMI TCP Connection(1037)-10.10.0.26] 2012-05-06 00:39:30,241 StorageService.java (line 2417) Endpoint /10.10.0.24 is down and will not receive data for re-replication of /10.10.0.22
>>>>>>>>> system.log.1: WARN [RMI TCP Connection(1037)-10.10.0.26] 2012-05-06 00:39:30,242 StorageService.java (line 2417) Endpoint /10.10.0.24 is down and will not receive data for re-replication of /10.10.0.22
>>>>>>>>> system.log.1: WARN [RMI TCP Connection(1037)-10.10.0.26] 2012-05-06 00:39:30,242 StorageService.java (line 2417) Endpoint /10.10.0.24 is down and will not receive data for re-replication of /10.10.0.22
>>>>>>>>> system.log.1: WARN [RMI TCP Connection(1037)-10.10.0.26] 2012-05-06 00:39:30,243 StorageService.java (line 2417) Endpoint /10.10.0.24 is down and will not receive data for re-replication of /10.10.0.22
>>>>>>>>> system.log.1: WARN [RMI TCP Connection(1037)-10.10.0.26] 2012-05-06 00:39:30,243 StorageService.java (line 2417) Endpoint /10.10.0.24 is down and will not receive data for re-replication of /10.10.0.22
>>>>>>>>> system.log.1: INFO [GossipStage:1] 2012-05-06 00:44:33,822 Gossiper.java (line 818) InetAddress /10.10.0.24 is now dead.
>>>>>>>>> system.log.1: INFO [GossipStage:1] 2012-05-06 04:25:23,894 Gossiper.java (line 818) InetAddress /10.10.0.24 is now dead.
>>>>>>>>> system.log.1: INFO [OptionalTasks:1] 2012-05-06 04:25:23,895 HintedHandOffManager.java (line 179) Deleting any stored hints for /10.10.0.24
>>>>>>>>> system.log.1: INFO [GossipStage:1] 2012-05-06 04:25:23,895 StorageService.java (line 1157) Removing token 127605887595351923798765477786913079296 for /10.10.0.24
>>>>>>>>> system.log.1: INFO [GossipStage:1] 2012-05-09 04:26:25,015 Gossiper.java (line 818) InetAddress /10.10.0.24 is now dead.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Maybe its the way I have removed nodes ? AFAIR I didn't used the decommission command. For each node I got the node down and then issue a remove token command.
>>>>>>>>> Here is what I can find in the log about when I removed one of them:
>>>>>>>>>
>>>>>>>>> system.log.1: INFO [GossipTasks:1] 2012-05-02 17:21:10,281 Gossiper.java (line 818) InetAddress /10.10.0.24 is now dead.
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 17:21:21,496 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [GossipStage:1] 2012-05-02 17:21:59,307 Gossiper.java (line 818) InetAddress /10.10.0.24 is now dead.
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 17:31:20,336 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 17:41:06,177 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 17:51:18,148 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 18:00:31,709 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 18:11:02,521 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 18:20:38,282 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 18:31:09,513 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 18:40:31,565 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 18:51:10,566 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 19:00:32,197 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 19:11:17,018 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [HintedHandoff:1] 2012-05-02 19:21:21,759 HintedHandOffManager.java (line 292) Endpoint /10.10.0.24 died before hint delivery, aborting
>>>>>>>>> system.log.1: INFO [GossipStage:1] 2012-05-02 20:05:57,281 Gossiper.java (line 818) InetAddress /10.10.0.24 is now dead.
>>>>>>>>> system.log.1: INFO [OptionalTasks:1] 2012-05-02 20:05:57,281 HintedHandOffManager.java (line 179) Deleting any stored hints for /10.10.0.24
>>>>>>>>> system.log.1: INFO [GossipStage:1] 2012-05-02 20:05:57,281 StorageService.java (line 1157) Removing token 145835300108973619103103718265651724288 for /10.10.0.24
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Nicolas
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> ----- Message d'origine -----
>>>>>>>>>> De : Nicolas Lalevée [nicolas.lalevee@hibnet.org]
>>>>>>>>>> Envoyé : 08/06/2012 19:26 ZE2
>>>>>>>>>> À : user@cassandra.apache.org
>>>>>>>>>> Objet : Re: Dead node still being pinged
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Le 8 juin 2012 à 15:17, Samuel CARRIERE a écrit :
>>>>>>>>>>
>>>>>>>>>>> What does nodetool ring says ? (Ask every node)
>>>>>>>>>>
>>>>>>>>>> currently, each of new node see only the tokens of the new nodes.
>>>>>>>>>>
>>>>>>>>>>> Have you checked that the list of seeds in every yaml is correct ?
>>>>>>>>>>
>>>>>>>>>> yes, it is correct, every of my new node point to the first of my new node
>>>>>>>>>>
>>>>>>>>>>> What version of cassandra are you using ?
>>>>>>>>>>
>>>>>>>>>> Sorry I should have wrote this in my first mail.
>>>>>>>>>> I use the 1.0.9
>>>>>>>>>>
>>>>>>>>>> Nicolas
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Samuel
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Nicolas Lalevée <ni...@hibnet.org>
>>>>>>>>>>> 08/06/2012 14:10
>>>>>>>>>>> Veuillez répondre à
>>>>>>>>>>> user@cassandra.apache.org
>>>>>>>>>>>
>>>>>>>>>>> A
>>>>>>>>>>> user@cassandra.apache.org
>>>>>>>>>>> cc
>>>>>>>>>>> Objet
>>>>>>>>>>> Dead node still being pinged
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I had a configuration where I had 4 nodes, data-1,4. We then bought 3 bigger machines, data-5,7. And we moved all data from data-1,4 to data-5,7.
>>>>>>>>>>> To move all the data without interruption of service, I added one new node at a time. And then I removed one by one the old machines via a "remove token".
>>>>>>>>>>>
>>>>>>>>>>> Everything was working fine. Until there was an expected load on our cluster, the machine started to swap and become unresponsive. We fixed the unexpected load and the three new machines were restarted. After that the new cassandra machines were stating that some old token were not assigned, namely from data-2 and data-4. To fix this I issued again some "remove token" commands.
>>>>>>>>>>>
>>>>>>>>>>> Everything seems to be back to normal, but on the network I still see some packet from the new cluster to the old machines. On the port 7000.
>>>>>>>>>>> How I can tell cassandra to completely forget about the old machines ?
>>>>>>>>>>>
>>>>>>>>>>> Nicolas
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>