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

kturner 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 3de72f63c4 fixes FateStarvationIT (#4278)
3de72f63c4 is described below

commit 3de72f63c434b977af43b7536d11e35123aa9347
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Fri Feb 16 16:45:19 2024 -0500

    fixes FateStarvationIT (#4278)
    
    This test had two problems.  First it was using the offline operation to
    check if all compactions were finished.  However compactations that were
    queued in fate when the offline started would start after the offline
    and potentially have fate locks for the check at the end of the test.
    Second the test starts a large number of compactions, but there was only
    one compactor process.  Started multiple compactor process to process
    compactions faster.
---
 .../accumulo/test/functional/FateStarvationIT.java | 43 ++++++++++++++++++++--
 1 file changed, 40 insertions(+), 3 deletions(-)

diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java 
b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
index 92a2d8251c..6b6da95020 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
@@ -23,21 +23,43 @@ import static 
org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.TestIngest.IngestParams;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
 
 public class FateStarvationIT extends AccumuloClusterHarness {
 
+  private static final Logger log = 
LoggerFactory.getLogger(FateStarvationIT.class);
+
   @Override
   protected Duration defaultTimeout() {
-    return Duration.ofMinutes(2);
+    return Duration.ofMinutes(4);
+  }
+
+  @Override
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration 
hadoopCoreSite) {
+    var groupName = "user_small";
+    // Add this check in case the config changes
+    Preconditions.checkState(
+        
Property.COMPACTION_SERVICE_DEFAULT_GROUPS.getDefaultValue().contains(groupName));
+    // This test creates around ~1300 compaction task, so start more 
compactors. There is randomness
+    // so the exact number of task varies.
+    cfg.getClusterServerConfiguration().addCompactorResourceGroup(groupName, 
4);
   }
 
   @Test
@@ -53,19 +75,34 @@ public class FateStarvationIT extends 
AccumuloClusterHarness {
       params.dataSize = 50;
       params.cols = 1;
       TestIngest.ingest(c, params);
+      log.debug("Ingest complete");
 
       c.tableOperations().flush(tableName, null, null, true);
+      log.debug("Flush complete");
 
       List<Text> splits = new ArrayList<>(TestIngest.getSplitPoints(0, 100000, 
67));
 
+      List<Future<?>> futures = new ArrayList<>();
+      var executor = Executors.newCachedThreadPool();
+
       for (int i = 0; i < 100; i++) {
         int idx1 = RANDOM.get().nextInt(splits.size() - 1);
         int idx2 = RANDOM.get().nextInt(splits.size() - (idx1 + 1)) + idx1 + 1;
 
-        c.tableOperations().compact(tableName, splits.get(idx1), 
splits.get(idx2), false, false);
+        var future = executor.submit(() -> {
+          c.tableOperations().compact(tableName, splits.get(idx1), 
splits.get(idx2), false, true);
+          return null;
+        });
+
+        futures.add(future);
       }
 
-      c.tableOperations().offline(tableName);
+      log.debug("Started compactions");
+
+      // wait for all compactions to complete
+      for (var future : futures) {
+        future.get();
+      }
 
       FunctionalTestUtils.assertNoDanglingFateLocks(getCluster());
     }

Reply via email to