You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Michael Lee (JIRA)" <ji...@apache.org> on 2010/01/21 03:54:55 UTC

[jira] Issue Comment Edited: (CASSANDRA-727) other nodes in the ring will throw exception when I use the same TokenID but different IP address for a new node bootstrapping

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

Michael Lee edited comment on CASSANDRA-727 at 1/21/10 2:54 AM:
----------------------------------------------------------------

we DON'T want to use remove token to stream a lot of data again and again. it's a kind of waste of I/O and network resource

a better way is replace bad node with a new one directly, not remove bad one first, and insert a new one

it's seem 'add a fresh node with same token and different ip ' is the nearest way, as we disscuss with Jonathan before.

-------------------------------------------------------------
发件人:Michael Lee
发送日期:2010-01-19 21:37:52
收件人:cassandra-user@incubator.apache.org
抄送:
主题:RE: Re: replace a bad node through bootstrapping

Sorry, My bad!

So, if use same token different IP, ' nodeprobe removetoken ' process is not needed, am I right? 

If so, I think the wiki page should state it clear, my advice....

And once again, the replaced node damage again, we replace the bad one with new node, this time we use the same token again, and use the 'old' IP For example:

Step1:  replace A (token=11111, ip=1.1.1.1) with B(token=11111, ip=2.2.2.2), according we discussed before, it will work
Step2: we fix A, format the whole file system, now B go to bad, we replace B(token=11111, ip=2.2.2.2) with A (token=11111, ip=1.1.1.1) because we only one backup IP and backup node.

Does step 2 still work?


---------END----------

-----Original Message-----
From: Jonathan Ellis [mailto:jbellis@gmail.com]
Sent: Tuesday, January 19, 2010 9:57 AM
To: cassandra-user@incubator.apache.org
Subject: Re: Re: replace a bad node through bootstrapping

This is described in the "Handling failure" section of the Operations page.

I believe it will work even if you use the same token as the old node, yes.

-Jonathan

2010/1/18 Michael Lee <ma...@gmail.com>:
> Even good node has the same token as bad one?
>
> Is it an 'documented' operation? I haven't seen it in wiki
links(Operation).
>
> ---------END----------
>
> -----Original Message-----
> From: Jonathan Ellis [mailto:jbellis@gmail.com]
> Sent: Monday, January 18, 2010 11:22 PM
> To: cassandra-user@incubator.apache.org
> Subject: Re: Re: Re: replace a bad node through bootstrapping
>
> yes
>
> On Mon, Jan 18, 2010 at 4:37 AM, XL.Pan <pa...@sina.com> wrote:
>> Hi Jonathan:
>>  "the old node can be the replacement, as long as you change its IP
> address"
>>
>>  Do you mean that the operations to replace a bad node is :
>> 1) choose a new machine which has the same configuration, eg.
> InitialToken, and has a different IP address;
>> 2) start the new machine, which will start boostrapping;
>> 3) After bootstrapping, the new machine will restore the data as before.
>>
>> (All nodes' InitialToken are set manully)
>>
>> I have tried in this way and that looks ok.  Is this a good way? :-) 
>> Thanks !!
>>
>>
>> ------------------
>> XL.Pan
>> 2010-01-18
>>
>> -------------------------------------------------------------
>> 发件人:Jonathan Ellis
>> 发送日期:2010-01-15 11:12:05
>> 收件人:cassandra-user
>> 抄送:
>> 主题:Re: Re: replace a bad node through bootstrapping
>>
>> On Thu, Jan 14, 2010 at 9:02 PM, Michael Lee 
>> <ma...@gmail.com> wrote:
>>> If a node's data has been damaged, you cannot use new node replace 
>>> old
> one directly, unless 'removetoken' first.
>>>
>>> But, (suppose node A is dead)
>>> 'removetoken' will complement missing replica due A's death first, 
>>> it
> will generate lot data on other nodes, say it's B, C, D
>>> After add new node and copy data from other node through 
>>> bootstrapping,
> you have to 'cleanup' data just
>>> generate from ' removetoken ' on B, C, D
>>>
>>> So, B/C/D will have heavy I/O load (half of them is waste) due to 
>>> repair
> A, in pan's case, it will be 5TB (and will cause days...)
>>>
>>> Pan try to invent a method to repair A directly through streaming, 
>>> and
> have less impact on other nodes.
>>
>> Thanks for clarifying that.
>>
>> I thought we agreed in your last thread about this that bootstrapping 
>> a replacement node (the old node can be the replacement, as long as 
>> you change its IP address) first, then removing the entry for the 
>> dead one, would be a reasonable procedure here.
>>
>> -Jonathan
>>
>
>



      was (Author: steel_mental):
    we DON'T want to use remove token to stream a lot of data again and again. it's a kind of waste of I/O and network resource

a better way is replace bad node with a new one directly, not remove bad one first, and insert a new one

it's seem 'add a fresh node with same token and different ip ' is the nearest way, as we disscuss with Jonathan before.

Hi  Jonathan :
    I find other nodes will throw exception when I use the same TokenID but different IP address for a new node while bootstrapping. My operations are that :
1) down nodeA which token is T, IP address  is IPA;
2) add a new node B which token is also T but IP address is IPB;
3) After nodeA knows nodeB is bootstrapping, A throw exception:

DEBUG [GMFD:2] 2010-01-19 22:22:36,667 StorageService.java (line 439) Node /10.81.37.52 state bootstrapping, token 136112946768375385385349842972707284580
ERROR [GMFD:2] 2010-01-19 22:22:36,667 DebuggableThreadPoolExecutor.java (line 157) Error in ThreadPoolExecutor
java.lang.RuntimeException: Bootstrap Token collision between /10.81.37.65 and /10.81.37.52 (token 136112946768375385385349842972707284580
        at org.apache.cassandra.locator.TokenMetadata.addBootstrapToken(TokenMetadata.java:136)
        at org.apache.cassandra.service.StorageService.handleStateBootstrap(StorageService.java:456)
        at org.apache.cassandra.service.StorageService.onChange(StorageService.java:419)
        at org.apache.cassandra.gms.Gossiper.doNotifications(Gossiper.java:692)
        at org.apache.cassandra.gms.Gossiper.applyApplicationStateLocally(Gossiper.java:657)
        at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:611)
        at org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(Gossiper.java:979)
        at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:38)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
        at java.lang.Thread.run(Thread.java:619)
ERROR [GMFD:2] 2010-01-19 22:22:36,667 CassandraDaemon.java (line 71) Fatal exception in thread Thread[GMFD:2,5,main]
java.lang.RuntimeException: Bootstrap Token collision between /10.81.37.65 and /10.81.37.52 (token 136112946768375385385349842972707284580
        at org.apache.cassandra.locator.TokenMetadata.addBootstrapToken(TokenMetadata.java:136)
        at org.apache.cassandra.service.StorageService.handleStateBootstrap(StorageService.java:456)
        at org.apache.cassandra.service.StorageService.onChange(StorageService.java:419)
        at org.apache.cassandra.gms.Gossiper.doNotifications(Gossiper.java:692)
        at org.apache.cassandra.gms.Gossiper.applyApplicationStateLocally(Gossiper.java:657)
        at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:611)
        at org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(Gossiper.java:979)
        at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:38)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
        at java.lang.Thread.run(Thread.java:619)

I traced the codes and find the reason is that :
    public void addBootstrapToken(Token token, InetAddress endpoint)
    {
             ......
            oldEndPoint = bootstrapTokens.get(token);
            if (oldEndPoint != null && !oldEndPoint.equals(endpoint))
                throw new RuntimeException("Bootstrap Token collision between " + oldEndPoint + " and " + endpoint + " (token " + token);
// the exception is here --|      :-)
                                   v
            oldEndPoint = tokenToEndPointMap.get(token);
            if (oldEndPoint != null && !oldEndPoint.equals(endpoint))
                throw new RuntimeException("Bootstrap Token collision between " + oldEndPoint + " and " + endpoint + " (token " + token);
           ......
    }

In my enviroment, A is still in the tokenToEndPointMap though it's down. As a result, A will not know B is bootstrapping and will not route any request to B until B changes to NORMAL.

I think the following changes may be one way to deal with this issue :

            if (oldEndPoint != null && !oldEndPoint.equals(endpoint) && FailureDetector.instance().isAlive(oldEndPoint))
                throw new RuntimeException("Bootstrap Token collision between " + oldEndPoint + " and " + endpoint + " (token " + token);

Will you please give me some advice? Thanks!


------------------				 
XL.Pan
2010-01-20

-------------------------------------------------------------
发件人:Michael Lee
发送日期:2010-01-19 21:37:52
收件人:cassandra-user@incubator.apache.org
抄送:
主题:RE: Re: replace a bad node through bootstrapping

Sorry, My bad!

So, if use same token different IP, ' nodeprobe removetoken ' process is not needed, am I right? 

If so, I think the wiki page should state it clear, my advice....

And once again, the replaced node damage again, we replace the bad one with new node, this time we use the same token again, and use the 'old' IP For example:

Step1:  replace A (token=11111, ip=1.1.1.1) with B(token=11111, ip=2.2.2.2), according we discussed before, it will work
Step2: we fix A, format the whole file system, now B go to bad, we replace B(token=11111, ip=2.2.2.2) with A (token=11111, ip=1.1.1.1) because we only one backup IP and backup node.

Does step 2 still work?


---------END----------

-----Original Message-----
From: Jonathan Ellis [mailto:jbellis@gmail.com]
Sent: Tuesday, January 19, 2010 9:57 AM
To: cassandra-user@incubator.apache.org
Subject: Re: Re: replace a bad node through bootstrapping

This is described in the "Handling failure" section of the Operations page.

I believe it will work even if you use the same token as the old node, yes.

-Jonathan

2010/1/18 Michael Lee <ma...@gmail.com>:
> Even good node has the same token as bad one?
>
> Is it an 'documented' operation? I haven't seen it in wiki
links(Operation).
>
> ---------END----------
>
> -----Original Message-----
> From: Jonathan Ellis [mailto:jbellis@gmail.com]
> Sent: Monday, January 18, 2010 11:22 PM
> To: cassandra-user@incubator.apache.org
> Subject: Re: Re: Re: replace a bad node through bootstrapping
>
> yes
>
> On Mon, Jan 18, 2010 at 4:37 AM, XL.Pan <pa...@sina.com> wrote:
>> Hi Jonathan:
>>  "the old node can be the replacement, as long as you change its IP
> address"
>>
>>  Do you mean that the operations to replace a bad node is :
>> 1) choose a new machine which has the same configuration, eg.
> InitialToken, and has a different IP address;
>> 2) start the new machine, which will start boostrapping;
>> 3) After bootstrapping, the new machine will restore the data as before.
>>
>> (All nodes' InitialToken are set manully)
>>
>> I have tried in this way and that looks ok.  Is this a good way? :-) 
>> Thanks !!
>>
>>
>> ------------------
>> XL.Pan
>> 2010-01-18
>>
>> -------------------------------------------------------------
>> 发件人:Jonathan Ellis
>> 发送日期:2010-01-15 11:12:05
>> 收件人:cassandra-user
>> 抄送:
>> 主题:Re: Re: replace a bad node through bootstrapping
>>
>> On Thu, Jan 14, 2010 at 9:02 PM, Michael Lee 
>> <ma...@gmail.com> wrote:
>>> If a node's data has been damaged, you cannot use new node replace 
>>> old
> one directly, unless 'removetoken' first.
>>>
>>> But, (suppose node A is dead)
>>> 'removetoken' will complement missing replica due A's death first, 
>>> it
> will generate lot data on other nodes, say it's B, C, D
>>> After add new node and copy data from other node through 
>>> bootstrapping,
> you have to 'cleanup' data just
>>> generate from ' removetoken ' on B, C, D
>>>
>>> So, B/C/D will have heavy I/O load (half of them is waste) due to 
>>> repair
> A, in pan's case, it will be 5TB (and will cause days...)
>>>
>>> Pan try to invent a method to repair A directly through streaming, 
>>> and
> have less impact on other nodes.
>>
>> Thanks for clarifying that.
>>
>> I thought we agreed in your last thread about this that bootstrapping 
>> a replacement node (the old node can be the replacement, as long as 
>> you change its IP address) first, then removing the entry for the 
>> dead one, would be a reasonable procedure here.
>>
>> -Jonathan
>>
>
>


  
> other nodes in the ring will throw exception when I use the same TokenID but different IP address for a new node bootstrapping
> ------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-727
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-727
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 0.5
>            Reporter: david.pan
>         Attachments: 727-allow_bootstrapping_with_the_same_token.patch
>
>   Original Estimate: 0.5h
>  Remaining Estimate: 0.5h
>
>  I find other nodes will throw exception when I use the same TokenID but different IP address for a new node while bootstrapping. My operations are that :
> 1) down nodeA which token is T, IP address  is IPA;
> 2) add a new node B which token is also T but IP address is IPB;
> 3) After nodeA knows nodeB is bootstrapping, A throw exception:
> DEBUG [GMFD:2] 2010-01-19 22:22:36,667 StorageService.java (line 439) Node /10.81.37.52 state bootstrapping, token 136112946768375385385349842972707284580
> ERROR [GMFD:2] 2010-01-19 22:22:36,667 DebuggableThreadPoolExecutor.java (line 157) Error in ThreadPoolExecutor
> java.lang.RuntimeException: Bootstrap Token collision between /10.81.37.65 and /10.81.37.52 (token 136112946768375385385349842972707284580
>         at org.apache.cassandra.locator.TokenMetadata.addBootstrapToken(TokenMetadata.java:136)
>         at org.apache.cassandra.service.StorageService.handleStateBootstrap(StorageService.java:456)
>         at org.apache.cassandra.service.StorageService.onChange(StorageService.java:419)
>         at org.apache.cassandra.gms.Gossiper.doNotifications(Gossiper.java:692)
>         at org.apache.cassandra.gms.Gossiper.applyApplicationStateLocally(Gossiper.java:657)
>         at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:611)
>         at org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(Gossiper.java:979)
>         at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:38)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>         at java.lang.Thread.run(Thread.java:619)
> ERROR [GMFD:2] 2010-01-19 22:22:36,667 CassandraDaemon.java (line 71) Fatal exception in thread Thread[GMFD:2,5,main]
> java.lang.RuntimeException: Bootstrap Token collision between /10.81.37.65 and /10.81.37.52 (token 136112946768375385385349842972707284580
>         at org.apache.cassandra.locator.TokenMetadata.addBootstrapToken(TokenMetadata.java:136)
>         at org.apache.cassandra.service.StorageService.handleStateBootstrap(StorageService.java:456)
>         at org.apache.cassandra.service.StorageService.onChange(StorageService.java:419)
>         at org.apache.cassandra.gms.Gossiper.doNotifications(Gossiper.java:692)
>         at org.apache.cassandra.gms.Gossiper.applyApplicationStateLocally(Gossiper.java:657)
>         at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:611)
>         at org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(Gossiper.java:979)
>         at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:38)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>         at java.lang.Thread.run(Thread.java:619)
> I traced the codes and find the reason is that :
>     public void addBootstrapToken(Token token, InetAddress endpoint)
>     {
>              ......
>             oldEndPoint = bootstrapTokens.get(token);
>             if (oldEndPoint != null && !oldEndPoint.equals(endpoint))
>                 throw new RuntimeException("Bootstrap Token collision between " + oldEndPoint + " and " + endpoint + " (token " + token);
> // the exception is here --|      :-)
>                                    v
>             oldEndPoint = tokenToEndPointMap.get(token);
>             if (oldEndPoint != null && !oldEndPoint.equals(endpoint))
>                 throw new RuntimeException("Bootstrap Token collision between " + oldEndPoint + " and " + endpoint + " (token " + token);
>            ......
>     }
> In my enviroment, A is still in the tokenToEndPointMap though it's down. As a result, A will not know B is bootstrapping and will not route any request to B until B changes to NORMAL.
> I think the following changes may be one way to deal with this issue :
>             if (oldEndPoint != null && !oldEndPoint.equals(endpoint) && FailureDetector.instance().isAlive(oldEndPoint))
>                 throw new RuntimeException("Bootstrap Token collision between " + oldEndPoint + " and " + endpoint + " (token " + token);

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