[ 
https://issues.apache.org/jira/browse/CASSANDRA-18042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17641585#comment-17641585
 ] 

Josh McKenzie commented on CASSANDRA-18042:
-------------------------------------------

I see it being differently vague with the _warned case:
{code}
sai_indexes_take_snapshots_warned: true
sai_indexes_take_snapshots: false
sai_indexes_warned: true
sai_indexes_enabled: false
{code}
In the above example, would we allow sai indexes and warn when used? Would we 
disallow them and not warn?

Whatever form we take, if we say the warning is contingent on the feature being 
enabled that's different than our other guardrails where we have an assumption 
that all features are enabled but they warn and fail at certain thresholds. 
This new paradigm (warning on feature flagged things) is overloading the notion 
of "warned" to something who's state is _dependent_ on another configuration 
option if I'm not mistaken.

So in the above example, if we're saying "sai_indexes will not be configured 
and the _warned option is dead unless the _enabled is flipped", there's an 
undocumented coupling between them that's also divergent from how the other 
guardrails use _warn and _fail.

{code}
sai_indexes_take_snapshots_if_enabled
{code}
I wasn't advocating for this format. Only specifically for "warn_if_enabled" 
for feature flags implemented through guardrails.

So, all this said: not a hill I'm willing to die on. I think we're kind of 
bikeshedding between a bunch of dissatisfying options here as we try to bolt 
something on to a feature (guardrails as feature flag) that was never intended 
to have the notion of "optionally warn if someone takes the extra step to turn 
this feature on".

> Implement a guardrail for not having zero default ttl on tables with TWCS
> -------------------------------------------------------------------------
>
>                 Key: CASSANDRA-18042
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-18042
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Feature/Guardrails, Legacy/Core
>            Reporter: Stefan Miklosovic
>            Assignee: Stefan Miklosovic
>            Priority: Normal
>             Fix For: 4.x
>
>          Time Spent: 3h
>  Remaining Estimate: 0h
>
> A user was surprised that his data have not started to expire after 90 days 
> on his TWCS, he noticed that default_time_to_live on the table was set to 0 
> (by accident from his side) and inserts were using TTL = 0 too.
> It is questionable why it it possible to create a table with TWCS and enable 
> a user to specify default_time_to_live to be zero.
> On the other hand, I would argue that having default_time_to_live set to 0 on 
> TWCS does not necessarily mean that such combination is illegal. It is about 
> people just using that with advantage very often so tables are compacted away 
> nicely. However, that does not have to mean that they could not use it with 
> 0. But I yet have to see a use-case where TWCS was used and default ttl was 
> set to 0 on purpose. Merely looking into Cassandra codebase, there are only 
> cases when this parameter is not 0.
> There are three approaches:
> 1) just reject such statements (for CreateTable and AlterTable statements) 
> where default_time_to_live = 0
> 2) Implement a guardrail for 1) so it can be enabled / disabled on demand
> 3) Leave possibility to set default_time_to_live to 0 on a table but make a 
> guardrail for UpdateStatement so it might reject queries for tables with 
> default_time_to_live is zero and for which its TTL (on that update statement) 
> is set to 0 too.
> I would be careful about making the current configuration illegal because of 
> backward compatibility. For that reason 2) makes the most sense to me.
> Maybe implementing 3) would make sense as well. There might be a table which 
> has default ttl set to 0 as it expects a user to supply TTL every time. 
> However, as it is not currently enforced anywhere, a client might still 
> insert TTLs to be set to 0 even by accident.
> POC for 2) is here 
> https://github.com/instaclustr/cassandra/commit/0b4dcc3d3deeffa393c02a3b80e27482007f9579



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to