You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jo...@apache.org on 2022/02/03 21:32:35 UTC

[cassandra] branch cassandra-3.0 updated: Fixes for intermittent in-JVM dtest failures

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

jonmeredith pushed a commit to branch cassandra-3.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-3.0 by this push:
     new 9ff28fc  Fixes for intermittent in-JVM dtest failures
9ff28fc is described below

commit 9ff28fc717e7926b5cf7c8acc85a35e0cfb43595
Author: Jon Meredith <ht...@apache.org>
AuthorDate: Wed Feb 2 17:09:42 2022 -0700

    Fixes for intermittent in-JVM dtest failures
    
    patch by Jon Meredith, Caleb Rackliffe, Marcus Eriksson, Benedict Elliott Smith <be...@apache.org>; reviewed by Caleb Rackliffe for CASSANDRA-17256
---
 src/java/org/apache/cassandra/service/StorageService.java  | 14 ++++++++++----
 .../apache/cassandra/distributed/impl/AbstractCluster.java |  8 +++++++-
 .../org/apache/cassandra/distributed/impl/Instance.java    | 10 ++++++++++
 .../org/apache/cassandra/distributed/test/GossipTest.java  |  3 ---
 .../distributed/upgrade/CompactStorage2to3UpgradeTest.java |  3 ++-
 .../apache/cassandra/streaming/StreamingTransferTest.java  |  2 +-
 6 files changed, 30 insertions(+), 10 deletions(-)

diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 78d1120..22040c1 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -4389,10 +4389,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             if (!isFinalShutdown)
                 setMode(Mode.DRAINING, "flushing column families", false);
 
-            // disable autocompaction - we don't want to start any new compactions while we are draining
-            for (Keyspace keyspace : Keyspace.all())
-                for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
-                    cfs.disableAutoCompaction();
+            disableAutoCompaction();
 
             // count CFs first, since forceFlush could block for the flushWriter to get a queue slot empty
             totalCFs = 0;
@@ -4465,6 +4462,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
     }
 
+    @VisibleForTesting
+    public void disableAutoCompaction()
+    {
+        // disable autocompaction - we don't want to start any new compactions while we are draining
+        for (Keyspace keyspace : Keyspace.all())
+            for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
+                cfs.disableAutoCompaction();
+    }
+
     /**
      * Some services are shutdown during draining and we should not attempt to start them again.
      *
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
index ede075f..83dbfbc 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
@@ -195,9 +195,15 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         @Override
         public synchronized void startup()
         {
+            startup(AbstractCluster.this);
+        }
+
+        @Override
+        public synchronized void startup(ICluster cluster)
+        {
             if (!isShutdown)
                 throw new IllegalStateException();
-            delegateForStartup().startup(AbstractCluster.this);
+            delegateForStartup().startup(cluster);
             isShutdown = false;
             updateMessagingVersions();
         }
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index aad46dd..776def3 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -550,6 +550,14 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
         sync(() -> {
             try
             {
+                if (config.has(GOSSIP))
+                {
+                    // TODO: hacky
+                    System.setProperty("cassandra.ring_delay_ms", "15000");
+                    System.setProperty("cassandra.consistent.rangemovement", "false");
+                    System.setProperty("cassandra.consistent.simultaneousmoves.allow", "true");
+                }
+
                 mkdirs();
 
                 assert config.networkTopology().contains(config.broadcastAddress());
@@ -768,6 +776,8 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
                 );
             }
 
+            error = parallelRun(error, executor, StorageService.instance::disableAutoCompaction);
+
             error = parallelRun(error, executor,
                                 () -> Gossiper.instance.stopShutdownAndWait(1L, MINUTES),
                                 CompactionManager.instance::forceShutdown,
diff --git a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java
index ba6027b..0cff2b0 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java
@@ -61,9 +61,6 @@ public class GossipTest extends TestBaseImpl
     public void nodeDownDuringMove() throws Throwable
     {
         int liveCount = 1;
-        System.setProperty("cassandra.ring_delay_ms", "5000"); // down from 30s default
-        System.setProperty("cassandra.consistent.rangemovement", "false");
-        System.setProperty("cassandra.consistent.simultaneousmoves.allow", "true");
         try (Cluster cluster = Cluster.build(2 + liveCount)
                                       .withConfig(config -> config.with(NETWORK).with(GOSSIP))
                                       .createWithoutStarting())
diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/CompactStorage2to3UpgradeTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/CompactStorage2to3UpgradeTest.java
index 9ea54c3..3cf9fff 100644
--- a/test/distributed/org/apache/cassandra/distributed/upgrade/CompactStorage2to3UpgradeTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/upgrade/CompactStorage2to3UpgradeTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.distributed.upgrade;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.junit.Ignore;
 import org.junit.Test;
 
 import org.apache.cassandra.distributed.UpgradeableCluster;
@@ -37,7 +38,7 @@ import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
 
-
+@Ignore // 2.2 branch is no longer maintained and using dtest-api 0.0.8 which is incompatible with current
 public class CompactStorage2to3UpgradeTest extends UpgradeTestBase
 {
     @Test
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index fb5e03f..a89754f 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -299,7 +299,7 @@ public class StreamingTransferTest
     {
         final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_INDEX);
-
+        cfs.disableAutoCompaction();
         List<String> keys = createAndTransfer(cfs, new Mutator()
         {
             public void mutate(String key, String col, long timestamp) throws Exception

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