You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by jm...@apache.org on 2023/11/16 16:10:25 UTC

(accumulo) branch elasticity updated: Remove property tserver.compaction.major.delay (#3950)

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

jmark99 pushed a commit to branch elasticity
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/elasticity by this push:
     new 67f3a7cdfa Remove property tserver.compaction.major.delay (#3950)
67f3a7cdfa is described below

commit 67f3a7cdfafad8745d5c808259f5a02aa568e116
Author: Mark Owens <jm...@apache.org>
AuthorDate: Thu Nov 16 11:10:20 2023 -0500

    Remove property tserver.compaction.major.delay (#3950)
    
    The tserver no longer does compactions, therefore the tserver.compation.major.delay property is no longer needed. This PR removes the property and associated references within the code.
    
    Updated Tablet.java and TabletServer.java with changes provided by @keith-turner.
    
    Refactored `configure` method in LargeSplitRowIT to remove two unnecessary lines.
---
 .../org/apache/accumulo/core/conf/Property.java    |  3 --
 .../accumulo/minicluster/MiniAccumuloRunner.java   |  4 +-
 .../miniclusterImpl/MiniAccumuloConfigImpl.java    |  3 --
 .../org/apache/accumulo/tserver/TabletServer.java  | 50 +++++++---------------
 .../org/apache/accumulo/tserver/tablet/Tablet.java |  5 +--
 .../org/apache/accumulo/test/LargeSplitRowIT.java  |  3 --
 .../BalanceInPresenceOfOfflineTableIT.java         |  1 -
 .../accumulo/test/functional/BatchScanSplitIT.java |  7 ---
 .../accumulo/test/functional/BigRootTabletIT.java  |  1 -
 .../accumulo/test/functional/BinaryStressIT.java   | 17 --------
 .../test/functional/BulkSplitOptimizationIT.java   | 27 +++---------
 .../test/functional/ChaoticBalancerIT.java         |  1 -
 .../accumulo/test/functional/CompactionIT.java     |  1 -
 .../accumulo/test/functional/ConcurrencyIT.java    |  1 -
 .../test/functional/DeleteEverythingIT.java        | 26 -----------
 .../test/functional/GarbageCollectorIT.java        |  1 -
 .../functional/GarbageCollectorTrashDefaultIT.java |  1 -
 .../functional/GarbageCollectorTrashEnabledIT.java |  1 -
 ...ageCollectorTrashEnabledWithCustomPolicyIT.java |  1 -
 .../accumulo/test/functional/LargeRowIT.java       | 17 --------
 .../test/functional/ManyWriteAheadLogsIT.java      | 19 +-------
 .../test/functional/MetadataMaxFilesIT.java        |  1 -
 .../accumulo/test/functional/MetadataSplitIT.java  |  8 ----
 .../accumulo/test/functional/RestartStressIT.java  |  1 -
 .../accumulo/test/functional/RowDeleteIT.java      |  1 -
 .../test/functional/SimpleBalancerFairnessIT.java  |  1 -
 .../apache/accumulo/test/functional/SplitIT.java   | 20 ---------
 .../accumulo/test/functional/WriteAheadLogIT.java  |  1 -
 28 files changed, 25 insertions(+), 198 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index a305a6846f..29710d76a8 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -581,9 +581,6 @@ public enum Property {
       "2.1.0"),
   TSERV_MIGRATE_MAXCONCURRENT("tserver.migrations.concurrent.max", "1", PropertyType.COUNT,
       "The maximum number of concurrent tablet migrations for a tablet server.", "1.3.5"),
-  // ELASTICITY_TODO look into removing this prop, may need to deprecate in 3.0
-  TSERV_MAJC_DELAY("tserver.compaction.major.delay", "30s", PropertyType.TIMEDURATION,
-      "Time a tablet server will sleep between checking which tablets need compaction.", "1.3.5"),
   TSERV_COMPACTION_SERVICE_PREFIX("tserver.compaction.major.service.", null, PropertyType.PREFIX,
       "Prefix for compaction services.", "2.1.0"),
   TSERV_COMPACTION_SERVICE_ROOT_PLANNER("tserver.compaction.major.service.root.planner",
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
index 31cd0898f9..f4a7bf24c0 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
@@ -105,9 +105,9 @@ public class MiniAccumuloRunner {
     System.out.println();
     System.out.println("# Configuration normally placed in accumulo.properties can be added using"
         + " a site.* prefix.");
-    System.out.println("# For example the following line will set tserver.compaction.major.delay");
+    System.out.println("# For example the following line will set tserver.compaction.warn.time");
     System.out.println();
-    System.out.println("#site.tserver.compaction.major.delay=60s");
+    System.out.println("#site.tserver.compaction.warn.time=10m");
 
   }
 
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
index 895b1b6f4a..95c089cad6 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -154,9 +154,6 @@ public class MiniAccumuloConfigImpl {
       mergeProp(Property.TSERV_MAXMEM.getKey(), "40M");
       mergeProp(Property.TSERV_WAL_MAX_SIZE.getKey(), "100M");
       mergeProp(Property.TSERV_NATIVEMAP_ENABLED.getKey(), "false");
-      // since there is a small amount of memory, check more frequently for majc... setting may not
-      // be needed in 1.5
-      mergeProp(Property.TSERV_MAJC_DELAY.getKey(), "3");
       mergeProp(Property.GC_CYCLE_DELAY.getKey(), "4s");
       mergeProp(Property.GC_CYCLE_START.getKey(), "0s");
       mergePropWithRandomPort(Property.MANAGER_CLIENTPORT.getKey());
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index f8208a3a04..34a24c7fb0 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -389,39 +389,6 @@ public class TabletServer extends AbstractServer implements TabletHostingServer
     return sessionManager.getSession(sessionId);
   }
 
-  private class MajorCompactor implements Runnable {
-
-    public MajorCompactor(ServerContext context) {
-      CompactionWatcher.startWatching(context);
-    }
-
-    @Override
-    public void run() {
-      while (true) {
-        try {
-          // TODO this property is misnamed, opened #3606
-          sleepUninterruptibly(getConfiguration().getTimeInMillis(Property.TSERV_MAJC_DELAY),
-              TimeUnit.MILLISECONDS);
-
-          final List<DfsLogger> closedCopy;
-
-          synchronized (closedLogs) {
-            closedCopy = List.copyOf(closedLogs);
-          }
-
-          // bail early now if we're shutting down
-          for (Entry<KeyExtent,Tablet> entry : getOnlineTablets().entrySet()) {
-            Tablet tablet = entry.getValue();
-            tablet.checkIfMinorCompactionNeededForLogs(closedCopy);
-          }
-        } catch (Exception t) {
-          log.error("Unexpected exception in {}", Thread.currentThread().getName(), t);
-          sleepUninterruptibly(1, TimeUnit.SECONDS);
-        }
-      }
-    }
-  }
-
   // add a message for the main thread to send back to the manager
   public void enqueueManagerMessage(ManagerMessage m) {
     managerMessages.addLast(m);
@@ -823,7 +790,7 @@ public class TabletServer extends AbstractServer implements TabletHostingServer
 
   private void config() {
     log.info("Tablet server starting on {}", getHostname());
-    Threads.createThread("Split/MajC initiator", new MajorCompactor(context)).start();
+    CompactionWatcher.startWatching(context);
 
     clientAddress = HostAndPort.fromParts(getHostname(), 0);
   }
@@ -1116,6 +1083,21 @@ public class TabletServer extends AbstractServer implements TabletHostingServer
       }
       log.info("Marking " + currentLog.getPath() + " as closed. Total closed logs " + clSize);
       walMarker.closeWal(getTabletSession(), currentLog.getPath());
+
+      // whenever a new log is added to the set of closed logs, go through all of the tablets and
+      // see if any need to minor compact
+      List<DfsLogger> closedCopy;
+      synchronized (closedLogs) {
+        closedCopy = List.copyOf(closedLogs);
+      }
+
+      int maxLogs = getConfiguration().getCount(Property.TSERV_WAL_MAX_REFERENCED);
+      if (closedCopy.size() >= maxLogs) {
+        for (Entry<KeyExtent,Tablet> entry : getOnlineTablets().entrySet()) {
+          Tablet tablet = entry.getValue();
+          tablet.checkIfMinorCompactionNeededForLogs(closedCopy, maxLogs);
+        }
+      }
     } else {
       log.info(
           "Marking " + currentLog.getPath() + " as unreferenced (skipping closed writes == 0)");
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 564f5be8d4..11c37ae5ea 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -1113,10 +1113,7 @@ public class Tablet extends TabletBase {
     candidates.removeAll(referencedLogs);
   }
 
-  public void checkIfMinorCompactionNeededForLogs(List<DfsLogger> closedLogs) {
-
-    // grab this outside of tablet lock.
-    int maxLogs = tableConfiguration.getCount(Property.TSERV_WAL_MAX_REFERENCED);
+  public void checkIfMinorCompactionNeededForLogs(List<DfsLogger> closedLogs, int maxLogs) {
 
     String reason = null;
     synchronized (this) {
diff --git a/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java b/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java
index 71a670ed39..343315ae88 100644
--- a/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java
@@ -68,9 +68,6 @@ public class LargeSplitRowIT extends ConfigurableMacBase {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
-
-    Map<String,String> siteConfig = Map.of(Property.TSERV_MAJC_DELAY.getKey(), "50ms");
-    cfg.setSiteConfig(siteConfig);
   }
 
   // User added split
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
index c80ad67f2e..fc2452d194 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
@@ -69,7 +69,6 @@ public class BalanceInPresenceOfOfflineTableIT extends AccumuloClusterHarness {
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> siteConfig = cfg.getSiteConfig();
     siteConfig.put(Property.TSERV_MAXMEM.getKey(), "10K");
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "50ms");
     cfg.setSiteConfig(siteConfig);
     // ensure we have two tservers
     if (cfg.getClusterServerConfiguration().getTabletServerConfiguration()
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
index 19b7eed3c2..c27128ec72 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
@@ -36,8 +36,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
@@ -51,11 +49,6 @@ public class BatchScanSplitIT extends AccumuloClusterHarness {
     return Duration.ofMinutes(2);
   }
 
-  @Override
-  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "50ms");
-  }
-
   @Test
   public void test() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
index 6b184c302c..5310d74203 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
@@ -49,7 +49,6 @@ public class BigRootTabletIT extends AccumuloClusterHarness {
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> siteConfig = cfg.getSiteConfig();
     siteConfig.put(Property.TABLE_SCAN_MAXMEM.getKey(), "1024");
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "60m");
     cfg.setSiteConfig(siteConfig);
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BinaryStressIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BinaryStressIT.java
index 629709365a..ebb2200b34 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BinaryStressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BinaryStressIT.java
@@ -41,7 +41,6 @@ import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
@@ -56,7 +55,6 @@ public class BinaryStressIT extends AccumuloClusterHarness {
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     cfg.setProperty(Property.TSERV_MAXMEM, "50K");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "50ms");
   }
 
   private String majcDelay, maxMem;
@@ -69,10 +67,8 @@ public class BinaryStressIT extends AccumuloClusterHarness {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       InstanceOperations iops = client.instanceOperations();
       Map<String,String> conf = iops.getSystemConfiguration();
-      majcDelay = conf.get(Property.TSERV_MAJC_DELAY.getKey());
       maxMem = conf.get(Property.TSERV_MAXMEM.getKey());
 
-      iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), "50ms");
       iops.setProperty(Property.TSERV_MAXMEM.getKey(), "50K");
 
       getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
@@ -80,19 +76,6 @@ public class BinaryStressIT extends AccumuloClusterHarness {
     }
   }
 
-  @AfterEach
-  public void resetConfig() throws Exception {
-    if (majcDelay != null) {
-      try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-        InstanceOperations iops = client.instanceOperations();
-        iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), majcDelay);
-        iops.setProperty(Property.TSERV_MAXMEM.getKey(), maxMem);
-      }
-      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
-    }
-  }
-
   @Test
   public void binaryStressTest() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index 1471c66766..de2ed8475f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -27,10 +27,8 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.accumulo.test.VerifyIngest.VerifyParams;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -49,34 +47,19 @@ public class BulkSplitOptimizationIT extends AccumuloClusterHarness {
     return Duration.ofMinutes(2);
   }
 
-  @Override
-  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "1s");
-  }
-
-  private String majcDelay;
-
   @BeforeEach
   public void alterConfig() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      majcDelay = client.instanceOperations().getSystemConfiguration()
-          .get(Property.TSERV_MAJC_DELAY.getKey());
-      if (!"1s".equals(majcDelay)) {
-        client.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), "1s");
-        getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-        getClusterControl().startAllServers(ServerType.TABLET_SERVER);
-      }
+      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
     }
   }
 
   @AfterEach
   public void resetConfig() throws Exception {
-    if (majcDelay != null) {
-      try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-        client.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), majcDelay);
-        getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-        getClusterControl().startAllServers(ServerType.TABLET_SERVER);
-      }
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
index c21baacbd8..1f38ad20df 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
@@ -48,7 +48,6 @@ public class ChaoticBalancerIT extends AccumuloClusterHarness {
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> siteConfig = cfg.getSiteConfig();
     siteConfig.put(Property.TSERV_MAXMEM.getKey(), "10K");
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "50ms");
     // ChaoticLoadBalancer balances across all tables
     siteConfig.put(Property.TABLE_LOAD_BALANCER.getKey(), ChaoticLoadBalancer.class.getName());
     cfg.setSiteConfig(siteConfig);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
index b45c0337bd..7928b511d5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
@@ -162,7 +162,6 @@ public class CompactionIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
     cfg.setProperty(Property.MANAGER_TABLET_GROUP_WATCHER_INTERVAL, "1s");
     cfg.setProperty(Property.COMPACTOR_MIN_JOB_WAIT_TIME, "100ms");
     cfg.setProperty(Property.COMPACTOR_MAX_JOB_WAIT_TIME, "1s");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
index 9fe5ae1e05..7a1eeac101 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
@@ -76,7 +76,6 @@ public class ConcurrencyIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> siteConfig = cfg.getSiteConfig();
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "1");
     cfg.setSiteConfig(siteConfig);
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
index 6140540460..f2465fde41 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
@@ -28,7 +28,6 @@ import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
@@ -37,8 +36,6 @@ import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
 public class DeleteEverythingIT extends AccumuloClusterHarness {
@@ -51,32 +48,9 @@ public class DeleteEverythingIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> siteConfig = cfg.getSiteConfig();
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "1s");
     cfg.setSiteConfig(siteConfig);
   }
 
-  private String majcDelay;
-
-  @BeforeEach
-  public void updateMajcDelay() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      majcDelay =
-          c.instanceOperations().getSystemConfiguration().get(Property.TSERV_MAJC_DELAY.getKey());
-      c.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), "1s");
-      if (getClusterType() == ClusterType.STANDALONE) {
-        // Gotta wait for the cluster to get out of the default sleep value
-        Thread.sleep(ConfigurationTypeHelper.getTimeInMillis(majcDelay));
-      }
-    }
-  }
-
-  @AfterEach
-  public void resetMajcDelay() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), majcDelay);
-    }
-  }
-
   @Test
   public void run() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index 2bbe6c783e..5806537a07 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -104,7 +104,6 @@ public class GarbageCollectorIT extends ConfigurableMacBase {
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
     cfg.setProperty(Property.GC_PORT, "0");
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
     // reduce the batch size significantly in order to cause the integration tests to have
     // to process many batches of deletion candidates.
     cfg.setProperty(Property.GC_CANDIDATE_BATCH_SIZE, "256K");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java
index f327761da2..52b387b1b3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java
@@ -61,7 +61,6 @@ public class GarbageCollectorTrashDefaultIT extends GarbageCollectorTrashBase {
     cfg.setProperty(Property.GC_PORT, "0");
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
     cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java
index 3d6d06e597..1ba12d0a56 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java
@@ -61,7 +61,6 @@ public class GarbageCollectorTrashEnabledIT extends GarbageCollectorTrashBase {
     cfg.setProperty(Property.GC_PORT, "0");
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
     cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "180s");
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java
index 7aafb3cfcf..5874e6f122 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java
@@ -79,7 +79,6 @@ public class GarbageCollectorTrashEnabledWithCustomPolicyIT extends GarbageColle
     cfg.setProperty(Property.GC_PORT, "0");
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
     cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "180s");
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java b/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java
index 3f9f7e8f2a..8aa3360be3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java
@@ -46,7 +46,6 @@ import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
@@ -67,7 +66,6 @@ public class LargeRowIT extends AccumuloClusterHarness {
     cfg.setMemory(ServerType.TABLET_SERVER, cfg.getMemory(ServerType.TABLET_SERVER) * 2,
         MemoryUnit.BYTE);
     Map<String,String> siteConfig = cfg.getSiteConfig();
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "10ms");
     cfg.setSiteConfig(siteConfig);
   }
 
@@ -89,21 +87,6 @@ public class LargeRowIT extends AccumuloClusterHarness {
     String[] names = getUniqueNames(2);
     REG_TABLE_NAME = names[0];
     PRE_SPLIT_TABLE_NAME = names[1];
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      tservMajcDelay =
-          c.instanceOperations().getSystemConfiguration().get(Property.TSERV_MAJC_DELAY.getKey());
-      c.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), "10ms");
-    }
-  }
-
-  @AfterEach
-  public void resetMajcDelay() throws Exception {
-    if (tservMajcDelay != null) {
-      try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-        client.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), tservMajcDelay);
-      }
-    }
   }
 
   @SuppressFBWarnings(value = {"PREDICTABLE_RANDOM", "DMI_RANDOM_USED_ONLY_ONCE"},
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
index b1d8d111d3..bc1dab2a12 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
@@ -44,7 +44,6 @@ import org.apache.accumulo.server.log.WalStateManager.WalState;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
@@ -54,7 +53,7 @@ public class ManyWriteAheadLogsIT extends AccumuloClusterHarness {
 
   private static final Logger log = LoggerFactory.getLogger(ManyWriteAheadLogsIT.class);
 
-  private String majcDelay, walSize;
+  private String walSize;
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
@@ -63,7 +62,6 @@ public class ManyWriteAheadLogsIT extends AccumuloClusterHarness {
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
     cfg.setProperty(Property.GC_CYCLE_START, "1");
     cfg.setProperty(Property.MANAGER_RECOVERY_DELAY, "1s");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     // idle compactions may addess the problem this test is creating, however they will not prevent
     // lots of closed WALs for all write patterns. This test ensures code that directly handles many
@@ -81,9 +79,7 @@ public class ManyWriteAheadLogsIT extends AccumuloClusterHarness {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       InstanceOperations iops = client.instanceOperations();
       Map<String,String> conf = iops.getSystemConfiguration();
-      majcDelay = conf.get(Property.TSERV_MAJC_DELAY.getKey());
       walSize = conf.get(Property.TSERV_WAL_MAX_SIZE.getKey());
-      iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), "1");
       iops.setProperty(Property.TSERV_WAL_MAX_SIZE.getKey(), "1M");
 
       getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
@@ -91,19 +87,6 @@ public class ManyWriteAheadLogsIT extends AccumuloClusterHarness {
     }
   }
 
-  @AfterEach
-  public void resetConfig() throws Exception {
-    if (majcDelay != null) {
-      try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-        InstanceOperations iops = client.instanceOperations();
-        iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), majcDelay);
-        iops.setProperty(Property.TSERV_WAL_MAX_SIZE.getKey(), walSize);
-      }
-      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
-    }
-  }
-
   /**
    * This creates a situation where many tablets reference many different write ahead logs. However
    * not single tablet references a lot of write ahead logs. Want to ensure the tablet server forces
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
index cf597b5709..d2c84b5265 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
@@ -54,7 +54,6 @@ public class MetadataMaxFilesIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
     cfg.setProperty(Property.TSERV_SCAN_MAX_OPENFILES, "10");
     cfg.setProperty(Property.TSERV_ASSIGNMENT_MAXCONCURRENT, "100");
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
index 60d864c07b..0d422f6430 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
@@ -23,14 +23,11 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
-import java.util.Collections;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
 import org.junit.jupiter.api.Test;
 
 public class MetadataSplitIT extends ConfigurableMacBase {
@@ -40,11 +37,6 @@ public class MetadataSplitIT extends ConfigurableMacBase {
     return Duration.ofMinutes(2);
   }
 
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "100ms"));
-  }
-
   @Test
   public void test() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
index 1915324ff8..58ecffd218 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
@@ -51,7 +51,6 @@ public class RestartStressIT extends AccumuloClusterHarness {
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> opts = cfg.getSiteConfig();
     opts.put(Property.TSERV_MAXMEM.getKey(), "100K");
-    opts.put(Property.TSERV_MAJC_DELAY.getKey(), "100ms");
     opts.put(Property.TSERV_WAL_MAX_SIZE.getKey(), "1M");
     opts.put(Property.INSTANCE_ZK_TIMEOUT.getKey(), "15s");
     opts.put(Property.MANAGER_RECOVERY_DELAY.getKey(), "1s");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RowDeleteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RowDeleteIT.java
index edc3e07826..d983884f72 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RowDeleteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RowDeleteIT.java
@@ -56,7 +56,6 @@ public class RowDeleteIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> siteConfig = cfg.getSiteConfig();
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "50ms");
     cfg.setSiteConfig(siteConfig);
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
index be01b927ab..6d8adf332f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
@@ -52,7 +52,6 @@ public class SimpleBalancerFairnessIT extends ConfigurableMacBase {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.TSERV_MAXMEM, "1K");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "50ms");
     cfg.setMemory(ServerType.TABLET_SERVER, cfg.getMemory(ServerType.TABLET_SERVER) * 3,
         MemoryUnit.BYTE);
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
index 852d2c2ec3..2d7a71aff3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
@@ -52,7 +52,6 @@ import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.rfile.RFile;
 import org.apache.accumulo.core.client.rfile.RFileWriter;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.TableId;
@@ -93,7 +92,6 @@ public class SplitIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "100ms");
   }
 
   private String tservMaxMem, tservMajcDelay;
@@ -105,11 +103,6 @@ public class SplitIT extends AccumuloClusterHarness {
       InstanceOperations iops = client.instanceOperations();
       Map<String,String> config = iops.getSystemConfiguration();
       tservMaxMem = config.get(Property.TSERV_MAXMEM.getKey());
-      tservMajcDelay = config.get(Property.TSERV_MAJC_DELAY.getKey());
-
-      if (!tservMajcDelay.equals("100ms")) {
-        iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), "100ms");
-      }
 
       // Property.TSERV_MAXMEM can't be altered on a running server
       boolean restarted = false;
@@ -119,14 +112,6 @@ public class SplitIT extends AccumuloClusterHarness {
         getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
         restarted = true;
       }
-
-      // If we restarted the tservers, we don't need to re-wait for the majc delay
-      if (!restarted) {
-        long millis = ConfigurationTypeHelper.getTimeInMillis(tservMajcDelay);
-        log.info("Waiting for majc delay period: {}ms", millis);
-        Thread.sleep(millis);
-        log.info("Finished waiting for majc delay period");
-      }
     }
   }
 
@@ -140,11 +125,6 @@ public class SplitIT extends AccumuloClusterHarness {
         getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
         getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
       }
-      if (tservMajcDelay != null) {
-        log.info("Resetting {}={}", Property.TSERV_MAJC_DELAY.getKey(), tservMajcDelay);
-        client.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), tservMajcDelay);
-        tservMajcDelay = null;
-      }
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
index 3f58bb88ad..528889aa36 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
@@ -45,7 +45,6 @@ public class WriteAheadLogIT extends AccumuloClusterHarness {
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
     cfg.setProperty(Property.GC_CYCLE_START, "1");
     cfg.setProperty(Property.MANAGER_RECOVERY_DELAY, "1s");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }