You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Paulo Motta (Jira)" <ji...@apache.org> on 2021/07/07 16:34:00 UTC
[jira] [Commented] (CASSANDRA-16789) Add TTL support to nodetool
snapshots
[ https://issues.apache.org/jira/browse/CASSANDRA-16789?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17376686#comment-17376686 ]
Paulo Motta commented on CASSANDRA-16789:
-----------------------------------------
Hi Abi,
Your initial working POC from [PR#1046|https://github.com/apache/cassandra/pull/1046] looks great so far, good job!
As we discussed in our last call let's make the following changes in order to make this production-ready:
* Use Instant instead of String on {{SnapshotDetails.createdAt}} and {{SnapshotDetails.expiresAt}}
* Rename {{SnapshotDetails}} -> {{SnapshotManifest}}, since a single keyspace snapshot can have contain multiple table manifests
* Create new inner class {{SnapshotDetails}} on {{SnapshotManager}}, which represents a set of {{SnapshotManifest}} of a given keyspace with the same name.
* Update SnapshotManager to use the following structure
{code:java}
class SnapshotManager {
ScheduledFuture cleanupTaskFuture;
Map<String, SnapshotDetails> activeSnapshots;
void start() {
populateActiveSnapshots();
cleanupTaskFuture = scheduleWithFixedDelay(this::clearExpiredSnapshots, 0, 1, TimeUnit.MINUTES);
}
void addSnapshot(String name, SnapshotManifest manifest) {
// Create new SnapshotDetails if it's a new snapshot, otherwise add to existing
}
void populateActiveSnapshots(){
// Read snapshots from disk and add it to activeSnapshots
}
void clearExpiredSnapshots() {
for (SnapshotDetails snapshot : activeSnapshots.values()) {
if (snapshot.isExpired()) {
logger.info("Clearing expired snapshot {}", snapshot);
snapshot.clear();
}
}
}
}
{code}
* Add unit test for each method of SnapshotManager on different scenarios
* MakeĀ {{ColumnFamilyStore.snapshotWithoutFlush}} add newly created snapshot manifests to {{SnapshotManager}} (via {{SnapshotManager.addSnapshot}}
* Add dtest with the following structure:
** create snapshot with 1 minute TTL
** stop node
** start node
** Wait *up to* 1 minute for snapshot expiration
** Check snapshot is removed from disk
> Add TTL support to nodetool snapshots
> -------------------------------------
>
> Key: CASSANDRA-16789
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16789
> Project: Cassandra
> Issue Type: Sub-task
> Components: Tool/nodetool
> Reporter: Paulo Motta
> Assignee: Abuli Palagashvili
> Priority: Normal
>
> Add new parameter {{--ttl}} to {{nodetool snapshot}} command. This parameter can be specified in human readable duration (ie. 30mins, 1h, 300d) and should not be lower than 1 minute.
> The expiration date should be added to the snapshot manifest in ISO format.
> A periodic thread should efficiently scan snapshots and automatically clear those past expiration date. The periodicity of the scan thread should be 1 minute by default but be overridable via a system property.
> The command {{nodetool listsnapshots}} should display the expiration date when the snapshot contains a TTL.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org