[ 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