[cassandra] Add the new servers in the production cluster
Now the servers are re-dispatched in the racks (2 servers per racks), we can update the datacenter topology to make cassandra aware of the new rack organization
https://cassandra.apache.org/doc/latest/cassandra/configuration/cass_rackdc_file.html
This operation is heavy as it seems the nodes need to be removed from the cluster and re-added one by one, with all the data movement it implies.
https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/operations/opsMoveNodeRack.html
GossipingPropertyFileSnitch
before updating the topology:
https://cassandra.apache.org/doc/4.1/cassandra/getting_started/production.html#configure-racks-and-snitch
The current replication class is:
cqlsh> describe keyspace swh;
CREATE KEYSPACE swh WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '3'} AND durable_writes = true;
...
https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/operations/opsChangeKSStrategy.html
A possible plan can be:
- Update the replication strategy of the keyspace
- Run a full repair (this step is probably optional as the topology will not change: all the servers are currently configured in the same DC and rack)
-
Add the first the new nodes configured
in the right rack, repair, cleanupand wait for the bootstrap - Add the second node, repair, cleanup, ...
-
for each "old nodes":
decommission the nodeupdate the rack configurationadd the cluster in the node, repair, cleanup, ...- cleanup
The link to the documentation to remove / add a new node: https://cassandra.apache.org/doc/4.0/cassandra/operating/topo_changes.html
Designs
- Show closed items
Activity
-
Newest first Oldest first
-
Show all activity Show comments only Show history only
- Vincent Sellier added activity::Deployment cassandra labels
added activity::Deployment cassandra labels
- Vincent Sellier marked this issue as related to #4741 (closed)
marked this issue as related to #4741 (closed)
- Vincent Sellier removed the relation with #4741 (closed)
removed the relation with #4741 (closed)
- Vincent Sellier marked this issue as related to #4737 (closed)
marked this issue as related to #4737 (closed)
- Vincent Sellier changed the description
changed the description
- Author Owner
I've tested the replication strategy change for the (small) reaper database. It seems the full repair is not needed as there is no topology change.
ALTER KEYSPACE reaper_db WITH replication = {'class': 'NetworkTopologyStrategy', 'sesi_rocquencourt': 3 } AND durable_writes = true;
The full repair doesn't see anything to repair after that.
- Author Owner
Configuration applied for the swh keyspace:
% cqlsh $(hostname) Connected to archive_production at cassandra01:9042 [cqlsh 6.0.0 | Cassandra 4.0.8 | CQL spec 3.4.5 | Native protocol v5] Use HELP for help. cqlsh> ALTER KEYSPACE swh WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'sesi_rocquencourt' : 3};
In the cassandra logs:
INFO [MigrationStage:1] 2023-03-10 12:56:48,731 NetworkTopologyStrategy.java:89 - Configured datacenter replicas are sesi_rocquencourt:rf(3) INFO [MigrationStage:1] 2023-03-10 12:56:48,731 NetworkTopologyStrategy.java:89 - Configured datacenter replicas are sesi_rocquencourt:rf(3) INFO [MigrationStage:1] 2023-03-10 12:56:48,734 ColumnFamilyStore.java:878 - Enqueuing flush of keyspaces: 0.843KiB (0%) on-heap, 0.000KiB (0%) off-heap INFO [MigrationStage:1] 2023-03-10 12:56:48,742 Keyspace.java:386 - Creating replication strategy swh params KeyspaceParams{durable_writes=true, replication=ReplicationParams{class=org.apache.cassandra.locator.NetworkTopologyStrategy, sesi_rocquencourt=3}} INFO [MigrationStage:1] 2023-03-10 12:56:48,742 NetworkTopologyStrategy.java:89 - Configured datacenter replicas are sesi_rocquencourt:rf(3) INFO [CompactionExecutor:54] 2023-03-10 12:56:48,742 CompactionTask.java:150 - Compacting (05166060-bf43-11ed-b436-e12421c9995e) [/srv/cassandra/instance1/data/system_schema/keyspaces-abac5682dea631c5b535b3d6cffd0fb6/nb-141-big-Data.db:level=0, /srv/cassandra/instance1/data/system_schema/keyspaces-abac5682dea631c5b535b3d6cffd0fb6/nb-142-big-Data.db:level=0, /srv/cassandra/instance1/data/system_schema/keyspaces-abac5682dea631c5b535b3d6cffd0fb6/nb-144-big-Data.db:level=0, /srv/cassandra/instance1/data/system_schema/keyspaces-abac5682dea631c5b535b3d6cffd0fb6/nb-143-big-Data.db:level=0, ] INFO [Native-Transport-Requests-52] 2023-03-10 12:56:48,749 NetworkTopologyStrategy.java:89 - Configured datacenter replicas are sesi_rocquencourt:rf(3)
In the replayer logs, which indicate the configuration seems to be correct:
cassandra-replay/storage-replayer-content-6694bd6d4c-lptm9[storage-replayer]: INFO:cassandra.policies:Using datacenter 'sesi_rocquencourt' for DCAwareRoundRobinPolicy (via host '192.168.100.183:9042'); if incorrect, please specify a local_dc to the constructor, or limit contact points to local cluster nodes
- Vincent Sellier changed the description
changed the description
- Vincent Sellier marked the checklist item Update the replication strategy of the keyspace as completed
marked the checklist item Update the replication strategy of the keyspace as completed
- Vincent Sellier marked the checklist item Run a full repair (this step is probably optional as the topology will not change: all the servers are currently configured in the same DC and rack) as completed
marked the checklist item Run a full repair (this step is probably optional as the topology will not change: all the servers are currently configured in the same DC and rack) as completed
- Vincent Sellier mentioned in commit swh/infra/puppet/puppet-swh-site@32116116
mentioned in commit swh/infra/puppet/puppet-swh-site@32116116
- Author Owner
cassandra07 was configured and started.
The bootstrap process directly streams the content from the other nodes:# /opt/cassandra/bin/nodetool -h cassandra01 -u cassandra --password [redacted] netstats | grep -v 100% Mode: NORMAL Bootstrap 8aaf17a0-bf46-11ed-aae5-b9c59c4d40de /192.168.100.187 Sending 347 files, 3460250391446 bytes total. Already sent 15 files (4.32%), 4213389894 bytes total (0.12%) /srv/cassandra/instance1/data/swh/directory_entry-b74d85d02d2e11ed970a612d80206516/nb-494616-big-Data.db 3640852480/53378494141 bytes (6%) sent to idx:0/192.168.100.187 Read Repair Statistics: Attempted: 0 Mismatch (Blocking): 1150 Mismatch (Background): 0 Pool Name Active Pending Completed Dropped Large messages n/a 0 750 1 Small messages n/a 1 426833786 1 Gossip messages n/a 0 268376 1055
Edited by Vincent Sellier - Author Owner
The initial bootstrap of cassandra07 failed due to lack of space.
It can be explained by cassandra trying to allocate a repliace of each token per racks so the new server which is alone is the rack can handle the amount of data.I tried to bootstrap cassandra08 but it dont want to bootstrap either with this error:
ERROR [main] 2023-03-12 19:29:46,113 CassandraDaemon.java:911 - Fatal configuration error org.apache.cassandra.exceptions.ConfigurationException: Token allocation failed: the number of racks 2 in datacenter sesi_rocquencourt is lower than its replication factor 3.
It due to the fact the number of racks need to be >= of the RF.
Other recommandation regarding the racks affinity make me think it's not something easy to manage and will add some constraints we won't be able to address:
- Number of racks must be >= of RF and preferably a factor of the RF (3/6/9/...)
- Each rack must have a equal number of nodes to have a balanced token allocation. If we follow this, we'll need to add 4 nodes each time we want to increase the cluster capacity
So finally, the new nodes will be added in the rack1 just as the old ones to simplify the deployment.
- Vincent Sellier changed title from [cassandra] Configure the new rack topology to [cassandra] Add the new servers in the cluster
changed title from [cassandra] Configure the new rack topology to [cassandra] Add the new servers in the cluster
- Vincent Sellier changed the description
changed the description
- Vincent Sellier mentioned in commit swh/infra/puppet/puppet-swh-site@41cc250e
mentioned in commit swh/infra/puppet/puppet-swh-site@41cc250e
- Author Owner
cassandra08 is up and bootstrapped:
root@cassandra08:~# /opt/cassandra/bin/nodetool -h cassandra08 -u cassandra --password [redacted] status Datacenter: sesi_rocquencourt ============================= Status=Up/Down |/ State=Normal/Leaving/Joining/Moving -- Address Load Tokens Owns (effective) Host ID Rack UN 192.168.100.184 10.24 TiB 16 43.2% 9c618479-7898-4d89-a8e0-dc1a23fce04e rack1 UN 192.168.100.181 9.94 TiB 16 42.5% cb0695ee-b7f1-4b31-ba5e-9ed7a068d993 rack1 UN 192.168.100.186 10.18 TiB 16 42.5% 557341c9-dc0c-4a37-99b3-bc71fb46b29c rack1 UN 192.168.100.188 8.64 TiB 16 43.2% 247cd9e3-a70c-465c-bca1-ea9d3af9609a rack1 UN 192.168.100.183 10.31 TiB 16 43.0% 4cc44367-67dc-41ea-accf-4ef8335eabad rack1 UN 192.168.100.185 10.21 TiB 16 42.9% ac5e4446-9b26-43e4-8203-b05cb34f2c35 rack1 UN 192.168.100.182 9.85 TiB 16 42.8% a3c89490-ee69-449a-acb1-c2aa6b3d6c71 rack1
Let's add cassandra07 now
Edited by Vincent Sellier - Vincent Sellier mentioned in issue #4737 (closed)
mentioned in issue #4737 (closed)
- Vincent Sellier assigned to @vsellier
assigned to @vsellier
- Author Owner
cassandra07 is up and running since yesterday (I forgot to validate the comment)
The cleanup of the other nodes (10->06, 08) is in progress.
It's running correctly for most of the nodes until the cleanup of the directory_entry.
There are 2 cleanup threads in parallel (the default is half of the compaction threads)Several counter-measures are in progress:
- Stop the compactions to reserve the space for the cleanup
root@pergamon:~# clush -b -w @cassandra "/opt/cassandra/bin/nodetool -u cassandra --password [redacted] disableautocompaction"
- launch a first cleanup on all the tables except
directory_entry
to retrieve the maximum of space before cleaning updirectory_entry
% /opt/cassandra/bin/nodetool -h cassandra03 -u cassandra --password [redacted] cleanup swh metadata_authority metadata_fetcher skipped_content_by_sha1 skipped_content_by_sha1_git skipped_content_by_blake2s256 skipped_content_by_sha256 skipped_content raw_extrinsic_metadata_by_id release extid_by_target extid snapshot origin origin_visit origin_visit_status raw_extrinsic_metadata revision_parent snapshot_branch content_by_sha1 content_by_sha1_git directory content_by_sha256 content_by_blake2s256 revision content
followed by
% /opt/cassandra/bin/nodetool -h cassandra03 -u cassandra --password [redacted] cleanup directory_entry
- At a last resorts, if 2 cleanups on very big sstables are in progress, it's still possible to kill one of then (it will be caught later)
% /opt/cassandra/bin/nodetool -h cassandra01 -u cassandra --password [redacted] stop -id 74ac6cb0-c251-11ed-b2bf-cb51bbd5c275 CLEANUP
- Antoine R. Dumont changed title from [cassandra] Add the new servers in the cluster to [cassandra] Add the new servers in the production cluster
changed title from [cassandra] Add the new servers in the cluster to [cassandra] Add the new servers in the production cluster
- Author Owner
current status:
-
The cleanup is still running only for cassandra03. All theothersnodes are cleaned
pergamon ~ # clush -b -w @cassandra df -h /srv/cassandra/instance1/data --------------- cassandra[01,07] (2) --------------- Filesystem Size Used Avail Use% Mounted on mixeduse/cassandra-instance1-data 12T 8.3T 3.0T 74% /srv/cassandra/instance1/data --------------- cassandra[02,05] (2) --------------- Filesystem Size Used Avail Use% Mounted on mixeduse/cassandra-instance1-data 12T 7.6T 3.8T 67% /srv/cassandra/instance1/data --------------- cassandra03 --------------- Filesystem Size Used Avail Use% Mounted on mixeduse/cassandra-instance1-data 12T 9.2T 2.2T 81% /srv/cassandra/instance1/data --------------- cassandra04 --------------- Filesystem Size Used Avail Use% Mounted on mixeduse/cassandra-instance1-data 12T 8.3T 3.1T 74% /srv/cassandra/instance1/data --------------- cassandra06 --------------- Filesystem Size Used Avail Use% Mounted on mixeduse/cassandra-instance1-data 12T 7.6T 3.7T 68% /srv/cassandra/instance1/data --------------- cassandra08 --------------- Filesystem Size Used Avail Use% Mounted on mixeduse/cassandra-instance1-data 12T 8.4T 3.0T 75% /srv/cassandra/instance1/data
The free space is now between 3.8To and 3To.
A full repair is in progress, there are a lot of ranges to fix after the rack affinity tests and the addition and (unclean) removal of cassandra07.
... INFO [RepairJobTask:25] 2023-03-20 08:05:17,886 RepairJob.java:160 - [repair #b8cf06e0-c667-11ed-88c8-d382e1d422cb] swh.skipped_content_by_sha1 is fully synced INFO [RepairJobTask:25] 2023-03-20 08:05:40,082 RepairJob.java:160 - [repair #b8cf06e0-c667-11ed-88c8-d382e1d422cb] swh.origin is fully synced INFO [RepairJobTask:28] 2023-03-20 08:07:58,776 RepairJob.java:160 - [repair #b8cf06e0-c667-11ed-88c8-d382e1d422cb] swh.revision_parent is fully synced INFO [RepairJobTask:31] 2023-03-20 08:32:26,685 LocalSyncTask.java:117 - [repair #b8cf06e0-c667-11ed-88c8-d382e1d422cb] Performing streaming repair of 400770 ranges with /192.168.100.188:7000 INFO [RepairJobTask:30] 2023-03-20 08:32:26,685 SymmetricRemoteSyncTask.java:58 - [repair #b8cf06e0-c667-11ed-88c8-d382e1d422cb] Forwarding streaming repair of 401059 ranges to /192.168.100.188:7000 (to be streamed with /192.168.100.183:7000) INFO [RepairJobTask:32] 2023-03-20 08:32:26,685 LocalSyncTask.java:117 - [repair #b8cf06e0-c667-11ed-88c8-d382e1d422cb] Performing streaming repair of 4294 ranges with /192.168.100.183:7000 ...
The progression of the repair can be seen in reaper.
It will progressively restart the replayer to see if it's usable as it or if we need to wait for the end of the repair.
Edited by Vincent Sellier -
- Vincent Sellier marked the checklist item Add the first the new nodes configured
in the right rack, repair, cleanupand wait for the bootstrap as completedmarked the checklist item Add the first the new nodes configured
in the right rack, repair, cleanupand wait for the bootstrap as completed - Vincent Sellier marked the checklist item Add the second node, repair, cleanup, ... as completed
marked the checklist item Add the second node, repair, cleanup, ... as completed
- Vincent Sellier marked the checklist item for each "old nodes": as completed
marked the checklist item for each "old nodes": as completed
- Vincent Sellier mentioned in commit swh/infra/ci-cd/swh-charts@f33ad3c5
mentioned in commit swh/infra/ci-cd/swh-charts@f33ad3c5
- Vincent Sellier mentioned in commit swh/infra/ci-cd/k8s-clusters-conf@d663b4db
mentioned in commit swh/infra/ci-cd/k8s-clusters-conf@d663b4db
- Author Owner
All the nodes are in the cluster, were cleaned and a full repair run, so this task
- Vincent Sellier closed
closed