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 2020/06/29 23:06:36 UTC

[GitHub] [cassandra] maedhroz opened a new pull request #659: A raft of minor optimizations for replica filtering protection

maedhroz opened a new pull request #659:
URL: https://github.com/apache/cassandra/pull/659


   


----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449534540



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE, false);

Review comment:
       We could use `REPLICAS * ROWS` instead of `REPLICAS * ROWS + 1` to test the max accepted value, as we do below with the failure 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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r457697875



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -289,6 +266,35 @@ public void close()
         };
     }
 
+    private void checkCachedRowThresholds()
+    {
+        if (rowsCached == cachedRowsFailThreshold + 1)
+        {
+            String message = String.format("Replica filtering protection has cached over %d rows during query %s. " +

Review comment:
       I'll try to expand the inline description in `cassandra.yaml` a bit, given the log message already refers the reader there.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on pull request #659:
URL: https://github.com/apache/cassandra/pull/659#issuecomment-661800812


   This is looking great. My only remaining concern is that we should record the metric for the per-query max cache size just once per query.


----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449538964



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(fullTableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // TODO: These should be able to use ICoordinator#executeWithResult() once CASSANDRA-15920 is resolved.
+        Object[] oldResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "old", ROWS));
+        Object[][] oldRows = (Object[][]) oldResponse[0];
+        assertRows(oldRows);
+        @SuppressWarnings("unchecked") List<String> oldWarnings = (List<String>) oldResponse[1];
+        assertEquals(shouldWarn, oldWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));

Review comment:
       It's probably not needed for testing the guardrails, but we could easily check here the SRP requests metric to verify that we are really generating the desired scenario, just in case:
   ```java
   assertEquals(3, rfpRequests(tableName));
   
   ...
   
   private static long rfpRequests(String tableName)
   {
       return cluster.get(1).callOnInstance(() -> Keyspace.open(KEYSPACE)
                                                          .getColumnFamilyStore(tableName)
                                                          .metric.replicaFilteringProtectionRequests
                                                          .getCount());
   }
   ```
   Also, we can consider renaming the metric from `ReplicaSideFilteringProtectionRequests` to `ReplicaFilteringProtectionRequests`, without the "side" part.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r459021125



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,122 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));

Review comment:
       Works for me.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r453737006



##########
File path: src/java/org/apache/cassandra/exceptions/TooManyCachedRowsException.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ReadCommand;
+
+import static org.apache.cassandra.exceptions.ExceptionCode.SERVER_ERROR;
+
+/**
+ * Exception thrown when {@link org.apache.cassandra.service.ReplicaFilteringProtection} caches
+ * the configured threshold number of row results from participating replicas.
+ */
+public class TooManyCachedRowsException extends RequestExecutionException

Review comment:
       The only problem w/ `OverloadedException` is that tools like `cqlsh` still don't seem to interpret the error usefully. (i.e. Still just dumps a `NoHostAvailable`.)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r453737006



##########
File path: src/java/org/apache/cassandra/exceptions/TooManyCachedRowsException.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ReadCommand;
+
+import static org.apache.cassandra.exceptions.ExceptionCode.SERVER_ERROR;
+
+/**
+ * Exception thrown when {@link org.apache.cassandra.service.ReplicaFilteringProtection} caches
+ * the configured threshold number of row results from participating replicas.
+ */
+public class TooManyCachedRowsException extends RequestExecutionException

Review comment:
       The only problem w/ `OverloadedException` is that tools like `cqlsh` still don't seem to interpret the error usefully. (i.e. Still just dumps a `NoHostAvailable`.)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449172162



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE);

Review comment:
       This test just makes sure we don't fail, but I haven't found an easy way to check the contents of the logs in a JVM dtest. (I verified manually, of course, but it would be nice to make an assertion here.)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r451843181



##########
File path: src/java/org/apache/cassandra/exceptions/TooManyCachedRowsException.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ReadCommand;
+
+/**
+ * Exception thrown when {@link org.apache.cassandra.service.ReplicaFilteringProtection} caches
+ * the configured threshold number of row results from participating replicas.
+ */
+public class TooManyCachedRowsException extends RuntimeException

Review comment:
       self nit: This needs to extend `RequestExecutionException`, or the error won't be packaged up properly for clients, people using cqlsh, etc.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448932911



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet

Review comment:
       We could explicitly say that the threshold values are for the number of rows. Also, that this number is per query (or per query page), and perhaps that they only live for the duration of the query, especially if we used "cached" instead of "materialized".




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] belliottsmith commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449151578



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       Given the existing usage of materialised on the project it might be best to pick something else again :/




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448953282



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       I don't know if using powers of two gives us any advantage here, and I'm a bit afraid it could make users think that this is somehow measuring memory sizes. Given that this seems more related to the query limit, perhaps we should use something like 1000, WDYT?
   
   Also, that is the number of rows in all replicas, so if we are reading from two on-sync replicas the threshold will start at 512 results, allowing a max limit/page size of 512 rows, won't it? Perhaps we should count the number of merged materialized/cached rows, ignoring the replicas, so the thresholds can be more easily related to the query limit.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r450413331



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(fullTableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // TODO: These should be able to use ICoordinator#executeWithResult() once CASSANDRA-15920 is resolved.
+        Object[] oldResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "old", ROWS));
+        Object[][] oldRows = (Object[][]) oldResponse[0];
+        assertRows(oldRows);
+        @SuppressWarnings("unchecked") List<String> oldWarnings = (List<String>) oldResponse[1];
+        assertEquals(shouldWarn, oldWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));

Review comment:
       Renaming done. Are you okay with the client warning checking implicitly verifying that RFP has occurred?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448684821



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -132,34 +147,33 @@
      */
     UnfilteredPartitionIterator queryProtectedPartitions(int source)
     {
-        UnfilteredPartitionIterator original = makeIterator(originalPartitions.get(source));
-        SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>> toFetch = rowsToFetch.get(source);
+        UnfilteredPartitionIterator original = makeIterator(originalPartitions.set(source, null));
+        SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>> toFetch = rowsToFetch.set(source, null);

Review comment:
       These two changes won't reduce the high watermark in terms of memory usage, but they will make the objects `original` and `toFetch` point to collectable a bit earlier.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449087897



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()

Review comment:
       Perhaps we should also test the case where the number of rows is the same than the 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.

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


[GitHub] [cassandra] belliottsmith commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449535302



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       If this is to be replica and page size dependent, perhaps it should be defined as a ratio of some multiple of these?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449165415



##########
File path: src/java/org/apache/cassandra/db/rows/EncodingStats.java
##########
@@ -107,6 +108,29 @@ public EncodingStats mergeWith(EncodingStats that)
         return new EncodingStats(minTimestamp, minDelTime, minTTL);
     }
 
+    /**
+     * Merge one or more EncodingStats, that are lazily materialized from some list of arbitrary type by the provided function
+     */
+    public static <V, F extends Function<V, EncodingStats>> EncodingStats merge(List<V> values, F function)
+    {
+        if (values.size() == 1)
+            return function.apply(values.get(0));
+
+        Collector collector = new Collector();
+        for (int i=0, isize=values.size(); i<isize; i++)

Review comment:
       Ouch. I'll fix the typos.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r457296929



##########
File path: test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java
##########
@@ -50,28 +50,28 @@ public void testAddMoreThanCapacity()
     @Test
     public void testIsEmptyAndSize()
     {
-        Accumulator<Integer> accu = new Accumulator(4);
+        Accumulator<Integer> accu = new Accumulator<>(4);
 
         assertTrue(accu.isEmpty());
         assertEquals(0, accu.size());
 
         accu.add(1);
         accu.add(2);
 
-        assertTrue(!accu.isEmpty());
+        assertFalse(accu.isEmpty());

Review comment:
       +1 :)

##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -650,9 +651,10 @@ private UnfilteredPartitionIterator extendWithShortReadProtection(UnfilteredPart
 
         private boolean partitionsFetched; // whether we've seen any new partitions since iteration start or last moreContents() call
 
-        private ShortReadPartitionsProtection(InetAddress source, DataLimits.Counter singleResultCounter, DataLimits.Counter mergedResultCounter)
+        private ShortReadPartitionsProtection(InetAddress source, Runnable preFetchCallback, DataLimits.Counter singleResultCounter, DataLimits.Counter mergedResultCounter)

Review comment:
       We could break this line:
   ```suggestion
           private ShortReadPartitionsProtection(InetAddress source, 
                                                 Runnable preFetchCallback, 
                                                 DataLimits.Counter singleResultCounter, 
                                                 DataLimits.Counter mergedResultCounter)
   ```

##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -596,14 +596,16 @@ else if (DROP_OVERSIZED_READ_REPAIR_MUTATIONS)
     }
 
     private UnfilteredPartitionIterator extendWithShortReadProtection(UnfilteredPartitionIterator partitions,
-                                                                      InetAddress source,
-                                                                      DataLimits.Counter mergedResultCounter)
+                                                                      ResolveContext context,
+                                                                      int i)
     {
         DataLimits.Counter singleResultCounter =
             command.limits().newCounter(command.nowInSec(), false, command.selectsFullPartition(), enforceStrictLiveness).onlyCount();
 
-        ShortReadPartitionsProtection protection =
-            new ShortReadPartitionsProtection(source, singleResultCounter, mergedResultCounter);
+        ShortReadPartitionsProtection protection = new ShortReadPartitionsProtection(context.sources[i],
+                                                                                     () -> responses.clearUnsafe(i),

Review comment:
       I think it's a nice improvement.

##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -150,46 +150,44 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
     {
         // Protecting against inconsistent replica filtering (some replica returning a row that is outdated but that
         // wouldn't be removed by normal reconciliation because up-to-date replica have filtered the up-to-date version
-        // of that row) works in 3 steps:
-        //   1) we read the full response just to collect rows that may be outdated (the ones we got from some
-        //      replica but didn't got any response for other; it could be those other replica have filtered a more
-        //      up-to-date result). In doing so, we do not count any of such "potentially outdated" row towards the
-        //      query limit. This simulate the worst case scenario where all those "potentially outdated" rows are
-        //      indeed outdated, and thus make sure we are guaranteed to read enough results (thanks to short read
-        //      protection).
-        //   2) we query all the replica/rows we need to rule out whether those "potentially outdated" rows are outdated
-        //      or not.
-        //   3) we re-read cached copies of each replica response using the "normal" read path merge with read-repair,
-        //      but where for each replica we use their original response _plus_ the additional rows queried in the
-        //      previous step (and apply the command#rowFilter() on the full result). Since the first phase has
-        //      pessimistically collected enough results for the case where all potentially outdated results are indeed
-        //      outdated, we shouldn't need further short-read protection requests during this phase.
+        // of that row) involves 3 main elements:
+        //   1) We combine short-read protection and a merge listener that identifies potentially "out-of-date"
+        //      rows to create an iterator that is guaranteed to produce enough valid row results to satisfy the query 
+        //      limit if enough actually exist. A row is considered out-of-date if its merged form is non-empty and we 
+        //      receive not response from at least one replica. In this case, it is possible that filtering at the
+        //      "silent" replica has produced a more up-to-date result.
+        //   2) This iterator is passed to the standard resolution process with read-repair, but is first wrapped in a 
+        //      response provider that lazily "completes" potentially out-of-date rows by directly querying them on the
+        //      replicas that were previously silent. As this iterator is consumed, it caches valid data for potentially
+        //      out-of-date rows, and this cached data is merged with the fetched data as rows are requested. If there
+        //      is no replica divergence, only rows in the partition being evalutated will be cached (then released
+        //      when the partition is consumed).
+        //   3) After a "complete" row is materialized, it must pass the row filter supplied by the original query 
+        //      before it counts against the limit.

Review comment:
       Nice update, +1

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -289,6 +266,35 @@ public void close()
         };
     }
 
+    private void checkCachedRowThresholds()
+    {
+        if (rowsCached == cachedRowsFailThreshold + 1)
+        {
+            String message = String.format("Replica filtering protection has cached over %d rows during query %s. " +

Review comment:
       What we would recommend to someone hitting this would be, beside changing the thresholds, improving the sync between replicas and/or lowering the page size. Should we mention this here, in the yaml, or in the documentation?

##########
File path: test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java
##########
@@ -99,32 +99,31 @@ public void testGetAndIterator()
     @Test
     public void testClearUnsafe()
     {
-        Accumulator<String> accu = new Accumulator<>(3);
+        Accumulator<String> accu = new Accumulator<>(5);
 
         accu.add("1");
         accu.add("2");
         accu.add("3");
 
-        accu.clearUnsafe();
+        accu.clearUnsafe(1);
 
-        assertEquals(0, accu.size());
-        assertFalse(accu.iterator().hasNext());
-        assertOutOfBonds(accu, 0);
+        assertEquals(3, accu.size());
+        assertTrue(accu.iterator().hasNext());
 
         accu.add("4");
         accu.add("5");
 
-        assertEquals(2, accu.size());
+        assertEquals(5, accu.size());
 
-        assertEquals("4", accu.get(0));
-        assertEquals("5", accu.get(1));
-        assertOutOfBonds(accu, 2);
+        assertEquals("4", accu.get(3));
+        assertEquals("5", accu.get(4));
+        assertOutOfBonds(accu, 5);
 
         Iterator<String> iter = accu.iterator();
         assertTrue(iter.hasNext());
-        assertEquals("4", iter.next());
-        assertEquals("5", iter.next());
-        assertFalse(iter.hasNext());
+        assertEquals("1", iter.next());
+        assertNull(iter.next());
+        assertTrue(iter.hasNext());

Review comment:
       We could also check the value of the next value:
   ```suggestion
           assertTrue(iter.hasNext());
           assertEquals("3", iter.next());
   ```

##########
File path: src/java/org/apache/cassandra/utils/concurrent/Accumulator.java
##########
@@ -138,14 +137,12 @@ public E get(int i)
     }
 
     /**
-     * Removes all of the elements from this accumulator.
+     * Removes element at the speficied index from this accumulator.
      *
      * This method is not thread-safe when used concurrently with {@link #add(Object)}.
      */
-    public void clearUnsafe()
+    public void clearUnsafe(int i)
     {
-        nextIndexUpdater.set(this, 0);
-        presentCountUpdater.set(this, 0);
-        Arrays.fill(values, null);
+        values[i] = null;

Review comment:
       I'd say this is safe with it's current usage

##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -723,6 +725,10 @@ public UnfilteredPartitionIterator moreContents()
             ColumnFamilyStore.metricsFor(command.metadata().cfId).shortReadProtectionRequests.mark();
             Tracing.trace("Requesting {} extra rows from {} for short read protection", toQuery, source);
 
+            // If we've arrived here, all responses have been consumed, and we're about to request more. Before that
+            // happens, clear the accumulator and allow garbage collection to free the resources they used.
+            preFetchCallback.run();

Review comment:
       Being the comment so specific about what the callback does, perhaps we could also be equally specific when naming it, perhaps something like `clearResources` instead of `preFetchCallback`. WDYT?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448680577



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       As always, I'm open to suggestions around naming and how I've described this.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r450412452



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -100,7 +110,9 @@
     ReplicaFilteringProtection(Keyspace keyspace,
                                ReadCommand command,
                                ConsistencyLevel consistency,
-                               InetAddress[] sources)
+                               InetAddress[] sources,
+                               int cachedRowsWarnThreshold,
+                               int cachedRowsFailThreshold)

Review comment:
       Note that looking at the client warnings implicitly checks that RFP has occurred.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458016410



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -148,6 +148,10 @@ private void testMissedUpdates(String tableName, int warnThreshold, int failThre
         
         // We should have made 3 row "completion" requests.
         assertEquals(ROWS, protectionQueryCount(cluster.get(1), tableName));
+
+        // In all cases above, the queries should be caching 1 row per partition, but 6 for the 

Review comment:
       Maybe we could clarify that it's 1 row per partition and replica




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] jrwest commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
jrwest commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r460585161



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -150,46 +150,46 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
     {
         // Protecting against inconsistent replica filtering (some replica returning a row that is outdated but that
         // wouldn't be removed by normal reconciliation because up-to-date replica have filtered the up-to-date version
-        // of that row) works in 3 steps:
-        //   1) we read the full response just to collect rows that may be outdated (the ones we got from some
-        //      replica but didn't got any response for other; it could be those other replica have filtered a more
-        //      up-to-date result). In doing so, we do not count any of such "potentially outdated" row towards the
-        //      query limit. This simulate the worst case scenario where all those "potentially outdated" rows are
-        //      indeed outdated, and thus make sure we are guaranteed to read enough results (thanks to short read
-        //      protection).
-        //   2) we query all the replica/rows we need to rule out whether those "potentially outdated" rows are outdated
-        //      or not.
-        //   3) we re-read cached copies of each replica response using the "normal" read path merge with read-repair,
-        //      but where for each replica we use their original response _plus_ the additional rows queried in the
-        //      previous step (and apply the command#rowFilter() on the full result). Since the first phase has
-        //      pessimistically collected enough results for the case where all potentially outdated results are indeed
-        //      outdated, we shouldn't need further short-read protection requests during this phase.
+        // of that row) involves 3 main elements:
+        //   1) We combine short-read protection and a merge listener that identifies potentially "out-of-date"
+        //      rows to create an iterator that is guaranteed to produce enough valid row results to satisfy the query 
+        //      limit if enough actually exist. A row is considered out-of-date if its merged form is non-empty and we 
+        //      receive not response from at least one replica. In this case, it is possible that filtering at the
+        //      "silent" replica has produced a more up-to-date result.
+        //   2) This iterator is passed to the standard resolution process with read-repair, but is first wrapped in a 
+        //      response provider that lazily "completes" potentially out-of-date rows by directly querying them on the
+        //      replicas that were previously silent. As this iterator is consumed, it caches valid data for potentially
+        //      out-of-date rows, and this cached data is merged with the fetched data as rows are requested. If there
+        //      is no replica divergence, only rows in the partition being evalutated will be cached (then released
+        //      when the partition is consumed).
+        //   3) After a "complete" row is materialized, it must pass the row filter supplied by the original query 
+        //      before it counts against the limit.
 
         // We could get more responses while this method runs, which is ok (we're happy to ignore any response not here
         // at the beginning of this method), so grab the response count once and use that through the method.
         int count = responses.size();
         // We need separate contexts, as each context has his own counter
         ResolveContext firstPhaseContext = new ResolveContext(count);
         ResolveContext secondPhaseContext = new ResolveContext(count);
-        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace, command, consistency, firstPhaseContext.sources);
+
+        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace,
+                                                                        command,
+                                                                        consistency,
+                                                                        firstPhaseContext.sources,
+                                                                        DatabaseDescriptor.getCachedReplicaRowsWarnThreshold(),
+                                                                        DatabaseDescriptor.getCachedReplicaRowsFailThreshold());
+
         PartitionIterator firstPhasePartitions = resolveInternal(firstPhaseContext,
                                                                  rfp.mergeController(),
                                                                  i -> shortReadProtectedResponse(i, firstPhaseContext),
                                                                  UnaryOperator.identity());
-
-        // Consume the first phase partitions to populate the replica filtering protection with both those materialized
-        // partitions and the primary keys to be fetched.
-        PartitionIterators.consume(firstPhasePartitions);
-        firstPhasePartitions.close();
-
-        // After reading the entire query results the protection helper should have cached all the partitions so we can
-        // clear the responses accumulator for the sake of memory usage, given that the second phase might take long if
-        // it needs to query replicas.
-        responses.clearUnsafe();
-
-        return resolveWithReadRepair(secondPhaseContext,
-                                     rfp::queryProtectedPartitions,
-                                     results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        PartitionIterator completedPartitions = resolveWithReadRepair(secondPhaseContext,
+                                                                      i -> rfp.queryProtectedPartitions(firstPhasePartitions, i),
+                                                                      results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        // Ensure that the RFP instance has a chance to record metrics when the iterator closes.

Review comment:
       I would have assumed initially that it was also because "we should close all iterators we open" but the additional comment was a good reminder that its important not just for convention but because actions are taken in close. 

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       Since we have the opportunity to revisit this part of the code is it worth changing this to an array instead of a list? These objects are per-request, so reducing overhead is beneficial and there are no operations on this field that can't be performed as an array. 

##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -218,7 +218,7 @@ private PartitionIterator resolveInternal(ResolveContext context,
 
         UnfilteredPartitionIterator merged = UnfilteredPartitionIterators.merge(results, command.nowInSec(), mergeListener);
         FilteredPartitions filtered =
-        FilteredPartitions.filter(merged, new Filter(command.nowInSec(), command.metadata().enforceStrictLiveness()));
+            FilteredPartitions.filter(merged, new Filter(command.nowInSec(), command.metadata().enforceStrictLiveness()));

Review comment:
       Really minor nit: instead of starting this statement on the next line break up arguments over multiple lines, per https://cassandra.apache.org/doc/latest/development/code_style.html#multiline-statements. 

##########
File path: test/distributed/org/apache/cassandra/distributed/impl/RowUtil.java
##########
@@ -41,7 +42,11 @@ public static SimpleQueryResult toQueryResult(ResultMessage res)
             ResultMessage.Rows rows = (ResultMessage.Rows) res;
             String[] names = getColumnNames(rows.result.metadata.names);
             Object[][] results = RowUtil.toObjects(rows);
-            return new SimpleQueryResult(names, results);
+            
+            // Warnings may be null here, due to ClientWarn#getWarnings() handing of empty warning lists.

Review comment:
       minor typo: "handling"

##########
File path: src/java/org/apache/cassandra/service/StorageService.java
##########
@@ -4811,6 +4811,26 @@ public void setTombstoneFailureThreshold(int threshold)
         DatabaseDescriptor.setTombstoneFailureThreshold(threshold);
     }
 
+    public int getCachedReplicaRowsWarnThreshold()
+    {
+        return DatabaseDescriptor.getCachedReplicaRowsWarnThreshold();
+    }
+
+    public void setCachedReplicaRowsWarnThreshold(int threshold)

Review comment:
       Minor nit: we don't do this by convention but it can be beneficial for later auditing to log the setting of these values via JMX. There are some other examples in StorageService. 

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,124 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
-
-            PartitionBuilder[] builders = new PartitionBuilder[sources.length];
-
-            for (int i = 0; i < sources.length; i++)
-                builders[i] = new PartitionBuilder(partitionKey, columns(versions), stats(versions));
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));
+            }
 
-            return new UnfilteredRowIterators.MergeListener()
+            @Override
+            public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List<UnfilteredRowIterator> versions)
             {
-                @Override
-                public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                PartitionBuilder[] builders = new PartitionBuilder[sources.length];
+                PartitionColumns columns = columns(versions);
+                EncodingStats stats = EncodingStats.merge(versions, NULL_TO_NO_STATS);
+                
+                for (int i = 0; i < sources.length; i++)
+                    builders[i] = new PartitionBuilder(partitionKey, sources[i], columns, stats);
+
+                return new UnfilteredRowIterators.MergeListener()
                 {
-                    // cache the deletion time versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].setDeletionTime(versions[i]);
-                }
+                    @Override
+                    public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                    {
+                        // cache the deletion time versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].setDeletionTime(versions[i]);
+                    }
 
-                @Override
-                public Row onMergedRows(Row merged, Row[] versions)
-                {
-                    // cache the row versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].addRow(versions[i]);
+                    @Override
+                    public Row onMergedRows(Row merged, Row[] versions)
+                    {
+                        // cache the row versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].addRow(versions[i]);
 
-                    if (merged.isEmpty())
-                        return merged;
+                        if (merged.isEmpty())
+                            return merged;
 
-                    boolean isPotentiallyOutdated = false;
-                    boolean isStatic = merged.isStatic();
-                    for (int i = 0; i < versions.length; i++)
-                    {
-                        Row version = versions[i];
-                        if (version == null || (isStatic && version.isEmpty()))
+                        boolean isPotentiallyOutdated = false;
+                        boolean isStatic = merged.isStatic();
+                        for (int i = 0; i < versions.length; i++)
                         {
-                            isPotentiallyOutdated = true;
-                            BTreeSet.Builder<Clustering> toFetch = getOrCreateToFetch(i, partitionKey);
-                            // Note that for static, we shouldn't add the clustering to the clustering set (the
-                            // ClusteringIndexNamesFilter we'll build from this later does not expect it), but the fact
-                            // we created a builder in the first place will act as a marker that the static row must be
-                            // fetched, even if no other rows are added for this partition.
-                            if (!isStatic)
-                                toFetch.add(merged.clustering());
+                            Row version = versions[i];
+                            if (version == null || (isStatic && version.isEmpty()))
+                            {
+                                isPotentiallyOutdated = true;
+                                builders[i].addToFetch(merged);
+                            }
                         }
-                    }
 
-                    // If the row is potentially outdated (because some replica didn't send anything and so it _may_ be
-                    // an outdated result that is only present because other replica have filtered the up-to-date result
-                    // out), then we skip the row. In other words, the results of the initial merging of results by this
-                    // protection assume the worst case scenario where every row that might be outdated actually is.
-                    // This ensures that during this first phase (collecting additional row to fetch) we are guaranteed
-                    // to look at enough data to ultimately fulfill the query limit.
-                    return isPotentiallyOutdated ? null : merged;
-                }
+                        // If the row is potentially outdated (because some replica didn't send anything and so it _may_ be
+                        // an outdated result that is only present because other replica have filtered the up-to-date result
+                        // out), then we skip the row. In other words, the results of the initial merging of results by this
+                        // protection assume the worst case scenario where every row that might be outdated actually is.
+                        // This ensures that during this first phase (collecting additional row to fetch) we are guaranteed
+                        // to look at enough data to ultimately fulfill the query limit.
+                        return isPotentiallyOutdated ? null : merged;
+                    }
 
-                @Override
-                public void onMergedRangeTombstoneMarkers(RangeTombstoneMarker merged, RangeTombstoneMarker[] versions)
-                {
-                    // cache the marker versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].addRangeTombstoneMarker(versions[i]);
-                }
+                    @Override
+                    public void onMergedRangeTombstoneMarkers(RangeTombstoneMarker merged, RangeTombstoneMarker[] versions)
+                    {
+                        // cache the marker versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].addRangeTombstoneMarker(versions[i]);
+                    }
 
-                @Override
-                public void close()
-                {
-                    for (int i = 0; i < sources.length; i++)
-                        originalPartitions.get(i).add(builders[i]);
-                }
-            };
+                    @Override
+                    public void close()
+                    {
+                        for (int i = 0; i < sources.length; i++)
+                            originalPartitions.get(i).add(builders[i]);
+                    }
+                };
+            }
         };
     }
 
+    private void incrementCachedRows()
+    {
+        currentRowsCached++;
+        
+        if (currentRowsCached == cachedRowsFailThreshold + 1)
+        {
+            String message = String.format("Replica filtering protection has cached over %d rows during query %s. " +
+                                           "(See 'cached_replica_rows_fail_threshold' in cassandra.yaml.)",
+                                           cachedRowsFailThreshold, command.toCQLString());
+
+            oneMinuteLogger.error(message);

Review comment:
       Wondering about suppressing failure messages using the `NoSpamLogger`. Thinking about cases where enough requests hit the warning threshold but only a few hit the error threshold. Do we want warnings to suppress those errors? 

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       Thats what I get for not trying it first myself. Apologies for not thinking of that. Since I don't evidence showing this to be a problem, I'm fine leaving it as is. 

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       Thats what I get for not trying it first myself. Apologies for not thinking of that. Since I don't have evidence showing this to be a problem, I'm fine leaving it as is. 




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449542415



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(fullTableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // TODO: These should be able to use ICoordinator#executeWithResult() once CASSANDRA-15920 is resolved.
+        Object[] oldResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "old", ROWS));
+        Object[][] oldRows = (Object[][]) oldResponse[0];
+        assertRows(oldRows);
+        @SuppressWarnings("unchecked") List<String> oldWarnings = (List<String>) oldResponse[1];
+        assertEquals(shouldWarn, oldWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));
+
+        Object[] newResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "new", ROWS));
+        Object[][] newRows = (Object[][]) newResponse[0];
+        assertRows(newRows, row(1, "new"), row(0, "new"), row(2, "new"));
+        @SuppressWarnings("unchecked") List<String> newWarnings = (List<String>) newResponse[1];
+        assertEquals(shouldWarn, newWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));

Review comment:
       Could we check that it only warns once per query?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458022634



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -178,8 +185,9 @@ private void testMissedUpdates(String tableName, int warnThreshold, int failThre
         // We sould have made 3 more row "completion" requests.
         assertEquals(ROWS * 2, protectionQueryCount(cluster.get(1), tableName));
 
-        // In all cases above, the queries should be caching 1 row per partition.
-        assertEquals(1L, maxRowsCachedPerPartition(cluster.get(1), tableName));
+        // In all cases above, the queries should be caching 1 row per partition, but 6 for the 
+        // whole query, given every row is potentially stale.
+        assertEquals(6L, maxRowsCachedPerQuery(cluster.get(1), tableName));

Review comment:
       We could also verify how many times we update the metric, something like:
   ```java
   assertEquals(4L, countRowsCachedPerQuery(cluster.get(1), tableName));
   (...)
   private long countRowsCachedPerQuery(IInvokableInstance instance, String tableName)
   {
       return instance.callOnInstance(() -> Keyspace.open(KEYSPACE)
                                                    .getColumnFamilyStore(tableName)
                                            .metric.rfpRowsCachedPerQuery.getCount());
   }
   ```




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449528815



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       I really like the nested properties approach. Adding the `replica_filtering_protection` section gives context to the properties and avoids any confusion with row caches, materialized views, etc., while also keeping the name length short.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449188343



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       At minimum, we'll need to explain that this is scoped to a (single page of) a query.
   
   I'll throw one more crazy idea (probably overkill) out there before I proceed with `cached_filtered_rows_warn_threshold` and a clear description of its per-query-ness: a namespace.
   
   Example: 
   
   ```
   replica_filtering_protection:
       cached_rows_warn_threshold: 1000
       cached_rows_fail_threshold: 16000
   ```




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458010659



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -340,7 +341,8 @@ public CFMetaData metadata()
             @Override
             public void close()
             {
-                // nothing to do here
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.

Review comment:
       I think this records the max cache size once per replica, when we want a single metric registry per query.
   
   I would say that the best place to put the metric saving is the `close` method of the `UnfilteredPartitionIterators.MergeListener` returned by `RFP#mergeController`. However, debug shows that that method is called before creating, using and closing the row merge listener, which is something that I don't understand.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r450787953



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(fullTableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // TODO: These should be able to use ICoordinator#executeWithResult() once CASSANDRA-15920 is resolved.
+        Object[] oldResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "old", ROWS));
+        Object[][] oldRows = (Object[][]) oldResponse[0];
+        assertRows(oldRows);
+        @SuppressWarnings("unchecked") List<String> oldWarnings = (List<String>) oldResponse[1];
+        assertEquals(shouldWarn, oldWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));
+
+        Object[] newResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "new", ROWS));
+        Object[][] newRows = (Object[][]) newResponse[0];
+        assertRows(newRows, row(1, "new"), row(0, "new"), row(2, "new"));
+        @SuppressWarnings("unchecked") List<String> newWarnings = (List<String>) newResponse[1];
+        assertEquals(shouldWarn, newWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));

Review comment:
       It seems we are missing this nit in the last commit.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r452982321



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,25 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;
+
+                        if (rowsCached == cachedRowsFailThreshold + 1)

Review comment:
       TODO: Both the warning and failure cases should be visible via tracing.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r453653370



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -132,34 +147,33 @@
      */
     UnfilteredPartitionIterator queryProtectedPartitions(int source)
     {
-        UnfilteredPartitionIterator original = makeIterator(originalPartitions.get(source));
-        SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>> toFetch = rowsToFetch.get(source);
+        UnfilteredPartitionIterator original = makeIterator(originalPartitions.set(source, null));
+        SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>> toFetch = rowsToFetch.set(source, null);
 
         if (toFetch.isEmpty())
             return original;
 
-        // TODO: this would be more efficient if we had multi-key queries internally
+        // TODO: This would be more efficient if we had multi-key queries internally (see CASSANDRA-15910)
         List<UnfilteredPartitionIterator> fetched = toFetch.keySet()
                                                            .stream()
-                                                           .map(k -> querySourceOnKey(source, k))
+                                                           .map(k -> querySourceOnKey(source, k, toFetch.get(k)))
                                                            .collect(Collectors.toList());

Review comment:
       @maedhroz Given that we don't have multi-key queries, we can easily reduce the number RFP queries in some cases if we use an iterator instead of building a list:
   ```java
   Iterator<UnfilteredPartitionIterator> fetched = toFetch.keySet()
                                                          .stream()
                                                          .map(k -> querySourceOnKey(source, k, toFetch.get(k)))
                                                          .iterator();
   ```
   For example, the following test would go down from four RFP queries to just one, although still needing four SRP requests:
   ```python
   self._prepare_cluster(
       create_table="CREATE TABLE t (k int PRIMARY KEY, v text)",
       create_index="CREATE INDEX ON t(v)",
       both_nodes=["INSERT INTO t (k, v) VALUES (1, 'old')"],
       only_node1=["UPDATE t SET v = 'new' WHERE k = 1",
                   "UPDATE t SET v = 'new' WHERE k = 2",
                   "UPDATE t SET v = 'new' WHERE k = 3",
                   "UPDATE t SET v = 'new' WHERE k = 4"])
   self._assert_one("SELECT * FROM t WHERE v = 'new' LIMIT 1", row=[1, 'new'])
   ```
   Changing to an iterator would also require modifying the method `UnfilteredPartitionIterator.concat` to accept an iterator instead of a list:
   ```java
   public static UnfilteredPartitionIterator concat(final Iterator<UnfilteredPartitionIterator> iterators)
   {
       assert iterators.hasNext();
       UnfilteredPartitionIterator first = iterators.next();
       if (!first.hasNext())
           return first;
       class Extend implements MorePartitions<UnfilteredPartitionIterator>
       {
           public UnfilteredPartitionIterator moreContents()
           {
               return iterators.hasNext() ? iterators.next() : null;
           }
       }
       return MorePartitions.extend(first, new Extend());
   }
   ```




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] belliottsmith commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448189804



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -48,8 +48,10 @@
     private static final boolean DROP_OVERSIZED_READ_REPAIR_MUTATIONS =
         Boolean.getBoolean("cassandra.drop_oversized_readrepair_mutations");
 
+    private static final UnaryOperator<PartitionIterator> IDENTITY = UnaryOperator.identity();

Review comment:
       👍  - just happened to notice this when I skimmed the patch description, so thought I'd mention it




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r450783936



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(fullTableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // TODO: These should be able to use ICoordinator#executeWithResult() once CASSANDRA-15920 is resolved.
+        Object[] oldResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "old", ROWS));
+        Object[][] oldRows = (Object[][]) oldResponse[0];
+        assertRows(oldRows);
+        @SuppressWarnings("unchecked") List<String> oldWarnings = (List<String>) oldResponse[1];
+        assertEquals(shouldWarn, oldWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));

Review comment:
       Sure, we can rely on the thresholds. For `testMissedUpdatesBelowCachingWarnThreshold` in isolation we don't have that implicit verification, but we know that the test scenario is properly created because the other tests work.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449178443



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,21 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;
+
+                        if (rowsCached == cachedRowsFailThreshold)
+                        {
+                            throw new TooManyCachedRowsException(cachedRowsFailThreshold, command);
+                        }
+                        else if (rowsCached == cachedRowsWarnThreshold)
+                        {
+                            logger.warn("Replica filtering protection has cached {} rows during query {}. " +

Review comment:
       Agreed. I'll try to graft that in here...




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449139164



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       > perhaps we should use something like 1000, WDYT?
   
   Yup. No particular attachment to powers of two.
   
   > so if we are reading from two on-sync replicas the threshold will start at 512 results, allowing a max limit/page size of 512 rows, won't it
   
   Correct.
   
   > Perhaps we should count the number of merged materialized/cached rows, ignoring the replicas, so the thresholds can be more easily related to the query limit.
   
   I agree that it would be a bit easier to relate that to the limit, but my worry is that it would make the same threshold values behave differently with different read consistency levels. That also brings up another question, which is whether, even in this scheme, we should count the merged row as well.
   
   If we ultimately don't need this to be directly relatable to the limit, and I propose we don't (see `tombstone_failure_threshold`), it may also make sense to go down to the level of cells. It doesn't actually look much more expensive to get the number of cells from `Row#columnCount()`, but it adds some complexity (and doesn't look like it includes the clustering).




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r453154216



##########
File path: src/java/org/apache/cassandra/exceptions/TooManyCachedRowsException.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ReadCommand;
+
+import static org.apache.cassandra.exceptions.ExceptionCode.SERVER_ERROR;
+
+/**
+ * Exception thrown when {@link org.apache.cassandra.service.ReplicaFilteringProtection} caches
+ * the configured threshold number of row results from participating replicas.
+ */
+public class TooManyCachedRowsException extends RequestExecutionException

Review comment:
       @jasonstack pointed out to me that #2 actually might make the most sense. `OverloadedException` is actually used to indicate too many in-flight (on-heap) hints and can fail a write operation. This patch is doing something very similar, just for reads.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458023863



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.exceptions.OverloadedException;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.config.ReplicaFilteringProtectionOptions.DEFAULT_FAIL_THRESHOLD;
+import static org.apache.cassandra.config.ReplicaFilteringProtectionOptions.DEFAULT_WARN_THRESHOLD;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+
+        // Make sure we start w/ the correct defaults:
+        cluster.get(1).runOnInstance(() -> assertEquals(DEFAULT_WARN_THRESHOLD, StorageService.instance.getCachedReplicaRowsWarnThreshold()));
+        cluster.get(1).runOnInstance(() -> assertEquals(DEFAULT_FAIL_THRESHOLD, StorageService.instance.getCachedReplicaRowsFailThreshold()));
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(tableName, REPLICAS * ROWS, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(tableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(tableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(tableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), OverloadedException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String tableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        String fullTableName = KEYSPACE + '.' + tableName;
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date.
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // The replica that missed the results creates a mismatch at every row, and we threfore cache a version

Review comment:
       ```suggestion
           // The replica that missed the results creates a mismatch at every row, and we therefore cache a version
   ```




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r450897888



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(fullTableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // TODO: These should be able to use ICoordinator#executeWithResult() once CASSANDRA-15920 is resolved.
+        Object[] oldResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "old", ROWS));
+        Object[][] oldRows = (Object[][]) oldResponse[0];
+        assertRows(oldRows);
+        @SuppressWarnings("unchecked") List<String> oldWarnings = (List<String>) oldResponse[1];
+        assertEquals(shouldWarn, oldWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));
+
+        Object[] newResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "new", ROWS));
+        Object[][] newRows = (Object[][]) newResponse[0];
+        assertRows(newRows, row(1, "new"), row(0, "new"), row(2, "new"));
+        @SuppressWarnings("unchecked") List<String> newWarnings = (List<String>) newResponse[1];
+        assertEquals(shouldWarn, newWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));

Review comment:
       @adelapena Forgot to push my last local commit ;)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448967271



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,21 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;
+
+                        if (rowsCached == cachedRowsFailThreshold)

Review comment:
       I wonder if it would be better to use `>` instead of `==`. If we used `>` and we were counting merged materialized/cached rows instead of per replica, we could guarantee that, without conflicts, a threshold of N rows allows a max limit of N rows. Also, I think that `tombstone_failure_threshold` uses `>`, so it will be more consistent with that meaning of threshold value. WDYT?

##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       I don't know if using powers of two gives us any advantage here, and I'm a bit afraid it could make users thing that this is somehow measuring memory sizes. Given that this seems more related to the query limit, perhaps we should use something like 1000, WDYT?
   
   Also, that is the number of rows in all replicas, so we are reading from two on-sync replicas the threshold will start at 512 results, won't it? Perhaps we should count the number of merged materialized/cached rows, ignoring the replicas, so the thresholds can be more easily related to the query limit.

##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet

Review comment:
       We could explicitly say that the threshold are for the number of rows. Also, that this number is per query (or per query page), and perhaps that they only live for the duration of the query, especially if we used "cached" instead of "materialized".

##########
File path: src/java/org/apache/cassandra/db/rows/EncodingStats.java
##########
@@ -107,6 +108,29 @@ public EncodingStats mergeWith(EncodingStats that)
         return new EncodingStats(minTimestamp, minDelTime, minTTL);
     }
 
+    /**
+     * Merge one or more EncodingStats, that are lazily materialized from some list of arbitrary type by the provided function
+     */
+    public static <V, F extends Function<V, EncodingStats>> EncodingStats merge(List<V> values, F function)
+    {
+        if (values.size() == 1)
+            return function.apply(values.get(0));
+
+        Collector collector = new Collector();
+        for (int i=0, isize=values.size(); i<isize; i++)

Review comment:
       Nit:
   ```suggestion
           for (int i = 0, isize = values.size(); i < isize; i++)
   ```
   Also, perhaps `size` or `iSize` instead of `isize`?

##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       Nice description. I think I'd prefer "materialized" rows rather than "cached" rows, since the later could be a bit misleading and make the unaware reader think that this is doing something different to what it actually does. WDYT?
   
   Also we could include a reference to the RFP ticket (CASSANDRA-8272) for further details, like the ones we have for another tickets in this file.

##########
File path: src/java/org/apache/cassandra/db/rows/EncodingStats.java
##########
@@ -107,6 +108,29 @@ public EncodingStats mergeWith(EncodingStats that)
         return new EncodingStats(minTimestamp, minDelTime, minTTL);
     }
 
+    /**
+     * Merge one or more EncodingStats, that are lazily materialized from some list of arbitrary type by the provided function
+     */
+    public static <V, F extends Function<V, EncodingStats>> EncodingStats merge(List<V> values, F function)
+    {
+        if (values.size() == 1)
+            return function.apply(values.get(0));
+
+        Collector collector = new Collector();
+        for (int i=0, isize=values.size(); i<isize; i++)
+        {
+            V v = values.get(i);
+            EncodingStats stats = function.apply(v);
+            if (stats.minTimestamp != TIMESTAMP_EPOCH)
+                collector.updateTimestamp(stats.minTimestamp);
+            if(stats.minLocalDeletionTime != DELETION_TIME_EPOCH)
+                collector.updateLocalDeletionTime(stats.minLocalDeletionTime);
+            if(stats.minTTL != TTL_EPOCH)

Review comment:
       ```suggestion
               if (stats.minTTL != TTL_EPOCH)
   ```

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,21 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;

Review comment:
       Are these limited by `tombstone_failure_threshold`?

##########
File path: src/java/org/apache/cassandra/db/rows/EncodingStats.java
##########
@@ -107,6 +108,29 @@ public EncodingStats mergeWith(EncodingStats that)
         return new EncodingStats(minTimestamp, minDelTime, minTTL);
     }
 
+    /**
+     * Merge one or more EncodingStats, that are lazily materialized from some list of arbitrary type by the provided function
+     */
+    public static <V, F extends Function<V, EncodingStats>> EncodingStats merge(List<V> values, F function)
+    {
+        if (values.size() == 1)
+            return function.apply(values.get(0));
+
+        Collector collector = new Collector();
+        for (int i=0, isize=values.size(); i<isize; i++)
+        {
+            V v = values.get(i);
+            EncodingStats stats = function.apply(v);
+            if (stats.minTimestamp != TIMESTAMP_EPOCH)
+                collector.updateTimestamp(stats.minTimestamp);
+            if(stats.minLocalDeletionTime != DELETION_TIME_EPOCH)

Review comment:
       ```suggestion
               if (stats.minLocalDeletionTime != DELETION_TIME_EPOCH)
   ```

##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -171,7 +171,14 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
         // We need separate contexts, as each context has his own counter
         ResolveContext firstPhaseContext = new ResolveContext(count);
         ResolveContext secondPhaseContext = new ResolveContext(count);
-        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace, command, consistency, firstPhaseContext.sources);
+
+        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace,
+                                                                        command,
+                                                                        consistency,
+                                                                        firstPhaseContext.sources,
+                                                                        DatabaseDescriptor.getCachedReplicaRowsWarnThreshold(),
+                                                                        DatabaseDescriptor.getCachedReplicaRowsFailThreshold());
+

Review comment:
       Have you intentionally withdrew the caching of `UnaryOperator.identity()` below?

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -132,34 +147,33 @@
      */
     UnfilteredPartitionIterator queryProtectedPartitions(int source)
     {
-        UnfilteredPartitionIterator original = makeIterator(originalPartitions.get(source));
-        SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>> toFetch = rowsToFetch.get(source);
+        UnfilteredPartitionIterator original = makeIterator(originalPartitions.set(source, null));
+        SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>> toFetch = rowsToFetch.set(source, null);
 
         if (toFetch.isEmpty())
             return original;
 
-        // TODO: this would be more efficient if we had multi-key queries internally
+        // TODO: This would be more efficient if we had multi-key queries internally (see CASSANDRA-15910)

Review comment:
       +1




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] smiklosovic closed pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
smiklosovic closed pull request #659:
URL: https://github.com/apache/cassandra/pull/659


   


-- 
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


[GitHub] [cassandra] belliottsmith commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448037610



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -48,8 +48,10 @@
     private static final boolean DROP_OVERSIZED_READ_REPAIR_MUTATIONS =
         Boolean.getBoolean("cassandra.drop_oversized_readrepair_mutations");
 
+    private static final UnaryOperator<PartitionIterator> IDENTITY = UnaryOperator.identity();

Review comment:
       Are you sure this is being allocated when you invoke it? The lambda defined at source should only be instantiated once, so this looks to be an unnecessary obfuscation.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448682273



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       My intuition from a few years of index query testing is that we shouldn't go much higher on the warn threshold here. Both of these are subject to change though, and we should do some basic stress testing before we commit.
   
   One potentially interesting interaction here is that it might be possible for the failure threshold to be set below the query limit. The default value is high enough here that it probably won't affect very many people at all, but it's worth mentioning.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r461517069



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,124 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
-
-            PartitionBuilder[] builders = new PartitionBuilder[sources.length];
-
-            for (int i = 0; i < sources.length; i++)
-                builders[i] = new PartitionBuilder(partitionKey, columns(versions), stats(versions));
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));
+            }
 
-            return new UnfilteredRowIterators.MergeListener()
+            @Override
+            public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List<UnfilteredRowIterator> versions)
             {
-                @Override
-                public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                PartitionBuilder[] builders = new PartitionBuilder[sources.length];
+                PartitionColumns columns = columns(versions);
+                EncodingStats stats = EncodingStats.merge(versions, NULL_TO_NO_STATS);
+                
+                for (int i = 0; i < sources.length; i++)
+                    builders[i] = new PartitionBuilder(partitionKey, sources[i], columns, stats);
+
+                return new UnfilteredRowIterators.MergeListener()
                 {
-                    // cache the deletion time versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].setDeletionTime(versions[i]);
-                }
+                    @Override
+                    public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                    {
+                        // cache the deletion time versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].setDeletionTime(versions[i]);
+                    }
 
-                @Override
-                public Row onMergedRows(Row merged, Row[] versions)
-                {
-                    // cache the row versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].addRow(versions[i]);
+                    @Override
+                    public Row onMergedRows(Row merged, Row[] versions)
+                    {
+                        // cache the row versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].addRow(versions[i]);
 
-                    if (merged.isEmpty())
-                        return merged;
+                        if (merged.isEmpty())
+                            return merged;
 
-                    boolean isPotentiallyOutdated = false;
-                    boolean isStatic = merged.isStatic();
-                    for (int i = 0; i < versions.length; i++)
-                    {
-                        Row version = versions[i];
-                        if (version == null || (isStatic && version.isEmpty()))
+                        boolean isPotentiallyOutdated = false;
+                        boolean isStatic = merged.isStatic();
+                        for (int i = 0; i < versions.length; i++)
                         {
-                            isPotentiallyOutdated = true;
-                            BTreeSet.Builder<Clustering> toFetch = getOrCreateToFetch(i, partitionKey);
-                            // Note that for static, we shouldn't add the clustering to the clustering set (the
-                            // ClusteringIndexNamesFilter we'll build from this later does not expect it), but the fact
-                            // we created a builder in the first place will act as a marker that the static row must be
-                            // fetched, even if no other rows are added for this partition.
-                            if (!isStatic)
-                                toFetch.add(merged.clustering());
+                            Row version = versions[i];
+                            if (version == null || (isStatic && version.isEmpty()))
+                            {
+                                isPotentiallyOutdated = true;
+                                builders[i].addToFetch(merged);
+                            }
                         }
-                    }
 
-                    // If the row is potentially outdated (because some replica didn't send anything and so it _may_ be
-                    // an outdated result that is only present because other replica have filtered the up-to-date result
-                    // out), then we skip the row. In other words, the results of the initial merging of results by this
-                    // protection assume the worst case scenario where every row that might be outdated actually is.
-                    // This ensures that during this first phase (collecting additional row to fetch) we are guaranteed
-                    // to look at enough data to ultimately fulfill the query limit.
-                    return isPotentiallyOutdated ? null : merged;
-                }
+                        // If the row is potentially outdated (because some replica didn't send anything and so it _may_ be
+                        // an outdated result that is only present because other replica have filtered the up-to-date result
+                        // out), then we skip the row. In other words, the results of the initial merging of results by this
+                        // protection assume the worst case scenario where every row that might be outdated actually is.
+                        // This ensures that during this first phase (collecting additional row to fetch) we are guaranteed
+                        // to look at enough data to ultimately fulfill the query limit.
+                        return isPotentiallyOutdated ? null : merged;
+                    }
 
-                @Override
-                public void onMergedRangeTombstoneMarkers(RangeTombstoneMarker merged, RangeTombstoneMarker[] versions)
-                {
-                    // cache the marker versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].addRangeTombstoneMarker(versions[i]);
-                }
+                    @Override
+                    public void onMergedRangeTombstoneMarkers(RangeTombstoneMarker merged, RangeTombstoneMarker[] versions)
+                    {
+                        // cache the marker versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].addRangeTombstoneMarker(versions[i]);
+                    }
 
-                @Override
-                public void close()
-                {
-                    for (int i = 0; i < sources.length; i++)
-                        originalPartitions.get(i).add(builders[i]);
-                }
-            };
+                    @Override
+                    public void close()
+                    {
+                        for (int i = 0; i < sources.length; i++)
+                            originalPartitions.get(i).add(builders[i]);
+                    }
+                };
+            }
         };
     }
 
+    private void incrementCachedRows()
+    {
+        currentRowsCached++;
+        
+        if (currentRowsCached == cachedRowsFailThreshold + 1)
+        {
+            String message = String.format("Replica filtering protection has cached over %d rows during query %s. " +
+                                           "(See 'cached_replica_rows_fail_threshold' in cassandra.yaml.)",
+                                           cachedRowsFailThreshold, command.toCQLString());
+
+            oneMinuteLogger.error(message);

Review comment:
       Makes sense to me

##########
File path: src/java/org/apache/cassandra/metrics/TableMetrics.java
##########
@@ -652,7 +662,8 @@ public Long getValue()
 
         readRepairRequests = createTableMeter("ReadRepairRequests");
         shortReadProtectionRequests = createTableMeter("ShortReadProtectionRequests");
-        replicaSideFilteringProtectionRequests = createTableMeter("ReplicaSideFilteringProtectionRequests");
+        replicaFilteringProtectionRequests = createTableMeter("ReplicaFilteringProtectionRequests");

Review comment:
       I think we never documented this metric, so I guess it's ok to change the name of the metric even though we have releases with the old name (3.0.21 and 3.11.7). I think that for 3.11 and trunk PRs we should add `ReplicaFilteringProtectionRequests` and `ReplicaFilteringProtectionRowsCachedPerQuery` to [`metrics.rst`](https://github.com/apache/cassandra/blob/cassandra-3.11/doc/source/operating/metrics.rst#table-metrics).




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449554758



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       The ratio sounds like a good idea for usability, since it allows to define how much overload are we willing to accept in a per-query basis. The downside is that it limits the capacity of administrators (who edit the yaml) to limit the damage that users can do if they decide to use a very high page size, I think. That would be ideal IMO if we had a way to also limit the page size (I think we haven't it).




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] belliottsmith commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449201020



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       honestly we should do this with more parameters - quick, muck things up on 15234 with this suggestion (actually I'm serious - this might clean up our config file a lot)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449086088



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE);

Review comment:
       Shouldn't we check that the warning is logged?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449178030



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ConsistencyLevel.ALL, i);
+        }
+
+        Object[][] initialRows =
+            cluster.coordinator(1).execute("SELECT * FROM " + fullTableName + " WHERE v = 'old' LIMIT ? ALLOW FILTERING",
+                                           ConsistencyLevel.ALL, ROWS);
+        assertRows(initialRows,
+                   row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        Object[][] emptyOldMatches =
+            cluster.coordinator(1).execute("SELECT * FROM " + fullTableName + " WHERE v = 'old' LIMIT ? ALLOW FILTERING",
+                                           ConsistencyLevel.ALL, ROWS);
+        assertRows(emptyOldMatches);
+
+        Object[][] newMatches =
+            cluster.coordinator(1).execute("SELECT * FROM " + fullTableName + " WHERE v = 'new' LIMIT ? ALLOW FILTERING",
+                                           ConsistencyLevel.ALL, ROWS);
+        assertRows(newMatches,

Review comment:
       They'll fail exactly the same way, so it's alway the first. I figured it would be okay to leave it this way, given we want to verify both for the non-failing cases. WDYT?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449203046



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()

Review comment:
       I've added a similar thing to the test for the fail 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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r450993845



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.config.ReplicaFilteringProtectionOptions.DEFAULT_FAIL_THRESHOLD;
+import static org.apache.cassandra.config.ReplicaFilteringProtectionOptions.DEFAULT_WARN_THRESHOLD;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+
+        // Make sure we start w/ the correct defaults:
+        cluster.get(1).runOnInstance(() -> assertEquals(DEFAULT_WARN_THRESHOLD, StorageService.instance.getCachedReplicaRowsWarnThreshold()));
+        cluster.get(1).runOnInstance(() -> assertEquals(DEFAULT_FAIL_THRESHOLD, StorageService.instance.getCachedReplicaRowsFailThreshold()));
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned

Review comment:
       Nit: we forgot to update this comment when we changed the threshold behaviour
   ```suggestion
           // The warning threshold provided is exactly the total number of rows returned
   ```




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458728397



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,122 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));
+            }
 
-            PartitionBuilder[] builders = new PartitionBuilder[sources.length];
+            public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List<UnfilteredRowIterator> versions)
+            {
+                PartitionBuilder[] builders = new PartitionBuilder[sources.length];
 
-            for (int i = 0; i < sources.length; i++)
-                builders[i] = new PartitionBuilder(partitionKey, columns(versions), stats(versions));
+                for (int i = 0; i < sources.length; i++)
+                    builders[i] = new PartitionBuilder(partitionKey, sources[i], columns(versions), EncodingStats.merge(versions, NULL_TO_NO_STATS));

Review comment:
       I think that both the `PartitionColumns` and the `EncodingStats` are immutable, we could create them right before the loop and reuse them for every version of the same partition:
   ```suggestion
                   PartitionColumns columns = columns(versions);
                   EncodingStats stats = EncodingStats.merge(versions, NULL_TO_NO_STATS);
                   for (int i = 0; i < sources.length; i++)
                       builders[i] = new PartitionBuilder(partitionKey, sources[i], columns, stats);
   ```

##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -150,46 +150,46 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
     {
         // Protecting against inconsistent replica filtering (some replica returning a row that is outdated but that
         // wouldn't be removed by normal reconciliation because up-to-date replica have filtered the up-to-date version
-        // of that row) works in 3 steps:
-        //   1) we read the full response just to collect rows that may be outdated (the ones we got from some
-        //      replica but didn't got any response for other; it could be those other replica have filtered a more
-        //      up-to-date result). In doing so, we do not count any of such "potentially outdated" row towards the
-        //      query limit. This simulate the worst case scenario where all those "potentially outdated" rows are
-        //      indeed outdated, and thus make sure we are guaranteed to read enough results (thanks to short read
-        //      protection).
-        //   2) we query all the replica/rows we need to rule out whether those "potentially outdated" rows are outdated
-        //      or not.
-        //   3) we re-read cached copies of each replica response using the "normal" read path merge with read-repair,
-        //      but where for each replica we use their original response _plus_ the additional rows queried in the
-        //      previous step (and apply the command#rowFilter() on the full result). Since the first phase has
-        //      pessimistically collected enough results for the case where all potentially outdated results are indeed
-        //      outdated, we shouldn't need further short-read protection requests during this phase.
+        // of that row) involves 3 main elements:
+        //   1) We combine short-read protection and a merge listener that identifies potentially "out-of-date"
+        //      rows to create an iterator that is guaranteed to produce enough valid row results to satisfy the query 
+        //      limit if enough actually exist. A row is considered out-of-date if its merged form is non-empty and we 
+        //      receive not response from at least one replica. In this case, it is possible that filtering at the
+        //      "silent" replica has produced a more up-to-date result.
+        //   2) This iterator is passed to the standard resolution process with read-repair, but is first wrapped in a 
+        //      response provider that lazily "completes" potentially out-of-date rows by directly querying them on the
+        //      replicas that were previously silent. As this iterator is consumed, it caches valid data for potentially
+        //      out-of-date rows, and this cached data is merged with the fetched data as rows are requested. If there
+        //      is no replica divergence, only rows in the partition being evalutated will be cached (then released
+        //      when the partition is consumed).
+        //   3) After a "complete" row is materialized, it must pass the row filter supplied by the original query 
+        //      before it counts against the limit.
 
         // We could get more responses while this method runs, which is ok (we're happy to ignore any response not here
         // at the beginning of this method), so grab the response count once and use that through the method.
         int count = responses.size();
         // We need separate contexts, as each context has his own counter
         ResolveContext firstPhaseContext = new ResolveContext(count);
         ResolveContext secondPhaseContext = new ResolveContext(count);
-        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace, command, consistency, firstPhaseContext.sources);
+
+        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace,
+                                                                        command,
+                                                                        consistency,
+                                                                        firstPhaseContext.sources,
+                                                                        DatabaseDescriptor.getCachedReplicaRowsWarnThreshold(),
+                                                                        DatabaseDescriptor.getCachedReplicaRowsFailThreshold());
+
         PartitionIterator firstPhasePartitions = resolveInternal(firstPhaseContext,
                                                                  rfp.mergeController(),
                                                                  i -> shortReadProtectedResponse(i, firstPhaseContext),
                                                                  UnaryOperator.identity());
-
-        // Consume the first phase partitions to populate the replica filtering protection with both those materialized
-        // partitions and the primary keys to be fetched.
-        PartitionIterators.consume(firstPhasePartitions);
-        firstPhasePartitions.close();
-
-        // After reading the entire query results the protection helper should have cached all the partitions so we can
-        // clear the responses accumulator for the sake of memory usage, given that the second phase might take long if
-        // it needs to query replicas.
-        responses.clearUnsafe();
-
-        return resolveWithReadRepair(secondPhaseContext,
-                                     rfp::queryProtectedPartitions,
-                                     results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        PartitionIterator completedPartitions = resolveWithReadRepair(secondPhaseContext,
+                                                                      i -> rfp.queryProtectedPartitions(firstPhasePartitions, i),
+                                                                      results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        // Ensure that the RFP instance has a chance to record metrics when the iterator closes.

Review comment:
       Not sure whether we need to be so specific about the metrics detail, we are just closing a `Closeable` iterator which is something that we should always do. I'm not against leaving the comment as it is, though.

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,122 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));
+            }
 
-            PartitionBuilder[] builders = new PartitionBuilder[sources.length];
+            public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List<UnfilteredRowIterator> versions)
+            {
+                PartitionBuilder[] builders = new PartitionBuilder[sources.length];
 
-            for (int i = 0; i < sources.length; i++)
-                builders[i] = new PartitionBuilder(partitionKey, columns(versions), stats(versions));
+                for (int i = 0; i < sources.length; i++)
+                    builders[i] = new PartitionBuilder(partitionKey, sources[i], columns(versions), EncodingStats.merge(versions, NULL_TO_NO_STATS));
 
-            return new UnfilteredRowIterators.MergeListener()
-            {
-                @Override
-                public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                return new UnfilteredRowIterators.MergeListener()
                 {
-                    // cache the deletion time versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].setDeletionTime(versions[i]);
-                }
+                    @Override
+                    public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                    {
+                        // cache the deletion time versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].setDeletionTime(versions[i]);
+                    }
 
-                @Override
-                public Row onMergedRows(Row merged, Row[] versions)
-                {
-                    // cache the row versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].addRow(versions[i]);
+                    @Override
+                    public Row onMergedRows(Row merged, Row[] versions)
+                    {
+                        // cache the row versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                        {
+                            builders[i].addRow(versions[i]);
+                            currentRowsCached++;
+                            checkCachedRowThresholds();

Review comment:
       @maedhroz Currently the listener increases the cached rows count while the builder decreases it. It's also the builder how adds and removes things from the cache. Also, the listener accesses directly to `partitionRowsCached` while the builder  does it through a method. We could easily move these two lines to `RFP.PartitionBuilder#addRow` and so make the row merge listener free from all cache management stuff. Also we could have a `RFP#increaseCachedRows()` method combining these two lines, which would be the counterpart of `RFP#releaseCachedRows(int)`. You can see [here](https://github.com/adelapena/cassandra/commit/38780dda5721839333cdcb79b5c0a54e2d0fc845) how it would look like. Feel free to ignore it if you don't like it.

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,122 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));

Review comment:
       I think it would be slightly less efficient but slightly easier to read and a bit less error prone if we just updated `maxRowsCached` when we increase `currentRowsCached`, so `maxRowsCached` is always updated. Not saying that we should do it, I'm not sure if doing so would worth it.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r455560930



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -860,6 +858,10 @@ public UnfilteredRowIterator moreContents()
                 ColumnFamilyStore.metricsFor(metadata.cfId).shortReadProtectionRequests.mark();
                 Tracing.trace("Requesting {} extra rows from {} for short read protection", lastQueried, source);
 
+                // If we've arrived here, all responses have been consumed, and we're about to request more. Before that
+                // happens, clear the accumulator and allow garbage collection to free the resources they used.
+                responses.clearUnsafe();

Review comment:
       Perhaps it would actually make more sense to just null out the accumulator entries as they're retrieved in `shortReadProtectedResponse()`.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r453754945



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -132,34 +147,33 @@
      */
     UnfilteredPartitionIterator queryProtectedPartitions(int source)
     {
-        UnfilteredPartitionIterator original = makeIterator(originalPartitions.get(source));
-        SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>> toFetch = rowsToFetch.get(source);
+        UnfilteredPartitionIterator original = makeIterator(originalPartitions.set(source, null));
+        SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>> toFetch = rowsToFetch.set(source, null);
 
         if (toFetch.isEmpty())
             return original;
 
-        // TODO: this would be more efficient if we had multi-key queries internally
+        // TODO: This would be more efficient if we had multi-key queries internally (see CASSANDRA-15910)
         List<UnfilteredPartitionIterator> fetched = toFetch.keySet()
                                                            .stream()
-                                                           .map(k -> querySourceOnKey(source, k))
+                                                           .map(k -> querySourceOnKey(source, k, toFetch.get(k)))
                                                            .collect(Collectors.toList());

Review comment:
       @adelapena Indeed. Let's be lazy.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] belliottsmith commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449181348



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       `temporary` rather than `cached` perhaps?
   
   I've not looked closely at the semantics, honestly, so just throwing it out there.  I often look in a thesaurus for inspiration, otherwise, when struggling!
   
   But yes, it's very unclear that this property relates to something entirely unrelated to materialised views




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449188343



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       I'll throw one more crazy idea (probably overkill) out there before I proceed with `cached_filtered_rows_warn_threshold` and a clear description of its per-query-ness: a namespace.
   
   Example: 
   
   ```
   replica_filtering_protection:
       cached_rows_warn_threshold: 1000
       cached_rows_fail_threshold: 16000
   ```
   
   It's about as descriptive as we can reasonably be...just the extra layer to deal with.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449188343



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       I'll throw one more crazy idea (probably overkill) out there before I proceed with `cached_filtered_rows_warn_threshold` and a clear description of its per-query-ness: a namespace.
   
   Example: 
   
   ```
   replica_filtering_protection:
       cached_rows_warn_threshold: 1000
       cached_rows_fail_threshold: 16000
   ```
   
   It's about as descriptive as we possibly can be...just the extra layer to deal with.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r455911721



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -596,14 +596,16 @@ else if (DROP_OVERSIZED_READ_REPAIR_MUTATIONS)
     }
 
     private UnfilteredPartitionIterator extendWithShortReadProtection(UnfilteredPartitionIterator partitions,
-                                                                      InetAddress source,
-                                                                      DataLimits.Counter mergedResultCounter)
+                                                                      ResolveContext context,
+                                                                      int i)
     {
         DataLimits.Counter singleResultCounter =
             command.limits().newCounter(command.nowInSec(), false, command.selectsFullPartition(), enforceStrictLiveness).onlyCount();
 
-        ShortReadPartitionsProtection protection =
-            new ShortReadPartitionsProtection(source, singleResultCounter, mergedResultCounter);
+        ShortReadPartitionsProtection protection = new ShortReadPartitionsProtection(context.sources[i],
+                                                                                     () -> responses.clearUnsafe(i),

Review comment:
       @adelapena So this clears the responses only from the relevant source, and when the entire payload is consumed. WDYT? (Another benefit of doing it this way is that I think it will apply cleanly to `trunk`, where the short read protection classes have been moved out of `DataResolver`.)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449188343



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       At minimum, we'll need to explain that this is scoped to a (single page of) a query.
   
   I'll throw one more crazy idea (probably overkill) out there before I proceed with `cached_filtered_rows_warn_threshold` and a clear description of its per-query-ness: a namespace.
   
   Example: 
   
   ```
   replica_filtering_protection:
       cached_rows_warn_threshold: 1000
       cached_rows_fail_threshold: 16000
   ```
   
   It's about as descriptive as we possibly can be...just the extra layer to deal with.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449554758



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       The ratio sounds like a good idea for usability, since it allows to define how much overload are we willing to accept in a per-query basis. The downside is that it limits the capacity of administrators (who edit the yaml) to limit the damage that users can do if they decide to use a very high page size, I think. That would be ideal IMO if we had a way to also limit the page size (I think we don't have it).




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448683322



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,21 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;

Review comment:
       As you can see, we only count rows toward the limit here, and not range tombstones as well. I'm not sure if this makes a difference.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r459012235



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,122 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));

Review comment:
       I'm comfortable with avoiding a potentially huge number of updates to `maxRowsCached` w/ a large partition, given the comment here is pretty clear.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458310857



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -340,7 +341,8 @@ public CFMetaData metadata()
             @Override
             public void close()
             {
-                // nothing to do here
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.

Review comment:
       Ended up adding a `doOnClose()` method to `PartitionIterators`, which seems to work well.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449112217



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,21 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;
+
+                        if (rowsCached == cachedRowsFailThreshold)
+                        {
+                            throw new TooManyCachedRowsException(cachedRowsFailThreshold, command);
+                        }
+                        else if (rowsCached == cachedRowsWarnThreshold)
+                        {
+                            logger.warn("Replica filtering protection has cached {} rows during query {}. " +

Review comment:
       We could issue also a client warning here, as we do with `tombstone_warn_threshold` in [`ReadCommand`](https://github.com/apache/cassandra/blob/48c54e49a636480b07dde5b12fe83cb8abfab2cd/src/java/org/apache/cassandra/db/ReadCommand.java#L543).




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r454651316



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,92 +78,67 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+
+    private int rowsCached = 0;
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;
 
     ReplicaFilteringProtection(Keyspace keyspace,
                                ReadCommand command,
                                ConsistencyLevel consistency,
-                               InetAddress[] sources)
+                               InetAddress[] sources,
+                               int cachedRowsWarnThreshold,
+                               int cachedRowsFailThreshold)
     {
         this.keyspace = keyspace;
         this.command = command;
         this.consistency = consistency;
         this.sources = sources;
-        this.rowsToFetch = new ArrayList<>(sources.length);
         this.originalPartitions = new ArrayList<>(sources.length);
 
         for (InetAddress ignored : sources)
         {
-            rowsToFetch.add(new TreeMap<>());
-            originalPartitions.add(new ArrayList<>());
+            originalPartitions.add(new ArrayDeque<>());

Review comment:
       The initial size of the deque is 16, which is obviously overkill for single partition queries. Not sure if it's worth doing anything about that though.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r453154216



##########
File path: src/java/org/apache/cassandra/exceptions/TooManyCachedRowsException.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ReadCommand;
+
+import static org.apache.cassandra.exceptions.ExceptionCode.SERVER_ERROR;
+
+/**
+ * Exception thrown when {@link org.apache.cassandra.service.ReplicaFilteringProtection} caches
+ * the configured threshold number of row results from participating replicas.
+ */
+public class TooManyCachedRowsException extends RequestExecutionException

Review comment:
       @jasonstack pointed out to me that using `OverloadedException` actually might make the most sense. Currently, it's used to indicate too many in-flight (on-heap) hints and can fail a write operation. This patch is doing something very similar, just for reads.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r459008953



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -150,46 +150,46 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
     {
         // Protecting against inconsistent replica filtering (some replica returning a row that is outdated but that
         // wouldn't be removed by normal reconciliation because up-to-date replica have filtered the up-to-date version
-        // of that row) works in 3 steps:
-        //   1) we read the full response just to collect rows that may be outdated (the ones we got from some
-        //      replica but didn't got any response for other; it could be those other replica have filtered a more
-        //      up-to-date result). In doing so, we do not count any of such "potentially outdated" row towards the
-        //      query limit. This simulate the worst case scenario where all those "potentially outdated" rows are
-        //      indeed outdated, and thus make sure we are guaranteed to read enough results (thanks to short read
-        //      protection).
-        //   2) we query all the replica/rows we need to rule out whether those "potentially outdated" rows are outdated
-        //      or not.
-        //   3) we re-read cached copies of each replica response using the "normal" read path merge with read-repair,
-        //      but where for each replica we use their original response _plus_ the additional rows queried in the
-        //      previous step (and apply the command#rowFilter() on the full result). Since the first phase has
-        //      pessimistically collected enough results for the case where all potentially outdated results are indeed
-        //      outdated, we shouldn't need further short-read protection requests during this phase.
+        // of that row) involves 3 main elements:
+        //   1) We combine short-read protection and a merge listener that identifies potentially "out-of-date"
+        //      rows to create an iterator that is guaranteed to produce enough valid row results to satisfy the query 
+        //      limit if enough actually exist. A row is considered out-of-date if its merged form is non-empty and we 
+        //      receive not response from at least one replica. In this case, it is possible that filtering at the
+        //      "silent" replica has produced a more up-to-date result.
+        //   2) This iterator is passed to the standard resolution process with read-repair, but is first wrapped in a 
+        //      response provider that lazily "completes" potentially out-of-date rows by directly querying them on the
+        //      replicas that were previously silent. As this iterator is consumed, it caches valid data for potentially
+        //      out-of-date rows, and this cached data is merged with the fetched data as rows are requested. If there
+        //      is no replica divergence, only rows in the partition being evalutated will be cached (then released
+        //      when the partition is consumed).
+        //   3) After a "complete" row is materialized, it must pass the row filter supplied by the original query 
+        //      before it counts against the limit.
 
         // We could get more responses while this method runs, which is ok (we're happy to ignore any response not here
         // at the beginning of this method), so grab the response count once and use that through the method.
         int count = responses.size();
         // We need separate contexts, as each context has his own counter
         ResolveContext firstPhaseContext = new ResolveContext(count);
         ResolveContext secondPhaseContext = new ResolveContext(count);
-        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace, command, consistency, firstPhaseContext.sources);
+
+        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace,
+                                                                        command,
+                                                                        consistency,
+                                                                        firstPhaseContext.sources,
+                                                                        DatabaseDescriptor.getCachedReplicaRowsWarnThreshold(),
+                                                                        DatabaseDescriptor.getCachedReplicaRowsFailThreshold());
+
         PartitionIterator firstPhasePartitions = resolveInternal(firstPhaseContext,
                                                                  rfp.mergeController(),
                                                                  i -> shortReadProtectedResponse(i, firstPhaseContext),
                                                                  UnaryOperator.identity());
-
-        // Consume the first phase partitions to populate the replica filtering protection with both those materialized
-        // partitions and the primary keys to be fetched.
-        PartitionIterators.consume(firstPhasePartitions);
-        firstPhasePartitions.close();
-
-        // After reading the entire query results the protection helper should have cached all the partitions so we can
-        // clear the responses accumulator for the sake of memory usage, given that the second phase might take long if
-        // it needs to query replicas.
-        responses.clearUnsafe();
-
-        return resolveWithReadRepair(secondPhaseContext,
-                                     rfp::queryProtectedPartitions,
-                                     results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        PartitionIterator completedPartitions = resolveWithReadRepair(secondPhaseContext,
+                                                                      i -> rfp.queryProtectedPartitions(firstPhasePartitions, i),
+                                                                      results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        // Ensure that the RFP instance has a chance to record metrics when the iterator closes.

Review comment:
       Fair, although I'd like to leave the reader with more information, assuming we don't think it's actually confusing.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #659:
URL: https://github.com/apache/cassandra/pull/659#issuecomment-652707717


   Preliminary test run: https://app.circleci.com/pipelines/github/maedhroz/cassandra/18/workflows/6f2be093-f000-4444-901d-ce80ebb58075


----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449169993



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,21 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;
+
+                        if (rowsCached == cachedRowsFailThreshold)

Review comment:
       The main reason why I used `==` here was to make sure we only logged the warning once. I could just as easily make this `rowsCached == cachedRowsFailThreshold + 1`. You're right that "threshold" usually means the level that must be _exceeded_ for some action to take place :)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448087497



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -48,8 +48,10 @@
     private static final boolean DROP_OVERSIZED_READ_REPAIR_MUTATIONS =
         Boolean.getBoolean("cassandra.drop_oversized_readrepair_mutations");
 
+    private static final UnaryOperator<PartitionIterator> IDENTITY = UnaryOperator.identity();

Review comment:
       I think you're [right](https://stackoverflow.com/questions/23983832/is-method-reference-caching-a-good-idea-in-java-8). We shouldn't end up with multiple instances from the same call site below. In any case, I'll likely revert this if we proceed w/ the [guardrails approach|https://issues.apache.org/jira/browse/CASSANDRA-15907?focusedCommentId=17148207&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17148207].




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #659:
URL: https://github.com/apache/cassandra/pull/659#issuecomment-664653253


   @adelapena @jrwest Pushed up one more commit to address the most recent review feedback.


----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449161522



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet

Review comment:
       Yeah, I'm leaning toward keeping the "cached" terminology, but explaining clearly that it's scoped to pages of a single query.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] belliottsmith commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449608064



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       True.  I hope we have some limit on page sizes anyway, but I guess all of these things are terrible tools really, and all poor proxies for the real problems.  I am just suspicious of guesstimating RFs and page sizes when picking defaults.
   
   I look forward to the day this can all be dropped in favour of proper resource management.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #659:
URL: https://github.com/apache/cassandra/pull/659#issuecomment-661602369


   For the benefit of anyone starting review, the latest CI run for this branch is [here](https://app.circleci.com/pipelines/github/maedhroz/cassandra/66/workflows/c3e60efe-a50e-47ff-a8a7-51de36deb17b).


----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz removed a comment on pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #659:
URL: https://github.com/apache/cassandra/pull/659#issuecomment-652707717


   Preliminary test run: https://app.circleci.com/pipelines/github/maedhroz/cassandra/18/workflows/6f2be093-f000-4444-901d-ce80ebb58075


----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449171140



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -171,7 +171,14 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
         // We need separate contexts, as each context has his own counter
         ResolveContext firstPhaseContext = new ResolveContext(count);
         ResolveContext secondPhaseContext = new ResolveContext(count);
-        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace, command, consistency, firstPhaseContext.sources);
+
+        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace,
+                                                                        command,
+                                                                        consistency,
+                                                                        firstPhaseContext.sources,
+                                                                        DatabaseDescriptor.getCachedReplicaRowsWarnThreshold(),
+                                                                        DatabaseDescriptor.getCachedReplicaRowsFailThreshold());
+

Review comment:
       Yes. See https://github.com/apache/cassandra/pull/659#discussion_r448037610.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449139164



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       > perhaps we should use something like 1000, WDYT?
   
   Yup. No particular attachment to powers of two.
   
   > so if we are reading from two on-sync replicas the threshold will start at 512 results, allowing a max limit/page size of 512 rows, won't it
   
   Correct.
   
   > Perhaps we should count the number of merged materialized/cached rows, ignoring the replicas, so the thresholds can be more easily related to the query limit.
   
   I agree that it would be a bit easier to relate that to the limit, but my worry is that it would make the same threshold values behave differently with different read consistency levels. That also brings up another question, which is whether, even in this scheme, we should count the merged row as well. (I'd say we don't have to, given the merged row is already being kept on heap outside `RFP`.)
   
   If we ultimately don't need this to be directly relatable to the limit, and I propose we don't (see `tombstone_failure_threshold`), it may also make sense to go down to the level of cells. It doesn't actually look much more expensive to get the number of cells from `Row#columnCount()`, but it adds some complexity (and doesn't look like it includes the clustering).




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r451829716



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,25 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;
+
+                        if (rowsCached == cachedRowsFailThreshold + 1)
+                        {
+                            throw new TooManyCachedRowsException(cachedRowsFailThreshold, command);
+                        }
+                        else if (rowsCached == cachedRowsWarnThreshold + 1)
+                        {
+                            String message =
+                                String.format("Replica filtering protection materialized %d rows during query %s. " +

Review comment:
       self nit: I'll probably change this to say "...materialized more than %d rows...", using `cachedRowsWarnThreshold` instead of `rowsCached`. At least the former indicates the threshold clearly.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r455612508



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -860,6 +858,10 @@ public UnfilteredRowIterator moreContents()
                 ColumnFamilyStore.metricsFor(metadata.cfId).shortReadProtectionRequests.mark();
                 Tracing.trace("Requesting {} extra rows from {} for short read protection", lastQueried, source);
 
+                // If we've arrived here, all responses have been consumed, and we're about to request more. Before that
+                // happens, clear the accumulator and allow garbage collection to free the resources they used.
+                responses.clearUnsafe();

Review comment:
       ...or at least clear only the response from this source, and then probably in `ShortReadPartitionsProtection#moreContents()`. There's another wrinkle here, which is that the short read protection classes are moved out of `DataResolver` in `trunk`. (Maybe we can provide a callback that clears the responses for the appropriate source to the SRP objects...)
   
   CC @adelapena 




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449100561



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ConsistencyLevel.ALL, i);
+        }
+
+        Object[][] initialRows =
+            cluster.coordinator(1).execute("SELECT * FROM " + fullTableName + " WHERE v = 'old' LIMIT ? ALLOW FILTERING",
+                                           ConsistencyLevel.ALL, ROWS);
+        assertRows(initialRows,
+                   row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        Object[][] emptyOldMatches =

Review comment:
       We can reduce the line breaking if we declare the query string before:
   ```java
   String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
   
   Object[][] emptyOldMatches = cluster.coordinator(1).execute(query, ConsistencyLevel.ALL,  "old", ROWS);
   assertRows(emptyOldMatches);
   
   Object[][] newMatches = cluster.coordinator(1).execute(query, ConsistencyLevel.ALL,  "new", ROWS);
   assertRows(newMatches, row(1, "new"), row(0, "new"), row(2, "new"));
   ```




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449538964



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(fullTableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // TODO: These should be able to use ICoordinator#executeWithResult() once CASSANDRA-15920 is resolved.
+        Object[] oldResponse = cluster.get(1).callOnInstance(() -> executeInternal(query, "old", ROWS));
+        Object[][] oldRows = (Object[][]) oldResponse[0];
+        assertRows(oldRows);
+        @SuppressWarnings("unchecked") List<String> oldWarnings = (List<String>) oldResponse[1];
+        assertEquals(shouldWarn, oldWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));

Review comment:
       It's probably not needed for testing the guardrails, but we could easily check here the SRP requests metric to verify that we are really generating the desired scenario, just in case:
   ```java
   assertEquals(3, rfpRequests(tableName));
   
   ...
   
   private static long rfpRequests(String tableName)
   {
       return cluster.get(1).callOnInstance(() -> Keyspace.open(KEYSPACE)
                                                          .getColumnFamilyStore(tableName)
                                                          .metric.replicaFilteringProtectionRequests
                                                          .getCount());
   }
   ```




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458007711



##########
File path: conf/cassandra.yaml
##########
@@ -675,6 +675,9 @@ replica_filtering_protection:
     # "cached_replica_rows_fail_threshold" is the per-query threshold at which the query will fail.
     #
     # These thresholds may also be adjusted at runtime using the StorageService mbean.
+    # 
+    # If the failure threshold is breeched, it is likely that either the current page/fetch size
+    # is too large or one or more replicas is severely out-of-sync and in need of repair.

Review comment:
       It looks perfect to me, +1




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458345471



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -150,46 +150,46 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
     {
         // Protecting against inconsistent replica filtering (some replica returning a row that is outdated but that
         // wouldn't be removed by normal reconciliation because up-to-date replica have filtered the up-to-date version
-        // of that row) works in 3 steps:
-        //   1) we read the full response just to collect rows that may be outdated (the ones we got from some
-        //      replica but didn't got any response for other; it could be those other replica have filtered a more
-        //      up-to-date result). In doing so, we do not count any of such "potentially outdated" row towards the
-        //      query limit. This simulate the worst case scenario where all those "potentially outdated" rows are
-        //      indeed outdated, and thus make sure we are guaranteed to read enough results (thanks to short read
-        //      protection).
-        //   2) we query all the replica/rows we need to rule out whether those "potentially outdated" rows are outdated
-        //      or not.
-        //   3) we re-read cached copies of each replica response using the "normal" read path merge with read-repair,
-        //      but where for each replica we use their original response _plus_ the additional rows queried in the
-        //      previous step (and apply the command#rowFilter() on the full result). Since the first phase has
-        //      pessimistically collected enough results for the case where all potentially outdated results are indeed
-        //      outdated, we shouldn't need further short-read protection requests during this phase.
+        // of that row) involves 3 main elements:
+        //   1) We combine short-read protection and a merge listener that identifies potentially "out-of-date"
+        //      rows to create an iterator that is guaranteed to produce enough valid row results to satisfy the query 
+        //      limit if enough actually exist. A row is considered out-of-date if its merged form is non-empty and we 
+        //      receive not response from at least one replica. In this case, it is possible that filtering at the
+        //      "silent" replica has produced a more up-to-date result.
+        //   2) This iterator is passed to the standard resolution process with read-repair, but is first wrapped in a 
+        //      response provider that lazily "completes" potentially out-of-date rows by directly querying them on the
+        //      replicas that were previously silent. As this iterator is consumed, it caches valid data for potentially
+        //      out-of-date rows, and this cached data is merged with the fetched data as rows are requested. If there
+        //      is no replica divergence, only rows in the partition being evalutated will be cached (then released
+        //      when the partition is consumed).
+        //   3) After a "complete" row is materialized, it must pass the row filter supplied by the original query 
+        //      before it counts against the limit.
 
         // We could get more responses while this method runs, which is ok (we're happy to ignore any response not here
         // at the beginning of this method), so grab the response count once and use that through the method.
         int count = responses.size();
         // We need separate contexts, as each context has his own counter
         ResolveContext firstPhaseContext = new ResolveContext(count);
         ResolveContext secondPhaseContext = new ResolveContext(count);
-        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace, command, consistency, firstPhaseContext.sources);
+
+        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace,
+                                                                        command,
+                                                                        consistency,
+                                                                        firstPhaseContext.sources,
+                                                                        DatabaseDescriptor.getCachedReplicaRowsWarnThreshold(),
+                                                                        DatabaseDescriptor.getCachedReplicaRowsFailThreshold());
+
         PartitionIterator firstPhasePartitions = resolveInternal(firstPhaseContext,
                                                                  rfp.mergeController(),
                                                                  i -> shortReadProtectedResponse(i, firstPhaseContext),
                                                                  UnaryOperator.identity());
-
-        // Consume the first phase partitions to populate the replica filtering protection with both those materialized
-        // partitions and the primary keys to be fetched.
-        PartitionIterators.consume(firstPhasePartitions);
-        firstPhasePartitions.close();
-
-        // After reading the entire query results the protection helper should have cached all the partitions so we can
-        // clear the responses accumulator for the sake of memory usage, given that the second phase might take long if
-        // it needs to query replicas.
-        responses.clearUnsafe();
-
-        return resolveWithReadRepair(secondPhaseContext,
-                                     rfp::queryProtectedPartitions,
-                                     results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        PartitionIterator completedPartitions = resolveWithReadRepair(secondPhaseContext,
+                                                                      i -> rfp.queryProtectedPartitions(firstPhasePartitions, i),
+                                                                      results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        // Ensure that the RFP instance has a chance to record metrics when the iterator closes.
+        return PartitionIterators.doOnClose(completedPartitions, rfp::close);

Review comment:
       I like the new `doOnClose` method. However, we are not closing the first phase iterator, and I think we should. We could either close it in `ReplicaFilteringProtection#close` or right here with a lambda, but I think it would be easier/cleaner if we move the `ReplicaFilteringProtection#close` method to the `UnfilteredPartitionIterators.MergeListener` returned by `ReplicaFilteringProtection#mergeController()`, and here we simply close the 1st phase iterator:
   ```suggestion
           return PartitionIterators.doOnClose(completedPartitions, firstPhasePartitions::close);
   ```
   Closing the 1st phase iterator here feels natural IMO, and allows us to abstract the `DataResolver` from the RFP metrics stuff. 
   
   Alternatively, if you don't agree and want to keep the `ReplicaFilteringProtection#close()` method, we could make `ReplicaFilteringProtection` implement `Closeable`, although I'd prefer using the `close()` method of the listener. WDYT?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r457711619



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -340,7 +341,8 @@ public CFMetaData metadata()
             @Override
             public void close()
             {
-                // nothing to do here
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.

Review comment:
       @adelapena Double-check me here :)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458338573



##########
File path: src/java/org/apache/cassandra/metrics/TableMetrics.java
##########
@@ -652,7 +662,8 @@ public Long getValue()
 
         readRepairRequests = createTableMeter("ReadRepairRequests");
         shortReadProtectionRequests = createTableMeter("ShortReadProtectionRequests");
-        replicaSideFilteringProtectionRequests = createTableMeter("ReplicaSideFilteringProtectionRequests");
+        replicaFilteringProtectionRequests = createTableMeter("ReplicaFilteringProtectionRequests");
+        rfpRowsCachedPerQuery = createHistogram("ReplicaFilteringProtectionRowsCachedPerPartition", true);

Review comment:
       ```suggestion
           rfpRowsCachedPerQuery = createHistogram("ReplicaFilteringProtectionRowsCachedPerQuery", true);
   ```




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449168260



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       >RE "cached" vs. "materialized", is the danger that readers will perceive the former as meaning rows are cached past the lifetime of the individual query?
   
   Yes, right, although I might be wrong. `cached_filtered_rows_warn_threshold` or `materialized_filtered_rows_warn_threshold` also feels a bit more descriptive to me, underlying the relationship with filtering. The rows are cached/materialized in the coordinator, and perhaps one might think the opposite with that "replica_rows".
   
   >Given the existing usage of materialised on the project it might be best to pick something else again :/
   
   Is that because readers might confuse materialized replica/filtered rows with materialized views? That could happen using the same adjective, I guess. Perhaps "loaded_filtered_rows", or "accumulated"? I have no better ideas, so it's ok for me if we go with "cached".




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r450785612



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;

Review comment:
       I think this was missed in the last commit, can we pass the unqualified table name to `testMissedUpdates`, assuming you agree?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449536391



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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.distributed.Cluster;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;

Review comment:
       Not a big deal, but the fully qualified table name is only used in `testMissedUpdates`, so we could just pass the `tableName` to it and calculate the full name internally, so we save us a line in every test. Also, the unqualified table name can be useful in `testMissedUpdates` if we want to take a look at the RFP requests metric on it.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449093016



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(fullTableName, 1, REPLICAS * ROWS - 1);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), TooManyCachedRowsException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String fullTableName, int warnThreshold, int failThreshold)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ConsistencyLevel.ALL, i);
+        }
+
+        Object[][] initialRows =
+            cluster.coordinator(1).execute("SELECT * FROM " + fullTableName + " WHERE v = 'old' LIMIT ? ALLOW FILTERING",
+                                           ConsistencyLevel.ALL, ROWS);
+        assertRows(initialRows,
+                   row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        Object[][] emptyOldMatches =
+            cluster.coordinator(1).execute("SELECT * FROM " + fullTableName + " WHERE v = 'old' LIMIT ? ALLOW FILTERING",
+                                           ConsistencyLevel.ALL, ROWS);
+        assertRows(emptyOldMatches);
+
+        Object[][] newMatches =
+            cluster.coordinator(1).execute("SELECT * FROM " + fullTableName + " WHERE v = 'new' LIMIT ? ALLOW FILTERING",
+                                           ConsistencyLevel.ALL, ROWS);
+        assertRows(newMatches,

Review comment:
       We can't know which query between the one for `old` and the one for `new` is triggering the 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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r457709697



##########
File path: conf/cassandra.yaml
##########
@@ -675,6 +675,9 @@ replica_filtering_protection:
     # "cached_replica_rows_fail_threshold" is the per-query threshold at which the query will fail.
     #
     # These thresholds may also be adjusted at runtime using the StorageService mbean.
+    # 
+    # If the failure threshold is breeched, it is likely that either the current page/fetch size
+    # is too large or one or more replicas is severely out-of-sync and in need of repair.

Review comment:
       @adelapena If someone actually sees an error, it should be pretty easy for them to navigate back here and get a better understanding. WDYT?




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r452970318



##########
File path: src/java/org/apache/cassandra/exceptions/TooManyCachedRowsException.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ReadCommand;
+
+import static org.apache.cassandra.exceptions.ExceptionCode.SERVER_ERROR;
+
+/**
+ * Exception thrown when {@link org.apache.cassandra.service.ReplicaFilteringProtection} caches
+ * the configured threshold number of row results from participating replicas.
+ */
+public class TooManyCachedRowsException extends RequestExecutionException

Review comment:
       @adelapena @jrwest This exception is problematic. As a `RequestExecutionException` using the `SERVER_ERROR` code, it isn't logged anywhere. (i.e. It's ignored in `QueryMessage#execute()`.) At the same time, when decoded by tools like `cqlsh`, it presents as a pretty useless `NoHostAvailable`. I'm fishing for options in terms of how to make this work reasonably:
   
   1.) Just leave this as a `RuntimeException`, which makes sure we at least fail and log the exception. The protocol level error ends up being `ServerError`, which doesn't really indicate a read execution problem.
   
   2.) Have it extend `OverloadedException` and log the error explicitly in `ReplicaFilteringProtection`. This would roughly communicate to the client that we breeched some resource usage threshold during a read, but it may not be the intended use for the `OVERLOADED` code?
   
   3.) Leave it as is but make `TooManyCachedRowsException` a first class citizen at the protocol level (as new code and changes to the codec). This would probably still require explicit logging, as w/ #2 above. 
   
   The least painful of the options above seems like #1. Thoughts?
   
   (Note that I never seriously considered `ReadFailureException`, given it has so much extraneous information, and doesn't really indicate an error in coordinator processing.)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449312026



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       @adelapena Unless you object, I think I'll try the namespace format from my last comment. Once that's in place, I think this is ready for a more official review.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r454650383



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.exceptions.OverloadedException;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.config.ReplicaFilteringProtectionOptions.DEFAULT_FAIL_THRESHOLD;
+import static org.apache.cassandra.config.ReplicaFilteringProtectionOptions.DEFAULT_WARN_THRESHOLD;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+
+        // Make sure we start w/ the correct defaults:
+        cluster.get(1).runOnInstance(() -> assertEquals(DEFAULT_WARN_THRESHOLD, StorageService.instance.getCachedReplicaRowsWarnThreshold()));
+        cluster.get(1).runOnInstance(() -> assertEquals(DEFAULT_FAIL_THRESHOLD, StorageService.instance.getCachedReplicaRowsFailThreshold()));
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(tableName, REPLICAS * ROWS, Integer.MAX_VALUE, false);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(tableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE, true);
+    }
+
+    @Test
+    public void testMissedUpdatesAroundCachingFailThreshold()
+    {
+        String tableName = "missed_updates_cache_fail";
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The failure threshold provided is exactly the total number of rows returned
+        // to the coordinator from all replicas and therefore should just warn.
+        testMissedUpdates(tableName, 1, REPLICAS * ROWS, true);
+
+        try
+        {
+            // The failure threshold provided is one less than the total number of rows returned
+            // to the coordinator from all replicas and therefore should fail the query.
+            testMissedUpdates(tableName, 1, REPLICAS * ROWS - 1, true);
+        }
+        catch (RuntimeException e)
+        {
+            assertEquals(e.getCause().getClass().getName(), OverloadedException.class.getName());
+        }
+    }
+
+    private void testMissedUpdates(String tableName, int warnThreshold, int failThreshold, boolean shouldWarn)
+    {
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsWarnThreshold(warnThreshold));
+        cluster.get(1).runOnInstance(() -> StorageService.instance.setCachedReplicaRowsFailThreshold(failThreshold));
+
+        String fullTableName = KEYSPACE + '.' + tableName;
+
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.coordinator(1).execute("INSERT INTO " + fullTableName + "(k, v) VALUES (?, 'old')", ALL, i);
+        }
+
+        String query = "SELECT * FROM " + fullTableName + " WHERE v = ? LIMIT ? ALLOW FILTERING";
+
+        Object[][] initialRows = cluster.coordinator(1).execute(query, ALL, "old", ROWS);
+        assertRows(initialRows, row(1, "old"), row(0, "old"), row(2, "old"));
+
+        // Update all rows on only one replica, leaving the entire dataset of the remaining replica out-of-date:
+        for (int i = 0; i < ROWS; i++)
+        {
+            cluster.get(1).executeInternal("UPDATE " + fullTableName + " SET v = 'new' WHERE k = ?", i);
+        }
+
+        // The replica that missed the results creates a mismatch at every row, and we threfore cache a version
+        // of that row for all replicas.
+        SimpleQueryResult oldResult = cluster.coordinator(1).executeWithResult(query, ALL, "old", ROWS);
+        assertRows(oldResult.toObjectArrays());
+        List<String> oldWarnings = oldResult.warnings();
+        assertEquals(shouldWarn, oldWarnings.stream().anyMatch(w -> w.contains("cached_replica_rows_warn_threshold")));
+        assertEquals(shouldWarn ? 1 : 0, oldWarnings.size());
+

Review comment:
       @adelapena These are the new bits after moving to lazy partition consumption/caching...




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #659:
URL: https://github.com/apache/cassandra/pull/659#issuecomment-653167124


   Pushed up a commit with some fixes for the less controversial items.


----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449180795



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       I like `cached_filtered_rows_warn_threshold`. It at least avoids confusion about the rows being cached at the replica instead of the coordinator. If we want to make that explicit, `coordinator_cached_rows_warn_threshold` could work, but I'd rather have this hint at the relationship w/ filtering. I'll go with `cached_filtered_rows_warn_threshold` if there are no other objections.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448682562



##########
File path: src/java/org/apache/cassandra/db/rows/EncodingStats.java
##########
@@ -107,6 +108,29 @@ public EncodingStats mergeWith(EncodingStats that)
         return new EncodingStats(minTimestamp, minDelTime, minTTL);
     }
 
+    /**
+     * Merge one or more EncodingStats, that are lazily materialized from some list of arbitrary type by the provided function
+     */
+    public static <V, F extends Function<V, EncodingStats>> EncodingStats merge(List<V> values, F function)
+    {
+        if (values.size() == 1)
+            return function.apply(values.get(0));
+
+        Collector collector = new Collector();
+        for (int i=0, isize=values.size(); i<isize; i++)
+        {
+            V v = values.get(i);
+            EncodingStats stats = function.apply(v);
+            if (stats.minTimestamp != TIMESTAMP_EPOCH)
+                collector.updateTimestamp(stats.minTimestamp);
+            if(stats.minLocalDeletionTime != DELETION_TIME_EPOCH)
+                collector.updateLocalDeletionTime(stats.minLocalDeletionTime);
+            if(stats.minTTL != TTL_EPOCH)
+                collector.updateTTL(stats.minTTL);
+        }
+        return collector.get();
+    }

Review comment:
       This is back-ported verbatim from CASSANDRA-14654, along with its tests.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449526250



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       >I agree that it would be a bit easier to relate that to the limit, but my worry is that it would make the same threshold values behave differently with different read consistency levels.
   
   Makes sense. The other way around, I understand that our intention is to provide a default threshold value that keeps heap usage reasonably low while allowing a reasonably high page size for the most common number of replicas. I think that the number of replicas is two, since for one replica RFP is not used. In that case, a default threshold value of 1000 means that the max allowed page size without warnings will be 500. Perhaps we should use a threshold value of 2000 to use a more round number (1000) in the page size, or it's too high? I don't really now what can be considered the maximum reasonable page size. For example the Spark driver uses 1000 as default, but with `LOCAL_ONE` consistency so it's not affected by this.
   
   >That also brings up another question, which is whether, even in this scheme, we should count the merged row as well. (I'd say we don't have to, given the merged row is already being kept on heap outside RFP.)
   
   Agree, the merged rows are not cached but consumed from an iterator as usual, so I wouldn't count them.
   
   >If we ultimately don't need this to be directly relatable to the limit, and I propose we don't (see tombstone_failure_threshold), it may also make sense to go down to the level of cells. It doesn't actually look much more expensive to get the number of cells from Row#columnCount(), but it adds some complexity (and doesn't look like it includes the clustering).
   
   The size of the cells is so dependant on the data they contain as the rows, so I'm not sure that would give us more control about the amount of heap usage. In both cases, and while we don't count actual payload size, users are going to need to think about what data are they storing/querying to get an estimate of the memory pressure. Calculating it from rows or cells doesn't seem to make a big difference, although I think I'd prefer rows.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449310929



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.exceptions.TooManyCachedRowsException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Exercises the functionality of {@link org.apache.cassandra.service.ReplicaFilteringProtection}, the
+ * mechanism that ensures distributed index and filtering queries at read consistency levels > ONE/LOCAL_ONE
+ * avoid stale replica results.
+ */
+public class ReplicaFilteringProtectionTest extends TestBaseImpl
+{
+    private static final int REPLICAS = 2;
+    private static final int ROWS = 3;
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(REPLICAS)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
+                                                          .set("commitlog_sync", "batch")
+                                                          .set("num_tokens", 1)).start());
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testMissedUpdatesBelowCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_no_warning";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one more than the total number of rows returned
+        // to the coordinator from all replicas and therefore should not be triggered.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS + 1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testMissedUpdatesAboveCachingWarnThreshold()
+    {
+        String tableName = "missed_updates_cache_warn";
+        String fullTableName = KEYSPACE + '.' + tableName;
+        cluster.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int PRIMARY KEY, v text)"));
+
+        // The warning threshold provided is one less than the total number of rows returned
+        // to the coordinator from all replicas and therefore should be triggered but not fail the query.
+        testMissedUpdates(fullTableName, REPLICAS * ROWS - 1, Integer.MAX_VALUE);

Review comment:
       @adelapena So instead of trying to figure out what's in the logs, I piggybacked on adding the new client warning. Even though not having CASSANDRA-15920 made it a little bit more complicated than I had hoped, I think the tests now do a pretty good job of making sure the new conditional logic in `RFP` works as expected.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r461177139



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -218,7 +218,7 @@ private PartitionIterator resolveInternal(ResolveContext context,
 
         UnfilteredPartitionIterator merged = UnfilteredPartitionIterators.merge(results, command.nowInSec(), mergeListener);
         FilteredPartitions filtered =
-        FilteredPartitions.filter(merged, new Filter(command.nowInSec(), command.metadata().enforceStrictLiveness()));
+            FilteredPartitions.filter(merged, new Filter(command.nowInSec(), command.metadata().enforceStrictLiveness()));

Review comment:
       I just made a local variable out of filter :)

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,124 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
-
-            PartitionBuilder[] builders = new PartitionBuilder[sources.length];
-
-            for (int i = 0; i < sources.length; i++)
-                builders[i] = new PartitionBuilder(partitionKey, columns(versions), stats(versions));
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));
+            }
 
-            return new UnfilteredRowIterators.MergeListener()
+            @Override
+            public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List<UnfilteredRowIterator> versions)
             {
-                @Override
-                public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                PartitionBuilder[] builders = new PartitionBuilder[sources.length];
+                PartitionColumns columns = columns(versions);
+                EncodingStats stats = EncodingStats.merge(versions, NULL_TO_NO_STATS);
+                
+                for (int i = 0; i < sources.length; i++)
+                    builders[i] = new PartitionBuilder(partitionKey, sources[i], columns, stats);
+
+                return new UnfilteredRowIterators.MergeListener()
                 {
-                    // cache the deletion time versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].setDeletionTime(versions[i]);
-                }
+                    @Override
+                    public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                    {
+                        // cache the deletion time versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].setDeletionTime(versions[i]);
+                    }
 
-                @Override
-                public Row onMergedRows(Row merged, Row[] versions)
-                {
-                    // cache the row versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].addRow(versions[i]);
+                    @Override
+                    public Row onMergedRows(Row merged, Row[] versions)
+                    {
+                        // cache the row versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].addRow(versions[i]);
 
-                    if (merged.isEmpty())
-                        return merged;
+                        if (merged.isEmpty())
+                            return merged;
 
-                    boolean isPotentiallyOutdated = false;
-                    boolean isStatic = merged.isStatic();
-                    for (int i = 0; i < versions.length; i++)
-                    {
-                        Row version = versions[i];
-                        if (version == null || (isStatic && version.isEmpty()))
+                        boolean isPotentiallyOutdated = false;
+                        boolean isStatic = merged.isStatic();
+                        for (int i = 0; i < versions.length; i++)
                         {
-                            isPotentiallyOutdated = true;
-                            BTreeSet.Builder<Clustering> toFetch = getOrCreateToFetch(i, partitionKey);
-                            // Note that for static, we shouldn't add the clustering to the clustering set (the
-                            // ClusteringIndexNamesFilter we'll build from this later does not expect it), but the fact
-                            // we created a builder in the first place will act as a marker that the static row must be
-                            // fetched, even if no other rows are added for this partition.
-                            if (!isStatic)
-                                toFetch.add(merged.clustering());
+                            Row version = versions[i];
+                            if (version == null || (isStatic && version.isEmpty()))
+                            {
+                                isPotentiallyOutdated = true;
+                                builders[i].addToFetch(merged);
+                            }
                         }
-                    }
 
-                    // If the row is potentially outdated (because some replica didn't send anything and so it _may_ be
-                    // an outdated result that is only present because other replica have filtered the up-to-date result
-                    // out), then we skip the row. In other words, the results of the initial merging of results by this
-                    // protection assume the worst case scenario where every row that might be outdated actually is.
-                    // This ensures that during this first phase (collecting additional row to fetch) we are guaranteed
-                    // to look at enough data to ultimately fulfill the query limit.
-                    return isPotentiallyOutdated ? null : merged;
-                }
+                        // If the row is potentially outdated (because some replica didn't send anything and so it _may_ be
+                        // an outdated result that is only present because other replica have filtered the up-to-date result
+                        // out), then we skip the row. In other words, the results of the initial merging of results by this
+                        // protection assume the worst case scenario where every row that might be outdated actually is.
+                        // This ensures that during this first phase (collecting additional row to fetch) we are guaranteed
+                        // to look at enough data to ultimately fulfill the query limit.
+                        return isPotentiallyOutdated ? null : merged;
+                    }
 
-                @Override
-                public void onMergedRangeTombstoneMarkers(RangeTombstoneMarker merged, RangeTombstoneMarker[] versions)
-                {
-                    // cache the marker versions to be able to regenerate the original row iterator
-                    for (int i = 0; i < versions.length; i++)
-                        builders[i].addRangeTombstoneMarker(versions[i]);
-                }
+                    @Override
+                    public void onMergedRangeTombstoneMarkers(RangeTombstoneMarker merged, RangeTombstoneMarker[] versions)
+                    {
+                        // cache the marker versions to be able to regenerate the original row iterator
+                        for (int i = 0; i < versions.length; i++)
+                            builders[i].addRangeTombstoneMarker(versions[i]);
+                    }
 
-                @Override
-                public void close()
-                {
-                    for (int i = 0; i < sources.length; i++)
-                        originalPartitions.get(i).add(builders[i]);
-                }
-            };
+                    @Override
+                    public void close()
+                    {
+                        for (int i = 0; i < sources.length; i++)
+                            originalPartitions.get(i).add(builders[i]);
+                    }
+                };
+            }
         };
     }
 
+    private void incrementCachedRows()
+    {
+        currentRowsCached++;
+        
+        if (currentRowsCached == cachedRowsFailThreshold + 1)
+        {
+            String message = String.format("Replica filtering protection has cached over %d rows during query %s. " +
+                                           "(See 'cached_replica_rows_fail_threshold' in cassandra.yaml.)",
+                                           cachedRowsFailThreshold, command.toCQLString());
+
+            oneMinuteLogger.error(message);

Review comment:
       @jrwest Yeah, I went back and forth on this a bit. It might be that there isn't much risk in allowing failure message to hit the regular log, given the system might simply not be able to process many queries like that concurrently anyway. I'm okay with changing this, assuming @adelapena has no objections.

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       Fair. I'll take a stab at this.

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       @jrwest `this.originalPartitions = new Queue<PartitionBuilder>[sources.length];` would actually be an illegal generic array creation, right?

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       @jrwest `this.originalPartitions = new Queue<PartitionBuilder>[sources.length];` would actually be an illegal generic array creation, right? (We can ignore that, of course...)

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       @jrwest `this.originalPartitions = new Queue[sources.length];` would actually be an illegal generic array creation, right? (We can ignore that, of course...)

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       @jrwest `this.originalPartitions = new Queue[sources.length];` would be a generic array creation, right? (We can ignore that, of course...)

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       @jrwest `this.originalPartitions = new Queue[sources.length];` would be a compile-time warning, right? (We can ignore that, of course, but generic array creation is illegal.)

##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -74,118 +78,58 @@
  * the rows in a replica response that don't have a corresponding row in other replica responses, and requests them by
  * primary key to the "silent" replicas in a second fetch round.
  * <p>
- * See CASSANDRA-8272 and CASSANDRA-8273 for further details.
+ * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
 class ReplicaFilteringProtection
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
+    private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static final Function<UnfilteredRowIterator, EncodingStats> NULL_TO_NO_STATS =
+        rowIterator -> rowIterator == null ? EncodingStats.NO_STATS : rowIterator.stats();
 
     private final Keyspace keyspace;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final InetAddress[] sources;
     private final TableMetrics tableMetrics;
 
-    /**
-     * Per-source primary keys of the rows that might be outdated so they need to be fetched.
-     * For outdated static rows we use an empty builder to signal it has to be queried.
-     */
-    private final List<SortedMap<DecoratedKey, BTreeSet.Builder<Clustering>>> rowsToFetch;
+    private final int cachedRowsWarnThreshold;
+    private final int cachedRowsFailThreshold;
+    
+    /** Tracks whether or not we've already hit the warning threshold while evaluating a partition. */
+    private boolean hitWarningThreshold = false;
+
+    private int currentRowsCached = 0; // tracks the current number of cached rows
+    private int maxRowsCached = 0; // tracks the high watermark for the number of cached rows
 
     /**
-     * Per-source list of all the partitions seen by the merge listener, to be merged with the extra fetched rows.
+     * Per-source list of the pending partitions seen by the merge listener, to be merged with the extra fetched rows.
      */
-    private final List<List<PartitionBuilder>> originalPartitions;
+    private final List<Queue<PartitionBuilder>> originalPartitions;

Review comment:
       Fair. I'll take a stab at this.

##########
File path: src/java/org/apache/cassandra/metrics/TableMetrics.java
##########
@@ -652,7 +662,8 @@ public Long getValue()
 
         readRepairRequests = createTableMeter("ReadRepairRequests");
         shortReadProtectionRequests = createTableMeter("ShortReadProtectionRequests");
-        replicaSideFilteringProtectionRequests = createTableMeter("ReplicaSideFilteringProtectionRequests");
+        replicaFilteringProtectionRequests = createTableMeter("ReplicaFilteringProtectionRequests");

Review comment:
       Thanks for the reminder.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] jrwest commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
jrwest commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r460583875



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -150,46 +150,46 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
     {
         // Protecting against inconsistent replica filtering (some replica returning a row that is outdated but that
         // wouldn't be removed by normal reconciliation because up-to-date replica have filtered the up-to-date version
-        // of that row) works in 3 steps:
-        //   1) we read the full response just to collect rows that may be outdated (the ones we got from some
-        //      replica but didn't got any response for other; it could be those other replica have filtered a more
-        //      up-to-date result). In doing so, we do not count any of such "potentially outdated" row towards the
-        //      query limit. This simulate the worst case scenario where all those "potentially outdated" rows are
-        //      indeed outdated, and thus make sure we are guaranteed to read enough results (thanks to short read
-        //      protection).
-        //   2) we query all the replica/rows we need to rule out whether those "potentially outdated" rows are outdated
-        //      or not.
-        //   3) we re-read cached copies of each replica response using the "normal" read path merge with read-repair,
-        //      but where for each replica we use their original response _plus_ the additional rows queried in the
-        //      previous step (and apply the command#rowFilter() on the full result). Since the first phase has
-        //      pessimistically collected enough results for the case where all potentially outdated results are indeed
-        //      outdated, we shouldn't need further short-read protection requests during this phase.
+        // of that row) involves 3 main elements:
+        //   1) We combine short-read protection and a merge listener that identifies potentially "out-of-date"
+        //      rows to create an iterator that is guaranteed to produce enough valid row results to satisfy the query 
+        //      limit if enough actually exist. A row is considered out-of-date if its merged form is non-empty and we 

Review comment:
       minor typo: "form" instead of "from"




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r459020906



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -150,46 +150,46 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
     {
         // Protecting against inconsistent replica filtering (some replica returning a row that is outdated but that
         // wouldn't be removed by normal reconciliation because up-to-date replica have filtered the up-to-date version
-        // of that row) works in 3 steps:
-        //   1) we read the full response just to collect rows that may be outdated (the ones we got from some
-        //      replica but didn't got any response for other; it could be those other replica have filtered a more
-        //      up-to-date result). In doing so, we do not count any of such "potentially outdated" row towards the
-        //      query limit. This simulate the worst case scenario where all those "potentially outdated" rows are
-        //      indeed outdated, and thus make sure we are guaranteed to read enough results (thanks to short read
-        //      protection).
-        //   2) we query all the replica/rows we need to rule out whether those "potentially outdated" rows are outdated
-        //      or not.
-        //   3) we re-read cached copies of each replica response using the "normal" read path merge with read-repair,
-        //      but where for each replica we use their original response _plus_ the additional rows queried in the
-        //      previous step (and apply the command#rowFilter() on the full result). Since the first phase has
-        //      pessimistically collected enough results for the case where all potentially outdated results are indeed
-        //      outdated, we shouldn't need further short-read protection requests during this phase.
+        // of that row) involves 3 main elements:
+        //   1) We combine short-read protection and a merge listener that identifies potentially "out-of-date"
+        //      rows to create an iterator that is guaranteed to produce enough valid row results to satisfy the query 
+        //      limit if enough actually exist. A row is considered out-of-date if its merged form is non-empty and we 
+        //      receive not response from at least one replica. In this case, it is possible that filtering at the
+        //      "silent" replica has produced a more up-to-date result.
+        //   2) This iterator is passed to the standard resolution process with read-repair, but is first wrapped in a 
+        //      response provider that lazily "completes" potentially out-of-date rows by directly querying them on the
+        //      replicas that were previously silent. As this iterator is consumed, it caches valid data for potentially
+        //      out-of-date rows, and this cached data is merged with the fetched data as rows are requested. If there
+        //      is no replica divergence, only rows in the partition being evalutated will be cached (then released
+        //      when the partition is consumed).
+        //   3) After a "complete" row is materialized, it must pass the row filter supplied by the original query 
+        //      before it counts against the limit.
 
         // We could get more responses while this method runs, which is ok (we're happy to ignore any response not here
         // at the beginning of this method), so grab the response count once and use that through the method.
         int count = responses.size();
         // We need separate contexts, as each context has his own counter
         ResolveContext firstPhaseContext = new ResolveContext(count);
         ResolveContext secondPhaseContext = new ResolveContext(count);
-        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace, command, consistency, firstPhaseContext.sources);
+
+        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace,
+                                                                        command,
+                                                                        consistency,
+                                                                        firstPhaseContext.sources,
+                                                                        DatabaseDescriptor.getCachedReplicaRowsWarnThreshold(),
+                                                                        DatabaseDescriptor.getCachedReplicaRowsFailThreshold());
+
         PartitionIterator firstPhasePartitions = resolveInternal(firstPhaseContext,
                                                                  rfp.mergeController(),
                                                                  i -> shortReadProtectedResponse(i, firstPhaseContext),
                                                                  UnaryOperator.identity());
-
-        // Consume the first phase partitions to populate the replica filtering protection with both those materialized
-        // partitions and the primary keys to be fetched.
-        PartitionIterators.consume(firstPhasePartitions);
-        firstPhasePartitions.close();
-
-        // After reading the entire query results the protection helper should have cached all the partitions so we can
-        // clear the responses accumulator for the sake of memory usage, given that the second phase might take long if
-        // it needs to query replicas.
-        responses.clearUnsafe();
-
-        return resolveWithReadRepair(secondPhaseContext,
-                                     rfp::queryProtectedPartitions,
-                                     results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        PartitionIterator completedPartitions = resolveWithReadRepair(secondPhaseContext,
+                                                                      i -> rfp.queryProtectedPartitions(firstPhasePartitions, i),
+                                                                      results -> command.rowFilter().filter(results, command.metadata(), command.nowInSec()));
+        
+        // Ensure that the RFP instance has a chance to record metrics when the iterator closes.

Review comment:
       I think this is not confusing, in the worst case it would be unnecessary, in the best it would be helpful, so I'm happy keeping it.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449169993



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,21 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;
+
+                        if (rowsCached == cachedRowsFailThreshold)

Review comment:
       The main reason why I used `==` here was to make sure we only logged the warning once. I could just as easily make this `rowsCached == cachedRowsFailThreshold + 1`. You're right that "threshold" usually means the level that must be _exceeded_ for some action to take place. (Either way, this depends on how we resolve the other discussion about whether to couple this to the limit, conceptually.)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r452977420



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,25 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;
+
+                        if (rowsCached == cachedRowsFailThreshold + 1)
+                        {
+                            throw new TooManyCachedRowsException(cachedRowsFailThreshold, command);
+                        }
+                        else if (rowsCached == cachedRowsWarnThreshold + 1)
+                        {
+                            String message =
+                                String.format("Replica filtering protection has cached over %d rows during query %s. " +
+                                              "(See 'cached_replica_rows_warn_threshold' in cassandra.yaml.)",
+                                              cachedRowsWarnThreshold, command.toCQLString());
+
+                            ClientWarn.instance.warn(message);
+                            logger.warn(message);

Review comment:
       @adelapena @jrwest Would this be a good candidate for the `NoSpamLogger`? I imagine we'd want to log all our failures, but between the thresholds, perhaps flooding the logs wouldn't be the best idea.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] belliottsmith commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449181348



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       `temporary` rather than `cached` perhaps?
   
   I've not looked closely at the semantics, honestly, so just throwing it out there.  I often look in a thesaurus for inspiration, otherwise, when struggling!




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449526834



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -171,7 +171,14 @@ private PartitionIterator resolveWithReplicaFilteringProtection()
         // We need separate contexts, as each context has his own counter
         ResolveContext firstPhaseContext = new ResolveContext(count);
         ResolveContext secondPhaseContext = new ResolveContext(count);
-        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace, command, consistency, firstPhaseContext.sources);
+
+        ReplicaFilteringProtection rfp = new ReplicaFilteringProtection(keyspace,
+                                                                        command,
+                                                                        consistency,
+                                                                        firstPhaseContext.sources,
+                                                                        DatabaseDescriptor.getCachedReplicaRowsWarnThreshold(),
+                                                                        DatabaseDescriptor.getCachedReplicaRowsFailThreshold());
+

Review comment:
       Oh, right, I missed that one




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449554758



##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -280,6 +280,9 @@
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
 
+    public volatile int cached_replica_rows_warn_threshold = 1024;
+    public volatile int cached_replica_rows_fail_threshold = 16384;

Review comment:
       The ratio sounds like a good idea for usability, since it allows to define how much overload are we willing to accept in a per-query basis. The downside is that it limits the capacity of administrators (who edit the yaml) to limit the damage that users can do if they decide to use a very high page size, I think.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r452970318



##########
File path: src/java/org/apache/cassandra/exceptions/TooManyCachedRowsException.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ReadCommand;
+
+import static org.apache.cassandra.exceptions.ExceptionCode.SERVER_ERROR;
+
+/**
+ * Exception thrown when {@link org.apache.cassandra.service.ReplicaFilteringProtection} caches
+ * the configured threshold number of row results from participating replicas.
+ */
+public class TooManyCachedRowsException extends RequestExecutionException

Review comment:
       @adelapena @jrwest This exception is problematic. As a `RequestExecutionException` using the `SERVER_ERROR` code, it isn't logged anywhere. (i.e. It's ignored in `QueryMessage#execute()`.) At the same time, when decoded by tools like `cqlsh`, it presents as a pretty useless `NoHostAvailable`. I'm fishing for options in terms of how to make this work reasonably:
   
   1.) Just leave this as a `RuntimeException`, which makes sure we at least fail and log the exception. The protocol level error ends up being `ServerError`, which doesn't really indicate a read execution problem.
   
   2.) Have it extend `OverloadedException` and log the error explicitly in `ReplicaFilteringProtection`. This would roughly communicate to the client that we breeched some resource usage threshold during a read, but it may not be the intended use for the `OVERLOADED` code?
   
   3.) Leave it as is but make `TooManyCachedRowsException` a first class citizen at the protocol level (a new error code and changes to the codec). This would probably still require explicit logging, as w/ #2 above.
   
   The least painful of the options above seems like #1. Thoughts?
   
   (Note that I never seriously considered `ReadFailureException`, given it has so much extraneous information, and doesn't really indicate an error in coordinator processing.)




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449129871



##########
File path: conf/cassandra.yaml
##########
@@ -661,6 +661,23 @@ auto_snapshot: true
 tombstone_warn_threshold: 1000
 tombstone_failure_threshold: 100000
 
+# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a
+# mechanism called replica filtering protection to ensure that results from stale replicas do
+# not violate consistency. This mechanism, which operates in two phases, caches replica
+# results on-heap at the coordinator. The more stale results returned by the former, the more
+# rows cached at the latter.
+#
+# These thresholds exist to limit the damage very stale replicas can cause during these queries.
+# They essentially define the memory budget individal index and filtering queries have to meet
+# the client-desired consistency level.
+#
+# "cached_replica_rows_warn_threshold" is the threshold at which a warning will be logged.
+# "cached_replica_rows_fail_threshold" is the threshold at which the query will fail.
+#
+# These thresholds may also be adjusted at runtime using the StorageService mbean.
+cached_replica_rows_warn_threshold: 1024
+cached_replica_rows_fail_threshold: 16384

Review comment:
       Definitely agree on throwing the ticket number in there for reference.
   
   RE "cached" vs. "materialized", is the danger that readers will perceive the former as meaning rows are cached past the lifetime of the individual query? I'm definitely okay w/ changing it to "materialized" in the description, although the name is starting to get really long. What would you say to something like `materialized_rows_warn_threshold`. It isn't the most specific name, but it's also not less specific than `tombstone_warn_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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r449158599



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -239,7 +253,21 @@ public Row onMergedRows(Row merged, Row[] versions)
                 {
                     // cache the row versions to be able to regenerate the original row iterator
                     for (int i = 0; i < versions.length; i++)
+                    {
                         builders[i].addRow(versions[i]);
+                        rowsCached++;

Review comment:
       @adelapena I think that's at the replica? Even so, I think you're right that it provides some protection.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r457689435



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -723,6 +725,10 @@ public UnfilteredPartitionIterator moreContents()
             ColumnFamilyStore.metricsFor(command.metadata().cfId).shortReadProtectionRequests.mark();
             Tracing.trace("Requesting {} extra rows from {} for short read protection", toQuery, source);
 
+            // If we've arrived here, all responses have been consumed, and we're about to request more. Before that
+            // happens, clear the accumulator and allow garbage collection to free the resources they used.
+            preFetchCallback.run();

Review comment:
       Yeah, I'll either do that or make it more generic, with this specific comment moving further up to where we instantiate `ShortReadPartitionsProtection`.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448682954



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -100,7 +110,9 @@
     ReplicaFilteringProtection(Keyspace keyspace,
                                ReadCommand command,
                                ConsistencyLevel consistency,
-                               InetAddress[] sources)
+                               InetAddress[] sources,
+                               int cachedRowsWarnThreshold,
+                               int cachedRowsFailThreshold)

Review comment:
       I didn't see any reason to couple `ReplicaFilteringProtection` to `DatabaseDescriptor` just to get these values. We don't want them to change in the middle of a query anyway.




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] maedhroz commented on a change in pull request #659: A raft of minor optimizations for replica filtering protection

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r448087497



##########
File path: src/java/org/apache/cassandra/service/DataResolver.java
##########
@@ -48,8 +48,10 @@
     private static final boolean DROP_OVERSIZED_READ_REPAIR_MUTATIONS =
         Boolean.getBoolean("cassandra.drop_oversized_readrepair_mutations");
 
+    private static final UnaryOperator<PartitionIterator> IDENTITY = UnaryOperator.identity();

Review comment:
       I think you're [right](https://stackoverflow.com/questions/23983832/is-method-reference-caching-a-good-idea-in-java-8). We shouldn't end up with multiple instances from the same call site below. In any case, I'll likely revert this if we proceed w/ the [guardrails approach](https://issues.apache.org/jira/browse/CASSANDRA-15907?focusedCommentId=17148207&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17148207).




----------------------------------------------------------------
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.

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


[GitHub] [cassandra] adelapena commented on a change in pull request #659: CASSANDRA-15907 Operational Improvements & Hardening for Replica Filtering Protection

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #659:
URL: https://github.com/apache/cassandra/pull/659#discussion_r458728397



##########
File path: src/java/org/apache/cassandra/service/ReplicaFilteringProtection.java
##########
@@ -212,83 +156,122 @@ private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, InetAddr
      * <p>
      * The listener will track both the accepted data and the primary keys of the rows that are considered as outdated.
      * That way, once the query results would have been merged using this listener, further calls to
-     * {@link #queryProtectedPartitions(int)} will use the collected data to return a copy of the
+     * {@link #queryProtectedPartitions(PartitionIterator, int)} will use the collected data to return a copy of the
      * data originally collected from the specified replica, completed with the potentially outdated rows.
      */
     UnfilteredPartitionIterators.MergeListener mergeController()
     {
-        return (partitionKey, versions) -> {
+        return new UnfilteredPartitionIterators.MergeListener()
+        {
+            @Override
+            public void close()
+            {
+                // If we hit the failure threshold before consuming a single partition, record the current rows cached.
+                tableMetrics.rfpRowsCachedPerQuery.update(Math.max(currentRowsCached, maxRowsCached));
+            }
 
-            PartitionBuilder[] builders = new PartitionBuilder[sources.length];
+            public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List<UnfilteredRowIterator> versions)
+            {
+                PartitionBuilder[] builders = new PartitionBuilder[sources.length];
 
-            for (int i = 0; i < sources.length; i++)
-                builders[i] = new PartitionBuilder(partitionKey, columns(versions), stats(versions));
+                for (int i = 0; i < sources.length; i++)
+                    builders[i] = new PartitionBuilder(partitionKey, sources[i], columns(versions), EncodingStats.merge(versions, NULL_TO_NO_STATS));

Review comment:
       I think that both the `PartitionColumns` and the `EncodingStats` are equals for each partition version and immutable, so we could create them right before the loop and reuse them for every version partition builder:
   ```suggestion
                   PartitionColumns columns = columns(versions);
                   EncodingStats stats = EncodingStats.merge(versions, NULL_TO_NO_STATS);
                   for (int i = 0; i < sources.length; i++)
                       builders[i] = new PartitionBuilder(partitionKey, sources[i], columns, stats);
   ```




----------------------------------------------------------------
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.

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