Merge branch 'cassandra-4.1' into cassandra-5.0.0

* cassandra-4.1:
  Use default commitlog settings in test YAMLs
diff --git a/CHANGES.txt b/CHANGES.txt
index 7e061bd..061fa7c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,6 +1,8 @@
 5.0-rc2
  * Fix direct IO support always being evaluated to false upon the first start of a node (CASSANDRA-19779)
  * Memtable allocation type unslabbed_heap_buffers_logged will cause an assertion error for TrieMemtables and SegmentedTrieMemtables (CASSANDRA-19835)
+Merged from 4.0:
+ * Use default commitlog settings in test YAMLs (CASSANDRA-19830)
 
 
 5.0-rc1
diff --git a/test/conf/cassandra-mtls-backward-compatibility.yaml b/test/conf/cassandra-mtls-backward-compatibility.yaml
index 7e5a2fd..66c8b21 100644
--- a/test/conf/cassandra-mtls-backward-compatibility.yaml
+++ b/test/conf/cassandra-mtls-backward-compatibility.yaml
@@ -21,7 +21,8 @@
 #
 cluster_name: Test Cluster
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 cdc_raw_directory: build/test/cassandra/cdc_raw
diff --git a/test/conf/cassandra-mtls.yaml b/test/conf/cassandra-mtls.yaml
index 84c7f9b..d6f1b3e 100644
--- a/test/conf/cassandra-mtls.yaml
+++ b/test/conf/cassandra-mtls.yaml
@@ -21,7 +21,8 @@
 #
 cluster_name: Test Cluster
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 cdc_raw_directory: build/test/cassandra/cdc_raw
diff --git a/test/conf/cassandra-murmur.yaml b/test/conf/cassandra-murmur.yaml
index 32170a1..2e5828f 100644
--- a/test/conf/cassandra-murmur.yaml
+++ b/test/conf/cassandra-murmur.yaml
@@ -4,7 +4,8 @@
 #
 cluster_name: Test Cluster
 memtable_allocation_type: heap_buffers
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 cdc_raw_directory: build/test/cassandra/cdc_raw
diff --git a/test/conf/cassandra-old.yaml b/test/conf/cassandra-old.yaml
index 05d1b64..b8c3b02 100644
--- a/test/conf/cassandra-old.yaml
+++ b/test/conf/cassandra-old.yaml
@@ -5,7 +5,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size_in_mb: 5
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/conf/cassandra-pem-jks-sslcontextfactory.yaml b/test/conf/cassandra-pem-jks-sslcontextfactory.yaml
index f0115b4..0bd034d 100644
--- a/test/conf/cassandra-pem-jks-sslcontextfactory.yaml
+++ b/test/conf/cassandra-pem-jks-sslcontextfactory.yaml
@@ -23,7 +23,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/conf/cassandra-pem-sslcontextfactory-invalidconfiguration.yaml b/test/conf/cassandra-pem-sslcontextfactory-invalidconfiguration.yaml
index fb1cf31..16cffb5 100644
--- a/test/conf/cassandra-pem-sslcontextfactory-invalidconfiguration.yaml
+++ b/test/conf/cassandra-pem-sslcontextfactory-invalidconfiguration.yaml
@@ -23,7 +23,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/conf/cassandra-pem-sslcontextfactory-mismatching-passwords.yaml b/test/conf/cassandra-pem-sslcontextfactory-mismatching-passwords.yaml
index 90ca4c5..7a96f24 100644
--- a/test/conf/cassandra-pem-sslcontextfactory-mismatching-passwords.yaml
+++ b/test/conf/cassandra-pem-sslcontextfactory-mismatching-passwords.yaml
@@ -23,7 +23,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/conf/cassandra-pem-sslcontextfactory-unencryptedkeys.yaml b/test/conf/cassandra-pem-sslcontextfactory-unencryptedkeys.yaml
index 9a66c28..8221c78 100644
--- a/test/conf/cassandra-pem-sslcontextfactory-unencryptedkeys.yaml
+++ b/test/conf/cassandra-pem-sslcontextfactory-unencryptedkeys.yaml
@@ -23,7 +23,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/conf/cassandra-pem-sslcontextfactory.yaml b/test/conf/cassandra-pem-sslcontextfactory.yaml
index 029fc41..229a0b0 100644
--- a/test/conf/cassandra-pem-sslcontextfactory.yaml
+++ b/test/conf/cassandra-pem-sslcontextfactory.yaml
@@ -23,7 +23,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/conf/cassandra-seeds.yaml b/test/conf/cassandra-seeds.yaml
index 6b3814f..53f82dd 100644
--- a/test/conf/cassandra-seeds.yaml
+++ b/test/conf/cassandra-seeds.yaml
@@ -5,7 +5,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 cdc_raw_directory: build/test/cassandra/cdc_raw
diff --git a/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml b/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
index cfe6f55..6b74883 100644
--- a/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
+++ b/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
@@ -23,7 +23,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/conf/cassandra-sslcontextfactory.yaml b/test/conf/cassandra-sslcontextfactory.yaml
index 177459f..a20d26e 100644
--- a/test/conf/cassandra-sslcontextfactory.yaml
+++ b/test/conf/cassandra-sslcontextfactory.yaml
@@ -23,7 +23,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml
index d04fa43..e9ba02c 100644
--- a/test/conf/cassandra.yaml
+++ b/test/conf/cassandra.yaml
@@ -5,7 +5,8 @@
 cluster_name: Test Cluster
 memtable_allocation_type: heap_buffers
 # memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 commitlog_disk_access_mode: legacy
diff --git a/test/conf/unit-test-conf/test-native-port.yaml b/test/conf/unit-test-conf/test-native-port.yaml
index 0cf6fb2..2d0b184 100644
--- a/test/conf/unit-test-conf/test-native-port.yaml
+++ b/test/conf/unit-test-conf/test-native-port.yaml
@@ -5,7 +5,8 @@
 cluster_name: Test Cluster
 # memtable_allocation_type: heap_buffers
 memtable_allocation_type: offheap_objects
-commitlog_sync: batch
+commitlog_sync: periodic
+commitlog_sync_period: 10s
 commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
index 1388954..ae891cf 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
@@ -101,7 +101,8 @@
                 .set("memtable_flush_writers", 1)
                 .set("concurrent_compactors", 1)
                 .set("memtable_heap_space", "10MiB")
-                .set("commitlog_sync", "batch")
+                .set("commitlog_sync", "periodic")
+                .set("commitlog_sync_period_in_ms", 10000)
                 .set("storage_port", storage_port)
                 .set("native_transport_port", native_transport_port)
                 .set("endpoint_snitch", DistributedTestSnitch.class.getName())
diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
index 9eab100..fd8110c 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java
@@ -57,8 +57,7 @@
     {
         cluster = init(Cluster.build()
                               .withNodes(REPLICAS)
-                              .withConfig(config -> config.set("hinted_handoff_enabled", false)
-                                                          .set("commitlog_sync", "batch")).start());
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false)).start());
 
         // Make sure we start w/ the correct defaults:
         cluster.get(1).runOnInstance(() -> assertEquals(DEFAULT_WARN_THRESHOLD, StorageService.instance.getCachedReplicaRowsWarnThreshold()));
diff --git a/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java b/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
index 92e4834..f7f9346 100644
--- a/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
+++ b/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
@@ -51,6 +51,7 @@
 import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableConsumer;
 import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableRunnable;
 import org.apache.cassandra.distributed.impl.DirectStreamingConnectionFactory;
+import org.apache.cassandra.distributed.impl.InstanceConfig;
 import org.apache.cassandra.distributed.impl.IsolatedExecutor;
 import org.apache.cassandra.io.compress.LZ4Compressor;
 import org.apache.cassandra.io.util.FileSystems;
@@ -686,7 +687,17 @@
                                    .set("file_cache_size", "16MiB")
                                    .set("use_deterministic_table_id", true)
                                    .set("disk_access_mode", disk_access_mode)
-                                   .set("failure_detector", SimulatedFailureDetector.Instance.class.getName());
+                                   .set("failure_detector", SimulatedFailureDetector.Instance.class.getName())
+                                   .set("commitlog_sync", "batch");
+
+                             // TODO: Add remove() to IInstanceConfig
+                             if (config instanceof InstanceConfig)
+                             {
+                                 InstanceConfig instanceConfig = (InstanceConfig) config;
+                                 instanceConfig.remove("commitlog_sync_period_in_ms");
+                                 instanceConfig.remove("commitlog_sync_period");
+                             }
+
                              if (commitlogCompressed)
                                  config.set("commitlog_compression", new ParameterizedClass(LZ4Compressor.class.getName(), emptyMap()));
                              configUpdater.accept(threadAllocator.update(config));
diff --git a/test/unit/org/apache/cassandra/config/FailStartupDuplicateParamsTest.java b/test/unit/org/apache/cassandra/config/FailStartupDuplicateParamsTest.java
index 341a0e8..f14c68b 100644
--- a/test/unit/org/apache/cassandra/config/FailStartupDuplicateParamsTest.java
+++ b/test/unit/org/apache/cassandra/config/FailStartupDuplicateParamsTest.java
@@ -42,7 +42,8 @@
 {
     private static final List<String> baseConfig = ImmutableList.of(
         "cluster_name: Test Cluster",
-        "commitlog_sync: batch",
+        "commitlog_sync: periodic",
+        "commitlog_sync_period: 10s",
         "commitlog_directory: build/test/cassandra/commitlog",
         "hints_directory: build/test/cassandra/hints",
         "partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner",
diff --git a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java b/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
index 8554ed4..aab430b 100644
--- a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
@@ -80,7 +80,7 @@
         assertEquals(new DataRateSpec.LongBytesPerSecondBound(24L  * 1024L * 1024L), config.inter_dc_stream_throughput_outbound);
         assertNull(config.commitlog_total_space);
         assertEquals(new DurationSpec.IntMillisecondsBound(0.0, TimeUnit.MILLISECONDS), config.commitlog_sync_group_window);
-        assertEquals(new DurationSpec.IntMillisecondsBound(0), config.commitlog_sync_period);
+        assertEquals(new DurationSpec.IntMillisecondsBound(10000), config.commitlog_sync_period);
         assertEquals(new DataStorageSpec.IntMebibytesBound(5), config.commitlog_segment_size);
         assertNull(config.periodic_commitlog_sync_lag_block);  //Integer
         assertNull(config.max_mutation_size);
diff --git a/test/unit/org/apache/cassandra/config/ParseAndConvertUnitsTest.java b/test/unit/org/apache/cassandra/config/ParseAndConvertUnitsTest.java
index 01b1bfc..7d28f09 100644
--- a/test/unit/org/apache/cassandra/config/ParseAndConvertUnitsTest.java
+++ b/test/unit/org/apache/cassandra/config/ParseAndConvertUnitsTest.java
@@ -53,7 +53,7 @@
         assertEquals(new DurationSpec.IntMillisecondsBound(2000), config.internode_tcp_connect_timeout);
         assertEquals(new DurationSpec.IntMillisecondsBound(30000), config.internode_tcp_user_timeout);
         assertEquals(new DurationSpec.IntMillisecondsBound(0), config.commitlog_sync_group_window);
-        assertEquals(new DurationSpec.IntMillisecondsBound(0), config.commitlog_sync_period);
+        assertEquals(new DurationSpec.IntMillisecondsBound(10000), config.commitlog_sync_period);
         assertNull(config.periodic_commitlog_sync_lag_block);
         assertEquals(new DurationSpec.IntMillisecondsBound(250), config.cdc_free_space_check_interval);
         assertEquals(new DurationSpec.IntMillisecondsBound(100), config.dynamic_snitch_update_interval);
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index be3aa01..6eade93 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -400,15 +400,20 @@
     @BeforeClass
     public static void setUpClass()
     {
+        prePrepareServer();
+
+        // Once per-JVM is enough
+        prepareServer();
+    }
+
+    protected static void prePrepareServer()
+    {
         CassandraRelevantProperties.SUPERUSER_SETUP_DELAY_MS.setLong(0);
         ServerTestUtils.daemonInitialization();
 
         if (ROW_CACHE_SIZE_IN_MIB > 0)
             DatabaseDescriptor.setRowCacheSizeInMiB(ROW_CACHE_SIZE_IN_MIB);
         StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
-
-        // Once per-JVM is enough
-        prepareServer();
     }
 
     @AfterClass
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java
index 02ce3f6..23440eb 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java
@@ -44,10 +44,17 @@
 public class CommitLogReaderTest extends CQLTester
 {
     @BeforeClass
-    public static void beforeClass()
+    public static void setUpClass()
     {
+        prePrepareServer();
+
         DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore);
         JVMStabilityInspector.replaceKiller(new KillerForTests(false));
+
+        DatabaseDescriptor.setCommitLogSync(Config.CommitLogSync.batch);
+
+        // Once per-JVM is enough
+        prepareServer();
     }
 
     @Before