Re: [EXTERNAL] Apache Cassandra upgrade path

2019-07-29 Thread Jai Bheemsen Rao Dhanwada
Thank you Romain

On Sat, Jul 27, 2019 at 1:42 AM Romain Hardouin 
wrote:

> Hi,
>
> Here are some upgrade options:
>   - Standard rolling upgrade: node by node
>
>   - Fast rolling upgrade: rack by rack.
> If clients use CL=LOCAL_ONE then it's OK as long as one rack is UP.
> For higher CL it's possible assuming you have no more than one replica per
> rack e.g. CL=LOCAL_QUORUM with RF=3 and 2 racks is a *BAD* setup. But RF=3
> with 3 rack is OK.
>   - Double write in another cluster: easy for short TTL data (e.g. TTL of
> few days)
> When possible, this option is not only the safest but also allows major
> change (e.g. Partitioner for legacy clusters).
> And of course it's a good opportunity to use new cloud instance type,
> change number of vnodes, etc.
>
> As Sean said, it's not possible for C* servers to stream data with other
> versions when Streaming versions are different. There is no workaround.
> You can check that here
> https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java#L35
> The community plans to work on this limitation to make streaming possible
> between different major versions starting from C*4.x
>
> Last but not least, don't forget to take snapshots (+ backup) and to
> prepare a rollback script.
> System keyspace will be automatically snapshotted by Cassandra when the
> new version will start: the rollback script should be based on that
> snapshot for the system part.
> New data (both commitlog and sstables flushed in 3.11 format) will be lost
> even with such a script but it's useful to test it and to have it ready for
> the D day.
> (See also snapshot_before_compaction setting but it might be useless
> depending on your procedure.)
>
> Romain
>
>
>
> Le vendredi 26 juillet 2019 à 23:51:52 UTC+2, Jai Bheemsen Rao Dhanwada <
> jaibheem...@gmail.com> a écrit :
>
>
> yes correct, it doesn't work for the servers. trying to see if any had any
> workaround for this issue? (may be changing the protocol version during the
> upgrade time?)
>
> On Fri, Jul 26, 2019 at 1:11 PM Durity, Sean R <
> sean_r_dur...@homedepot.com> wrote:
>
> This would handle client protocol, but not streaming protocol between
> nodes.
>
>
>
>
>
> Sean Durity – Staff Systems Engineer, Cassandra
>
>
>
> *From:* Alok Dwivedi 
> *Sent:* Friday, July 26, 2019 3:21 PM
> *To:* user@cassandra.apache.org
> *Subject:* Re: [EXTERNAL] Apache Cassandra upgrade path
>
>
>
> Hi Sean
>
> The recommended practice for upgrade is to explicitly control protocol
> version in your application during upgrade process. Basically the protocol
> version is negotiated on first connection and based on chance it can talk
> to an already upgraded node first which means it will negotiate a higher
> version that will not be compatible with those nodes which are still one
> lower Cassandra version. So initially you set it a lower version that is
> like lower common denominator for mixed mode cluster and then remove the
> call to explicit setting once upgrade has completed.
>
>
>
> Cluster cluster = Cluster.builder()
>
> .addContactPoint("127.0.0.1")
>
> .withProtocolVersion(ProtocolVersion.V2)
>
> .build();
>
>
>
> Refer here for more information if using Java driver
>
>
> https://docs.datastax.com/en/developer/java-driver/3.7/manual/native_protocol/#protocol-version-with-mixed-clusters
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__docs.datastax.com_en_developer_java-2Ddriver_3.7_manual_native-5Fprotocol_-23protocol-2Dversion-2Dwith-2Dmixed-2Dclusters=DwMFaQ=MtgQEAMQGqekjTjiAhkudQ=aC_gxC6z_4f9GLlbWiKzHm1vucZTtVYWDDvyLkh8IaQ=JUUAJpaOGj5fhLX2uWOwUVqUcHN3c24hEaDC1T8RZVQ=WLqlcmEjAYjj7TAAmvYA3NyPqe7ZqgFTNuRNZXryUQE=>
>
>
>
> Same thing applies to drivers in other languages.
>
>
>
> Thanks
>
> Alok Dwivedi
>
> Senior Consultant
>
> https://www.instaclustr.com/
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__www.instaclustr.com_=DwMFaQ=MtgQEAMQGqekjTjiAhkudQ=aC_gxC6z_4f9GLlbWiKzHm1vucZTtVYWDDvyLkh8IaQ=JUUAJpaOGj5fhLX2uWOwUVqUcHN3c24hEaDC1T8RZVQ=gQuE9u1lRiSA9uZsshvcKIuYih5Rvz3v6lhUOLZzvw4=>
>
>
>
>
>
> On Fri, 26 Jul 2019 at 20:03, Jai Bheemsen Rao Dhanwada <
> jaibheem...@gmail.com> wrote:
>
> Thanks Sean,
>
>
>
> In my use case all my clusters are multi DC, and I am trying my best
> effort to upgrade ASAP, however there is a chance since all machines are
> VMs. Also my key spaces are not uniform across DCs. some are replicated to
> all DCs and some of them are just one DC, so I am worried there.
>
>
>
> Is there a way

Re: [EXTERNAL] Apache Cassandra upgrade path

2019-07-27 Thread Romain Hardouin
 Hi,
Here are some upgrade options:  - Standard rolling upgrade: node by node    - 
Fast rolling upgrade: rack by rack.  If clients use CL=LOCAL_ONE then it's OK 
as long as one rack is UP. For higher CL it's possible assuming you have no 
more than one replica per rack e.g. CL=LOCAL_QUORUM with RF=3 and 2 racks is a 
*BAD* setup. But RF=3 with 3 rack is OK.   - Double write in another cluster: 
easy for short TTL data (e.g. TTL of few days) When possible, this option is 
not only the safest but also allows major change (e.g. Partitioner for legacy 
clusters). And of course it's a good opportunity to use new cloud instance 
type, change number of vnodes, etc.
As Sean said, it's not possible for C* servers to stream data with other 
versions when Streaming versions are different. There is no workaround.You can 
check that here 
https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java#L35The
 community plans to work on this limitation to make streaming possible between 
different major versions starting from C*4.x
Last but not least, don't forget to take snapshots (+ backup) and to prepare a 
rollback script.System keyspace will be automatically snapshotted by Cassandra 
when the new version will start: the rollback script should be based on that 
snapshot for the system part.New data (both commitlog and sstables flushed in 
3.11 format) will be lost even with such a script but it's useful to test it 
and to have it ready for the D day.(See also snapshot_before_compaction setting 
but it might be useless depending on your procedure.)
Romain


Le vendredi 26 juillet 2019 à 23:51:52 UTC+2, Jai Bheemsen Rao Dhanwada 
 a écrit :  
 
 yes correct, it doesn't work for the servers. trying to see if any had any 
workaround for this issue? (may be changing the protocol version during the 
upgrade time?)

On Fri, Jul 26, 2019 at 1:11 PM Durity, Sean R  
wrote:


This would handle client protocol, but not streaming protocol between nodes.

 

 

Sean Durity – Staff Systems Engineer, Cassandra

 

From: Alok Dwivedi  
Sent: Friday, July 26, 2019 3:21 PM
To: user@cassandra.apache.org
Subject: Re: [EXTERNAL] Apache Cassandra upgrade path

 

Hi Sean

The recommended practice for upgrade is to explicitly control protocol version 
in your application during upgrade process. Basically the protocol version is 
negotiated on first connection and based on chance it can talk to an already 
upgraded node first which means it will negotiate a higher version that will 
not be compatible with those nodes which are still one lower Cassandra version. 
So initially you set it a lower version that is like lower common denominator 
for mixed mode cluster and then remove the call to explicit setting once 
upgrade has completed. 

 

Clustercluster= Cluster.builder()

   .addContactPoint("127.0.0.1")

   .withProtocolVersion(ProtocolVersion.V2)

   .build();

 

Refer here for more information if using Java driver

https://docs.datastax.com/en/developer/java-driver/3.7/manual/native_protocol/#protocol-version-with-mixed-clusters

 

Same thing applies to drivers in other languages. 

 

Thanks

Alok Dwivedi

Senior Consultant 

https://www.instaclustr.com/

 

 

On Fri, 26 Jul 2019 at 20:03, Jai Bheemsen Rao Dhanwada  
wrote:


Thanks Sean,

 

In my use case all my clusters are multi DC, and I am trying my best effort to 
upgrade ASAP, however there is a chance since all machines are VMs. Also my key 
spaces are not uniform across DCs. some are replicated to all DCs and some of 
them are just one DC, so I am worried there.

 

Is there a way to override the protocol version until the upgrade is done and 
then change it back once the upgrade is completed?

 

On Fri, Jul 26, 2019 at 11:42 AM Durity, Sean R  
wrote:


What you have seen is totally expected. You can’t stream between different 
major versions of Cassandra. Get the upgrade done, then worry about any down 
hardware. If you are using DCs, upgrade one DC at a time, so that there is an 
available environment in case of any disasters.

 

My advice, though, is to get through the rolling upgrade process as quickly as 
possible. Don’t stay in a mixed state very long. The cluster will function fine 
in a mixed state – except for those streaming operations. No repairs, no 
bootstraps. 

 

 

Sean Durity – Staff Systems Engineer, Cassandra

 

From: Jai Bheemsen Rao Dhanwada 
Sent: Friday, July 26, 2019 2:24 PM
To: user@cassandra.apache.org
Subject: [EXTERNAL] Apache Cassandra upgrade path

 

Hello,

 

I am trying to upgrade Apache Cassandra from 2.1.16 to 3.11.3, the regular 
rolling upgrade process works fine without any issues.

 

However, I am running into an issue where if there is a node with older version 
dies (hardware failure) and a new node comes up and tries to bootstrap, it's 
failing.

 

I tried two combinations:

 

1. Joining replacement node with 2.1.16 version of cassandra 

In

Re: [EXTERNAL] Apache Cassandra upgrade path

2019-07-26 Thread Jai Bheemsen Rao Dhanwada
yes correct, it doesn't work for the servers. trying to see if any had any
workaround for this issue? (may be changing the protocol version during the
upgrade time?)

On Fri, Jul 26, 2019 at 1:11 PM Durity, Sean R 
wrote:

> This would handle client protocol, but not streaming protocol between
> nodes.
>
>
>
>
>
> Sean Durity – Staff Systems Engineer, Cassandra
>
>
>
> *From:* Alok Dwivedi 
> *Sent:* Friday, July 26, 2019 3:21 PM
> *To:* user@cassandra.apache.org
> *Subject:* Re: [EXTERNAL] Apache Cassandra upgrade path
>
>
>
> Hi Sean
>
> The recommended practice for upgrade is to explicitly control protocol
> version in your application during upgrade process. Basically the protocol
> version is negotiated on first connection and based on chance it can talk
> to an already upgraded node first which means it will negotiate a higher
> version that will not be compatible with those nodes which are still one
> lower Cassandra version. So initially you set it a lower version that is
> like lower common denominator for mixed mode cluster and then remove the
> call to explicit setting once upgrade has completed.
>
>
>
> Cluster cluster = Cluster.builder()
>
> .addContactPoint("127.0.0.1")
>
> .withProtocolVersion(ProtocolVersion.V2)
>
> .build();
>
>
>
> Refer here for more information if using Java driver
>
>
> https://docs.datastax.com/en/developer/java-driver/3.7/manual/native_protocol/#protocol-version-with-mixed-clusters
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__docs.datastax.com_en_developer_java-2Ddriver_3.7_manual_native-5Fprotocol_-23protocol-2Dversion-2Dwith-2Dmixed-2Dclusters=DwMFaQ=MtgQEAMQGqekjTjiAhkudQ=aC_gxC6z_4f9GLlbWiKzHm1vucZTtVYWDDvyLkh8IaQ=JUUAJpaOGj5fhLX2uWOwUVqUcHN3c24hEaDC1T8RZVQ=WLqlcmEjAYjj7TAAmvYA3NyPqe7ZqgFTNuRNZXryUQE=>
>
>
>
> Same thing applies to drivers in other languages.
>
>
>
> Thanks
>
> Alok Dwivedi
>
> Senior Consultant
>
> https://www.instaclustr.com/
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__www.instaclustr.com_=DwMFaQ=MtgQEAMQGqekjTjiAhkudQ=aC_gxC6z_4f9GLlbWiKzHm1vucZTtVYWDDvyLkh8IaQ=JUUAJpaOGj5fhLX2uWOwUVqUcHN3c24hEaDC1T8RZVQ=gQuE9u1lRiSA9uZsshvcKIuYih5Rvz3v6lhUOLZzvw4=>
>
>
>
>
>
> On Fri, 26 Jul 2019 at 20:03, Jai Bheemsen Rao Dhanwada <
> jaibheem...@gmail.com> wrote:
>
> Thanks Sean,
>
>
>
> In my use case all my clusters are multi DC, and I am trying my best
> effort to upgrade ASAP, however there is a chance since all machines are
> VMs. Also my key spaces are not uniform across DCs. some are replicated to
> all DCs and some of them are just one DC, so I am worried there.
>
>
>
> Is there a way to override the protocol version until the upgrade is done
> and then change it back once the upgrade is completed?
>
>
>
> On Fri, Jul 26, 2019 at 11:42 AM Durity, Sean R <
> sean_r_dur...@homedepot.com> wrote:
>
> What you have seen is totally expected. You can’t stream between different
> major versions of Cassandra. Get the upgrade done, then worry about any
> down hardware. If you are using DCs, upgrade one DC at a time, so that
> there is an available environment in case of any disasters.
>
>
>
> My advice, though, is to get through the rolling upgrade process as
> quickly as possible. Don’t stay in a mixed state very long. The cluster
> will function fine in a mixed state – except for those streaming
> operations. No repairs, no bootstraps.
>
>
>
>
>
> Sean Durity – Staff Systems Engineer, Cassandra
>
>
>
> *From:* Jai Bheemsen Rao Dhanwada 
> *Sent:* Friday, July 26, 2019 2:24 PM
> *To:* user@cassandra.apache.org
> *Subject:* [EXTERNAL] Apache Cassandra upgrade path
>
>
>
> Hello,
>
>
>
> I am trying to upgrade Apache Cassandra from 2.1.16 to 3.11.3, the regular
> rolling upgrade process works fine without any issues.
>
>
>
> However, I am running into an issue where if there is a node with older
> version dies (hardware failure) and a new node comes up and tries to
> bootstrap, it's failing.
>
>
>
> I tried two combinations:
>
>
>
> 1. Joining replacement node with 2.1.16 version of cassandra
>
> In this case nodes with 2.1.16 version are able to stream data to the new
> node, but the nodes with 3.11.3 version are failing with the below error.
>
>
>
> ERROR [STREAM-INIT-/10.x.x.x:40296] 2019-07-26 17:45:17,775
> IncomingStreamingConnection.java:80 - Error while reading from socket from
> /10.y.y.y:40296.
> java.io.IOException: Received stream using protocol version 2 (my version
> 4). Terminating connection
>
> 2. Join

RE: [EXTERNAL] Apache Cassandra upgrade path

2019-07-26 Thread Durity, Sean R
This would handle client protocol, but not streaming protocol between nodes.


Sean Durity – Staff Systems Engineer, Cassandra

From: Alok Dwivedi 
Sent: Friday, July 26, 2019 3:21 PM
To: user@cassandra.apache.org
Subject: Re: [EXTERNAL] Apache Cassandra upgrade path

Hi Sean
The recommended practice for upgrade is to explicitly control protocol version 
in your application during upgrade process. Basically the protocol version is 
negotiated on first connection and based on chance it can talk to an already 
upgraded node first which means it will negotiate a higher version that will 
not be compatible with those nodes which are still one lower Cassandra version. 
So initially you set it a lower version that is like lower common denominator 
for mixed mode cluster and then remove the call to explicit setting once 
upgrade has completed.

Cluster cluster = Cluster.builder()
.addContactPoint("127.0.0.1")
.withProtocolVersion(ProtocolVersion.V2)
.build();

Refer here for more information if using Java driver
https://docs.datastax.com/en/developer/java-driver/3.7/manual/native_protocol/#protocol-version-with-mixed-clusters<https://urldefense.proofpoint.com/v2/url?u=https-3A__docs.datastax.com_en_developer_java-2Ddriver_3.7_manual_native-5Fprotocol_-23protocol-2Dversion-2Dwith-2Dmixed-2Dclusters=DwMFaQ=MtgQEAMQGqekjTjiAhkudQ=aC_gxC6z_4f9GLlbWiKzHm1vucZTtVYWDDvyLkh8IaQ=JUUAJpaOGj5fhLX2uWOwUVqUcHN3c24hEaDC1T8RZVQ=WLqlcmEjAYjj7TAAmvYA3NyPqe7ZqgFTNuRNZXryUQE=>

Same thing applies to drivers in other languages.

Thanks
Alok Dwivedi
Senior Consultant
https://www.instaclustr.com/<https://urldefense.proofpoint.com/v2/url?u=https-3A__www.instaclustr.com_=DwMFaQ=MtgQEAMQGqekjTjiAhkudQ=aC_gxC6z_4f9GLlbWiKzHm1vucZTtVYWDDvyLkh8IaQ=JUUAJpaOGj5fhLX2uWOwUVqUcHN3c24hEaDC1T8RZVQ=gQuE9u1lRiSA9uZsshvcKIuYih5Rvz3v6lhUOLZzvw4=>


On Fri, 26 Jul 2019 at 20:03, Jai Bheemsen Rao Dhanwada 
mailto:jaibheem...@gmail.com>> wrote:
Thanks Sean,

In my use case all my clusters are multi DC, and I am trying my best effort to 
upgrade ASAP, however there is a chance since all machines are VMs. Also my key 
spaces are not uniform across DCs. some are replicated to all DCs and some of 
them are just one DC, so I am worried there.

Is there a way to override the protocol version until the upgrade is done and 
then change it back once the upgrade is completed?

On Fri, Jul 26, 2019 at 11:42 AM Durity, Sean R 
mailto:sean_r_dur...@homedepot.com>> wrote:
What you have seen is totally expected. You can’t stream between different 
major versions of Cassandra. Get the upgrade done, then worry about any down 
hardware. If you are using DCs, upgrade one DC at a time, so that there is an 
available environment in case of any disasters.

My advice, though, is to get through the rolling upgrade process as quickly as 
possible. Don’t stay in a mixed state very long. The cluster will function fine 
in a mixed state – except for those streaming operations. No repairs, no 
bootstraps.


Sean Durity – Staff Systems Engineer, Cassandra

From: Jai Bheemsen Rao Dhanwada 
mailto:jaibheem...@gmail.com>>
Sent: Friday, July 26, 2019 2:24 PM
To: user@cassandra.apache.org<mailto:user@cassandra.apache.org>
Subject: [EXTERNAL] Apache Cassandra upgrade path

Hello,

I am trying to upgrade Apache Cassandra from 2.1.16 to 3.11.3, the regular 
rolling upgrade process works fine without any issues.

However, I am running into an issue where if there is a node with older version 
dies (hardware failure) and a new node comes up and tries to bootstrap, it's 
failing.

I tried two combinations:

1. Joining replacement node with 2.1.16 version of cassandra
In this case nodes with 2.1.16 version are able to stream data to the new node, 
but the nodes with 3.11.3 version are failing with the below error.

ERROR [STREAM-INIT-/10.x.x.x:40296] 2019-07-26 17:45:17,775 
IncomingStreamingConnection.java:80 - Error while reading from socket from 
/10.y.y.y:40296.
java.io.IOException: Received stream using protocol version 2 (my version 4). 
Terminating connection
2. Joining replacement node with 3.11.3 version of cassandra
In this case the nodes with 3.11.3 version of cassandra are able to stream the 
data but it's not able to stream data from the 2.1.16 nodes and failing with 
the below error.

ERROR [STREAM-IN-/10.z.z.z:7000] 2019-07-26 18:08:10,380 StreamSession.java:593 
- [Stream #538c6900-afd0-11e9-a649-ab2e045ee53b] Streaming error occurred on 
session with peer 10.z.z.z
java.io.IOException: Connection reset by peer
   at sun.nio.ch.FileDispatcherImpl.read0(Native Method) ~[na:1.8.0_151]
   at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39) 
~[na:1.8.0_151]
   at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223) 
~[na:1.8.0_151]
   at sun.nio.ch.IOUtil.read(IOUtil.java:197) ~[na:1.8.0_151]
   at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380) 
~[na:1.8.

Re: [EXTERNAL] Apache Cassandra upgrade path

2019-07-26 Thread Alok Dwivedi
Hi Sean
The recommended practice for upgrade is to explicitly control protocol
version in your application during upgrade process. Basically the protocol
version is negotiated on first connection and based on chance it can talk
to an already upgraded node first which means it will negotiate a higher
version that will not be compatible with those nodes which are still one
lower Cassandra version. So initially you set it a lower version that is
like lower common denominator for mixed mode cluster and then remove the
call to explicit setting once upgrade has completed.

Cluster cluster = Cluster.builder() .addContactPoint("127.0.0.1") .
withProtocolVersion(ProtocolVersion.V2) .build();

Refer here for more information if using Java driver
https://docs.datastax.com/en/developer/java-driver/3.7/manual/native_protocol/#protocol-version-with-mixed-clusters

Same thing applies to drivers in other languages.

Thanks
Alok Dwivedi
Senior Consultant
https://www.instaclustr.com/


On Fri, 26 Jul 2019 at 20:03, Jai Bheemsen Rao Dhanwada <
jaibheem...@gmail.com> wrote:

> Thanks Sean,
>
> In my use case all my clusters are multi DC, and I am trying my best
> effort to upgrade ASAP, however there is a chance since all machines are
> VMs. Also my key spaces are not uniform across DCs. some are replicated to
> all DCs and some of them are just one DC, so I am worried there.
>
> Is there a way to override the protocol version until the upgrade is done
> and then change it back once the upgrade is completed?
>
> On Fri, Jul 26, 2019 at 11:42 AM Durity, Sean R <
> sean_r_dur...@homedepot.com> wrote:
>
>> What you have seen is totally expected. You can’t stream between
>> different major versions of Cassandra. Get the upgrade done, then worry
>> about any down hardware. If you are using DCs, upgrade one DC at a time, so
>> that there is an available environment in case of any disasters.
>>
>>
>>
>> My advice, though, is to get through the rolling upgrade process as
>> quickly as possible. Don’t stay in a mixed state very long. The cluster
>> will function fine in a mixed state – except for those streaming
>> operations. No repairs, no bootstraps.
>>
>>
>>
>>
>>
>> Sean Durity – Staff Systems Engineer, Cassandra
>>
>>
>>
>> *From:* Jai Bheemsen Rao Dhanwada 
>> *Sent:* Friday, July 26, 2019 2:24 PM
>> *To:* user@cassandra.apache.org
>> *Subject:* [EXTERNAL] Apache Cassandra upgrade path
>>
>>
>>
>> Hello,
>>
>>
>>
>> I am trying to upgrade Apache Cassandra from 2.1.16 to 3.11.3, the
>> regular rolling upgrade process works fine without any issues.
>>
>>
>>
>> However, I am running into an issue where if there is a node with older
>> version dies (hardware failure) and a new node comes up and tries to
>> bootstrap, it's failing.
>>
>>
>>
>> I tried two combinations:
>>
>>
>>
>> 1. Joining replacement node with 2.1.16 version of cassandra
>>
>> In this case nodes with 2.1.16 version are able to stream data to the new
>> node, but the nodes with 3.11.3 version are failing with the below error.
>>
>>
>>
>> ERROR [STREAM-INIT-/10.x.x.x:40296] 2019-07-26 17:45:17,775
>> IncomingStreamingConnection.java:80 - Error while reading from socket from
>> /10.y.y.y:40296.
>> java.io.IOException: Received stream using protocol version 2 (my version
>> 4). Terminating connection
>>
>> 2. Joining replacement node with 3.11.3 version of cassandra
>>
>> In this case the nodes with 3.11.3 version of cassandra are able to
>> stream the data but it's not able to stream data from the 2.1.16 nodes and
>> failing with the below error.
>>
>>
>>
>> ERROR [STREAM-IN-/10.z.z.z:7000] 2019-07-26 18:08:10,380
>> StreamSession.java:593 - [Stream #538c6900-afd0-11e9-a649-ab2e045ee53b]
>> Streaming error occurred on session with peer 10.z.z.z
>> java.io.IOException: Connection reset by peer
>>at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
>> ~[na:1.8.0_151]
>>at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
>> ~[na:1.8.0_151]
>>at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
>> ~[na:1.8.0_151]
>>at sun.nio.ch.IOUtil.read(IOUtil.java:197) ~[na:1.8.0_151]
>>at
>> sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
>> ~[na:1.8.0_151]
>>at
>> sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:206)
>> ~[na:1.8.0_151]
>>at
>> sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
>> ~[na:1.8.0_151]
>>at
>> java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:385)
>> ~[na:1.8.0_151]
>>at
>> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
>> ~[apache-cassandra-3.11.3.jar:3.11.3]
>>at
>> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:311)
>> ~[apache-cassandra-3.11.3.jar:3.11.3]
>>at java.lang.Thread.run(Thread.java:748) [na:1.8.0_151]
>>
>>
>>
>> Note: In both cases I 

Re: [EXTERNAL] Apache Cassandra upgrade path

2019-07-26 Thread Jai Bheemsen Rao Dhanwada
Thanks Sean,

In my use case all my clusters are multi DC, and I am trying my best effort
to upgrade ASAP, however there is a chance since all machines are VMs. Also
my key spaces are not uniform across DCs. some are replicated to all DCs
and some of them are just one DC, so I am worried there.

Is there a way to override the protocol version until the upgrade is done
and then change it back once the upgrade is completed?

On Fri, Jul 26, 2019 at 11:42 AM Durity, Sean R 
wrote:

> What you have seen is totally expected. You can’t stream between different
> major versions of Cassandra. Get the upgrade done, then worry about any
> down hardware. If you are using DCs, upgrade one DC at a time, so that
> there is an available environment in case of any disasters.
>
>
>
> My advice, though, is to get through the rolling upgrade process as
> quickly as possible. Don’t stay in a mixed state very long. The cluster
> will function fine in a mixed state – except for those streaming
> operations. No repairs, no bootstraps.
>
>
>
>
>
> Sean Durity – Staff Systems Engineer, Cassandra
>
>
>
> *From:* Jai Bheemsen Rao Dhanwada 
> *Sent:* Friday, July 26, 2019 2:24 PM
> *To:* user@cassandra.apache.org
> *Subject:* [EXTERNAL] Apache Cassandra upgrade path
>
>
>
> Hello,
>
>
>
> I am trying to upgrade Apache Cassandra from 2.1.16 to 3.11.3, the regular
> rolling upgrade process works fine without any issues.
>
>
>
> However, I am running into an issue where if there is a node with older
> version dies (hardware failure) and a new node comes up and tries to
> bootstrap, it's failing.
>
>
>
> I tried two combinations:
>
>
>
> 1. Joining replacement node with 2.1.16 version of cassandra
>
> In this case nodes with 2.1.16 version are able to stream data to the new
> node, but the nodes with 3.11.3 version are failing with the below error.
>
>
>
> ERROR [STREAM-INIT-/10.x.x.x:40296] 2019-07-26 17:45:17,775
> IncomingStreamingConnection.java:80 - Error while reading from socket from
> /10.y.y.y:40296.
> java.io.IOException: Received stream using protocol version 2 (my version
> 4). Terminating connection
>
> 2. Joining replacement node with 3.11.3 version of cassandra
>
> In this case the nodes with 3.11.3 version of cassandra are able to stream
> the data but it's not able to stream data from the 2.1.16 nodes and failing
> with the below error.
>
>
>
> ERROR [STREAM-IN-/10.z.z.z:7000] 2019-07-26 18:08:10,380
> StreamSession.java:593 - [Stream #538c6900-afd0-11e9-a649-ab2e045ee53b]
> Streaming error occurred on session with peer 10.z.z.z
> java.io.IOException: Connection reset by peer
>at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
> ~[na:1.8.0_151]
>at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
> ~[na:1.8.0_151]
>at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> ~[na:1.8.0_151]
>at sun.nio.ch.IOUtil.read(IOUtil.java:197) ~[na:1.8.0_151]
>at
> sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> ~[na:1.8.0_151]
>at
> sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:206)
> ~[na:1.8.0_151]
>at
> sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
> ~[na:1.8.0_151]
>at
> java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:385)
> ~[na:1.8.0_151]
>at
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
> ~[apache-cassandra-3.11.3.jar:3.11.3]
>at
> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:311)
> ~[apache-cassandra-3.11.3.jar:3.11.3]
>at java.lang.Thread.run(Thread.java:748) [na:1.8.0_151]
>
>
>
> Note: In both cases I am using replace_address to replace dead node, as I
> am running into some issues with "nodetool removenode" . I use ephemeral
> disk, so replacement node always comes up with empty data dir and bootstrap.
>
>
>
> Any other work around to mitigate this problem? I am worried about any
> nodes going down while we are in the process of upgrade, as it could take
> several hours to upgrade depending on the cluster size.
>
> --
>
> The information in this Internet Email is confidential and may be legally
> privileged. It is intended solely for the addressee. Access to this Email
> by anyone else is unauthorized. If you are not the intended recipient, any
> disclosure, copying, distribution or any action taken or omitted to be
> taken in reliance on it, is prohibited and may be unlawful. When addressed
> to our clients any opinions or advice contained in this Email are subject
> to the terms and conditions expressed in any applicable governing The Home
> Depot terms of business or client engagement letter. The Home Depot
> disclaims all responsibility and liability for the accuracy and content of
> this attachment and for any damages or losses arising from any
> inaccuracies, errors, 

RE: [EXTERNAL] Apache Cassandra upgrade path

2019-07-26 Thread Durity, Sean R
What you have seen is totally expected. You can’t stream between different 
major versions of Cassandra. Get the upgrade done, then worry about any down 
hardware. If you are using DCs, upgrade one DC at a time, so that there is an 
available environment in case of any disasters.

My advice, though, is to get through the rolling upgrade process as quickly as 
possible. Don’t stay in a mixed state very long. The cluster will function fine 
in a mixed state – except for those streaming operations. No repairs, no 
bootstraps.


Sean Durity – Staff Systems Engineer, Cassandra

From: Jai Bheemsen Rao Dhanwada 
Sent: Friday, July 26, 2019 2:24 PM
To: user@cassandra.apache.org
Subject: [EXTERNAL] Apache Cassandra upgrade path

Hello,

I am trying to upgrade Apache Cassandra from 2.1.16 to 3.11.3, the regular 
rolling upgrade process works fine without any issues.

However, I am running into an issue where if there is a node with older version 
dies (hardware failure) and a new node comes up and tries to bootstrap, it's 
failing.

I tried two combinations:

1. Joining replacement node with 2.1.16 version of cassandra
In this case nodes with 2.1.16 version are able to stream data to the new node, 
but the nodes with 3.11.3 version are failing with the below error.

ERROR [STREAM-INIT-/10.x.x.x:40296] 2019-07-26 17:45:17,775 
IncomingStreamingConnection.java:80 - Error while reading from socket from 
/10.y.y.y:40296.
java.io.IOException: Received stream using protocol version 2 (my version 4). 
Terminating connection
2. Joining replacement node with 3.11.3 version of cassandra
In this case the nodes with 3.11.3 version of cassandra are able to stream the 
data but it's not able to stream data from the 2.1.16 nodes and failing with 
the below error.

ERROR [STREAM-IN-/10.z.z.z:7000] 2019-07-26 18:08:10,380 StreamSession.java:593 
- [Stream #538c6900-afd0-11e9-a649-ab2e045ee53b] Streaming error occurred on 
session with peer 10.z.z.z
java.io.IOException: Connection reset by peer
   at sun.nio.ch.FileDispatcherImpl.read0(Native Method) ~[na:1.8.0_151]
   at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39) 
~[na:1.8.0_151]
   at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223) 
~[na:1.8.0_151]
   at sun.nio.ch.IOUtil.read(IOUtil.java:197) ~[na:1.8.0_151]
   at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380) 
~[na:1.8.0_151]
   at 
sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:206) 
~[na:1.8.0_151]
   at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103) 
~[na:1.8.0_151]
   at 
java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:385) 
~[na:1.8.0_151]
   at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
 ~[apache-cassandra-3.11.3.jar:3.11.3]
   at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:311)
 ~[apache-cassandra-3.11.3.jar:3.11.3]
   at java.lang.Thread.run(Thread.java:748) [na:1.8.0_151]

Note: In both cases I am using replace_address to replace dead node, as I am 
running into some issues with "nodetool removenode" . I use ephemeral disk, so 
replacement node always comes up with empty data dir and bootstrap.

Any other work around to mitigate this problem? I am worried about any nodes 
going down while we are in the process of upgrade, as it could take several 
hours to upgrade depending on the cluster size.



The information in this Internet Email is confidential and may be legally 
privileged. It is intended solely for the addressee. Access to this Email by 
anyone else is unauthorized. If you are not the intended recipient, any 
disclosure, copying, distribution or any action taken or omitted to be taken in 
reliance on it, is prohibited and may be unlawful. When addressed to our 
clients any opinions or advice contained in this Email are subject to the terms 
and conditions expressed in any applicable governing The Home Depot terms of 
business or client engagement letter. The Home Depot disclaims all 
responsibility and liability for the accuracy and content of this attachment 
and for any damages or losses arising from any inaccuracies, errors, viruses, 
e.g., worms, trojan horses, etc., or other items of a destructive nature, which 
may be contained in this attachment and shall not be liable for direct, 
indirect, consequential or special damages in connection with this e-mail 
message or its attachment.