CASSANDRA-18031: Add configurable snapshot_directory for cross-mount snapshot storage#4648
Open
sachinAtMS wants to merge 1 commit intoapache:cassandra-4.1from
Open
CASSANDRA-18031: Add configurable snapshot_directory for cross-mount snapshot storage#4648sachinAtMS wants to merge 1 commit intoapache:cassandra-4.1from
snapshot_directory for cross-mount snapshot storage#4648sachinAtMS wants to merge 1 commit intoapache:cassandra-4.1from
Conversation
c3842f8 to
0962da0
Compare
snapshot_directory for cross-mount snapshot storagesnapshot_directory for cross-mount snapshot storage
Author
|
This PR implements the feature requested in CASSANDRA-18031. |
0962da0 to
8e23a7d
Compare
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
JIRA: CASSANDRA-18031 — Make snapshot directory configurable
Motivation
When running Cassandra in managed infrastructure environments (e.g., Azure VMSS with Medusa backup tooling), a common deployment pattern uses a single RAID volume (e.g., 4TB) shared between Cassandra data files and backup snapshots. During the backup upload window, both the live SSTable data and the snapshot copies (which are hardlinks occupying logical space on the same filesystem) compete for the same disk. If total data exceeds 50% of the disk, the snapshot hardlinks plus live data can fill the volume entirely, causing Cassandra to halt writes or crash.
The root cause is that Cassandra's current snapshot mechanism always creates snapshots as hardlinks under
data_file_directories, making it impossible to redirect snapshot storage to a separate mount/volume without modifying application code.Real-World Scenario
Solution
This patch introduces a new optional
snapshot_directoryconfiguration parameter incassandra.yaml. When set:data_file_directories<snapshot_directory>/<keyspace>/<table-id>/snapshots/<tag>/nodetool snapshot,nodetool listsnapshots,nodetool clearsnapshotsnapshot_directoryis not set, behavior is identical to current Cassandra (hardlinks under data dirs)Trade-off: Copies vs Hardlinks
Using file copies instead of hardlinks means:
This trade-off is acceptable for the target use case where disk safety is more critical than snapshot speed.
Configuration
Add to
cassandra.yaml:Usage Examples
Taking a snapshot (unchanged CLI, new storage location)
Listing snapshots
$ nodetool listsnapshots Snapshot Details: Snapshot name Keyspace name Column family name True size Size on disk Creation time daily_backup_20260303 my_keyspace my_table 6.05 KiB 6.05 KiB 2026-03-03T10:00:00.000ZClearing a snapshot
Changes
Files Modified (8 files, ~300 lines added)
Config.javasnapshot_directoryfield with JavadocDatabaseDescriptor.javagetSnapshotDirectory()/hasSnapshotDirectory()accessor methodsDirectories.javagetSnapshotBaseDirectory(),getSnapshotSearchPaths()helpers; modifiedgetSnapshotDirectory(),listAllSnapshots(),listSnapshotDirsByTag(),snapshotExists(),trueSnapshotsSize(); new 4-argclearSnapshot()overloadColumnFamilyStore.javasnapshotWithoutMemtable()to useFiles.copy()when configured; newcopySSTableToDirectory()helper;clearSnapshot()passescheckSnapshotDirectory=trueSSTableReader.javacopyToDirectory()method (alternative tocreateLinks())SnapshotLoader.javaaddSnapshotDirectory()helper to includesnapshot_directoryin snapshot discovery pathsSnapshotManager.javaSnapshotLoader()constructor (which now includes snapshot_directory)Keyspace.javaclearSnapshot()passescheckSnapshotDirectory=trueto ensure snapshot files on separate mount are cleaned upDesign Principles
DatabaseDescriptor.hasSnapshotDirectory(); when not configured, every code path falls through to the original logic<keyspace>/<table-id>/snapshots/<tag>/hierarchy from data directoriesTest Evidence
Test Environment
cassandra-4.1)Test 1: Startup —
snapshot_directoryrecognizedPASS: Configuration is validated and logged at startup.
Test 2: Snapshot creation — files stored on separate mount
PASS: Snapshot files created in
snapshot_directory, data directory is clean.Test 3: Files are copies, not hardlinks
PASS: Different inodes confirm these are independent file copies, not hardlinks.
Test 4:
nodetool listsnapshotsdiscovers snapshots on separate mount$ nodetool listsnapshots Snapshot Details: Snapshot name Keyspace name Column family name True size Size on disk Creation time final_test test_ks test_table 6.05 KiB 6.05 KiB 2026-03-02T17:06:08.221ZPASS: Snapshots in
snapshot_directoryare correctly discovered and listed.Test 5:
nodetool clearsnapshotremoves files from separate mountPASS: Snapshot files deleted from separate mount; other snapshots unaffected.
Test 6:
nodetool listsnapshotsreflects cleared snapshotPASS: Cleared snapshot no longer appears in listing.
Test 7: Backward compatibility (no
snapshot_directoryset)When
snapshot_directoryis not configured incassandra.yaml:DatabaseDescriptor.hasSnapshotDirectory()returnsfalsedata_file_directoriesPASS: Feature is fully opt-in with zero impact on existing deployments.
Build Verification
Compatibility
snapshot_directoryconfigured) is identical to upstream