You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jm...@apache.org on 2021/08/27 19:12:51 UTC

[cassandra] branch trunk updated (33979e3 -> dfb465c)

This is an automated email from the ASF dual-hosted git repository.

jmckenzie pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git.


    from 33979e3  Merge branch 'cassandra-4.0' into trunk
     new 2b6799a  Harden PrunableArrayQueue against Pruner implementations that might throw exceptions
     new dfb465c  Merge branch 'cassandra-4.0' into trunk

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CHANGES.txt                                        |  5 +-
 .../apache/cassandra/net/PrunableArrayQueue.java   | 49 ++++++++++++++-
 .../cassandra/net/PrunableArrayQueueTest.java      | 73 ++++++++++++++++++++++
 3 files changed, 122 insertions(+), 5 deletions(-)

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


[cassandra] 02/02: Merge branch 'cassandra-4.0' into trunk

Posted by jm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jmckenzie pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit dfb465c2e08ff158bf9991ec0b9dbfe546c1c8a8
Merge: 33979e3 2b6799a
Author: Josh McKenzie <jm...@apache.org>
AuthorDate: Fri Aug 27 15:11:37 2021 -0400

    Merge branch 'cassandra-4.0' into trunk

 CHANGES.txt                                        |  5 +-
 .../apache/cassandra/net/PrunableArrayQueue.java   | 49 ++++++++++++++-
 .../cassandra/net/PrunableArrayQueueTest.java      | 73 ++++++++++++++++++++++
 3 files changed, 122 insertions(+), 5 deletions(-)

diff --cc CHANGES.txt
index 23ed6b1,b9d4aa1..84a0b29
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,24 -1,5 +1,25 @@@
 -4.0.1
 - * Harden PrunableArrayQueue against Pruner implementations that might throw exceptions (CASSANDRA-16866)
 +4.1
 + * Add client warnings and abort to tombstone and coordinator reads which go past a low/high watermark (CASSANDRA-16850)
 + * Add TTL support to nodetool snapshots (CASSANDRA-16789)
 + * Allow CommitLogSegmentReader to optionally skip sync marker CRC checks (CASSANDRA-16842)
 + * allow blocking IPs from updating metrics about traffic (CASSANDRA-16859)
 + * Request-Based Native Transport Rate-Limiting (CASSANDRA-16663)
 + * Implement nodetool getauditlog command (CASSANDRA-16725)
 + * Clean up repair code (CASSANDRA-13720)
 + * Background schedule to clean up orphaned hints files (CASSANDRA-16815)
 + * Modify SecondaryIndexManager#indexPartition() to retrieve only columns for which indexes are actually being built (CASSANDRA-16776)
 + * Batch the token metadata update to improve the speed (CASSANDRA-15291)
 + * Reduce the log level on "expected" repair exceptions (CASSANDRA-16775)
 + * Make JMXTimer expose attributes using consistent time unit (CASSANDRA-16760)
 + * Remove check on gossip status from DynamicEndpointSnitch::updateScores (CASSANDRA-11671)
 + * Fix AbstractReadQuery::toCQLString not returning valid CQL (CASSANDRA-16510)
 + * Log when compacting many tombstones (CASSANDRA-16780)
 + * Display bytes per level in tablestats for LCS tables (CASSANDRA-16799)
 + * Add isolated flush timer to CommitLogMetrics and ensure writes correspond to single WaitingOnCommit data points (CASSANDRA-16701)
 + * Add a system property to set hostId if not yet initialized (CASSANDRA-14582)
 + * GossiperTest.testHasVersion3Nodes didn't take into account trunk version changes, fixed to rely on latest version (CASSANDRA-16651)
 +Merged from 4.0:
++ * Harden PrunableArrayQueue against Pruner implementations that might throw exceptions (CASSANDRA-16866) 
   * Move RepairedDataInfo to the execution controller rather than the ReadCommand to avoid unintended sharing (CASSANDRA-16721)
   * Bump zstd-jni version to 1.5.0-4 (CASSANDRA-16884)
   * Remove assumption that all urgent messages are small (CASSANDRA-16877)

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


[cassandra] 01/02: Harden PrunableArrayQueue against Pruner implementations that might throw exceptions

Posted by jm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jmckenzie pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 2b6799a3941353444d964c769415937920099b18
Author: Alex Petrov <ol...@gmail.com>
AuthorDate: Fri Aug 13 12:02:50 2021 -0400

    Harden PrunableArrayQueue against Pruner implementations that might throw exceptions
    
    patch by Alex Petrov; reviewed by Caleb Rackliffe and Jon Meredith for CASSANDRA-16866
    
    Co-authored by Alex Petrov <ol...@gmail.com>
    Co-authored by Josh McKenzie <jm...@apache.org>
---
 CHANGES.txt                                        |  1 +
 .../apache/cassandra/net/PrunableArrayQueue.java   | 49 ++++++++++++++-
 .../cassandra/net/PrunableArrayQueueTest.java      | 73 ++++++++++++++++++++++
 3 files changed, 120 insertions(+), 3 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 22818c9..b9d4aa1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0.1
+ * Harden PrunableArrayQueue against Pruner implementations that might throw exceptions (CASSANDRA-16866)
  * Move RepairedDataInfo to the execution controller rather than the ReadCommand to avoid unintended sharing (CASSANDRA-16721)
  * Bump zstd-jni version to 1.5.0-4 (CASSANDRA-16884)
  * Remove assumption that all urgent messages are small (CASSANDRA-16877)
diff --git a/src/java/org/apache/cassandra/net/PrunableArrayQueue.java b/src/java/org/apache/cassandra/net/PrunableArrayQueue.java
index 1fca43c..9c91ca7 100644
--- a/src/java/org/apache/cassandra/net/PrunableArrayQueue.java
+++ b/src/java/org/apache/cassandra/net/PrunableArrayQueue.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.net;
 
 import java.util.function.Predicate;
 
+import org.apache.cassandra.utils.Throwables;
+
 /**
  * A growing array-based queue that allows efficient bulk in-place removal.
  *
@@ -103,10 +105,12 @@ final class PrunableArrayQueue<E>
      *
      * @return count of removed elements.
      */
+    @SuppressWarnings("ThrowFromFinallyBlock")
     int prune(Pruner<E> pruner)
     {
         E e;
         int removed = 0;
+        Throwable error = null;
 
         try
         {
@@ -120,11 +124,35 @@ final class PrunableArrayQueue<E>
                 int k = (tail - 1 - i) & mask;
                 e = buffer[k];
 
-                if (pruner.shouldPrune(e))
+                boolean shouldPrune = false;
+
+                // If any error has been thrown from the Pruner callbacks, don't bother asking the
+                // pruner. Just move any elements that need to be moved, correct the head, and rethrow.
+                if (error == null)
+                {
+                    try
+                    {
+                        shouldPrune = pruner.shouldPrune(e);
+                    }
+                    catch (Throwable t)
+                    {
+                        error = t;
+                    }
+                }
+
+                if (shouldPrune)
                 {
                     buffer[k] = null;
                     removed++;
-                    pruner.onPruned(e);
+
+                    try
+                    {
+                        pruner.onPruned(e);
+                    }
+                    catch (Throwable t)
+                    {
+                        error = t;
+                    }
                 }
                 else
                 {
@@ -133,13 +161,28 @@ final class PrunableArrayQueue<E>
                         buffer[(k + removed) & mask] = e;
                         buffer[k] = null;
                     }
-                    pruner.onKept(e);
+
+                    try
+                    {
+                        pruner.onKept(e);
+                    }
+                    catch (Throwable t)
+                    {
+                        if (error == null)
+                        {
+                            error = t;
+                        }
+                    }
                 }
             }
         }
         finally
         {
             head = (head + removed) & mask;
+
+            // Rethrow any error(s) from the Pruner callbacks, but only after the queue state is valid.
+            if (error != null)
+                throw Throwables.unchecked(error);
         }
 
         return removed;
diff --git a/test/unit/org/apache/cassandra/net/PrunableArrayQueueTest.java b/test/unit/org/apache/cassandra/net/PrunableArrayQueueTest.java
index c4fd55a..34f61a6 100644
--- a/test/unit/org/apache/cassandra/net/PrunableArrayQueueTest.java
+++ b/test/unit/org/apache/cassandra/net/PrunableArrayQueueTest.java
@@ -17,14 +17,21 @@
  */
 package org.apache.cassandra.net;
 
+import java.util.Random;
+
 import org.junit.Test;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.net.PrunableArrayQueue;
 
 import static org.junit.Assert.*;
 
 public class PrunableArrayQueueTest
 {
+    private static final Logger logger = LoggerFactory.getLogger(PrunableArrayQueueTest.class);
+
     private final PrunableArrayQueue<Integer> queue = new PrunableArrayQueue<>(8);
 
     @Test
@@ -127,4 +134,70 @@ public class PrunableArrayQueueTest
             assertEquals((Integer) i, queue.poll());
         assertTrue(queue.isEmpty());
     }
+
+    @Test
+    public void testUnreliablePruner()
+    {
+        long seed = System.currentTimeMillis();
+        Random rand = new Random(seed);
+
+        logger.info("Testing unreliable pruner with random seed {}...", seed);
+
+        int iteratons = 100;
+        int startingQueueSize = 1024;
+        double pruneChance = 0.1;
+        double errorOnKeptChance = 0.00005;
+        double errorOnPruneChance = 0.00002;
+
+        for (int i = 0; i < iteratons; i++)
+        {
+            int failureValue = rand.nextInt(startingQueueSize);
+
+            PrunableArrayQueue<Integer> testQueue = new PrunableArrayQueue<>(startingQueueSize);
+
+            for (int o = 0; o < startingQueueSize; o++)
+                testQueue.offer(o);
+
+            class UnreliablePruner implements PrunableArrayQueue.Pruner<Integer>
+            {
+                public boolean shouldPrune(Integer value)
+                {
+                    if (rand.nextDouble() < errorOnPruneChance)
+                        throw new RuntimeException("Failed on pruning check for value: " + value);
+
+                    return rand.nextDouble() < pruneChance;
+                }
+
+                public void onPruned(Integer value)
+                {
+                    if (value == failureValue)
+                        throw new RuntimeException("Failed on pruned value: " + value);
+                }
+
+                public void onKept(Integer value)
+                {
+                    if (rand.nextDouble() < errorOnKeptChance)
+                        throw new RuntimeException("Failed on retained value: " + value);
+                }
+            }
+
+            assertEquals(startingQueueSize, testQueue.size());
+
+            try
+            {
+                testQueue.prune(new UnreliablePruner());
+            }
+            catch (RuntimeException e)
+            {
+                logger.info("Expected pruning failure with seed {}", seed, e);
+            }
+
+            for (int p = 0, postPruneSize = testQueue.size(); p < postPruneSize; p++)
+            {
+                assertNotNull("Queue should contain no null elements after pruning. Seed: " + seed + ". Iteration: " + i, testQueue.poll());
+            }
+
+            assertEquals("Queue size should be zero after draining. Seed: " + seed + ". Iteration: " + i, 0, testQueue.size());
+        }
+    }
 }
\ No newline at end of file

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