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]