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 this case nodes