You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2021/12/15 13:39:22 UTC

[GitHub] [cassandra] adelapena commented on a change in pull request #1361: CASSANDRA-17189

adelapena commented on a change in pull request #1361:
URL: https://github.com/apache/cassandra/pull/1361#discussion_r769603707



##########
File path: conf/cassandra.yaml
##########
@@ -1602,3 +1602,8 @@ enable_drop_compact_storage: false
 #         disallowed: []
 # Guardrail to allow/disallow user-provided timestamps. Defaults to true.
 #     user_timestamps_enabled: true
+# Guardrail to warn or abort when user a page size greater than threshold.

Review comment:
       Nit: I think it's `using` instead of `user`:
   ```suggestion
   # Guardrail to warn or abort when using a page size greater than threshold.
   ```

##########
File path: conf/cassandra.yaml
##########
@@ -1602,3 +1602,8 @@ enable_drop_compact_storage: false
 #         disallowed: []
 # Guardrail to allow/disallow user-provided timestamps. Defaults to true.
 #     user_timestamps_enabled: true
+# Guardrail to warn or abort when user a page size greater than threshold.
+# The two thresholds default to -1 to disable.
+#     page_size:
+#       warn_threshold: 2000
+#       abort_threshold: 4000

Review comment:
       We should leave them disabled by default:
   ```suggestion
   #       warn_threshold: -1
   #       abort_threshold: -1
   ```

##########
File path: test/unit/org/apache/cassandra/db/guardrails/GuardrailPageSizeTest.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.guardrails;
+
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+
+/**
+ * Tests the guardrail for the page size, {@link Guardrails#pageSize}.
+ */
+public class GuardrailPageSizeTest extends ThresholdTester
+{
+    private static final int PAGE_SIZE_WARN_THRESHOLD = 5;
+    private static final int PAGE_SIZE_ABORT_THRESHOLD = 10;
+
+    public GuardrailPageSizeTest()
+    {
+        super(PAGE_SIZE_WARN_THRESHOLD,
+              PAGE_SIZE_ABORT_THRESHOLD,
+              DatabaseDescriptor.getGuardrailsConfig().getPageSize(),
+              Guardrails::setPageSizeThreshold,
+              Guardrails::getPageSizeWarnThreshold,
+              Guardrails::getPageSizeAbortThreshold);
+    }
+
+    @Before
+    public void setupTest()
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (k INT, c INT, v TEXT, PRIMARY KEY(k, c))");
+    }
+
+    @Test
+    public void testSelectStatementAgainstPageSize() throws Throwable
+    {
+        assertPagingValid("SELECT * FROM %s", 3);
+        assertPagingValid("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+
+        assertPagingWarns("SELECT * FROM %s", 6, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), 6, PAGE_SIZE_WARN_THRESHOLD));
+        assertPagingWarns("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), PAGE_SIZE_ABORT_THRESHOLD, PAGE_SIZE_WARN_THRESHOLD));
+
+        assertPagingAborts("SELECT * FROM %s", 11, format("Aborting %s table query, page size %s exceeds abort threshold of %s.", currentTable(), 11, PAGE_SIZE_ABORT_THRESHOLD));
+    }
+
+    @Test
+    public void testIgnoreInternalClients() throws Throwable
+    {
+        assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+        assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD);

Review comment:
       These queries won't trigger the intended guardrail because they use exactly the threshold value, so we can't verify that the superuser and the internal user are allowed to skip the guardrail (although the abort threshold check would trigger the warn threshold). I think we should use a higher value:
   ```suggestion
           assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD + 1);
           assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD + 1);
   ```

##########
File path: test/unit/org/apache/cassandra/db/guardrails/GuardrailPageSizeTest.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.guardrails;
+
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+
+/**
+ * Tests the guardrail for the page size, {@link Guardrails#pageSize}.
+ */
+public class GuardrailPageSizeTest extends ThresholdTester
+{
+    private static final int PAGE_SIZE_WARN_THRESHOLD = 5;
+    private static final int PAGE_SIZE_ABORT_THRESHOLD = 10;
+
+    public GuardrailPageSizeTest()
+    {
+        super(PAGE_SIZE_WARN_THRESHOLD,
+              PAGE_SIZE_ABORT_THRESHOLD,
+              DatabaseDescriptor.getGuardrailsConfig().getPageSize(),
+              Guardrails::setPageSizeThreshold,
+              Guardrails::getPageSizeWarnThreshold,
+              Guardrails::getPageSizeAbortThreshold);
+    }
+
+    @Before
+    public void setupTest()
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (k INT, c INT, v TEXT, PRIMARY KEY(k, c))");
+    }
+
+    @Test
+    public void testSelectStatementAgainstPageSize() throws Throwable
+    {
+        assertPagingValid("SELECT * FROM %s", 3);
+        assertPagingValid("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+
+        assertPagingWarns("SELECT * FROM %s", 6, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), 6, PAGE_SIZE_WARN_THRESHOLD));
+        assertPagingWarns("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), PAGE_SIZE_ABORT_THRESHOLD, PAGE_SIZE_WARN_THRESHOLD));
+
+        assertPagingAborts("SELECT * FROM %s", 11, format("Aborting %s table query, page size %s exceeds abort threshold of %s.", currentTable(), 11, PAGE_SIZE_ABORT_THRESHOLD));
+    }
+
+    @Test
+    public void testIgnoreInternalClients() throws Throwable

Review comment:
       Nit: we are also testing that superusers are excluded, not only internal queries. Thus, we could call the method `testExcludedUsers`, which is the name used in other similar tests.

##########
File path: test/unit/org/apache/cassandra/db/guardrails/GuardrailPageSizeTest.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.guardrails;
+
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+
+/**
+ * Tests the guardrail for the page size, {@link Guardrails#pageSize}.
+ */
+public class GuardrailPageSizeTest extends ThresholdTester
+{
+    private static final int PAGE_SIZE_WARN_THRESHOLD = 5;
+    private static final int PAGE_SIZE_ABORT_THRESHOLD = 10;
+
+    public GuardrailPageSizeTest()
+    {
+        super(PAGE_SIZE_WARN_THRESHOLD,
+              PAGE_SIZE_ABORT_THRESHOLD,
+              DatabaseDescriptor.getGuardrailsConfig().getPageSize(),
+              Guardrails::setPageSizeThreshold,
+              Guardrails::getPageSizeWarnThreshold,
+              Guardrails::getPageSizeAbortThreshold);
+    }
+
+    @Before
+    public void setupTest()
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (k INT, c INT, v TEXT, PRIMARY KEY(k, c))");
+    }
+
+    @Test
+    public void testSelectStatementAgainstPageSize() throws Throwable
+    {
+        assertPagingValid("SELECT * FROM %s", 3);
+        assertPagingValid("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+
+        assertPagingWarns("SELECT * FROM %s", 6, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), 6, PAGE_SIZE_WARN_THRESHOLD));
+        assertPagingWarns("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), PAGE_SIZE_ABORT_THRESHOLD, PAGE_SIZE_WARN_THRESHOLD));
+
+        assertPagingAborts("SELECT * FROM %s", 11, format("Aborting %s table query, page size %s exceeds abort threshold of %s.", currentTable(), 11, PAGE_SIZE_ABORT_THRESHOLD));
+    }
+
+    @Test
+    public void testIgnoreInternalClients() throws Throwable
+    {
+        assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+        assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD);
+    }
+
+    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);

Review comment:
       Nit: I was thinking that we don't really need the `message` argument here, we can build it inside the method and so simplify the calls to it:
   ```suggestion
           assertWarns(() -> executeWithPaging(userClientState, query, pageSize),
                       format("Query for table %s with page size %s exceeds warning threshold of %s.",
                              currentTable(), pageSize, PAGE_SIZE_WARN_THRESHOLD));
   ```

##########
File path: test/unit/org/apache/cassandra/db/guardrails/GuardrailPageSizeTest.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.guardrails;
+
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+
+/**
+ * Tests the guardrail for the page size, {@link Guardrails#pageSize}.
+ */
+public class GuardrailPageSizeTest extends ThresholdTester
+{
+    private static final int PAGE_SIZE_WARN_THRESHOLD = 5;
+    private static final int PAGE_SIZE_ABORT_THRESHOLD = 10;
+
+    public GuardrailPageSizeTest()
+    {
+        super(PAGE_SIZE_WARN_THRESHOLD,
+              PAGE_SIZE_ABORT_THRESHOLD,
+              DatabaseDescriptor.getGuardrailsConfig().getPageSize(),
+              Guardrails::setPageSizeThreshold,
+              Guardrails::getPageSizeWarnThreshold,
+              Guardrails::getPageSizeAbortThreshold);
+    }
+
+    @Before
+    public void setupTest()
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (k INT, c INT, v TEXT, PRIMARY KEY(k, c))");
+    }
+
+    @Test
+    public void testSelectStatementAgainstPageSize() throws Throwable
+    {
+        assertPagingValid("SELECT * FROM %s", 3);
+        assertPagingValid("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+
+        assertPagingWarns("SELECT * FROM %s", 6, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), 6, PAGE_SIZE_WARN_THRESHOLD));
+        assertPagingWarns("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), PAGE_SIZE_ABORT_THRESHOLD, PAGE_SIZE_WARN_THRESHOLD));
+
+        assertPagingAborts("SELECT * FROM %s", 11, format("Aborting %s table query, page size %s exceeds abort threshold of %s.", currentTable(), 11, PAGE_SIZE_ABORT_THRESHOLD));

Review comment:
       We could also include an aggregation query (e.g `SELECT COUNT(*)`, and probably also a `LIMIT` clause. The `LIMIT` case is interesting because if it's lesser than the page size the triggering of the guardrail would depend on where in `SelectStatement` we have put the guardrail check. So it could be something like:
   ```suggestion
           // regular query
           String query = "SELECT * FROM %s";
           assertPagingValid(query, 3);
           assertPagingValid(query, PAGE_SIZE_WARN_THRESHOLD);
           assertPagingWarns(query, 6);
           assertPagingWarns(query, PAGE_SIZE_ABORT_THRESHOLD);
           assertPagingAborts(query, 11);
   
           // aggregation query
           query = "SELECT COUNT(*) FROM %s WHERE k=0";
           assertPagingValid(query, 3);
           assertPagingValid(query, PAGE_SIZE_WARN_THRESHOLD);
           assertPagingWarns(query, 6);
           assertPagingWarns(query, PAGE_SIZE_ABORT_THRESHOLD);
           assertPagingAborts(query, 11);
   
           // query with limit over thresholds
           query = "SELECT * FROM %s LIMIT 100";
           assertPagingValid(query, 3);
           assertPagingValid(query, PAGE_SIZE_WARN_THRESHOLD);
           assertPagingWarns(query, 6);
           assertPagingWarns(query, PAGE_SIZE_ABORT_THRESHOLD);
           assertPagingAborts(query, 11);
   
           // query with limit under thresholds
           query = "SELECT * FROM %s LIMIT 1";
           assertPagingValid(query, 3);
           assertPagingValid(query, PAGE_SIZE_WARN_THRESHOLD);
           assertPagingValid(query, 6);
           assertPagingValid(query, PAGE_SIZE_ABORT_THRESHOLD);
           assertPagingValid(query, 11);
   ```
   Please note that `SELECT * FROM %s LIMIT 1` internally uses a lower page size because of the limit, so the guardrail isn't triggered. I think this behaviour is correct.

##########
File path: src/java/org/apache/cassandra/db/guardrails/Guardrails.java
##########
@@ -98,6 +98,18 @@
     new DisableFlag(state -> !CONFIG_PROVIDER.getOrCreate(state).getUserTimestampsEnabled(),
                     "User provided timestamps (USING TIMESTAMP)");
 
+    /**
+     * Guardrail on the number of elements returned within page.
+     */
+    public static final Threshold pageSize =
+    new Threshold(state -> CONFIG_PROVIDER.getOrCreate(state).getPageSize(),
+                  (isWarning, what, value, threshold) ->
+                  isWarning ? format("Query for table %s with page size %s exceeds warning threshold of %s.",
+                                     what, value, threshold)
+                            : format("Aborting %s table query, page size %s exceeds abort threshold of %s.",

Review comment:
       ```suggestion
                               : format("Aborting query for table %s, page size %s exceeds abort threshold of %s.",
   ```

##########
File path: test/unit/org/apache/cassandra/db/guardrails/GuardrailPageSizeTest.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.guardrails;
+
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+
+/**
+ * Tests the guardrail for the page size, {@link Guardrails#pageSize}.
+ */
+public class GuardrailPageSizeTest extends ThresholdTester
+{
+    private static final int PAGE_SIZE_WARN_THRESHOLD = 5;
+    private static final int PAGE_SIZE_ABORT_THRESHOLD = 10;
+
+    public GuardrailPageSizeTest()
+    {
+        super(PAGE_SIZE_WARN_THRESHOLD,
+              PAGE_SIZE_ABORT_THRESHOLD,
+              DatabaseDescriptor.getGuardrailsConfig().getPageSize(),
+              Guardrails::setPageSizeThreshold,
+              Guardrails::getPageSizeWarnThreshold,
+              Guardrails::getPageSizeAbortThreshold);
+    }
+
+    @Before
+    public void setupTest()
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (k INT, c INT, v TEXT, PRIMARY KEY(k, c))");
+    }
+
+    @Test
+    public void testSelectStatementAgainstPageSize() throws Throwable
+    {
+        assertPagingValid("SELECT * FROM %s", 3);
+        assertPagingValid("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+
+        assertPagingWarns("SELECT * FROM %s", 6, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), 6, PAGE_SIZE_WARN_THRESHOLD));
+        assertPagingWarns("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), PAGE_SIZE_ABORT_THRESHOLD, PAGE_SIZE_WARN_THRESHOLD));
+
+        assertPagingAborts("SELECT * FROM %s", 11, format("Aborting %s table query, page size %s exceeds abort threshold of %s.", currentTable(), 11, PAGE_SIZE_ABORT_THRESHOLD));
+    }
+
+    @Test
+    public void testIgnoreInternalClients() throws Throwable
+    {
+        assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+        assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD);
+    }
+
+    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);
+    }
+
+    //not used by page-size guardrail tests.
+    protected long currentValue()
+    {
+        return 0;

Review comment:
       Nit: We are supposed not to call this because it doesn't make sense for this particular guardrail, so I would throw an exception so we quickly detect an accidental call, wdyt?
   ```suggestion
           throw new UnsupportedOperationException();
   ```

##########
File path: test/unit/org/apache/cassandra/db/guardrails/GuardrailPageSizeTest.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.guardrails;
+
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+
+/**
+ * Tests the guardrail for the page size, {@link Guardrails#pageSize}.
+ */
+public class GuardrailPageSizeTest extends ThresholdTester
+{
+    private static final int PAGE_SIZE_WARN_THRESHOLD = 5;
+    private static final int PAGE_SIZE_ABORT_THRESHOLD = 10;
+
+    public GuardrailPageSizeTest()
+    {
+        super(PAGE_SIZE_WARN_THRESHOLD,
+              PAGE_SIZE_ABORT_THRESHOLD,
+              DatabaseDescriptor.getGuardrailsConfig().getPageSize(),
+              Guardrails::setPageSizeThreshold,
+              Guardrails::getPageSizeWarnThreshold,
+              Guardrails::getPageSizeAbortThreshold);
+    }
+
+    @Before
+    public void setupTest()
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (k INT, c INT, v TEXT, PRIMARY KEY(k, c))");
+    }
+
+    @Test
+    public void testSelectStatementAgainstPageSize() throws Throwable
+    {
+        assertPagingValid("SELECT * FROM %s", 3);
+        assertPagingValid("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+
+        assertPagingWarns("SELECT * FROM %s", 6, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), 6, PAGE_SIZE_WARN_THRESHOLD));
+        assertPagingWarns("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD, format("Query for table %s with page size %s exceeds warning threshold of %s.", currentTable(), PAGE_SIZE_ABORT_THRESHOLD, PAGE_SIZE_WARN_THRESHOLD));
+
+        assertPagingAborts("SELECT * FROM %s", 11, format("Aborting %s table query, page size %s exceeds abort threshold of %s.", currentTable(), 11, PAGE_SIZE_ABORT_THRESHOLD));
+    }
+
+    @Test
+    public void testIgnoreInternalClients() throws Throwable
+    {
+        assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_WARN_THRESHOLD);
+        assertPagingIgnored("SELECT * FROM %s", PAGE_SIZE_ABORT_THRESHOLD);
+    }
+
+    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);

Review comment:
       Same as before, we can get rid of the `message` argument:
   ```suggestion
           assertAborts(() -> executeWithPaging(userClientState, query, pageSize),
                        format("Aborting query for table %s, page size %s exceeds abort threshold of %s.",
                               currentTable(), pageSize, PAGE_SIZE_ABORT_THRESHOLD));
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org