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

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


The following commit(s) were added to refs/heads/main by this push:
     new 2d2780396c fixes SplitIT.bulkImportThatCantSplitHangsCompaction (#5148)
2d2780396c is described below

commit 2d2780396c64750af21d0badf51146d2cd174c6c
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Mon Dec 9 08:12:46 2024 -0500

    fixes SplitIT.bulkImportThatCantSplitHangsCompaction (#5148)
    
    This test was making a incorrect assumption that a table with no splits
    and data for a single row could not split.  What would actually happen
    is that the table would split once for the single row in the data and
    then still need to split but be unable to.  Added single split at table
    creation time to fix this.
    
    Before these changes the test would add data to a table and then
    immediately check that its splits were zero.  The table would eventually
    split, but usually the check in the test would happen prior to this.
    This race condition in the test hid the flawed assumption in the test.
    Added a wait to the test to fix this.
---
 .../apache/accumulo/server/split/SplitUtils.java   |  2 ++
 .../apache/accumulo/test/functional/SplitIT.java   | 25 +++++++++++++++++++---
 2 files changed, 24 insertions(+), 3 deletions(-)

diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/split/SplitUtils.java 
b/server/base/src/main/java/org/apache/accumulo/server/split/SplitUtils.java
index e844deb779..677f2b93c0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/split/SplitUtils.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/split/SplitUtils.java
@@ -283,6 +283,8 @@ public class SplitUtils {
       lastRow = key.getRowData();
     }
 
+    log.trace("numKeys:{} desiredSplits:{} splits:{}", numKeys, desiredSplits, 
splits);
+
     return splits;
   }
 
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 cee49c3756..f97c646ae4 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
@@ -39,6 +39,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.Callable;
@@ -58,6 +59,7 @@ 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.clientImpl.ClientContext;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -77,6 +79,7 @@ import 
org.apache.accumulo.server.util.CheckForMetadataProblems;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.accumulo.test.VerifyIngest.VerifyParams;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -487,8 +490,12 @@ public class SplitIT extends AccumuloClusterHarness {
     try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
 
-      c.tableOperations().create(tableName, new NewTableConfiguration()
-          .setProperties(singletonMap(Property.TABLE_SPLIT_THRESHOLD.getKey(), 
"10K")));
+      SortedSet<Text> initialSplits = new TreeSet<>(List.of(new Text("r1")));
+
+      c.tableOperations().create(tableName,
+          new NewTableConfiguration()
+              
.setProperties(singletonMap(Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K"))
+              .withSplits(initialSplits));
 
       var random = RANDOM.get();
       byte[] val = new byte[100];
@@ -511,8 +518,20 @@ public class SplitIT extends AccumuloClusterHarness {
       // import the file
       c.tableOperations().importDirectory(dir).to(tableName).load();
 
+      // wait for the tablet to be marked unsplittable
+      var ctx = (ClientContext) c;
+      Wait.waitFor(() -> {
+        var tableId = ctx.getTableId(tableName);
+        try (var tabletsMeta = 
ctx.getAmple().readTablets().forTable(tableId).build()) {
+          return tabletsMeta.stream()
+              .filter(tabletMetadata -> tabletMetadata.getUnSplittable() != 
null).count() == 1;
+        }
+      });
+
       // tablet should not be able to split
-      assertEquals(0, c.tableOperations().listSplits(tableName).size());
+      var splits = 
c.tableOperations().listSplits(tableName).stream().map(Text::toString)
+          .collect(Collectors.toSet());
+      assertEquals(Set.of("r1"), splits);
 
       Thread.sleep(1000);
 

Reply via email to