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

Andres de la Peña commented on CASSANDRA-18042:
-----------------------------------------------

I agree that the names of the guardrail properties should start by the 
guardrail name to keep consistency.

I like the `warn/fail` naming because it's easy to understand and it aligns 
well with the naming used for thresholds:
{code:java}
zero_default_ttl_on_twcs_warn: true
zero_default_ttl_on_twcs_fail: false
{code}
However, the downside of this approach is that it breaks the {{*_disabled}} 
convention that we are using on other enable flags. So if we wanted to add a 
warning to one of the existing enable flags, we would need to change the naming 
of the old property,

For example, lets take the currently existing guardrail to disable secondary 
indexes:
{code:java}
secondary_indexes_enabled: true
{code}
If at some point we wanted to also add a warning discouraging the use of 
secondary indexes, the naming with new convention would be:
{code:java}
secondary_indexes_warn: false
secondary_indexes_fail: false
{code}
But we can't replace the old property because of backward compatibility. That 
won't happen if the conventions for warn/fail boolean guardrails were:
{code:java}
secondary_indexes_warned: false
secondary_indexes_enabled: true
{code}
That's why I would consider using:
{code:java}
zero_ttl_on_twcs_warned: true
zero_ttl_on_twcs_enabled: true
{code}
So we have a naming convention that allows to easily transform fail-only 
boolean guardrails into warn-or-fail boolean guardrails.

wdyt?

Besides the naming discussion, I have left a few minor suggestion on the PR, 
which mostly looks good to me.

> 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: 2h 20m
>  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: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to