Skip to content

Commit 2d27803

Browse files
authored
fixes SplitIT.bulkImportThatCantSplitHangsCompaction (apache#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.
1 parent 88b108e commit 2d27803

File tree

2 files changed

+24
-3
lines changed
  • server/base/src/main/java/org/apache/accumulo/server/split
  • test/src/main/java/org/apache/accumulo/test/functional

2 files changed

+24
-3
lines changed

server/base/src/main/java/org/apache/accumulo/server/split/SplitUtils.java

+2
Original file line numberDiff line numberDiff line change
@@ -283,6 +283,8 @@ public static SortedSet<Text> findSplits(Iterable<Key> tabletIndexIterator, int
283283
lastRow = key.getRowData();
284284
}
285285

286+
log.trace("numKeys:{} desiredSplits:{} splits:{}", numKeys, desiredSplits, splits);
287+
286288
return splits;
287289
}
288290

test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java

+22-3
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@
3939
import java.util.Map;
4040
import java.util.Map.Entry;
4141
import java.util.Set;
42+
import java.util.SortedSet;
4243
import java.util.TreeMap;
4344
import java.util.TreeSet;
4445
import java.util.concurrent.Callable;
@@ -58,6 +59,7 @@
5859
import org.apache.accumulo.core.client.admin.NewTableConfiguration;
5960
import org.apache.accumulo.core.client.rfile.RFile;
6061
import org.apache.accumulo.core.client.rfile.RFileWriter;
62+
import org.apache.accumulo.core.clientImpl.ClientContext;
6163
import org.apache.accumulo.core.conf.Property;
6264
import org.apache.accumulo.core.data.Key;
6365
import org.apache.accumulo.core.data.Mutation;
@@ -77,6 +79,7 @@
7779
import org.apache.accumulo.test.TestIngest;
7880
import org.apache.accumulo.test.VerifyIngest;
7981
import org.apache.accumulo.test.VerifyIngest.VerifyParams;
82+
import org.apache.accumulo.test.util.Wait;
8083
import org.apache.hadoop.conf.Configuration;
8184
import org.apache.hadoop.fs.Path;
8285
import org.apache.hadoop.io.Text;
@@ -487,8 +490,12 @@ public void bulkImportThatCantSplitHangsCompaction() throws Exception {
487490
try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
488491
String tableName = getUniqueNames(1)[0];
489492

490-
c.tableOperations().create(tableName, new NewTableConfiguration()
491-
.setProperties(singletonMap(Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K")));
493+
SortedSet<Text> initialSplits = new TreeSet<>(List.of(new Text("r1")));
494+
495+
c.tableOperations().create(tableName,
496+
new NewTableConfiguration()
497+
.setProperties(singletonMap(Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K"))
498+
.withSplits(initialSplits));
492499

493500
var random = RANDOM.get();
494501
byte[] val = new byte[100];
@@ -511,8 +518,20 @@ public void bulkImportThatCantSplitHangsCompaction() throws Exception {
511518
// import the file
512519
c.tableOperations().importDirectory(dir).to(tableName).load();
513520

521+
// wait for the tablet to be marked unsplittable
522+
var ctx = (ClientContext) c;
523+
Wait.waitFor(() -> {
524+
var tableId = ctx.getTableId(tableName);
525+
try (var tabletsMeta = ctx.getAmple().readTablets().forTable(tableId).build()) {
526+
return tabletsMeta.stream()
527+
.filter(tabletMetadata -> tabletMetadata.getUnSplittable() != null).count() == 1;
528+
}
529+
});
530+
514531
// tablet should not be able to split
515-
assertEquals(0, c.tableOperations().listSplits(tableName).size());
532+
var splits = c.tableOperations().listSplits(tableName).stream().map(Text::toString)
533+
.collect(Collectors.toSet());
534+
assertEquals(Set.of("r1"), splits);
516535

517536
Thread.sleep(1000);
518537

0 commit comments

Comments
 (0)