49
49
import org .apache .accumulo .core .dataImpl .KeyExtent ;
50
50
import org .apache .accumulo .core .metadata .schema .TabletMetadata ;
51
51
import org .apache .accumulo .core .security .Authorizations ;
52
- import org .apache .accumulo .miniclusterImpl . MiniAccumuloConfigImpl ;
52
+ import org .apache .accumulo .harness . SharedMiniClusterBase ;
53
53
import org .apache .accumulo .server .split .SplitUtils ;
54
54
import org .apache .accumulo .test .fate .ManagerRepoIT ;
55
- import org .apache .accumulo .test .functional .ConfigurableMacBase ;
56
55
import org .apache .accumulo .test .util .Wait ;
57
- import org .apache .hadoop .conf .Configuration ;
58
56
import org .apache .hadoop .io .Text ;
57
+ import org .junit .jupiter .api .AfterAll ;
58
+ import org .junit .jupiter .api .BeforeAll ;
59
59
import org .junit .jupiter .api .Test ;
60
60
import org .junit .jupiter .api .Timeout ;
61
61
import org .slf4j .Logger ;
62
62
import org .slf4j .LoggerFactory ;
63
63
64
- public class LargeSplitRowIT extends ConfigurableMacBase {
64
+ public class LargeSplitRowIT extends SharedMiniClusterBase {
65
+
66
+ @ BeforeAll
67
+ public static void setup () throws Exception {
68
+ SharedMiniClusterBase .startMiniClusterWithConfig (
69
+ (cfg , coreSite ) -> cfg .getClusterServerConfiguration ().setNumDefaultTabletServers (1 ));
70
+ }
71
+
72
+ @ AfterAll
73
+ public static void teardown () {
74
+ SharedMiniClusterBase .stopMiniCluster ();
75
+ }
76
+
65
77
private static final Logger log = LoggerFactory .getLogger (LargeSplitRowIT .class );
66
78
67
79
@ Override
68
80
protected Duration defaultTimeout () {
69
81
return Duration .ofMinutes (1 );
70
82
}
71
83
72
- @ Override
73
- public void configure (MiniAccumuloConfigImpl cfg , Configuration hadoopCoreSite ) {
74
- cfg .getClusterServerConfiguration ().setNumDefaultTabletServers (1 );
75
- }
76
-
77
84
// User added split
78
85
@ Test
79
86
public void userAddedSplit () throws Exception {
@@ -82,7 +89,7 @@ public void userAddedSplit() throws Exception {
82
89
83
90
// make a table and lower the TABLE_END_ROW_MAX_SIZE property
84
91
final String tableName = getUniqueNames (1 )[0 ];
85
- try (AccumuloClient client = Accumulo .newClient ().from (getClientProperties ()).build ()) {
92
+ try (AccumuloClient client = Accumulo .newClient ().from (getClientProps ()).build ()) {
86
93
Map <String ,String > props = Map .of (Property .TABLE_MAX_END_ROW_SIZE .getKey (), "1000" );
87
94
client .tableOperations ().create (tableName , new NewTableConfiguration ().setProperties (props ));
88
95
@@ -128,7 +135,7 @@ public void automaticSplitWith250Same() throws Exception {
128
135
129
136
// make a table and lower the configuration properties
130
137
final String tableName = getUniqueNames (1 )[0 ];
131
- try (AccumuloClient client = Accumulo .newClient ().from (getClientProperties ()).build ()) {
138
+ try (AccumuloClient client = Accumulo .newClient ().from (getClientProps ()).build ()) {
132
139
// @formatter:off
133
140
Map <String ,String > props = Map .of (
134
141
Property .TABLE_SPLIT_THRESHOLD .getKey (), "10K" ,
@@ -164,14 +171,16 @@ public void automaticSplitWith250Same() throws Exception {
164
171
165
172
// Wait for the tablet to be marked as unsplittable due to the system split running
166
173
TableId tableId = TableId .of (client .tableOperations ().tableIdMap ().get (tableName ));
167
- Wait .waitFor (() -> getServerContext ().getAmple ()
168
- .readTablet (new KeyExtent (tableId , null , null )).getUnSplittable () != null ,
174
+ Wait .waitFor (
175
+ () -> getCluster ().getServerContext ().getAmple ()
176
+ .readTablet (new KeyExtent (tableId , null , null )).getUnSplittable () != null ,
169
177
Wait .MAX_WAIT_MILLIS , 100 );
170
178
171
179
// Verify that the unsplittable column is read correctly
172
180
TabletMetadata tm =
173
- getServerContext ().getAmple ().readTablet (new KeyExtent (tableId , null , null ));
174
- assertEquals (tm .getUnSplittable (), SplitUtils .toUnSplittable (getServerContext (), tm ));
181
+ getCluster ().getServerContext ().getAmple ().readTablet (new KeyExtent (tableId , null , null ));
182
+ assertEquals (tm .getUnSplittable (),
183
+ SplitUtils .toUnSplittable (getCluster ().getServerContext (), tm ));
175
184
176
185
// Make sure all the data that was put in the table is still correct
177
186
int count = 0 ;
@@ -199,7 +208,7 @@ public void automaticSplitWith250Same() throws Exception {
199
208
@ Timeout (60 )
200
209
public void automaticSplitWithGaps () throws Exception {
201
210
log .info ("Automatic Split With Gaps" );
202
- try (AccumuloClient client = Accumulo .newClient ().from (getClientProperties ()).build ()) {
211
+ try (AccumuloClient client = Accumulo .newClient ().from (getClientProps ()).build ()) {
203
212
automaticSplit (client , 30 , 2 );
204
213
}
205
214
}
@@ -209,7 +218,7 @@ public void automaticSplitWithGaps() throws Exception {
209
218
@ Timeout (60 )
210
219
public void automaticSplitWithoutGaps () throws Exception {
211
220
log .info ("Automatic Split Without Gaps" );
212
- try (AccumuloClient client = Accumulo .newClient ().from (getClientProperties ()).build ()) {
221
+ try (AccumuloClient client = Accumulo .newClient ().from (getClientProps ()).build ()) {
213
222
automaticSplit (client , 15 , 1 );
214
223
}
215
224
}
@@ -218,7 +227,7 @@ public void automaticSplitWithoutGaps() throws Exception {
218
227
@ Timeout (120 )
219
228
public void automaticSplitLater () throws Exception {
220
229
log .info ("Split later" );
221
- try (AccumuloClient client = Accumulo .newClient ().from (getClientProperties ()).build ()) {
230
+ try (AccumuloClient client = Accumulo .newClient ().from (getClientProps ()).build ()) {
222
231
// Generate large rows which have long common prefixes and therefore no split can be found.
223
232
// Setting max to 1 causes all rows to have long common prefixes. Setting a max of greater
224
233
// than 1 would generate a row with a short common prefix.
@@ -262,7 +271,7 @@ public void automaticSplitLater() throws Exception {
262
271
@ Timeout (60 )
263
272
public void testUnsplittableColumn () throws Exception {
264
273
log .info ("Unsplittable Column Test" );
265
- try (AccumuloClient client = Accumulo .newClient ().from (getClientProperties ()).build ()) {
274
+ try (AccumuloClient client = Accumulo .newClient ().from (getClientProps ()).build ()) {
266
275
// make a table and lower the configuration properties
267
276
// @formatter:off
268
277
var maxEndRow = 100 ;
@@ -298,15 +307,16 @@ public void testUnsplittableColumn() throws Exception {
298
307
299
308
// Wait for the tablets to be marked as unsplittable due to the system split running
300
309
TableId tableId = TableId .of (client .tableOperations ().tableIdMap ().get (tableName ));
301
- Wait .waitFor (() -> getServerContext ().getAmple ()
302
- .readTablet (new KeyExtent (tableId , null , null )).getUnSplittable () != null ,
310
+ Wait .waitFor (
311
+ () -> getCluster ().getServerContext ().getAmple ()
312
+ .readTablet (new KeyExtent (tableId , null , null )).getUnSplittable () != null ,
303
313
Wait .MAX_WAIT_MILLIS , 100 );
304
314
305
315
// Verify that the unsplittable column is read correctly
306
316
TabletMetadata tm =
307
- getServerContext ().getAmple ().readTablet (new KeyExtent (tableId , null , null ));
317
+ getCluster (). getServerContext ().getAmple ().readTablet (new KeyExtent (tableId , null , null ));
308
318
var unsplittable = tm .getUnSplittable ();
309
- assertEquals (unsplittable , SplitUtils .toUnSplittable (getServerContext (), tm ));
319
+ assertEquals (unsplittable , SplitUtils .toUnSplittable (getCluster (). getServerContext (), tm ));
310
320
311
321
// Make sure no splits occurred in the table
312
322
assertTrue (client .tableOperations ().listSplits (tableName ).isEmpty ());
@@ -318,13 +328,15 @@ public void testUnsplittableColumn() throws Exception {
318
328
319
329
// wait for the unsplittable marker to be set to a new value due to the property change
320
330
Wait .waitFor (() -> {
321
- var updatedUnsplittable = getServerContext ().getAmple ()
331
+ var updatedUnsplittable = getCluster (). getServerContext ().getAmple ()
322
332
.readTablet (new KeyExtent (tableId , null , null )).getUnSplittable ();
323
333
return updatedUnsplittable != null && !updatedUnsplittable .equals (unsplittable );
324
334
}, Wait .MAX_WAIT_MILLIS , 100 );
325
335
// recheck with the computed meta is correct after property update
326
- tm = getServerContext ().getAmple ().readTablet (new KeyExtent (tableId , null , null ));
327
- assertEquals (tm .getUnSplittable (), SplitUtils .toUnSplittable (getServerContext (), tm ));
336
+ tm = getCluster ().getServerContext ().getAmple ()
337
+ .readTablet (new KeyExtent (tableId , null , null ));
338
+ assertEquals (tm .getUnSplittable (),
339
+ SplitUtils .toUnSplittable (getCluster ().getServerContext (), tm ));
328
340
329
341
// Bump max end row size and verify split occurs and unsplittable column is cleaned up
330
342
client .tableOperations ().setProperty (tableName , Property .TABLE_MAX_END_ROW_SIZE .getKey (),
@@ -338,7 +350,7 @@ public void testUnsplittableColumn() throws Exception {
338
350
// Verify all tablets have no unsplittable metadata column
339
351
Wait .waitFor (() -> {
340
352
try (var tabletsMetadata =
341
- getServerContext ().getAmple ().readTablets ().forTable (tableId ).build ()) {
353
+ getCluster (). getServerContext ().getAmple ().readTablets ().forTable (tableId ).build ()) {
342
354
return tabletsMetadata .stream ()
343
355
.allMatch (tabletMetadata -> tabletMetadata .getUnSplittable () == null );
344
356
}
@@ -355,7 +367,7 @@ public void testUnsplittableColumn() throws Exception {
355
367
@ Timeout (60 )
356
368
public void testUnsplittableCleanup () throws Exception {
357
369
log .info ("Unsplittable Column Cleanup" );
358
- try (AccumuloClient client = Accumulo .newClient ().from (getClientProperties ()).build ()) {
370
+ try (AccumuloClient client = Accumulo .newClient ().from (getClientProps ()).build ()) {
359
371
// make a table and lower the configuration properties
360
372
// @formatter:off
361
373
Map <String ,String > props = Map .of (
@@ -394,7 +406,7 @@ public void testUnsplittableCleanup() throws Exception {
394
406
// as unsplittable due to the same end row for all keys after the default tablet is split
395
407
Wait .waitFor (() -> {
396
408
try (var tabletsMetadata =
397
- getServerContext ().getAmple ().readTablets ().forTable (tableId ).build ()) {
409
+ getCluster (). getServerContext ().getAmple ().readTablets ().forTable (tableId ).build ()) {
398
410
return tabletsMetadata .stream ().anyMatch (tm -> tm .getUnSplittable () != null );
399
411
}
400
412
}, Wait .MAX_WAIT_MILLIS , 100 );
@@ -409,7 +421,7 @@ public void testUnsplittableCleanup() throws Exception {
409
421
// same number of splits as before
410
422
Wait .waitFor (() -> {
411
423
try (var tabletsMetadata =
412
- getServerContext ().getAmple ().readTablets ().forTable (tableId ).build ()) {
424
+ getCluster (). getServerContext ().getAmple ().readTablets ().forTable (tableId ).build ()) {
413
425
return tabletsMetadata .stream ().allMatch (tm -> tm .getUnSplittable () == null );
414
426
}
415
427
}, Wait .MAX_WAIT_MILLIS , 100 );
0 commit comments