Re: replace dead node vs remove node

2018-03-25 Thread kurt greaves
Didn't read the blog but it's worth noting that if you replace the node and
give it a *different* ip address repairs will not be necessary as it will
receive writes during replacement. This works as long as you start up the
replacement node before HH window ends.

https://issues.apache.org/jira/browse/CASSANDRA-12344 and
https://issues.apache.org/jira/browse/CASSANDRA-11559 fixes this for same
address replacements (hopefully in 4.0)

On Fri., 23 Mar. 2018, 15:11 Anthony Grasso, <anthony.gra...@gmail.com>
wrote:

> Hi Peng,
>
> Correct, you would want to repair in either case.
>
> Regards,
> Anthony
>
>
> On Fri, 23 Mar 2018 at 14:09, Peng Xiao <2535...@qq.com> wrote:
>
>> Hi Anthony,
>>
>> there is a problem with replacing dead node as per the blog,if the
>> replacement process takes longer than max_hint_window_in_ms,we must run
>> repair to make the replaced node consistent again, since it missed ongoing
>> writes during bootstrapping.but for a great cluster,repair is a painful
>> process.
>>
>> Thanks,
>> Peng Xiao
>>
>>
>>
>> -- 原始邮件 --
>> *发件人:* "Anthony Grasso"<anthony.gra...@gmail.com>;
>> *发送时间:* 2018年3月22日(星期四) 晚上7:13
>> *收件人:* "user"<user@cassandra.apache.org>;
>> *主题:* Re: replace dead node vs remove node
>>
>> Hi Peng,
>>
>> Depending on the hardware failure you can do one of two things:
>>
>> 1. If the disks are intact and uncorrupted you could just use the disks
>> with the current data on them in the new node. Even if the IP address
>> changes for the new node that is fine. In that case all you need to do is
>> run repair on the new node. The repair will fix any writes the node missed
>> while it was down. This process is similar to the scenario in this blog
>> post:
>> http://thelastpickle.com/blog/2018/02/21/replace-node-without-bootstrapping.html
>>
>> 2. If the disks are inaccessible or corrupted, then use the method as
>> described in the blogpost you linked to. The operation is similar to
>> bootstrapping a new node. There is no need to perform any other remove or
>> join operation on the failed or new nodes. As per the blog post, you
>> definitely want to run repair on the new node as soon as it joins the
>> cluster. In this case here, the data on the failed node is effectively lost
>> and replaced with data from other nodes in the cluster.
>>
>> Hope this helps.
>>
>> Regards,
>> Anthony
>>
>>
>> On Thu, 22 Mar 2018 at 20:52, Peng Xiao <2535...@qq.com> wrote:
>>
>>> Dear All,
>>>
>>> when one node failure with hardware errors,it will be in DN status in
>>> the cluster.Then if we are not able to handle this error in three hours(max
>>> hints window),we will loss data,right?we have to run repair to keep the
>>> consistency.
>>> And as per
>>> https://blog.alteroot.org/articles/2014-03-12/replace-a-dead-node-in-cassandra.html,we
>>> can replace this dead node,is it the same as bootstrap new node?that means
>>> we don't need to remove node and rejoin?
>>> Could anyone please advise?
>>>
>>> Thanks,
>>> Peng Xiao
>>>
>>>
>>>
>>>
>>>


Re: replace dead node vs remove node

2018-03-22 Thread Anthony Grasso
Hi Peng,

Correct, you would want to repair in either case.

Regards,
Anthony


On Fri, 23 Mar 2018 at 14:09, Peng Xiao <2535...@qq.com> wrote:

> Hi Anthony,
>
> there is a problem with replacing dead node as per the blog,if the
> replacement process takes longer than max_hint_window_in_ms,we must run
> repair to make the replaced node consistent again, since it missed ongoing
> writes during bootstrapping.but for a great cluster,repair is a painful
> process.
>
> Thanks,
> Peng Xiao
>
>
>
> -- 原始邮件 --
> *发件人:* "Anthony Grasso"<anthony.gra...@gmail.com>;
> *发送时间:* 2018年3月22日(星期四) 晚上7:13
> *收件人:* "user"<user@cassandra.apache.org>;
> *主题:* Re: replace dead node vs remove node
>
> Hi Peng,
>
> Depending on the hardware failure you can do one of two things:
>
> 1. If the disks are intact and uncorrupted you could just use the disks
> with the current data on them in the new node. Even if the IP address
> changes for the new node that is fine. In that case all you need to do is
> run repair on the new node. The repair will fix any writes the node missed
> while it was down. This process is similar to the scenario in this blog
> post:
> http://thelastpickle.com/blog/2018/02/21/replace-node-without-bootstrapping.html
>
> 2. If the disks are inaccessible or corrupted, then use the method as
> described in the blogpost you linked to. The operation is similar to
> bootstrapping a new node. There is no need to perform any other remove or
> join operation on the failed or new nodes. As per the blog post, you
> definitely want to run repair on the new node as soon as it joins the
> cluster. In this case here, the data on the failed node is effectively lost
> and replaced with data from other nodes in the cluster.
>
> Hope this helps.
>
> Regards,
> Anthony
>
>
> On Thu, 22 Mar 2018 at 20:52, Peng Xiao <2535...@qq.com> wrote:
>
>> Dear All,
>>
>> when one node failure with hardware errors,it will be in DN status in the
>> cluster.Then if we are not able to handle this error in three hours(max
>> hints window),we will loss data,right?we have to run repair to keep the
>> consistency.
>> And as per
>> https://blog.alteroot.org/articles/2014-03-12/replace-a-dead-node-in-cassandra.html,we
>> can replace this dead node,is it the same as bootstrap new node?that means
>> we don't need to remove node and rejoin?
>> Could anyone please advise?
>>
>> Thanks,
>> Peng Xiao
>>
>>
>>
>>
>>


Re: replace dead node vs remove node

2018-03-22 Thread Jonathan Haddad
Ah sorry - I misread the original post - for some reason I had it in my
head the question was about bootstrap.

Carry on.

On Thu, Mar 22, 2018 at 8:35 PM Jonathan Haddad <j...@jonhaddad.com> wrote:

> Under normal circumstances this is not true.
>
> Take a look at org.apache.cassandra.service.StorageProxy#performWrite, it
> grabs both the natural endpoints and the pending endpoints (new nodes).
> They're eventually passed through
> to 
> org.apache.cassandra.locator.AbstractReplicationStrategy#getWriteResponseHandler,
> which keeps track of both the current endpoints and the pending ones.
> Later, it gets to the actual work:
>
> performer.apply(mutation, Iterables.concat(naturalEndpoints, 
> pendingEndpoints), responseHandler, localDataCenter, consistency_level);
>
> The signature of this method is:
>
> public interface WritePerformer
> {
> public void apply(IMutation mutation,
>   Iterable targets,
>   AbstractWriteResponseHandler responseHandler,
>   String localDataCenter,
>   ConsistencyLevel consistencyLevel) throws 
> OverloadedException;
> }
>
> Notice the targets?  That's the list of all current owners and pending
> owners.  The list is a concatenation of the natural endpoints and the
> pending ones.
>
> Pending owners are listed in org.apache.cassandra.locator.TokenMetadata
>
> // this is a cache of the calculation from {tokenToEndpointMap, 
> bootstrapTokens, leavingEndpoints}
> private final ConcurrentMap<String, PendingRangeMaps> pendingRanges = new 
> ConcurrentHashMap<String, PendingRangeMaps>();
>
>
> TL;DR: mutations are sent to nodes being bootstrapped.
>
> Jon
>
>
> On Thu, Mar 22, 2018 at 8:09 PM Peng Xiao <2535...@qq.com> wrote:
>
>> Hi Anthony,
>>
>> there is a problem with replacing dead node as per the blog,if the
>> replacement process takes longer than max_hint_window_in_ms,we must run
>> repair to make the replaced node consistent again, since it missed ongoing
>> writes during bootstrapping.but for a great cluster,repair is a painful
>> process.
>>
>> Thanks,
>> Peng Xiao
>>
>>
>>
>> -- 原始邮件 --
>> *发件人:* "Anthony Grasso"<anthony.gra...@gmail.com>;
>> *发送时间:* 2018年3月22日(星期四) 晚上7:13
>> *收件人:* "user"<user@cassandra.apache.org>;
>> *主题:* Re: replace dead node vs remove node
>>
>> Hi Peng,
>>
>> Depending on the hardware failure you can do one of two things:
>>
>> 1. If the disks are intact and uncorrupted you could just use the disks
>> with the current data on them in the new node. Even if the IP address
>> changes for the new node that is fine. In that case all you need to do is
>> run repair on the new node. The repair will fix any writes the node missed
>> while it was down. This process is similar to the scenario in this blog
>> post:
>> http://thelastpickle.com/blog/2018/02/21/replace-node-without-bootstrapping.html
>>
>> 2. If the disks are inaccessible or corrupted, then use the method as
>> described in the blogpost you linked to. The operation is similar to
>> bootstrapping a new node. There is no need to perform any other remove or
>> join operation on the failed or new nodes. As per the blog post, you
>> definitely want to run repair on the new node as soon as it joins the
>> cluster. In this case here, the data on the failed node is effectively lost
>> and replaced with data from other nodes in the cluster.
>>
>> Hope this helps.
>>
>> Regards,
>> Anthony
>>
>>
>> On Thu, 22 Mar 2018 at 20:52, Peng Xiao <2535...@qq.com> wrote:
>>
>>> Dear All,
>>>
>>> when one node failure with hardware errors,it will be in DN status in
>>> the cluster.Then if we are not able to handle this error in three hours(max
>>> hints window),we will loss data,right?we have to run repair to keep the
>>> consistency.
>>> And as per
>>> https://blog.alteroot.org/articles/2014-03-12/replace-a-dead-node-in-cassandra.html,we
>>> can replace this dead node,is it the same as bootstrap new node?that means
>>> we don't need to remove node and rejoin?
>>> Could anyone please advise?
>>>
>>> Thanks,
>>> Peng Xiao
>>>
>>>
>>>
>>>
>>>


Re: replace dead node vs remove node

2018-03-22 Thread Jonathan Haddad
Under normal circumstances this is not true.

Take a look at org.apache.cassandra.service.StorageProxy#performWrite, it
grabs both the natural endpoints and the pending endpoints (new nodes).
They're eventually passed through
to 
org.apache.cassandra.locator.AbstractReplicationStrategy#getWriteResponseHandler,
which keeps track of both the current endpoints and the pending ones.
Later, it gets to the actual work:

performer.apply(mutation, Iterables.concat(naturalEndpoints,
pendingEndpoints), responseHandler, localDataCenter,
consistency_level);

The signature of this method is:

public interface WritePerformer
{
public void apply(IMutation mutation,
  Iterable targets,
  AbstractWriteResponseHandler responseHandler,
  String localDataCenter,
  ConsistencyLevel consistencyLevel) throws
OverloadedException;
}

Notice the targets?  That's the list of all current owners and pending
owners.  The list is a concatenation of the natural endpoints and the
pending ones.

Pending owners are listed in org.apache.cassandra.locator.TokenMetadata

// this is a cache of the calculation from {tokenToEndpointMap,
bootstrapTokens, leavingEndpoints}
private final ConcurrentMap<String, PendingRangeMaps> pendingRanges =
new ConcurrentHashMap<String, PendingRangeMaps>();


TL;DR: mutations are sent to nodes being bootstrapped.

Jon


On Thu, Mar 22, 2018 at 8:09 PM Peng Xiao <2535...@qq.com> wrote:

> Hi Anthony,
>
> there is a problem with replacing dead node as per the blog,if the
> replacement process takes longer than max_hint_window_in_ms,we must run
> repair to make the replaced node consistent again, since it missed ongoing
> writes during bootstrapping.but for a great cluster,repair is a painful
> process.
>
> Thanks,
> Peng Xiao
>
>
>
> -- 原始邮件 --
> *发件人:* "Anthony Grasso"<anthony.gra...@gmail.com>;
> *发送时间:* 2018年3月22日(星期四) 晚上7:13
> *收件人:* "user"<user@cassandra.apache.org>;
> *主题:* Re: replace dead node vs remove node
>
> Hi Peng,
>
> Depending on the hardware failure you can do one of two things:
>
> 1. If the disks are intact and uncorrupted you could just use the disks
> with the current data on them in the new node. Even if the IP address
> changes for the new node that is fine. In that case all you need to do is
> run repair on the new node. The repair will fix any writes the node missed
> while it was down. This process is similar to the scenario in this blog
> post:
> http://thelastpickle.com/blog/2018/02/21/replace-node-without-bootstrapping.html
>
> 2. If the disks are inaccessible or corrupted, then use the method as
> described in the blogpost you linked to. The operation is similar to
> bootstrapping a new node. There is no need to perform any other remove or
> join operation on the failed or new nodes. As per the blog post, you
> definitely want to run repair on the new node as soon as it joins the
> cluster. In this case here, the data on the failed node is effectively lost
> and replaced with data from other nodes in the cluster.
>
> Hope this helps.
>
> Regards,
> Anthony
>
>
> On Thu, 22 Mar 2018 at 20:52, Peng Xiao <2535...@qq.com> wrote:
>
>> Dear All,
>>
>> when one node failure with hardware errors,it will be in DN status in the
>> cluster.Then if we are not able to handle this error in three hours(max
>> hints window),we will loss data,right?we have to run repair to keep the
>> consistency.
>> And as per
>> https://blog.alteroot.org/articles/2014-03-12/replace-a-dead-node-in-cassandra.html,we
>> can replace this dead node,is it the same as bootstrap new node?that means
>> we don't need to remove node and rejoin?
>> Could anyone please advise?
>>
>> Thanks,
>> Peng Xiao
>>
>>
>>
>>
>>


Re: 回复: replace dead node vs remove node

2018-03-22 Thread Jeff Jirsa
Subrange repair of only the neighbors is sufficient

Break the range covering the dead node into ~100 splits and repair those splits 
individually in sequence. You don’t have to repair the whole range all at once



-- 
Jeff Jirsa


> On Mar 22, 2018, at 8:08 PM, Peng Xiao <2535...@qq.com> wrote:
> 
> Hi Anthony,
> 
> there is a problem with replacing dead node as per the blog,if the 
> replacement process takes longer than max_hint_window_in_ms,we must run 
> repair to make the replaced node consistent again, since it missed ongoing 
> writes during bootstrapping.but for a great cluster,repair is a painful 
> process.
>  
> Thanks,
> Peng Xiao
> 
> 
> 
> -- 原始邮件 --
> 发件人: "Anthony Grasso"<anthony.gra...@gmail.com>;
> 发送时间: 2018年3月22日(星期四) 晚上7:13
> 收件人: "user"<user@cassandra.apache.org>;
> 主题: Re: replace dead node vs remove node
> 
> Hi Peng,
> 
> Depending on the hardware failure you can do one of two things:
> 
> 1. If the disks are intact and uncorrupted you could just use the disks with 
> the current data on them in the new node. Even if the IP address changes for 
> the new node that is fine. In that case all you need to do is run repair on 
> the new node. The repair will fix any writes the node missed while it was 
> down. This process is similar to the scenario in this blog post: 
> http://thelastpickle.com/blog/2018/02/21/replace-node-without-bootstrapping.html
> 
> 2. If the disks are inaccessible or corrupted, then use the method as 
> described in the blogpost you linked to. The operation is similar to 
> bootstrapping a new node. There is no need to perform any other remove or 
> join operation on the failed or new nodes. As per the blog post, you 
> definitely want to run repair on the new node as soon as it joins the 
> cluster. In this case here, the data on the failed node is effectively lost 
> and replaced with data from other nodes in the cluster.
> 
> Hope this helps.
> 
> Regards,
> Anthony
> 
> 
>> On Thu, 22 Mar 2018 at 20:52, Peng Xiao <2535...@qq.com> wrote:
>> Dear All,
>> 
>> when one node failure with hardware errors,it will be in DN status in the 
>> cluster.Then if we are not able to handle this error in three hours(max 
>> hints window),we will loss data,right?we have to run repair to keep the 
>> consistency.
>> And as per 
>> https://blog.alteroot.org/articles/2014-03-12/replace-a-dead-node-in-cassandra.html,we
>>  can replace this dead node,is it the same as bootstrap new node?that means 
>> we don't need to remove node and rejoin?
>> Could anyone please advise?
>> 
>> Thanks,
>> Peng Xiao
>> 
>>  
>> 
>> 


?????? replace dead node vs remove node

2018-03-22 Thread Peng Xiao
Hi Anthony,


there is a problem with replacing dead node as per the blog,if the replacement 
process takes longer than max_hint_window_in_ms,we must run repair to make the 
replaced node consistent again, since it missed ongoing writes during 
bootstrapping.but for a great cluster,repair is a painful process.
 
Thanks,
Peng Xiao






--  --
??: "Anthony Grasso"<anthony.gra...@gmail.com>;
: 2018??3??22??(??) 7:13
??: "user"<user@cassandra.apache.org>;

: Re: replace dead node vs remove node



Hi Peng,

Depending on the hardware failure you can do one of two things:



1. If the disks are intact and uncorrupted you could just use the disks with 
the current data on them in the new node. Even if the IP address changes for 
the new node that is fine. In that case all you need to do is run repair on the 
new node. The repair will fix any writes the node missed while it was down. 
This process is similar to the scenario in this blog post: 
http://thelastpickle.com/blog/2018/02/21/replace-node-without-bootstrapping.html


2. If the disks are inaccessible or corrupted, then use the method as described 
in the blogpost you linked to. The operation is similar to bootstrapping a new 
node. There is no need to perform any other remove or join operation on the 
failed or new nodes. As per the blog post, you definitely want to run repair on 
the new node as soon as it joins the cluster. In this case here, the data on 
the failed node is effectively lost and replaced with data from other nodes in 
the cluster.


Hope this helps.


Regards,
Anthony


On Thu, 22 Mar 2018 at 20:52, Peng Xiao <2535...@qq.com> wrote:

Dear All,


when one node failure with hardware errors,it will be in DN status in the 
cluster.Then if we are not able to handle this error in three hours(max hints 
window),we will loss data,right?we have to run repair to keep the consistency.
And as per 
https://blog.alteroot.org/articles/2014-03-12/replace-a-dead-node-in-cassandra.html,we
 can replace this dead node,is it the same as bootstrap new node?that means we 
don't need to remove node and rejoin?
Could anyone please advise?


Thanks,
Peng Xiao

Re: replace dead node vs remove node

2018-03-22 Thread Anthony Grasso
Hi Peng,

Depending on the hardware failure you can do one of two things:

1. If the disks are intact and uncorrupted you could just use the disks
with the current data on them in the new node. Even if the IP address
changes for the new node that is fine. In that case all you need to do is
run repair on the new node. The repair will fix any writes the node missed
while it was down. This process is similar to the scenario in this blog
post:
http://thelastpickle.com/blog/2018/02/21/replace-node-without-bootstrapping.html

2. If the disks are inaccessible or corrupted, then use the method as
described in the blogpost you linked to. The operation is similar to
bootstrapping a new node. There is no need to perform any other remove or
join operation on the failed or new nodes. As per the blog post, you
definitely want to run repair on the new node as soon as it joins the
cluster. In this case here, the data on the failed node is effectively lost
and replaced with data from other nodes in the cluster.

Hope this helps.

Regards,
Anthony


On Thu, 22 Mar 2018 at 20:52, Peng Xiao <2535...@qq.com> wrote:

> Dear All,
>
> when one node failure with hardware errors,it will be in DN status in the
> cluster.Then if we are not able to handle this error in three hours(max
> hints window),we will loss data,right?we have to run repair to keep the
> consistency.
> And as per
> https://blog.alteroot.org/articles/2014-03-12/replace-a-dead-node-in-cassandra.html,we
> can replace this dead node,is it the same as bootstrap new node?that means
> we don't need to remove node and rejoin?
> Could anyone please advise?
>
> Thanks,
> Peng Xiao
>
>
>
>
>


replace dead node vs remove node

2018-03-22 Thread Peng Xiao
Dear All,


when one node failure with hardware errors,it will be in DN status in the 
cluster.Then if we are not able to handle this error in three hours(max hints 
window),we will loss data,right?we have to run repair to keep the consistency.
And as per 
https://blog.alteroot.org/articles/2014-03-12/replace-a-dead-node-in-cassandra.html,we
 can replace this dead node,is it the same as bootstrap new node?that means we 
don't need to remove node and rejoin?
Could anyone please advise?


Thanks,
Peng Xiao