[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17460900#comment-17460900 ] Andres de la Peña commented on CASSANDRA-17189: --- Committed to {{trunk}} as [bf1446cd85ca476ca3e6f53ed3e13e18697acfbc|https://github.com/apache/cassandra/commit/bf1446cd85ca476ca3e6f53ed3e13e18697acfbc]. Thanks for the patch and the second review. > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > Time Spent: 2h 20m > Remaining Estimate: 0h > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17460859#comment-17460859 ] Andres de la Peña commented on CASSANDRA-17189: --- Great, thank you :). I'll commit in a bit. > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > Time Spent: 2h 20m > Remaining Estimate: 0h > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17460834#comment-17460834 ] Brandon Williams commented on CASSANDRA-17189: -- All the CI failures are known, patch looks good. +1 > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > Time Spent: 2h 20m > Remaining Estimate: 0h > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17460819#comment-17460819 ] Andres de la Peña commented on CASSANDRA-17189: --- I'd say that the CI failures in the run above are not related to the changes. The PR with the last changes looks good to me, +1. Now we need the approval of a second committer. > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > Time Spent: 2h 20m > Remaining Estimate: 0h > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17460644#comment-17460644 ] Andres de la Peña commented on CASSANDRA-17189: --- Here are CI runs for [j8 |https://app.circleci.com/pipelines/github/adelapena/cassandra/1213/workflows/594052af-cabc-4add-8a2d-5cabe5817796] and [j11|https://app.circleci.com/pipelines/github/adelapena/cassandra/1213/workflows/96341b28-4296-4ede-9a1a-09ece1faeee2]. > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > Time Spent: 2h 20m > Remaining Estimate: 0h > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17459944#comment-17459944 ] Andres de la Peña commented on CASSANDRA-17189: --- You are very welcome, and thanks for the PR with tests :) The PR mostly looks good to me, I have left a few minor suggestions on it. > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > Time Spent: 10m > Remaining Estimate: 0h > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17459501#comment-17459501 ] Bartlomiej commented on CASSANDRA-17189: Thanks a lot [~adelapena] for help with tests (especially with those executeWithPaging method - it would take me ages to make it working like this). I also applied your changes (sorry for not applying code-formatting - I just realized that Intellij config has it out-of-the-box). Here is the PR [https://github.com/apache/cassandra/pull/1361] (so it will be easier to comment :) ) once again thanks for all your help ;) > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17458504#comment-17458504 ] Andres de la Peña commented on CASSANDRA-17189: --- Great, thanks for the patch, it mostly looks good to me. I have left a few suggestions about typos, formatting, etc. [here|https://github.com/adelapena/cassandra/commit/d5a91df61e232ae65049871cdab9985a31261f3b]. If you create a pull request from your fork we can comment the code in place. We don't directly merge PRs in the end, but they might be useful during reviews, as you prefer. {quote}I also had to move guard call from *public execute()* to *private execute()* because otherwise *executeInternal()* would stay without guard check or this guard call would have be duplicated. Now, after reading the code, I understand that all Select statements will be handled by that guard. {quote} Good point. I think that {{executeInternal}} is only used for internal queries and queries with {{NODE_LOCAL}} consistency. The internal queries are not a problem since they are ignored by guardrails, but probably we want to have the check with the special consistency level. {quote}Please say what you think about my patch, if it is "lets say ok" I will write test {quote} It's definitively looking good, let's go for testing :). I think that the test could extend {{{}ThresholdTester{}}}, just being careful of throwing an {{UnsupportedException}} in the implementation of {{ThresholdTester#currentValue}} because we don't have a way to check the current guarded value. Many utility methods for running queries on {{ThresholdTester}} and it's parent {{GuardrailTester}} don't consider paging, so I think that for out test we could define our own utility methods using paging, for example: {code:java} private void assertPagingValid(String query, int pageSize) throws Throwable { assertValid(() -> executeWithPaging(userClientState, query, pageSize)); } private void assertPagingIgnored(String query, int pageSize) throws Throwable { assertValid(() -> executeWithPaging(superClientState, query, pageSize)); assertValid(() -> executeWithPaging(systemClientState, query, pageSize)); } private void assertPagingWarns(String query, int pageSize, String message) throws Throwable { assertWarns(() -> executeWithPaging(userClientState, query, pageSize), message); } private void assertPagingAborts(String query, int pageSize, String message) throws Throwable { assertAborts(() -> executeWithPaging(userClientState, query, pageSize), message); } private void executeWithPaging(ClientState state, String query, int pageSize) { QueryState queryState = new QueryState(state); String formattedQuery = formatQuery(query); CQLStatement statement = QueryProcessor.parseStatement(formattedQuery, queryState.getClientState()); statement.validate(state); QueryOptions options = QueryOptions.create(ConsistencyLevel.ONE, Collections.emptyList(), false, pageSize, null, null, ProtocolVersion.CURRENT, KEYSPACE); statement.executeLocally(queryState, options); } {code} > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously c
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17457359#comment-17457359 ] Bartlomiej commented on CASSANDRA-17189: {code:java} Why would it crash?{code} that was because, in forPaging method, state was null {code:java} public DataLimits forPaging(int pageSize) { Guardrails.pageSize.guard(pageSize, "?", null); return new CQLLimits(pageSize, perPartitionLimit, isDistinct); }{code} so guard was also applied for super user {code:java} (state == null || state.isOrdinaryUser()); {code} Now it is not the case anymore because in execute I am able to get state and super users are not aborded :) {code:java} By the way, that SelectStatement#columnFamily method uses the old terminology for tables, which were originally called column families. Maybe we can rename the method to table(). {code} as you suggested I changed two method to table() :) I also had to move guard call from {code:java} public ResultMessage.Rows execute() {code} to {code:java} private ResultMessage.Rows execute(){code} because otherwise {code:java} public ResultMessage.Rows executeInternal() {code} would stay without guard check or this guard call would have be duplicated. Now, after reading the code, I understand that all Select statements will be handled by that guard. Please check what you think about my change, if it is "lets say ok" I will write tests :) Thanks ! > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.diff > > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17457222#comment-17457222 ] Andres de la Peña commented on CASSANDRA-17189: --- Hi [~bkowalczyyk], bq. Should I extract guard outside forPaging ?(for example in SelectStatement.execute?) Right, I also think that [{{SelectStatement#execute}}|https://github.com/apache/cassandra/blob/e99a8da161ed599c1a22a853c9c7f9caf6c1eb79/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java#L246] might be a better choice for placing for the call to the guardrail. There you can easily get the table name and use it as the {{what}} argument, with something like: {code:java} Guardrails.keyspaces.guard(pageSize, columnFamily(), state.getClientState()); {code} By the way, that {{SelectStatement#columnFamily}} method uses the old terminology for tables, which were originally called column families. Maybe we can rename the method to {{{}table(){}}}. {quote}I also wonder, should we care about too small value for abort_threshold ? If someone will set for example 100, cassandra will crash. {quote} Why would it crash? High page limits can be a huge problem because they put too much pressure on memory, but IIRC queries with a very low page size are just slow, and could even make some sense if the rows are huge (although other guardrails will try to limit the size of rows). > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.txt > > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17456819#comment-17456819 ] Bartlomiej commented on CASSANDRA-17189: Hi [~brandon.williams] I will have a question, I made the change following steps from the ticket [^CASSANDRA-17189-trunk.txt] , but I don't know what to do here: {code:java} public DataLimits forPaging(int pageSize) { Guardrails.pageSize.guard(pageSize, "?", null); return new CQLLimits(pageSize, perPartitionLimit, isDistinct); } public DataLimits forPaging(int pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) { Guardrails.pageSize.guard(pageSize, "?", null); return new CQLPagingLimits(pageSize, perPartitionLimit, isDistinct, lastReturnedKey, lastReturnedKeyRemaining); } {code} atm that is how messages look {code:java} WARN Quering page size with size 2200 exceeds warning threshold of 2000. WARN Quering page size with size 2400 exceeds warning threshold of 2000. {code} problem is that there is no table name in the log(what makes it difficult to figure out what query exceed the threshold), also guard requires 'what' argument (it is is place for table name), but I have no idea how to make it good :) should I pass table name to forPaging ? that doesn't looks good for me (or no?). Should I extract guard outside forPaging ?(for example in SelectStatement.execute?) - problem is that there are multiple places that forPagining is executed. * I also wonder, should we care about too small value for abort_threshold ? If someone will set for example 100, cassandra will crash. Thanks ! > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > Attachments: CASSANDRA-17189-trunk.txt > > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17455915#comment-17455915 ] Brandon Williams commented on CASSANDRA-17189: -- Assigned to you, please do let us know if you need help, and good luck! > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Assignee: Bartlomiej >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-17189) Guardrail for page size
[ https://issues.apache.org/jira/browse/CASSANDRA-17189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17455913#comment-17455913 ] Bartlomiej commented on CASSANDRA-17189: Hi, I would like to try implement this (hope it will not overwhelm me :D ), thanks ! > Guardrail for page size > --- > > Key: CASSANDRA-17189 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17189 > Project: Cassandra > Issue Type: New Feature > Components: Feature/Guardrails >Reporter: Andres de la Peña >Priority: Normal > Labels: AdventCalendar2021, lhf > Fix For: 4.1 > > > Add guardrail limiting the query page size, for example: > {code} > # Guardrail to warn about or reject page sizes greater than threshold. > # The two thresholds default to -1 to disable. > page_size: > warn_threshold: -1 > abort_threshold: -1 > {code} > Initially this can be based on the specified number of rows used as page > size, although it would be ideal to also limit the actual size in bytes of > the returned pages. > +Additional information for newcomers:+ > # Add the configuration for the new guardrail on page size in the guardrails > section of cassandra.yaml. > # Add a getPageSize method in GuardrailsConfig returning a Threshold.Config > object > # Implement that method in GuardrailsOptions, which is the default yaml-based > implementation of GuardrailsConfig > # Add a Threshold guardrail named pageSize in Guardrails, using the > previously created config > # Define JMX-friendly getters and setters for the previously created config > in GuardrailsMBean > # Implement the JMX-friendly getters and setters in Guardrails > # Now that we have the guardrail ready, it’s time to use it. We should search > for a place to invoke the Guardrails.pageSize#guard method with the page size > that each query is going to use. The DataLimits#forPaging methods look like > good candidates for this. > # Finally, add some tests for the new guardrail. Given that the new guardrail > is a Threshold, our new test should probably extend ThresholdTester. -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org