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

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


The following commit(s) were added to refs/heads/trunk by this push:
     new ffe4d85df2 Use WithProperties in try-with-resources to improve 
properties handling in tests
ffe4d85df2 is described below

commit ffe4d85df23e22be78b8047e91e4a065c5c73c06
Author: Bernardo Botella Corbi <[email protected]>
AuthorDate: Thu May 18 16:21:16 2023 -0700

    Use WithProperties in try-with-resources to improve properties handling in 
tests
    
    patch by Bernardo Botella Corbi; reviewed by Stefan Miklosovic, Maxim 
Muzafarov and Jacek Lewandowski for CASSANDRA-18453
---
 checkstyle_test.xml                                |  7 +++
 .../cassandra/distributed/action/GossipHelper.java |  2 +-
 .../test/BootstrapBinaryDisabledTest.java          | 10 ++--
 .../test/DataResurrectionCheckTest.java            | 11 ++--
 .../test/FailingResponseDoesNotLogTest.java        | 17 +++---
 .../distributed/test/MessageForwardingTest.java    | 18 ++-----
 .../distributed/test/MigrationCoordinatorTest.java |  6 +--
 .../distributed/test/NativeMixedVersionTest.java   | 13 +++--
 .../distributed/test/PaxosRepair2Test.java         | 18 +++----
 .../distributed/test/VirtualTableLogsTest.java     | 24 ++++-----
 .../distributed/test/ring/BootstrapTest.java       | 20 +++----
 .../cassandra/auth/PasswordAuthenticatorTest.java  | 11 +---
 .../config/CassandraRelevantPropertiesTest.java    | 63 +++++-----------------
 .../cassandra/config/DatabaseDescriptorTest.java   | 11 +---
 .../cql3/validation/operations/TTLTest.java        |  8 +--
 .../cassandra/db/commitlog/CommitLogTest.java      | 57 ++++++++------------
 .../org/apache/cassandra/gms/ShadowRoundTest.java  | 15 ++----
 .../org/apache/cassandra/io/sstable/ScrubTest.java |  7 ++-
 .../org/apache/cassandra/net/ConnectionTest.java   |  8 +--
 .../CustomSslContextFactoryConfigTest.java         |  7 ++-
 .../CustomSslContextFactoryInvalidConfigTest.java  |  6 ++-
 .../security/DefaultSslContextFactoryTest.java     | 16 +++---
 .../security/FileBasedSslContextFactoryTest.java   |  6 ++-
 .../PEMBasedSslContextFactoryConfigTest.java       |  7 ++-
 ...tFactoryConfigWithMismatchingPasswordsTest.java |  7 ++-
 ...ontextFactoryConfigWithUnencryptedKeysTest.java | 11 ++--
 ...PEMBasedSslContextFactoryInvalidConfigTest.java |  7 ++-
 .../security/PEMBasedSslContextFactoryTest.java    | 16 +++---
 .../PEMJKSSslContextFactoryConfigTest.java         |  7 ++-
 .../AbstractFilesystemOwnershipCheckTest.java      |  7 ++-
 .../apache/cassandra/service/ClientStateTest.java  |  7 ++-
 .../service/StorageServiceServerTest.java          |  7 +--
 ...ropertiesBasedFileSystemOwnershipCheckTest.java |  5 +-
 .../service/reads/range/RangeCommandsTest.java     |  7 ++-
 .../tools/StandaloneUpgraderOnSStablesTest.java    |  7 ++-
 .../tools/StandaloneVerifierOnSSTablesTest.java    |  7 ++-
 36 files changed, 202 insertions(+), 261 deletions(-)

diff --git a/checkstyle_test.xml b/checkstyle_test.xml
index d487a16c4c..720bc81cef 100644
--- a/checkstyle_test.xml
+++ b/checkstyle_test.xml
@@ -118,6 +118,13 @@
       <property name="message" value="Use the CassandraRelevantProperties or 
CassandraRelevantEnv instead." />
     </module>
 
+    <module name="RegexpSinglelineJava">
+      <property name="id" value="clearValueSystemPropertyUsage"/>
+      <property name="format" value="\.clearValue\("/>
+      <property name="ignoreComments" value="true"/>
+      <property name="message" value="Please use WithProperties in 
try-with-resources instead. See CASSANDRA-18453." />
+    </module>
+
     <module name="RedundantImport"/>
     <module name="UnusedImports"/>
   </module>
diff --git 
a/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java 
b/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java
index b3acccaa2d..7a46935646 100644
--- a/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java
+++ b/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java
@@ -483,7 +483,7 @@ public class GossipHelper
         finally
         {
             if (prev == null)
-                prop.clearValue();
+                prop.clearValue(); // checkstyle: suppress nearby 
'clearValueSystemPropertyUsage'
             else
                 prop.setString(prev);
         }
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/BootstrapBinaryDisabledTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/BootstrapBinaryDisabledTest.java
index 1706c13b4f..627b6956eb 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/BootstrapBinaryDisabledTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/BootstrapBinaryDisabledTest.java
@@ -38,6 +38,7 @@ import org.apache.cassandra.distributed.api.SimpleQueryResult;
 import org.apache.cassandra.distributed.api.TokenSupplier;
 import org.apache.cassandra.distributed.shared.Byteman;
 import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.utils.Shared;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.RESET_BOOTSTRAP_PROGRESS;
@@ -49,22 +50,19 @@ import static 
org.apache.cassandra.config.CassandraRelevantProperties.TEST_WRITE
  */
 public class BootstrapBinaryDisabledTest extends TestBaseImpl
 {
-    static Boolean originalResetBootstrapProgress = null;
+    static WithProperties properties;
 
     @BeforeClass
     public static void beforeClass() throws Throwable
     {
         TestBaseImpl.beforeClass();
-        originalResetBootstrapProgress = 
RESET_BOOTSTRAP_PROGRESS.setBoolean(false);
+        properties = new WithProperties().set(RESET_BOOTSTRAP_PROGRESS, false);
     }
 
     @AfterClass
     public static void afterClass()
     {
-        if (originalResetBootstrapProgress == null)
-            RESET_BOOTSTRAP_PROGRESS.clearValue();
-        else
-            
RESET_BOOTSTRAP_PROGRESS.setBoolean(originalResetBootstrapProgress);
+        properties.close();
     }
 
     @Test
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/DataResurrectionCheckTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/DataResurrectionCheckTest.java
index b1a3d208f5..f48c872552 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/DataResurrectionCheckTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/DataResurrectionCheckTest.java
@@ -29,6 +29,7 @@ import org.apache.cassandra.config.StartupChecksOptions;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.distributed.api.IIsolatedExecutor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.StartupException;
 import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.service.DataResurrectionCheck;
@@ -59,11 +60,9 @@ public class DataResurrectionCheckTest extends TestBaseImpl
     @Test
     public void testDataResurrectionCheck() throws Exception
     {
-        try
+        // set it to 1 hour so check will be not updated after it is written, 
for test purposes
+        try (WithProperties properties = new 
WithProperties().set(CHECK_DATA_RESURRECTION_HEARTBEAT_PERIOD, 3600000))
         {
-            // set it to 1 hour so check will be not updated after it is 
written, for test purposes
-            CHECK_DATA_RESURRECTION_HEARTBEAT_PERIOD.setInt(3600000);
-
             // start the node with the check enabled, it will just pass fine 
as there are not any user tables yet
             // and system tables are young enough
             try (Cluster cluster = build().withNodes(1)
@@ -129,10 +128,6 @@ public class DataResurrectionCheckTest extends TestBaseImpl
                                                    
EXCLUDED_KEYSPACES_CONFIG_PROPERTY, "ks3"));
             }
         }
-        finally
-        {
-            CHECK_DATA_RESURRECTION_HEARTBEAT_PERIOD.clearValue();
-        }
     }
 
     private Throwable executeChecksOnInstance(IInvokableInstance instance, 
final String... config)
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/FailingResponseDoesNotLogTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/FailingResponseDoesNotLogTest.java
index 8a710241bd..4032cc7752 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/FailingResponseDoesNotLogTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/FailingResponseDoesNotLogTest.java
@@ -40,6 +40,7 @@ import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.Feature;
 import org.apache.cassandra.distributed.api.LogAction;
 import org.apache.cassandra.distributed.api.LogResult;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
@@ -70,13 +71,15 @@ public class FailingResponseDoesNotLogTest extends 
TestBaseImpl
     @Test
     public void dispatcherErrorDoesNotLock() throws IOException
     {
-        
CUSTOM_QUERY_HANDLER_CLASS.setString(AlwaysRejectErrorQueryHandler.class.getName());
-        try (Cluster cluster = Cluster.build(1)
+        try (WithProperties properties = new 
WithProperties().set(CUSTOM_QUERY_HANDLER_CLASS, 
AlwaysRejectErrorQueryHandler.class.getName());
+             Cluster cluster = Cluster.build(1)
                                       .withConfig(c -> 
c.with(Feature.NATIVE_PROTOCOL, Feature.GOSSIP)
-                                                        
.set("client_error_reporting_exclusions", ImmutableMap.of("subnets", 
Collections.singletonList("127.0.0.1")))
-                                      )
-                                      .start())
+                                                        
.set("client_error_reporting_exclusions", ImmutableMap.of("subnets", 
Collections.singletonList("127.0.0.1"))))
+                                      .start();
+
+        )
         {
+
             try (SimpleClient client = SimpleClient.builder("127.0.0.1", 
9042).build().connect(false))
             {
                 client.execute("SELECT * FROM system.peers", 
ConsistencyLevel.ONE);
@@ -94,10 +97,6 @@ public class FailingResponseDoesNotLogTest extends 
TestBaseImpl
             matches = logs.grep("Unexpected exception during request");
             Assertions.assertThat(matches.getResult()).isEmpty();
         }
-        finally
-        {
-            CUSTOM_QUERY_HANDLER_CLASS.clearValue();
-        }
     }
 
     public static class AlwaysRejectErrorQueryHandler implements QueryHandler
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/MessageForwardingTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/MessageForwardingTest.java
index 7969c77c24..700679534b 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/MessageForwardingTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/MessageForwardingTest.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.impl.IsolatedExecutor;
 import org.apache.cassandra.distributed.impl.TracingUtil;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.utils.TimeUUID;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.WAIT_FOR_TRACING_EVENTS_TIMEOUT_SECS;
@@ -49,18 +50,16 @@ public class MessageForwardingTest extends TestBaseImpl
     @Test
     public void mutationsForwardedToAllReplicasTest()
     {
-        // Set up the wait for tracing time system property, returning the 
previous value.
-        // Handles being called again to reset with the original value, 
replacing the null
-        // with the default value.
-        Integer originalTraceTimeout = 
WAIT_FOR_TRACING_EVENTS_TIMEOUT_SECS.setInt(1);
         final int numInserts = 100;
         Map<InetAddress, Integer> forwardFromCounts = new HashMap<>();
         Map<InetAddress, Integer> commitCounts = new HashMap<>();
 
-        try (Cluster cluster = (Cluster) init(builder()
+        try (WithProperties properties = new 
WithProperties().set(WAIT_FOR_TRACING_EVENTS_TIMEOUT_SECS, 1);
+             Cluster cluster = (Cluster) init(builder()
                                               .withDC("dc0", 1)
                                               .withDC("dc1", 3)
-                                              .start()))
+                                              .start());
+             )
         {
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, 
ck int, v text, PRIMARY KEY (pk, ck))");
 
@@ -119,12 +118,5 @@ public class MessageForwardingTest extends TestBaseImpl
         {
             Assert.fail("Threw exception: " + e);
         }
-        finally
-        {
-            if (originalTraceTimeout == null)
-                WAIT_FOR_TRACING_EVENTS_TIMEOUT_SECS.clearValue();
-            else
-                
WAIT_FOR_TRACING_EVENTS_TIMEOUT_SECS.setInt(originalTraceTimeout);
-        }
     }
 }
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/MigrationCoordinatorTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/MigrationCoordinatorTest.java
index 9478c75b01..e75783d497 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/MigrationCoordinatorTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/MigrationCoordinatorTest.java
@@ -43,9 +43,9 @@ public class MigrationCoordinatorTest extends TestBaseImpl
     @Before
     public void setUp()
     {
-        REPLACE_ADDRESS.clearValue();
-        CONSISTENT_RANGE_MOVEMENT.clearValue();
-        IGNORED_SCHEMA_CHECK_VERSIONS.clearValue();
+        REPLACE_ADDRESS.clearValue(); // checkstyle: suppress nearby 
'clearValueSystemPropertyUsage'
+        CONSISTENT_RANGE_MOVEMENT.clearValue(); // checkstyle: suppress nearby 
'clearValueSystemPropertyUsage'
+        IGNORED_SCHEMA_CHECK_VERSIONS.clearValue(); // checkstyle: suppress 
nearby 'clearValueSystemPropertyUsage'
     }
     /**
      * We shouldn't wait on versions only available from a node being replaced
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/NativeMixedVersionTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/NativeMixedVersionTest.java
index bdabf55789..29605f0c1f 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/NativeMixedVersionTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/NativeMixedVersionTest.java
@@ -30,6 +30,7 @@ import com.datastax.driver.core.Session;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.Feature;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.assertj.core.api.Assertions;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.IO_NETTY_EVENTLOOP_THREADS;
@@ -41,15 +42,17 @@ public class NativeMixedVersionTest extends TestBaseImpl
     {
         // make sure to limit the netty thread pool to size 1, this will make 
the test determanistic as all work
         // will happen on the single thread.
-        IO_NETTY_EVENTLOOP_THREADS.setInt(1);
-        try (Cluster cluster = Cluster.build(1)
+        try (WithProperties properties = new 
WithProperties().set(IO_NETTY_EVENTLOOP_THREADS, 1);
+             Cluster cluster = Cluster.build(1)
                                       .withConfig(c ->
                                                   c.with(Feature.values())
                                                    
.set("read_thresholds_enabled", true)
                                                    
.set("local_read_size_warn_threshold", "1KiB")
                                       )
-                                      .start())
+                                      .start();
+             )
         {
+
             init(cluster);
             cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, 
ck1 int, value blob, PRIMARY KEY (pk, ck1))"));
             IInvokableInstance node = cluster.get(1);
@@ -80,9 +83,5 @@ public class NativeMixedVersionTest extends TestBaseImpl
             List<String> result = node.logs().grep("Warnings present in 
message with version less than").getResult();
             Assertions.assertThat(result).isEmpty();
         }
-        finally
-        {
-            IO_NETTY_EVENTLOOP_THREADS.clearValue();
-        }
     }
 }
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java 
b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java
index 13405162c3..40050bf63c 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java
@@ -27,6 +27,8 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Sets;
+
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.awaitility.Awaitility;
 import org.junit.Assert;
 import org.junit.Test;
@@ -333,13 +335,12 @@ public class PaxosRepair2Test extends TestBaseImpl
     @Test
     public void paxosAutoRepair() throws Throwable
     {
-        AUTO_REPAIR_FREQUENCY_SECONDS.setInt(1);
-        DISABLE_PAXOS_AUTO_REPAIRS.setBoolean(true);
-        try (Cluster cluster = init(Cluster.create(3, cfg -> cfg
+        try (WithProperties properties = new 
WithProperties().set(AUTO_REPAIR_FREQUENCY_SECONDS, 
1).set(DISABLE_PAXOS_AUTO_REPAIRS, true);
+             Cluster cluster = init(Cluster.create(3, cfg -> cfg
                                                              
.set("paxos_variant", "v2")
                                                              
.set("paxos_repair_enabled", true)
-                                                             
.set("truncate_request_timeout_in_ms", 1000L)))
-        )
+                                                             
.set("truncate_request_timeout_in_ms", 1000L)));
+             )
         {
             cluster.forEach(i -> {
                 
Assert.assertFalse(CassandraRelevantProperties.CLOCK_GLOBAL.isPresent());
@@ -372,7 +373,7 @@ public class PaxosRepair2Test extends TestBaseImpl
             for (int i=0; i<20; i++)
             {
                 if (!cluster.get(1).callsOnInstance(() -> 
PaxosState.uncommittedTracker().hasInflightAutoRepairs()).call()
-                 && !cluster.get(2).callsOnInstance(() -> 
PaxosState.uncommittedTracker().hasInflightAutoRepairs()).call())
+                    && !cluster.get(2).callsOnInstance(() -> 
PaxosState.uncommittedTracker().hasInflightAutoRepairs()).call())
                     break;
                 logger.info("Waiting for auto repairs to finish...");
                 Thread.sleep(1000);
@@ -380,11 +381,6 @@ public class PaxosRepair2Test extends TestBaseImpl
             assertUncommitted(cluster.get(1), KEYSPACE, TABLE, 0);
             assertUncommitted(cluster.get(2), KEYSPACE, TABLE, 0);
         }
-        finally
-        {
-            AUTO_REPAIR_FREQUENCY_SECONDS.clearValue();
-            DISABLE_PAXOS_AUTO_REPAIRS.clearValue();
-        }
     }
 
     @Test
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/VirtualTableLogsTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/VirtualTableLogsTest.java
index b0062ce3e5..1fc29589df 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/VirtualTableLogsTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/VirtualTableLogsTest.java
@@ -29,6 +29,7 @@ import 
org.apache.cassandra.db.virtual.LogMessagesTable.LogMessage;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.Feature;
 import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.utils.logging.VirtualTableAppender;
 
@@ -50,31 +51,28 @@ public class VirtualTableLogsTest extends TestBaseImpl
     @Test
     public void testVTableOutput() throws Throwable
     {
-        
LOGBACK_CONFIGURATION_FILE.setString("test/conf/logback-dtest_with_vtable_appender.xml");
-
-        try (Cluster cluster = Cluster.build(1)
+        try (WithProperties properties = new 
WithProperties().set(LOGBACK_CONFIGURATION_FILE, 
"test/conf/logback-dtest_with_vtable_appender.xml");
+             Cluster cluster = Cluster.build(1)
                                       .withConfig(c -> 
c.with(Feature.values()))
-                                      .start())
+                                      .start();
+             )
         {
             List<TestingLogMessage> rows = getRows(cluster);
             assertFalse(rows.isEmpty());
 
             rows.forEach(message -> 
assertTrue(Level.toLevel(message.level).isGreaterOrEqual(Level.INFO)));
         }
-        finally
-        {
-            LOGBACK_CONFIGURATION_FILE.clearValue();
-        }
     }
 
     @Test
     public void testMultipleAppendersFailToStartNode() throws Throwable
     {
         
LOGBACK_CONFIGURATION_FILE.setString("test/conf/logback-dtest_with_vtable_appender_invalid.xml");
-
-        try (Cluster ignored = Cluster.build(1)
+        try (WithProperties properties = new 
WithProperties().set(LOGBACK_CONFIGURATION_FILE, 
"test/conf/logback-dtest_with_vtable_appender_invalid.xml");
+             Cluster ignored = Cluster.build(1)
                                       .withConfig(c -> 
c.with(Feature.values()))
-                                      .start())
+                                      .start();
+             )
         {
             fail("Node should not start as there is supposed to be invalid 
logback configuration file.");
         }
@@ -85,10 +83,6 @@ public class VirtualTableLogsTest extends TestBaseImpl
                                 VirtualTableAppender.class.getName()),
                          ex.getMessage());
         }
-        finally
-        {
-            LOGBACK_CONFIGURATION_FILE.clearValue();
-        }
     }
 
     private List<TestingLogMessage> getRows(Cluster cluster)
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
index c4245eeab8..f31e3a8806 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
@@ -32,7 +32,6 @@ import org.junit.AssumptionViolatedException;
 import org.junit.Before;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.SystemKeyspace;
@@ -45,11 +44,13 @@ import org.apache.cassandra.distributed.api.IInstanceConfig;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.distributed.api.TokenSupplier;
 import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.distributed.test.TestBaseImpl;
 import org.apache.cassandra.schema.SchemaConstants;
 
 import static java.util.Arrays.asList;
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.JOIN_RING;
+import static 
org.apache.cassandra.config.CassandraRelevantProperties.MIGRATION_DELAY;
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.RESET_BOOTSTRAP_PROGRESS;
 import static org.apache.cassandra.distributed.action.GossipHelper.bootstrap;
 import static 
org.apache.cassandra.distributed.action.GossipHelper.pullSchemaFrom;
@@ -62,7 +63,7 @@ public class BootstrapTest extends TestBaseImpl
 {
     private long savedMigrationDelay;
 
-    static Boolean originalResetBootstrapProgress = null;
+    static WithProperties properties;
 
     @Before
     public void beforeTest()
@@ -73,20 +74,13 @@ public class BootstrapTest extends TestBaseImpl
         // When we are running multiple test cases in the class, where each 
starts a node but in the same JVM, the
         // up-time will be more or less relevant only for the first test. In 
order to enforce the startup-like behaviour
         // for each test case, the MIGRATION_DELAY time is adjusted accordingly
-        savedMigrationDelay = 
CassandraRelevantProperties.MIGRATION_DELAY.getLong();
-        
CassandraRelevantProperties.MIGRATION_DELAY.setLong(ManagementFactory.getRuntimeMXBean().getUptime()
 + savedMigrationDelay);
-
-        originalResetBootstrapProgress = RESET_BOOTSTRAP_PROGRESS.getBoolean();
+        properties = new WithProperties().set(MIGRATION_DELAY, 
ManagementFactory.getRuntimeMXBean().getUptime() + savedMigrationDelay);
     }
 
     @After
     public void afterTest()
     {
-        
CassandraRelevantProperties.MIGRATION_DELAY.setLong(savedMigrationDelay);
-        if (originalResetBootstrapProgress == null)
-            RESET_BOOTSTRAP_PROGRESS.clearValue();
-        else
-            
RESET_BOOTSTRAP_PROGRESS.setBoolean(originalResetBootstrapProgress);
+        properties.close();
     }
 
     @Test
@@ -110,7 +104,7 @@ public class BootstrapTest extends TestBaseImpl
     @Test
     public void bootstrapUnspecifiedResumeTest() throws Throwable
     {
-        RESET_BOOTSTRAP_PROGRESS.clearValue();
+        RESET_BOOTSTRAP_PROGRESS.clearValue(); // checkstyle: suppress nearby 
'clearValueSystemPropertyUsage'
         bootstrapTest();
     }
 
@@ -126,7 +120,7 @@ public class BootstrapTest extends TestBaseImpl
     @Test
     public void bootstrapUnspecifiedFailsOnResumeTest() throws Throwable
     {
-        RESET_BOOTSTRAP_PROGRESS.clearValue();
+        RESET_BOOTSTRAP_PROGRESS.clearValue(); // checkstyle: suppress nearby 
'clearValueSystemPropertyUsage'
 
         // Need our partitioner active for rangeToBytes conversion below
         Config c = DatabaseDescriptor.loadConfig();
diff --git a/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java 
b/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java
index 3349c8c33a..0a8efc25cc 100644
--- a/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java
+++ b/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java
@@ -26,6 +26,7 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.mindrot.jbcrypt.BCrypt;
 
 import com.datastax.driver.core.Authenticator;
@@ -118,19 +119,11 @@ public class PasswordAuthenticatorTest extends CQLTester
 
     private void executeSaltRoundsPropertyTest(Integer rounds)
     {
-        Integer oldProperty = AUTH_BCRYPT_GENSALT_LOG2_ROUNDS.setInt(rounds);
-        try
+        try (WithProperties properties = new 
WithProperties().set(AUTH_BCRYPT_GENSALT_LOG2_ROUNDS, rounds))
         {
             getGensaltLogRounds();
             Assert.fail("Property " + AUTH_BCRYPT_GENSALT_LOG2_ROUNDS.getKey() 
+ " must be in interval [4,30]");
         }
-        finally
-        {
-            if (oldProperty != null)
-                AUTH_BCRYPT_GENSALT_LOG2_ROUNDS.setInt(oldProperty);
-            else
-                AUTH_BCRYPT_GENSALT_LOG2_ROUNDS.clearValue();
-        }
     }
 
     @Test(expected = AuthenticationException.class)
diff --git 
a/test/unit/org/apache/cassandra/config/CassandraRelevantPropertiesTest.java 
b/test/unit/org/apache/cassandra/config/CassandraRelevantPropertiesTest.java
index ee8d2cb632..7a3e0cd9be 100644
--- a/test/unit/org/apache/cassandra/config/CassandraRelevantPropertiesTest.java
+++ b/test/unit/org/apache/cassandra/config/CassandraRelevantPropertiesTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.config;
 import org.junit.Before;
 import org.junit.Test;
 
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.assertj.core.api.Assertions;
 
@@ -40,29 +41,19 @@ public class CassandraRelevantPropertiesTest
 
     @Test
     public void testSystemPropertyisSet() {
-        try
+        try (WithProperties properties = new 
WithProperties().set(TEST_CASSANDRA_RELEVANT_PROPERTIES, "test"))
         {
-            TEST_CASSANDRA_RELEVANT_PROPERTIES.setString("test");
             
Assertions.assertThat(System.getProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey())).isEqualTo("test");
 // checkstyle: suppress nearby 'blockSystemPropertyUsage'
         }
-        finally
-        {
-            TEST_CASSANDRA_RELEVANT_PROPERTIES.clearValue();
-        }
     }
 
     @Test
     public void testString()
     {
-        try
+        try (WithProperties properties = new 
WithProperties().set(TEST_CASSANDRA_RELEVANT_PROPERTIES, "some-string"))
         {
-            System.setProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey(), 
"some-string");
             
Assertions.assertThat(TEST_CASSANDRA_RELEVANT_PROPERTIES.getString()).isEqualTo("some-string");
         }
-        finally
-        {
-            System.clearProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey());
-        }
     }
 
     @Test
@@ -88,92 +79,66 @@ public class CassandraRelevantPropertiesTest
     @Test
     public void testBoolean_null()
     {
-        try
+        try (WithProperties properties = new WithProperties())
         {
             TEST_CASSANDRA_RELEVANT_PROPERTIES.getBoolean();
-        }
-        finally
-        {
-            System.clearProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey());
+            
Assertions.assertThat(TEST_CASSANDRA_RELEVANT_PROPERTIES.getBoolean()).isFalse();
         }
     }
 
     @Test
     public void testDecimal()
     {
-        try
+        try (WithProperties properties = new 
WithProperties().set(TEST_CASSANDRA_RELEVANT_PROPERTIES, "123456789"))
         {
-            System.setProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey(), 
"123456789");
             
Assertions.assertThat(TEST_CASSANDRA_RELEVANT_PROPERTIES.getInt()).isEqualTo(123456789);
         }
-        finally
-        {
-            System.clearProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey());
-        }
     }
 
     @Test
     public void testHexadecimal()
     {
-        try
+        try (WithProperties properties = new 
WithProperties().set(TEST_CASSANDRA_RELEVANT_PROPERTIES, "0x1234567a"))
         {
-            System.setProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey(), 
"0x1234567a");
             
Assertions.assertThat(TEST_CASSANDRA_RELEVANT_PROPERTIES.getInt()).isEqualTo(305419898);
         }
-        finally
-        {
-            System.clearProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey());
-        }
     }
 
     @Test
     public void testOctal()
     {
-        try
+        try (WithProperties properties = new 
WithProperties().set(TEST_CASSANDRA_RELEVANT_PROPERTIES, "01234567"))
         {
-            System.setProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey(), 
"01234567");
             
Assertions.assertThat(TEST_CASSANDRA_RELEVANT_PROPERTIES.getInt()).isEqualTo(342391);
         }
-        finally
-        {
-            System.clearProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey());
-        }
     }
 
     @Test(expected = ConfigurationException.class)
     public void testInteger_empty()
     {
-        try
+        try (WithProperties properties = new 
WithProperties().set(TEST_CASSANDRA_RELEVANT_PROPERTIES, ""))
         {
-            System.setProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey(), 
"");
             
Assertions.assertThat(TEST_CASSANDRA_RELEVANT_PROPERTIES.getInt()).isEqualTo(342391);
         }
-        finally
-        {
-            System.clearProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey());
-        }
     }
 
     @Test(expected = ConfigurationException.class)
     public void testInteger_null()
     {
-        try
+        try (WithProperties properties = new WithProperties())
         {
             TEST_CASSANDRA_RELEVANT_PROPERTIES.getInt();
         }
-        finally
-        {
-            System.clearProperty(TEST_CASSANDRA_RELEVANT_PROPERTIES.getKey());
-        }
     }
 
     @Test
     public void testClearProperty()
     {
         assertNull(TEST_CASSANDRA_RELEVANT_PROPERTIES.getString());
-        TEST_CASSANDRA_RELEVANT_PROPERTIES.setString("test");
-        assertEquals("test", TEST_CASSANDRA_RELEVANT_PROPERTIES.getString());
-        TEST_CASSANDRA_RELEVANT_PROPERTIES.clearValue();
+        try (WithProperties properties = new 
WithProperties().set(TEST_CASSANDRA_RELEVANT_PROPERTIES, "test"))
+        {
+            assertEquals("test", 
TEST_CASSANDRA_RELEVANT_PROPERTIES.getString());
+        }
         assertNull(TEST_CASSANDRA_RELEVANT_PROPERTIES.getString());
     }
 }
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java 
b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index e4a44452e6..c6f30b42d9 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -34,6 +34,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.assertj.core.api.Assertions;
 
@@ -236,8 +237,7 @@ public class DatabaseDescriptorTest
     @Test
     public void testInvalidPartitionPropertyOverride() throws Exception
     {
-        String previous = PARTITIONER.setString("ThisDoesNotExist");
-        try
+        try (WithProperties properties = new WithProperties().set(PARTITIONER, 
"ThisDoesNotExist"))
         {
             Config testConfig = DatabaseDescriptor.loadConfig();
             testConfig.partitioner = "Murmur3Partitioner";
@@ -257,13 +257,6 @@ public class DatabaseDescriptorTest
                 
Assert.assertEquals("org.apache.cassandra.dht.ThisDoesNotExist", 
cause.getMessage());
             }
         }
-        finally
-        {
-            if (previous == null)
-                PARTITIONER.clearValue();
-            else
-                PARTITIONER.setString(previous);
-        }
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java 
b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 9100a69b6f..7b66e3137d 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@ -35,6 +35,7 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.ExpirationDateOverflowHandling;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.rows.AbstractCell;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.sstable.IScrubber;
 import org.apache.cassandra.io.util.File;
@@ -404,8 +405,7 @@ public class TTLTest extends CQLTester
         }
         if (runSStableScrub)
         {
-            TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST.setBoolean(true);
-            try
+            try (WithProperties properties = new 
WithProperties().set(TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST, true))
             {
                 ToolResult tool;
                 if (reinsertOverflowedTTL)
@@ -420,10 +420,6 @@ public class TTLTest extends CQLTester
                 else
                     Assertions.assertThat(tool.getStdout()).contains("No valid 
partitions found while scrubbing");
             }
-            finally
-            {
-                TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST.clearValue();
-            }
         }
 
         try
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java 
b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 764a497913..d53c1d46d2 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.cassandra.db.commitlog;
 
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.io.util.File;
 
 import java.io.ByteArrayOutputStream;
@@ -216,7 +218,7 @@ public abstract class CommitLogTest
     public void afterTest()
     {
         CommitLogSegmentReader.setAllowSkipSyncMarkerCrc(false);
-        COMMIT_LOG_REPLAY_LIST.clearValue();
+        COMMIT_LOG_REPLAY_LIST.clearValue(); // checkstyle: suppress nearby 
'clearValueSystemPropertyUsage'
         testKiller.reset();
     }
 
@@ -337,15 +339,10 @@ public abstract class CommitLogTest
     @Test
     public void testRecoveryWithGarbageLog_ignoredByProperty() throws Exception
     {
-        try
+        try (WithProperties properties = new 
WithProperties().set(COMMITLOG_IGNORE_REPLAY_ERRORS, "true"))
         {
-            COMMITLOG_IGNORE_REPLAY_ERRORS.setBoolean(true);
             testRecoveryWithGarbageLog();
         }
-        finally
-        {
-            COMMITLOG_IGNORE_REPLAY_ERRORS.clearValue();
-        }
     }
 
     @Test
@@ -820,22 +817,22 @@ public abstract class CommitLogTest
     public void testReplayListProperty() throws Throwable
     {
         // only keyspace
-        assertReplay(2, () -> 
COMMIT_LOG_REPLAY_LIST.setString(KEYSPACE1_REPLAY));
+        assertReplay(2, COMMIT_LOG_REPLAY_LIST, KEYSPACE1_REPLAY);
 
         // only keyspaces
-        assertReplay(3, () -> COMMIT_LOG_REPLAY_LIST.setString(format("%s,%s", 
KEYSPACE1_REPLAY, KEYSPACE2_REPLAY)));
+        assertReplay(3, COMMIT_LOG_REPLAY_LIST, format("%s,%s", 
KEYSPACE1_REPLAY, KEYSPACE2_REPLAY));
 
         // only table with keyspace
-        assertReplay(1, () -> COMMIT_LOG_REPLAY_LIST.setString(format("%s.%s", 
KEYSPACE1_REPLAY, KEYSPACE1_REPLAY_TABLE1)));
+        assertReplay(1, COMMIT_LOG_REPLAY_LIST, format("%s.%s", 
KEYSPACE1_REPLAY, KEYSPACE1_REPLAY_TABLE1));
 
         // mix of keyspace and tables with keyspaces
-        assertReplay(2, () -> 
COMMIT_LOG_REPLAY_LIST.setString(format("%s.%s,%s", KEYSPACE1_REPLAY, 
KEYSPACE1_REPLAY_TABLE1, KEYSPACE2_REPLAY)));
+        assertReplay(2, COMMIT_LOG_REPLAY_LIST, format("%s.%s,%s", 
KEYSPACE1_REPLAY, KEYSPACE1_REPLAY_TABLE1, KEYSPACE2_REPLAY));
 
         // only tables with keyspaces
-        assertReplay(2, () -> 
COMMIT_LOG_REPLAY_LIST.setString(format("%s.%s,%s.%s", KEYSPACE1_REPLAY, 
KEYSPACE1_REPLAY_TABLE1, KEYSPACE2_REPLAY, KEYSPACE2_REPLAY_TABLE2)));
+        assertReplay(2, COMMIT_LOG_REPLAY_LIST, format("%s.%s,%s.%s", 
KEYSPACE1_REPLAY, KEYSPACE1_REPLAY_TABLE1, KEYSPACE2_REPLAY, 
KEYSPACE2_REPLAY_TABLE2));
 
         // mix of keyspace and tables with keyspaces within same keyspace.
-        assertReplay(2, () -> 
COMMIT_LOG_REPLAY_LIST.setString(format("%s.%s,%s", KEYSPACE1_REPLAY, 
KEYSPACE1_REPLAY_TABLE1, KEYSPACE1_REPLAY)));
+        assertReplay(2, COMMIT_LOG_REPLAY_LIST, format("%s.%s,%s", 
KEYSPACE1_REPLAY, KEYSPACE1_REPLAY_TABLE1, KEYSPACE1_REPLAY));
 
         // test for wrong formats
 
@@ -844,7 +841,7 @@ public abstract class CommitLogTest
         try
         {
             assertReplay(2,
-                         () -> 
COMMIT_LOG_REPLAY_LIST.setString(invalidFormat));
+                         COMMIT_LOG_REPLAY_LIST, invalidFormat);
             fail(format("replay should fail on -D%s=%s as it is in invalid 
format",
                         COMMIT_LOG_REPLAY_LIST.getKey(), invalidFormat));
         }
@@ -861,7 +858,7 @@ public abstract class CommitLogTest
         try
         {
             assertReplay(2,
-                         () -> 
COMMIT_LOG_REPLAY_LIST.setString(invalidFormat2));
+                         COMMIT_LOG_REPLAY_LIST, invalidFormat2);
             fail(format("replay should fail on -D%s=%s as it is in invalid 
format",
                         COMMIT_LOG_REPLAY_LIST.getKey(), invalidFormat2));
         }
@@ -894,12 +891,10 @@ public abstract class CommitLogTest
         }
     }
 
-    private void assertReplay(int expectedReplayedMutations, Runnable 
systemPropertySetter) throws Throwable
+    private void assertReplay(int expectedReplayedMutations, 
CassandraRelevantProperties property, String propertyValue) throws Throwable
     {
-        try
+        try (WithProperties properties = new WithProperties().set(property, 
propertyValue))
         {
-            systemPropertySetter.run();
-
             CommitLog.instance.resetUnsafe(true);
 
             ColumnFamilyStore ks1tb1 = 
Keyspace.open(KEYSPACE1_REPLAY).getColumnFamilyStore(KEYSPACE1_REPLAY_TABLE1);
@@ -934,10 +929,6 @@ public abstract class CommitLogTest
 
             assertEquals(expectedReplayedMutations, replayer.count);
         }
-        finally
-        {
-            System.clearProperty(COMMIT_LOG_REPLAY_LIST.getKey());
-        }
     }
 
     @Test
@@ -1128,11 +1119,12 @@ public abstract class CommitLogTest
         }
 
         CommitLog.instance.sync(true);
-        COMMIT_LOG_REPLAY_LIST.setString(KEYSPACE1 + '.' + STANDARD1);
-        // Currently we don't attempt to re-flush a memtable that failed, thus 
make sure data is replayed by commitlog.
-        // If retries work subsequent flushes should clear up error and this 
should change to expect 0.
-        assertEquals(1, CommitLog.instance.resetUnsafe(false));
-        COMMIT_LOG_REPLAY_LIST.clearValue();
+        try (WithProperties properties = new 
WithProperties().set(COMMIT_LOG_REPLAY_LIST, KEYSPACE1 + '.' + STANDARD1))
+        {
+            // Currently we don't attempt to re-flush a memtable that failed, 
thus make sure data is replayed by commitlog.
+            // If retries work subsequent flushes should clear up error and 
this should change to expect 0.
+            assertEquals(1, CommitLog.instance.resetUnsafe(false));
+        }
     }
 
     public void testOutOfOrderFlushRecovery(BiConsumer<ColumnFamilyStore, 
Memtable> flushAction, boolean performCompaction)
@@ -1239,16 +1231,11 @@ public abstract class CommitLogTest
 
         int replayed = 0;
 
-        try
+        try (WithProperties properties = new 
WithProperties().set(COMMITLOG_IGNORE_REPLAY_ERRORS, true))
         {
-            COMMITLOG_IGNORE_REPLAY_ERRORS.setBoolean(true);
             replayed = CommitLog.instance.resetUnsafe(false);
         }
-        finally
-        {
-            COMMITLOG_IGNORE_REPLAY_ERRORS.clearValue();
-        }
-
+        
         assertEquals(replayed, 1);
     }
 }
diff --git a/test/unit/org/apache/cassandra/gms/ShadowRoundTest.java 
b/test/unit/org/apache/cassandra/gms/ShadowRoundTest.java
index b948dd5bbf..33d9a1206a 100644
--- a/test/unit/org/apache/cassandra/gms/ShadowRoundTest.java
+++ b/test/unit/org/apache/cassandra/gms/ShadowRoundTest.java
@@ -37,6 +37,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.InetAddressAndPort;
@@ -172,8 +173,7 @@ public class ShadowRoundTest
                 }, 1);
 
 
-        AUTO_BOOTSTRAP.setBoolean(false);
-        try
+        try (WithProperties properties = new 
WithProperties().set(AUTO_BOOTSTRAP, false))
         {
             
StorageService.instance.checkForEndpointCollision(SystemKeyspace.getOrInitializeLocalHostId(),
 SystemKeyspace.loadHostIds().keySet());
         }
@@ -181,10 +181,6 @@ public class ShadowRoundTest
         {
             assertEquals("Unable to gossip with any peers", e.getMessage());
         }
-        finally
-        {
-            AUTO_BOOTSTRAP.clearValue();
-        }
     }
 
     @Test
@@ -213,15 +209,10 @@ public class ShadowRoundTest
                 }, 1);
 
 
-        AUTO_BOOTSTRAP.setBoolean(false);
-        try
+        try (WithProperties properties = new 
WithProperties().set(AUTO_BOOTSTRAP, false))
         {
             
StorageService.instance.checkForEndpointCollision(SystemKeyspace.getOrInitializeLocalHostId(),
 SystemKeyspace.loadHostIds().keySet());
         }
-        finally
-        {
-            AUTO_BOOTSTRAP.clearValue();
-        }
     }
 
 }
diff --git a/test/unit/org/apache/cassandra/io/sstable/ScrubTest.java 
b/test/unit/org/apache/cassandra/io/sstable/ScrubTest.java
index bb024aaa5c..ad52262081 100644
--- a/test/unit/org/apache/cassandra/io/sstable/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/ScrubTest.java
@@ -79,6 +79,7 @@ import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
@@ -132,6 +133,8 @@ public class ScrubTest
 
     private static final AtomicInteger seq = new AtomicInteger();
 
+    static WithProperties properties;
+
     String ksName;
     Keyspace keyspace;
 
@@ -160,13 +163,13 @@ public class ScrubTest
         keyspace = Keyspace.open(ksName);
 
         CompactionManager.instance.disableAutoCompaction();
-        TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST.setBoolean(true);
+        properties = new 
WithProperties().set(TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST, true);
     }
 
     @AfterClass
     public static void clearClassEnv()
     {
-        TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST.clearValue();
+        properties.close();
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/net/ConnectionTest.java 
b/test/unit/org/apache/cassandra/net/ConnectionTest.java
index a0bd2a56e1..53bc7c5d97 100644
--- a/test/unit/org/apache/cassandra/net/ConnectionTest.java
+++ b/test/unit/org/apache/cassandra/net/ConnectionTest.java
@@ -58,6 +58,7 @@ import io.netty.channel.ChannelPromise;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.exceptions.UnknownColumnException;
 import org.apache.cassandra.io.IVersionedAsymmetricSerializer;
@@ -573,8 +574,7 @@ public class ConnectionTest
     @Test
     public void 
testPendingOutboundConnectionUpdatesMessageVersionOnReconnectAttempt() throws 
Throwable
     {
-        final Integer originalStoragePort = SSL_STORAGE_PORT.setInt(7011);
-        try
+        try (WithProperties properties = new 
WithProperties().set(SSL_STORAGE_PORT, 7011))
         {
             // Set up an inbound connection listening *only* on the SSL 
storage port to
             // replicate a 3.x node.  Force the messaging version to be 
incorrectly set to 4.0
@@ -651,10 +651,6 @@ public class ConnectionTest
         {
             
MessagingService.instance().versions.set(FBUtilities.getBroadcastAddressAndPort(),
                                                      current_version);
-            if (originalStoragePort != null)
-                SSL_STORAGE_PORT.setInt(originalStoragePort);
-            else
-                SSL_STORAGE_PORT.clearValue();
         }
     }
 
diff --git 
a/test/unit/org/apache/cassandra/security/CustomSslContextFactoryConfigTest.java
 
b/test/unit/org/apache/cassandra/security/CustomSslContextFactoryConfigTest.java
index f7c391a402..daa0f16539 100644
--- 
a/test/unit/org/apache/cassandra/security/CustomSslContextFactoryConfigTest.java
+++ 
b/test/unit/org/apache/cassandra/security/CustomSslContextFactoryConfigTest.java
@@ -25,21 +25,24 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG;
 
 public class CustomSslContextFactoryConfigTest
 {
+    static WithProperties properties;
+
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
-        CASSANDRA_CONFIG.setString("cassandra-sslcontextfactory.yaml");
+        properties = new WithProperties().set(CASSANDRA_CONFIG, 
"cassandra-sslcontextfactory.yaml");
     }
 
     @AfterClass
     public static void tearDownDatabaseDescriptor() {
-        CASSANDRA_CONFIG.clearValue();
+        properties.close();
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/security/CustomSslContextFactoryInvalidConfigTest.java
 
b/test/unit/org/apache/cassandra/security/CustomSslContextFactoryInvalidConfigTest.java
index 4e99e40fd1..dfadea7e60 100644
--- 
a/test/unit/org/apache/cassandra/security/CustomSslContextFactoryInvalidConfigTest.java
+++ 
b/test/unit/org/apache/cassandra/security/CustomSslContextFactoryInvalidConfigTest.java
@@ -24,20 +24,22 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG;
 
 public class CustomSslContextFactoryInvalidConfigTest
 {
+    static WithProperties properties;
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
-        
CASSANDRA_CONFIG.setString("cassandra-sslcontextfactory-invalidconfiguration.yaml");
+        properties = new WithProperties().set(CASSANDRA_CONFIG, 
"cassandra-sslcontextfactory-invalidconfiguration.yaml");
     }
 
     @AfterClass
     public static void tearDownDatabaseDescriptor() {
-        CASSANDRA_CONFIG.clearValue();
+        properties.close();
     }
 
     @Test(expected = IllegalArgumentException.class)
diff --git 
a/test/unit/org/apache/cassandra/security/DefaultSslContextFactoryTest.java 
b/test/unit/org/apache/cassandra/security/DefaultSslContextFactoryTest.java
index d6bd258942..bcd13efd4d 100644
--- a/test/unit/org/apache/cassandra/security/DefaultSslContextFactoryTest.java
+++ b/test/unit/org/apache/cassandra/security/DefaultSslContextFactoryTest.java
@@ -33,6 +33,7 @@ import io.netty.handler.ssl.OpenSslContext;
 import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslProvider;
 import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.distributed.shared.WithProperties;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_TCACTIVE_OPENSSL;
 
@@ -220,12 +221,13 @@ public class DefaultSslContextFactoryTest
     public void testDisableOpenSslForInJvmDtests() {
         // The configuration name below is hard-coded intentionally to make 
sure we don't break the contract without
         // changing the documentation appropriately
-        DISABLE_TCACTIVE_OPENSSL.setBoolean(true);
-        Map<String,Object> config = new HashMap<>();
-        config.putAll(commonConfig);
-
-        DefaultSslContextFactory defaultSslContextFactoryImpl = new 
DefaultSslContextFactory(config);
-        Assert.assertEquals(SslProvider.JDK, 
defaultSslContextFactoryImpl.getSslProvider());
-        DISABLE_TCACTIVE_OPENSSL.clearValue();
+        try (WithProperties properties = new 
WithProperties().set(DISABLE_TCACTIVE_OPENSSL, true))
+        {
+            Map<String,Object> config = new HashMap<>();
+            config.putAll(commonConfig);
+
+            DefaultSslContextFactory defaultSslContextFactoryImpl = new 
DefaultSslContextFactory(config);
+            Assert.assertEquals(SslProvider.JDK, 
defaultSslContextFactoryImpl.getSslProvider());
+        }
     }
 }
diff --git 
a/test/unit/org/apache/cassandra/security/FileBasedSslContextFactoryTest.java 
b/test/unit/org/apache/cassandra/security/FileBasedSslContextFactoryTest.java
index 5641a17b0a..fc84b3a00b 100644
--- 
a/test/unit/org/apache/cassandra/security/FileBasedSslContextFactoryTest.java
+++ 
b/test/unit/org/apache/cassandra/security/FileBasedSslContextFactoryTest.java
@@ -32,6 +32,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.distributed.shared.WithProperties;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG;
 
@@ -41,16 +42,19 @@ public class FileBasedSslContextFactoryTest
 
     private EncryptionOptions.ServerEncryptionOptions encryptionOptions;
 
+    static WithProperties properties;
+
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
         CASSANDRA_CONFIG.reset();
+        properties = new WithProperties();
     }
 
     @AfterClass
     public static void tearDownDatabaseDescriptor()
     {
-        CASSANDRA_CONFIG.clearValue();
+        properties.close();
     }
 
     @Before
diff --git 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigTest.java
 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigTest.java
index 206908dfc6..e27be4e625 100644
--- 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigTest.java
+++ 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigTest.java
@@ -27,21 +27,24 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG;
 
 public class PEMBasedSslContextFactoryConfigTest
 {
+    static WithProperties properties;
+
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
-        CASSANDRA_CONFIG.setString("cassandra-pem-sslcontextfactory.yaml");
+        properties = new WithProperties().set(CASSANDRA_CONFIG, 
"cassandra-pem-sslcontextfactory.yaml");
     }
 
     @AfterClass
     public static void tearDownDatabaseDescriptor()
     {
-        CASSANDRA_CONFIG.clearValue();
+        properties.close();
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithMismatchingPasswordsTest.java
 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithMismatchingPasswordsTest.java
index 7eadba5a35..698e7a7bb0 100644
--- 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithMismatchingPasswordsTest.java
+++ 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithMismatchingPasswordsTest.java
@@ -27,22 +27,25 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG;
 
 public class PEMBasedSslContextFactoryConfigWithMismatchingPasswordsTest
 {
+    static WithProperties properties;
+
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
-        
CASSANDRA_CONFIG.setString("cassandra-pem-sslcontextfactory-mismatching-passwords.yaml");
+        properties = new WithProperties().set(CASSANDRA_CONFIG, 
"cassandra-pem-sslcontextfactory-mismatching-passwords.yaml");
     }
 
     @AfterClass
     public static void tearDownDatabaseDescriptor()
     {
-        CASSANDRA_CONFIG.clearValue();
+        properties.close();
     }
 
     @Test(expected = ConfigurationException.class)
diff --git 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithUnencryptedKeysTest.java
 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithUnencryptedKeysTest.java
index 2d816ca51b..2b63d056de 100644
--- 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithUnencryptedKeysTest.java
+++ 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithUnencryptedKeysTest.java
@@ -27,24 +27,27 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG;
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_TCACTIVE_OPENSSL;
 
 public class PEMBasedSslContextFactoryConfigWithUnencryptedKeysTest
 {
+    static WithProperties properties;
+
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
-        
CASSANDRA_CONFIG.setString("cassandra-pem-sslcontextfactory-unencryptedkeys.yaml");
-        DISABLE_TCACTIVE_OPENSSL.setBoolean(true);
+        properties = new WithProperties()
+                     .set(CASSANDRA_CONFIG, 
"cassandra-pem-sslcontextfactory-unencryptedkeys.yaml")
+                     .set(DISABLE_TCACTIVE_OPENSSL, true);
     }
 
     @AfterClass
     public static void tearDownDatabaseDescriptor()
     {
-        CASSANDRA_CONFIG.clearValue();
-        DISABLE_TCACTIVE_OPENSSL.clearValue();
+        properties.close();
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryInvalidConfigTest.java
 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryInvalidConfigTest.java
index cd3040b35c..bb157bb6bb 100644
--- 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryInvalidConfigTest.java
+++ 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryInvalidConfigTest.java
@@ -27,22 +27,25 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG;
 
 public class PEMBasedSslContextFactoryInvalidConfigTest
 {
+    static WithProperties properties;
+
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
-        
CASSANDRA_CONFIG.setString("cassandra-pem-sslcontextfactory-invalidconfiguration.yaml");
+        properties = new WithProperties().set(CASSANDRA_CONFIG, 
"cassandra-pem-sslcontextfactory-invalidconfiguration.yaml");
     }
 
     @AfterClass
     public static void tearDownDatabaseDescriptor()
     {
-        CASSANDRA_CONFIG.clearValue();
+        properties.close();
     }
 
     @Test(expected = ConfigurationException.class)
diff --git 
a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryTest.java 
b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryTest.java
index a0be3918d6..ee9c6104c8 100644
--- a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryTest.java
+++ b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryTest.java
@@ -35,6 +35,7 @@ import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslProvider;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.distributed.shared.WithProperties;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_TCACTIVE_OPENSSL;
 import static 
org.apache.cassandra.security.PEMBasedSslContextFactory.ConfigKey.ENCODED_CERTIFICATES;
@@ -414,13 +415,14 @@ public class PEMBasedSslContextFactoryTest
     {
         // The configuration name below is hard-coded intentionally to make 
sure we don't break the contract without
         // changing the documentation appropriately
-        DISABLE_TCACTIVE_OPENSSL.setBoolean(true);
-        Map<String, Object> config = new HashMap<>();
-        config.putAll(commonConfig);
-
-        PEMBasedSslContextFactory sslContextFactory = new 
PEMBasedSslContextFactory(config);
-        Assert.assertEquals(SslProvider.JDK, 
sslContextFactory.getSslProvider());
-        DISABLE_TCACTIVE_OPENSSL.clearValue();
+        try (WithProperties properties = new 
WithProperties().set(DISABLE_TCACTIVE_OPENSSL, true))
+        {
+            Map<String, Object> config = new HashMap<>();
+            config.putAll(commonConfig);
+
+            PEMBasedSslContextFactory sslContextFactory = new 
PEMBasedSslContextFactory(config);
+            Assert.assertEquals(SslProvider.JDK, 
sslContextFactory.getSslProvider());
+        }
     }
 
     @Test(expected = IllegalArgumentException.class)
diff --git 
a/test/unit/org/apache/cassandra/security/PEMJKSSslContextFactoryConfigTest.java
 
b/test/unit/org/apache/cassandra/security/PEMJKSSslContextFactoryConfigTest.java
index 834fe44b77..f187e2c4f5 100644
--- 
a/test/unit/org/apache/cassandra/security/PEMJKSSslContextFactoryConfigTest.java
+++ 
b/test/unit/org/apache/cassandra/security/PEMJKSSslContextFactoryConfigTest.java
@@ -27,20 +27,23 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG;
 
 public class PEMJKSSslContextFactoryConfigTest
 {
+    static WithProperties properties;
+
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
-        CASSANDRA_CONFIG.setString("cassandra-pem-jks-sslcontextfactory.yaml");
+        properties = new WithProperties().set(CASSANDRA_CONFIG, 
"cassandra-pem-jks-sslcontextfactory.yaml");
     }
 
     @AfterClass
     public static void tearDownDatabaseDescriptor() {
-        CASSANDRA_CONFIG.clearValue();
+        properties.close();
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/service/AbstractFilesystemOwnershipCheckTest.java
 
b/test/unit/org/apache/cassandra/service/AbstractFilesystemOwnershipCheckTest.java
index 607dbd0b61..cae7de060b 100644
--- 
a/test/unit/org/apache/cassandra/service/AbstractFilesystemOwnershipCheckTest.java
+++ 
b/test/unit/org/apache/cassandra/service/AbstractFilesystemOwnershipCheckTest.java
@@ -36,6 +36,7 @@ import org.junit.Test;
 import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.StartupChecksOptions;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.StartupException;
 import org.apache.cassandra.io.util.File;
 
@@ -66,11 +67,14 @@ public abstract class AbstractFilesystemOwnershipCheckTest
 
     protected StartupChecksOptions options = new StartupChecksOptions();
 
+    static WithProperties properties;
+
     protected void setup()
     {
         cleanTempDir();
         tempDir = new File(com.google.common.io.Files.createTempDir());
         token = makeRandomString(10);
+        properties = new WithProperties();
         
System.clearProperty(CassandraRelevantProperties.FILE_SYSTEM_CHECK_OWNERSHIP_FILENAME.getKey());
         
System.clearProperty(CassandraRelevantProperties.FILE_SYSTEM_CHECK_OWNERSHIP_TOKEN.getKey());
         
System.clearProperty(CassandraRelevantProperties.FILE_SYSTEM_CHECK_ENABLE.getKey());
@@ -188,6 +192,7 @@ public abstract class AbstractFilesystemOwnershipCheckTest
     public void teardown() throws IOException
     {
         cleanTempDir();
+        properties.close();
     }
 
     // tests for enabling/disabling/configuring the check
@@ -220,7 +225,7 @@ public abstract class AbstractFilesystemOwnershipCheckTest
     public void checkEnabledButClusterPropertyIsUnset()
     {
         
Assume.assumeFalse(options.getConfig(check_filesystem_ownership).containsKey("ownership_token"));
-        
CassandraRelevantProperties.FILE_SYSTEM_CHECK_OWNERSHIP_TOKEN.clearValue();
+        
CassandraRelevantProperties.FILE_SYSTEM_CHECK_OWNERSHIP_TOKEN.clearValue(); // 
checkstyle: suppress nearby 'clearValueSystemPropertyUsage'
         
AbstractFilesystemOwnershipCheckTest.executeAndFail(AbstractFilesystemOwnershipCheckTest.checker(tempDir),
 options, MISSING_PROPERTY, 
CassandraRelevantProperties.FILE_SYSTEM_CHECK_OWNERSHIP_TOKEN.getKey());
     }
 
diff --git a/test/unit/org/apache/cassandra/service/ClientStateTest.java 
b/test/unit/org/apache/cassandra/service/ClientStateTest.java
index 6e8cfd799c..04bddb4169 100644
--- a/test/unit/org/apache/cassandra/service/ClientStateTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientStateTest.java
@@ -36,6 +36,7 @@ import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.auth.Roles;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.TableMetadata;
@@ -46,10 +47,12 @@ import static org.junit.Assert.fail;
 
 public class ClientStateTest
 {
+    static WithProperties properties;
+
     @BeforeClass
     public static void beforeClass()
     {
-        ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true);
+        properties = new 
WithProperties().set(ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION, true);
         SchemaLoader.prepareServer();
         DatabaseDescriptor.setAuthFromRoot(true);
         // create the system_auth keyspace so the IRoleManager can function as 
normal
@@ -63,7 +66,7 @@ public class ClientStateTest
     @AfterClass
     public static void afterClass()
     {
-        ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.clearValue();
+        properties.close();
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java 
b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 39f3216303..d4cf4504f8 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -41,6 +41,7 @@ import org.apache.cassandra.dht.Murmur3Partitioner.LongToken;
 import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -635,7 +636,7 @@ public class StorageServiceServerTest
     @Test
     public void isReplacingSameHostAddressAndHostIdTest() throws 
UnknownHostException
     {
-        try
+        try (WithProperties properties = new WithProperties())
         {
             UUID differentHostId = UUID.randomUUID();
             
Assert.assertFalse(StorageService.instance.isReplacingSameHostAddressAndHostId(differentHostId));
@@ -656,9 +657,5 @@ public class StorageServiceServerTest
             REPLACE_ADDRESS.setString("unresolvable.host.local.");
             
Assert.assertFalse(StorageService.instance.isReplacingSameHostAddressAndHostId(differentHostId));
         }
-        finally
-        {
-            REPLACE_ADDRESS.clearValue();
-        }
     }
 }
\ No newline at end of file
diff --git 
a/test/unit/org/apache/cassandra/service/SystemPropertiesBasedFileSystemOwnershipCheckTest.java
 
b/test/unit/org/apache/cassandra/service/SystemPropertiesBasedFileSystemOwnershipCheckTest.java
index 5101fc9d54..557b26dc95 100644
--- 
a/test/unit/org/apache/cassandra/service/SystemPropertiesBasedFileSystemOwnershipCheckTest.java
+++ 
b/test/unit/org/apache/cassandra/service/SystemPropertiesBasedFileSystemOwnershipCheckTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.service;
 import org.junit.Before;
 
 import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.distributed.shared.WithProperties;
 
 public class SystemPropertiesBasedFileSystemOwnershipCheckTest extends 
AbstractFilesystemOwnershipCheckTest
 {
@@ -28,7 +29,7 @@ public class 
SystemPropertiesBasedFileSystemOwnershipCheckTest extends AbstractF
     public void setup()
     {
         super.setup();
-        
CassandraRelevantProperties.FILE_SYSTEM_CHECK_OWNERSHIP_TOKEN.setString(token);
-        CassandraRelevantProperties.FILE_SYSTEM_CHECK_ENABLE.setBoolean(true);
+        properties = new 
WithProperties().set(CassandraRelevantProperties.FILE_SYSTEM_CHECK_OWNERSHIP_TOKEN,
 token)
+                                         
.set(CassandraRelevantProperties.FILE_SYSTEM_CHECK_ENABLE, true);
     }
 }
\ No newline at end of file
diff --git 
a/test/unit/org/apache/cassandra/service/reads/range/RangeCommandsTest.java 
b/test/unit/org/apache/cassandra/service/reads/range/RangeCommandsTest.java
index 2b15ebe5cf..32ebeff14f 100644
--- a/test/unit/org/apache/cassandra/service/reads/range/RangeCommandsTest.java
+++ b/test/unit/org/apache/cassandra/service/reads/range/RangeCommandsTest.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.PartitionRangeReadCommand;
 import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.partitions.CachedPartition;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.index.StubIndex;
 import org.apache.cassandra.schema.IndexMetadata;
@@ -48,18 +49,20 @@ import static org.junit.Assert.assertEquals;
  */
 public class RangeCommandsTest extends CQLTester
 {
+
+    static WithProperties properties;
     private static final int MAX_CONCURRENCY_FACTOR = 4;
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
-        MAX_CONCURRENT_RANGE_REQUESTS.setInt(MAX_CONCURRENCY_FACTOR);
+        properties = new WithProperties().set(MAX_CONCURRENT_RANGE_REQUESTS, 
MAX_CONCURRENCY_FACTOR);
     }
 
     @AfterClass
     public static void cleanup()
     {
-        MAX_CONCURRENT_RANGE_REQUESTS.clearValue();
+        properties.close();
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java 
b/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java
index 709b404d2b..13705f9b34 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java
@@ -30,6 +30,7 @@ import org.junit.Test;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.StartupException;
 import org.apache.cassandra.io.sstable.LegacySSTableTest;
@@ -48,19 +49,21 @@ import static org.junit.Assert.assertEquals;
  */
 public class StandaloneUpgraderOnSStablesTest
 {
+    static WithProperties properties;
+
     String legacyId = 
LegacySSTableTest.legacyVersions[LegacySSTableTest.legacyVersions.length - 1];
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
         LegacySSTableTest.defineSchema();
-        TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST.setBoolean(true);
+        properties = new 
WithProperties().set(TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST, true);
     }
 
     @AfterClass
     public static void clearClassEnv()
     {
-        TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST.clearValue();
+        properties.close();
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java 
b/test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
index 009f450fc4..b1eb139004 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.distributed.shared.WithProperties;
 import org.apache.cassandra.io.sstable.VerifyTest;
 import org.apache.cassandra.io.sstable.format.SSTableFormat.Components;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -61,13 +62,15 @@ import static org.junit.Assert.assertEquals;
  */
 public class StandaloneVerifierOnSSTablesTest extends OfflineToolUtils
 {
+    static WithProperties properties;
+
     @BeforeClass
     public static void setup()
     {
         // since legacy tables test data uses ByteOrderedPartitioner that's 
what we need
         // for the check version to work
         
CassandraRelevantProperties.PARTITIONER.setString("org.apache.cassandra.dht.ByteOrderedPartitioner");
-        TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST.setBoolean(true);
+        properties = new 
WithProperties().set(TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST, true);
         SchemaLoader.loadSchema();
         StorageService.instance.initServer();
     }
@@ -76,7 +79,7 @@ public class StandaloneVerifierOnSSTablesTest extends 
OfflineToolUtils
     public static void teardown() throws Exception
     {
         SchemaLoader.cleanupAndLeaveDirs();
-        TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST.clearValue();
+        properties.close();
     }
 
     @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to