You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bd...@apache.org on 2019/04/25 17:27:07 UTC

[cassandra] branch cassandra-3.0 updated: Fix assorted gossip races and add related runtime checks

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

bdeggleston 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 c3ce32e  Fix assorted gossip races and add related runtime checks
c3ce32e is described below

commit c3ce32e239b1ba41faf1d58a942465b9bf45b986
Author: Blake Eggleston <bd...@gmail.com>
AuthorDate: Thu Mar 21 13:01:49 2019 -0700

    Fix assorted gossip races and add related runtime checks
    
    Patch by Blake Eggleston; Reviewed by Ariel Weisberg for CASSANDRA-15059
---
 CHANGES.txt                                        |   1 +
 build.xml                                          |   1 +
 ide/idea/workspace.xml                             |   2 +-
 .../cassandra/config/DatabaseDescriptor.java       |   8 +
 src/java/org/apache/cassandra/gms/Gossiper.java    | 192 +++++++++++++++------
 .../apache/cassandra/service/StorageService.java   |  10 +-
 .../cassandra/distributed/impl/Instance.java       |  21 ++-
 .../org/apache/cassandra/gms/GossiperTest.java     |   1 +
 .../gms/PendingRangeCalculatorServiceTest.java     |   1 +
 .../cassandra/locator/CloudstackSnitchTest.java    |   1 +
 .../apache/cassandra/locator/EC2SnitchTest.java    |   1 +
 .../cassandra/locator/GoogleCloudSnitchTest.java   |   1 +
 .../cassandra/locator/PropertyFileSnitchTest.java  |   1 +
 13 files changed, 170 insertions(+), 71 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 4f76c70..596d902 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.19
+ * Fix assorted gossip races and add related runtime checks (CASSANDRA-15059)
  * Fix mixed mode partition range scans with limit (CASSANDRA-15072)
  * cassandra-stress works with frozen collections: list and set (CASSANDRA-14907)
  * For nodetool listsnapshots output, put spaces between columns, and increase snapshot padding (CASSANDRA-14876)
diff --git a/build.xml b/build.xml
index 593b1c3..8ef8c67 100644
--- a/build.xml
+++ b/build.xml
@@ -1224,6 +1224,7 @@
         <jvmarg value="-Djava.security.egd=file:/dev/urandom" />
         <jvmarg value="-Dcassandra.testtag=@{testtag}"/>
         <jvmarg value="-Dcassandra.keepBriefBrief=${cassandra.keepBriefBrief}" />
+        <jvmarg value="-Dcassandra.strict.runtime.checks=true" />
 	<optjvmargs/>
         <classpath>
           <pathelement path="${java.class.path}"/>
diff --git a/ide/idea/workspace.xml b/ide/idea/workspace.xml
index e632b5f..3424af3 100644
--- a/ide/idea/workspace.xml
+++ b/ide/idea/workspace.xml
@@ -168,7 +168,7 @@
       <option name="MAIN_CLASS_NAME" value="" />
       <option name="METHOD_NAME" value="" />
       <option name="TEST_OBJECT" value="class" />
-      <option name="VM_PARAMETERS" value="-Dcassandra.config=file://$PROJECT_DIR$/test/conf/cassandra.yaml -Dlogback.configurationFile=file://$PROJECT_DIR$/test/conf/logback-test.xml -Dcassandra.logdir=$PROJECT_DIR$/build/test/logs -ea -XX:MaxMetaspaceSize=256M -XX:SoftRefLRUPolicyMSPerMB=0" />
+      <option name="VM_PARAMETERS" value="-Dcassandra.config=file://$PROJECT_DIR$/test/conf/cassandra.yaml -Dlogback.configurationFile=file://$PROJECT_DIR$/test/conf/logback-test.xml -Dcassandra.logdir=$PROJECT_DIR$/build/test/logs -ea -XX:MaxMetaspaceSize=256M -XX:SoftRefLRUPolicyMSPerMB=0 -Dcassandra.strict.runtime.checks=true" />
       <option name="PARAMETERS" value="" />
       <option name="WORKING_DIRECTORY" value="" />
       <option name="ENV_VARIABLES" />
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 8f4b338..db55c20 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -106,6 +106,9 @@ public class DatabaseDescriptor
 
     private static boolean daemonInitialized;
 
+    // turns some warnings into exceptions for testing
+    private static final boolean strictRuntimeChecks = Boolean.getBoolean("cassandra.strict.runtime.checks");
+
     public static boolean isDaemonInitialized()
     {
         return daemonInitialized;
@@ -2105,4 +2108,9 @@ public class DatabaseDescriptor
     {
         return conf.gc_warn_threshold_in_ms;
     }
+
+    public static boolean strictRuntimeChecks()
+    {
+        return strictRuntimeChecks;
+    }
 }
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index 59b366a..4ea0a4a 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -25,11 +25,15 @@ import java.util.concurrent.*;
 import java.util.concurrent.locks.ReentrantLock;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.ListenableFutureTask;
 import com.google.common.util.concurrent.Uninterruptibles;
 
+import io.netty.util.concurrent.FastThreadLocal;
 import org.apache.cassandra.utils.MBeanWrapper;
+import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -59,11 +63,17 @@ import org.apache.cassandra.utils.JVMStabilityInspector;
  * of the three above mentioned messages updates the Failure Detector with the liveness information.
  * Upon hearing a GossipShutdownMessage, this module will instantly mark the remote node as down in
  * the Failure Detector.
+ *
+ * This class is not threadsafe and any state changes should happen in the gossip stage.
  */
 
 public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 {
     public static final String MBEAN_NAME = "org.apache.cassandra.net:type=Gossiper";
+    public static class Props
+    {
+        public static final String DISABLE_THREAD_VALIDATION = "cassandra.gossip.disable_thread_validation";
+    }
 
     private static final DebuggableScheduledThreadPoolExecutor executor = new DebuggableScheduledThreadPoolExecutor("GossipTasks");
 
@@ -82,6 +92,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     public final static int intervalInMillis = 1000;
     public final static int QUARANTINE_DELAY = StorageService.RING_DELAY * 2;
     private static final Logger logger = LoggerFactory.getLogger(Gossiper.class);
+    private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 15L, TimeUnit.MINUTES);
     public static final Gossiper instance = new Gossiper();
 
     // Timestamp to prevent processing any in-flight messages for we've not send any SYN yet, see CASSANDRA-12653.
@@ -130,6 +141,37 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
     private volatile long lastProcessedMessageAt = System.currentTimeMillis();
 
+    private static FastThreadLocal<Boolean> isGossipStage = new FastThreadLocal<>();
+
+    private static final boolean disableThreadValidation = Boolean.getBoolean(Props.DISABLE_THREAD_VALIDATION);
+
+    private static boolean isInGossipStage()
+    {
+        Boolean isGossip = isGossipStage.get();
+        if (isGossip == null)
+        {
+            isGossip = Thread.currentThread().getName().contains(Stage.GOSSIP.getJmxName());
+            isGossipStage.set(isGossip);
+        }
+        return isGossip;
+    }
+
+    private static void checkProperThreadForStateMutation()
+    {
+        if (disableThreadValidation || isInGossipStage())
+            return;
+
+        IllegalStateException e = new IllegalStateException("Attempting gossip state mutation from illegal thread: " + Thread.currentThread().getName());
+        if (DatabaseDescriptor.strictRuntimeChecks())
+        {
+            throw e;
+        }
+        else
+        {
+            noSpamLogger.getStatement(Throwables.getStackTraceAsString(e)).error(e.getMessage(), e);
+        }
+    }
+
     private class GossipTask implements Runnable
     {
         public void run()
@@ -317,6 +359,27 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         return state.equals(VersionedValue.SHUTDOWN);
     }
 
+    public static void runInGossipStageBlocking(Runnable runnable)
+    {
+        // run immediately if we're already in the gossip stage
+        if (isInGossipStage())
+        {
+            runnable.run();
+            return;
+        }
+
+        ListenableFutureTask task = ListenableFutureTask.create(runnable, null);
+        StageManager.getStage(Stage.GOSSIP).execute(task);
+        try
+        {
+            task.get();
+        }
+        catch (InterruptedException | ExecutionException e)
+        {
+            throw new AssertionError(e);
+        }
+    }
+
     /**
      * This method is part of IFailureDetectionEventListener interface. This is invoked
      * by the Failure Detector when it convicts an end point.
@@ -325,24 +388,26 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      */
     public void convict(InetAddress endpoint, double phi)
     {
-        EndpointState epState = endpointStateMap.get(endpoint);
-        if (epState == null)
-            return;
+        runInGossipStageBlocking(() -> {
+            EndpointState epState = endpointStateMap.get(endpoint);
+            if (epState == null)
+                return;
 
-        if (!epState.isAlive())
-            return;
+            if (!epState.isAlive())
+                return;
 
-        logger.debug("Convicting {} with status {} - alive {}", endpoint, getGossipStatus(epState), epState.isAlive());
+            logger.debug("Convicting {} with status {} - alive {}", endpoint, getGossipStatus(epState), epState.isAlive());
 
 
-        if (isShutdown(endpoint))
-        {
-            markAsShutdown(endpoint);
-        }
-        else
-        {
-            markDead(endpoint, epState);
-        }
+            if (isShutdown(endpoint))
+            {
+                markAsShutdown(endpoint);
+            }
+            else
+            {
+                markDead(endpoint, epState);
+            }
+        });
     }
 
     /**
@@ -351,6 +416,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      */
     protected void markAsShutdown(InetAddress endpoint)
     {
+        checkProperThreadForStateMutation();
         EndpointState epState = endpointStateMap.get(endpoint);
         if (epState == null)
             return;
@@ -382,6 +448,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      */
     private void evictFromMembership(InetAddress endpoint)
     {
+        checkProperThreadForStateMutation();
         unreachableEndpoints.remove(endpoint);
         endpointStateMap.remove(endpoint);
         expireTimeEndpointMap.remove(endpoint);
@@ -396,6 +463,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      */
     public void removeEndpoint(InetAddress endpoint)
     {
+        checkProperThreadForStateMutation();
         // do subscribers first so anything in the subscriber that depends on gossiper state won't get confused
         for (IEndpointStateChangeSubscriber subscriber : subscribers)
             subscriber.onRemove(endpoint);
@@ -456,6 +524,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      */
     public void replacedEndpoint(InetAddress endpoint)
     {
+        checkProperThreadForStateMutation();
         removeEndpoint(endpoint);
         evictFromMembership(endpoint);
         replacementQuarantine(endpoint);
@@ -568,49 +637,51 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     public void assassinateEndpoint(String address) throws UnknownHostException
     {
         InetAddress endpoint = InetAddress.getByName(address);
-        EndpointState epState = endpointStateMap.get(endpoint);
-        Collection<Token> tokens = null;
-        logger.warn("Assassinating {} via gossip", endpoint);
+        runInGossipStageBlocking(() -> {
+            EndpointState epState = endpointStateMap.get(endpoint);
+            Collection<Token> tokens = null;
+            logger.warn("Assassinating {} via gossip", endpoint);
 
-        if (epState == null)
-        {
-            epState = new EndpointState(new HeartBeatState((int) ((System.currentTimeMillis() + 60000) / 1000), 9999));
-        }
-        else
-        {
-            int generation = epState.getHeartBeatState().getGeneration();
-            int heartbeat = epState.getHeartBeatState().getHeartBeatVersion();
-            logger.info("Sleeping for {}ms to ensure {} does not change", StorageService.RING_DELAY, endpoint);
-            Uninterruptibles.sleepUninterruptibly(StorageService.RING_DELAY, TimeUnit.MILLISECONDS);
-            // make sure it did not change
-            EndpointState newState = endpointStateMap.get(endpoint);
-            if (newState == null)
-                logger.warn("Endpoint {} disappeared while trying to assassinate, continuing anyway", endpoint);
-            else if (newState.getHeartBeatState().getGeneration() != generation)
-                throw new RuntimeException("Endpoint still alive: " + endpoint + " generation changed while trying to assassinate it");
-            else if (newState.getHeartBeatState().getHeartBeatVersion() != heartbeat)
-                throw new RuntimeException("Endpoint still alive: " + endpoint + " heartbeat changed while trying to assassinate it");
-            epState.updateTimestamp(); // make sure we don't evict it too soon
-            epState.getHeartBeatState().forceNewerGenerationUnsafe();
-        }
+            if (epState == null)
+            {
+                epState = new EndpointState(new HeartBeatState((int) ((System.currentTimeMillis() + 60000) / 1000), 9999));
+            }
+            else
+            {
+                int generation = epState.getHeartBeatState().getGeneration();
+                int heartbeat = epState.getHeartBeatState().getHeartBeatVersion();
+                logger.info("Sleeping for {}ms to ensure {} does not change", StorageService.RING_DELAY, endpoint);
+                Uninterruptibles.sleepUninterruptibly(StorageService.RING_DELAY, TimeUnit.MILLISECONDS);
+                // make sure it did not change
+                EndpointState newState = endpointStateMap.get(endpoint);
+                if (newState == null)
+                    logger.warn("Endpoint {} disappeared while trying to assassinate, continuing anyway", endpoint);
+                else if (newState.getHeartBeatState().getGeneration() != generation)
+                    throw new RuntimeException("Endpoint still alive: " + endpoint + " generation changed while trying to assassinate it");
+                else if (newState.getHeartBeatState().getHeartBeatVersion() != heartbeat)
+                    throw new RuntimeException("Endpoint still alive: " + endpoint + " heartbeat changed while trying to assassinate it");
+                epState.updateTimestamp(); // make sure we don't evict it too soon
+                epState.getHeartBeatState().forceNewerGenerationUnsafe();
+            }
 
-        try
-        {
-            tokens = StorageService.instance.getTokenMetadata().getTokens(endpoint);
-        }
-        catch (Throwable th)
-        {
-            JVMStabilityInspector.inspectThrowable(th);
-            // TODO this is broken
-            logger.warn("Unable to calculate tokens for {}.  Will use a random one", address);
-            tokens = Collections.singletonList(StorageService.instance.getTokenMetadata().partitioner.getRandomToken());
-        }
+            try
+            {
+                tokens = StorageService.instance.getTokenMetadata().getTokens(endpoint);
+            }
+            catch (Throwable th)
+            {
+                JVMStabilityInspector.inspectThrowable(th);
+                // TODO this is broken
+                logger.warn("Unable to calculate tokens for {}.  Will use a random one", address);
+                tokens = Collections.singletonList(StorageService.instance.getTokenMetadata().partitioner.getRandomToken());
+            }
 
-        // do not pass go, do not collect 200 dollars, just gtfo
-        epState.addApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.left(tokens, computeExpireTime()));
-        handleMajorStateChange(endpoint, epState);
-        Uninterruptibles.sleepUninterruptibly(intervalInMillis * 4, TimeUnit.MILLISECONDS);
-        logger.warn("Finished assassinating {}", endpoint);
+            // do not pass go, do not collect 200 dollars, just gtfo
+            epState.addApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.left(tokens, computeExpireTime()));
+            handleMajorStateChange(endpoint, epState);
+            Uninterruptibles.sleepUninterruptibly(intervalInMillis * 4, TimeUnit.MILLISECONDS);
+            logger.warn("Finished assassinating {}", endpoint);
+        });
     }
 
     public boolean isKnownEndpoint(InetAddress endpoint)
@@ -772,8 +843,10 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
                     && TimeUnit.NANOSECONDS.toMillis(nowNano - epState.getUpdateTimestamp()) > fatClientTimeout)
                 {
                     logger.info("FatClient {} has been silent for {}ms, removing from gossip", endpoint, fatClientTimeout);
-                    removeEndpoint(endpoint); // will put it in justRemovedEndpoints to respect quarantine delay
-                    evictFromMembership(endpoint); // can get rid of the state immediately
+                    runInGossipStageBlocking(() -> {
+                        removeEndpoint(endpoint); // will put it in justRemovedEndpoints to respect quarantine delay
+                        evictFromMembership(endpoint); // can get rid of the state immediately
+                    });
                 }
 
                 // check for dead state removal
@@ -974,7 +1047,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
             public void response(MessageIn msg)
             {
-                realMarkAlive(addr, localState);
+                runInGossipStageBlocking(() -> realMarkAlive(addr, localState));
             }
         };
 
@@ -984,6 +1057,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     @VisibleForTesting
     public void realMarkAlive(final InetAddress addr, final EndpointState localState)
     {
+        checkProperThreadForStateMutation();
         if (logger.isTraceEnabled())
             logger.trace("marking as alive {}", addr);
         localState.markAlive();
@@ -1002,6 +1076,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     @VisibleForTesting
     public void markDead(InetAddress addr, EndpointState localState)
     {
+        checkProperThreadForStateMutation();
         if (logger.isTraceEnabled())
             logger.trace("marking as down {}", addr);
         localState.markDead();
@@ -1022,6 +1097,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      */
     private void handleMajorStateChange(InetAddress ep, EndpointState epState)
     {
+        checkProperThreadForStateMutation();
         EndpointState localEpState = endpointStateMap.get(ep);
         if (!isDeadState(epState))
         {
@@ -1093,6 +1169,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
     void applyStateLocally(Map<InetAddress, EndpointState> epStateMap)
     {
+        checkProperThreadForStateMutation();
         for (Entry<InetAddress, EndpointState> entry : epStateMap.entrySet())
         {
             InetAddress ep = entry.getKey();
@@ -1409,6 +1486,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      */
     public void addSavedEndpoint(InetAddress ep)
     {
+        checkProperThreadForStateMutation();
         if (ep.equals(FBUtilities.getBroadcastAddress()))
         {
             logger.debug("Attempt to add self as saved endpoint");
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index b07401a..a1f361d 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -635,7 +635,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 {
                     if (loadedHostIds.containsKey(ep))
                         tokenMetadata.updateHostId(loadedHostIds.get(ep), ep);
-                    Gossiper.instance.addSavedEndpoint(ep);
+                    Gossiper.runInGossipStageBlocking(() -> Gossiper.instance.addSavedEndpoint(ep));
                 }
             }
         }
@@ -962,8 +962,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 // remove the existing info about the replaced node.
                 if (!current.isEmpty())
                 {
-                    for (InetAddress existing : current)
-                        Gossiper.instance.replacedEndpoint(existing);
+                    Gossiper.runInGossipStageBlocking(() -> {
+                        for (InetAddress existing : current)
+                            Gossiper.instance.replacedEndpoint(existing);
+                    });
                 }
             }
             else
@@ -2349,7 +2351,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     /** unlike excise we just need this endpoint gone without going through any notifications **/
     private void removeEndpoint(InetAddress endpoint)
     {
-        Gossiper.instance.removeEndpoint(endpoint);
+        Gossiper.runInGossipStageBlocking(() -> Gossiper.instance.removeEndpoint(endpoint));
         SystemKeyspace.removeEndpoint(endpoint);
     }
 
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index dddabf8..94dbc96 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -348,15 +348,18 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
             for (int i = 0; i < tokens.size(); i++)
             {
                 InetAddressAndPort ep = hosts.get(i);
-                Gossiper.instance.initializeNodeUnsafe(ep.address, hostIds.get(i), 1);
-                Gossiper.instance.injectApplicationState(ep.address,
-                        ApplicationState.TOKENS,
-                        new VersionedValue.VersionedValueFactory(partitioner).tokens(Collections.singleton(tokens.get(i))));
-                storageService.onChange(ep.address,
-                        ApplicationState.STATUS,
-                        new VersionedValue.VersionedValueFactory(partitioner).normal(Collections.singleton(tokens.get(i))));
-                Gossiper.instance.realMarkAlive(ep.address, Gossiper.instance.getEndpointStateForEndpoint(ep.address));
-
+                UUID hostId = hostIds.get(i);
+                Token token = tokens.get(i);
+                Gossiper.runInGossipStageBlocking(() -> {
+                    Gossiper.instance.initializeNodeUnsafe(ep.address, hostId, 1);
+                    Gossiper.instance.injectApplicationState(ep.address,
+                                                             ApplicationState.TOKENS,
+                                                             new VersionedValue.VersionedValueFactory(partitioner).tokens(Collections.singleton(token)));
+                    storageService.onChange(ep.address,
+                                            ApplicationState.STATUS,
+                                            new VersionedValue.VersionedValueFactory(partitioner).normal(Collections.singleton(token)));
+                    Gossiper.instance.realMarkAlive(ep.address, Gossiper.instance.getEndpointStateForEndpoint(ep.address));
+                });
                 int version = Math.min(MessagingService.current_version, cluster.get(ep).getMessagingVersion());
                 MessagingService.instance().setVersion(ep.address, version);
             }
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index eb01305..f23c016 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -47,6 +47,7 @@ public class GossiperTest
 {
     static
     {
+        System.setProperty(Gossiper.Props.DISABLE_THREAD_VALIDATION, "true");
         DatabaseDescriptor.setDaemonInitialized();
     }
     static final IPartitioner partitioner = new RandomPartitioner();
diff --git a/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java b/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
index 90bbf1d..7892de4 100644
--- a/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
+++ b/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
@@ -55,6 +55,7 @@ public class PendingRangeCalculatorServiceTest
     @BeforeClass
     public static void setUp() throws ConfigurationException
     {
+        System.setProperty(Gossiper.Props.DISABLE_THREAD_VALIDATION, "true");
         SchemaLoader.prepareServer();
         StorageService.instance.initServer();
     }
diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 37b5fa9..5dc34df 100644
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@ -45,6 +45,7 @@ public class CloudstackSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        System.setProperty(Gossiper.Props.DISABLE_THREAD_VALIDATION, "true");
         DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
diff --git a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
index 1a0c8fe..9d078ce 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -50,6 +50,7 @@ public class EC2SnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        System.setProperty(Gossiper.Props.DISABLE_THREAD_VALIDATION, "true");
         DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
index ef64f21..04b71e9 100644
--- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
@@ -46,6 +46,7 @@ public class GoogleCloudSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        System.setProperty(Gossiper.Props.DISABLE_THREAD_VALIDATION, "true");
         DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
diff --git a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
index e9a307b..29ea4d5 100644
--- a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
@@ -64,6 +64,7 @@ public class PropertyFileSnitchTest
     @Before
     public void setup() throws ConfigurationException, IOException
     {
+        System.setProperty(Gossiper.Props.DISABLE_THREAD_VALIDATION, "true");
         String confFile = FBUtilities.resourceToFile(PropertyFileSnitch.SNITCH_PROPERTIES_FILENAME);
         effectiveFile = Paths.get(confFile);
         backupFile = Paths.get(confFile + ".bak");


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