HBASE-23723 Ensure MOB compaction works in optimized mode after snapshot clone (...
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / client / TestAdmin1.java
blob57c57cedc39d87466fcdbeea87bbd13ee9373a9a
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.client;
20 import static org.junit.Assert.assertEquals;
21 import static org.junit.Assert.assertFalse;
22 import static org.junit.Assert.assertTrue;
23 import static org.junit.Assert.fail;
25 import java.io.IOException;
26 import java.util.ArrayList;
27 import java.util.List;
28 import java.util.concurrent.ExecutionException;
29 import java.util.concurrent.TimeUnit;
30 import java.util.concurrent.atomic.AtomicInteger;
31 import org.apache.hadoop.hbase.HBaseClassTestRule;
32 import org.apache.hadoop.hbase.HConstants;
33 import org.apache.hadoop.hbase.HRegionLocation;
34 import org.apache.hadoop.hbase.MetaTableAccessor;
35 import org.apache.hadoop.hbase.ServerName;
36 import org.apache.hadoop.hbase.TableName;
37 import org.apache.hadoop.hbase.TableNotFoundException;
38 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
39 import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
40 import org.apache.hadoop.hbase.regionserver.HRegion;
41 import org.apache.hadoop.hbase.regionserver.HStore;
42 import org.apache.hadoop.hbase.regionserver.HStoreFile;
43 import org.apache.hadoop.hbase.testclassification.ClientTests;
44 import org.apache.hadoop.hbase.testclassification.LargeTests;
45 import org.apache.hadoop.hbase.util.Bytes;
46 import org.apache.hadoop.hbase.util.FSUtils;
47 import org.apache.hadoop.hbase.util.FutureUtils;
48 import org.apache.hadoop.hbase.util.Pair;
49 import org.apache.hadoop.hbase.util.Threads;
50 import org.junit.ClassRule;
51 import org.junit.Test;
52 import org.junit.experimental.categories.Category;
53 import org.slf4j.Logger;
54 import org.slf4j.LoggerFactory;
56 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
57 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
59 /**
60 * Class to test HBaseAdmin. Spins up the minicluster once at test start and then takes it down
61 * afterward. Add any testing of HBaseAdmin functionality here.
63 @Category({ LargeTests.class, ClientTests.class })
64 public class TestAdmin1 extends TestAdminBase {
66 @ClassRule
67 public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestAdmin1.class);
69 private static final Logger LOG = LoggerFactory.getLogger(TestAdmin1.class);
71 @Test
72 public void testSplitFlushCompactUnknownTable() throws InterruptedException {
73 final TableName unknowntable = TableName.valueOf(name.getMethodName());
74 Exception exception = null;
75 try {
76 ADMIN.compact(unknowntable);
77 } catch (IOException e) {
78 exception = e;
80 assertTrue(exception instanceof TableNotFoundException);
82 exception = null;
83 try {
84 ADMIN.flush(unknowntable);
85 } catch (IOException e) {
86 exception = e;
88 assertTrue(exception instanceof TableNotFoundException);
90 exception = null;
91 try {
92 ADMIN.split(unknowntable);
93 } catch (IOException e) {
94 exception = e;
96 assertTrue(exception instanceof TableNotFoundException);
99 @Test
100 public void testCompactionTimestamps() throws Exception {
101 TableName tableName = TableName.valueOf(name.getMethodName());
102 TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
103 .setColumnFamily(ColumnFamilyDescriptorBuilder.of("fam1")).build();
104 ADMIN.createTable(htd);
105 Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
106 long ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
107 assertEquals(0, ts);
108 Put p = new Put(Bytes.toBytes("row1"));
109 p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
110 table.put(p);
111 ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
112 // no files written -> no data
113 assertEquals(0, ts);
115 ADMIN.flush(tableName);
116 ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
117 // still 0, we flushed a file, but no major compaction happened
118 assertEquals(0, ts);
120 byte[] regionName;
121 try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
122 regionName = l.getAllRegionLocations().get(0).getRegion().getRegionName();
124 long ts1 = ADMIN.getLastMajorCompactionTimestampForRegion(regionName);
125 assertEquals(ts, ts1);
126 p = new Put(Bytes.toBytes("row2"));
127 p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
128 table.put(p);
129 ADMIN.flush(tableName);
130 ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
131 // make sure the region API returns the same value, as the old file is still around
132 assertEquals(ts1, ts);
134 TEST_UTIL.compact(tableName, true);
135 table.put(p);
136 // forces a wait for the compaction
137 ADMIN.flush(tableName);
138 ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
139 // after a compaction our earliest timestamp will have progressed forward
140 assertTrue(ts > ts1);
142 // region api still the same
143 ts1 = ADMIN.getLastMajorCompactionTimestampForRegion(regionName);
144 assertEquals(ts, ts1);
145 table.put(p);
146 ADMIN.flush(tableName);
147 ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
148 assertEquals(ts, ts1);
149 table.close();
152 @Test(expected = IllegalArgumentException.class)
153 public void testColumnValidName() {
154 ColumnFamilyDescriptorBuilder.of("\\test\\abc");
157 @Test
158 public void testTableExist() throws IOException {
159 final TableName table = TableName.valueOf(name.getMethodName());
160 boolean exist;
161 exist = ADMIN.tableExists(table);
162 assertEquals(false, exist);
163 TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
164 exist = ADMIN.tableExists(table);
165 assertEquals(true, exist);
169 * Tests forcing split from client and having scanners successfully ride over split.
171 @Test
172 public void testForceSplit() throws Exception {
173 byte[][] familyNames = new byte[][] { Bytes.toBytes("cf") };
174 int[] rowCounts = new int[] { 6000 };
175 int numVersions = ColumnFamilyDescriptorBuilder.DEFAULT_MAX_VERSIONS;
176 int blockSize = 256;
177 splitTest(null, familyNames, rowCounts, numVersions, blockSize, true);
179 byte[] splitKey = Bytes.toBytes(3500);
180 splitTest(splitKey, familyNames, rowCounts, numVersions, blockSize, true);
181 // test regionSplitSync
182 splitTest(splitKey, familyNames, rowCounts, numVersions, blockSize, false);
186 * Multi-family scenario. Tests forcing split from client and having scanners successfully ride
187 * over split.
189 @Test
190 public void testForceSplitMultiFamily() throws Exception {
191 int numVersions = ColumnFamilyDescriptorBuilder.DEFAULT_MAX_VERSIONS;
193 // use small HFile block size so that we can have lots of blocks in HFile
194 // Otherwise, if there is only one block,
195 // HFileBlockIndex.midKey()'s value == startKey
196 int blockSize = 256;
197 byte[][] familyNames = new byte[][] { Bytes.toBytes("cf1"), Bytes.toBytes("cf2") };
199 // one of the column families isn't splittable
200 int[] rowCounts = new int[] { 6000, 1 };
201 splitTest(null, familyNames, rowCounts, numVersions, blockSize, true);
203 rowCounts = new int[] { 1, 6000 };
204 splitTest(null, familyNames, rowCounts, numVersions, blockSize, true);
206 // one column family has much smaller data than the other
207 // the split key should be based on the largest column family
208 rowCounts = new int[] { 6000, 300 };
209 splitTest(null, familyNames, rowCounts, numVersions, blockSize, true);
211 rowCounts = new int[] { 300, 6000 };
212 splitTest(null, familyNames, rowCounts, numVersions, blockSize, true);
215 private int count(ResultScanner scanner) throws IOException {
216 int rows = 0;
217 while (scanner.next() != null) {
218 rows++;
220 return rows;
223 private void splitTest(byte[] splitPoint, byte[][] familyNames, int[] rowCounts, int numVersions,
224 int blockSize, boolean async) throws Exception {
225 TableName tableName = TableName.valueOf("testForceSplit");
226 StringBuilder sb = new StringBuilder();
227 // Add tail to String so can see better in logs where a test is running.
228 for (int i = 0; i < rowCounts.length; i++) {
229 sb.append("_").append(Integer.toString(rowCounts[i]));
231 assertFalse(ADMIN.tableExists(tableName));
232 try (final Table table = TEST_UTIL.createTable(tableName, familyNames, numVersions, blockSize);
233 final RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
235 int rowCount = 0;
236 byte[] q = new byte[0];
238 // insert rows into column families. The number of rows that have values
239 // in a specific column family is decided by rowCounts[familyIndex]
240 for (int index = 0; index < familyNames.length; index++) {
241 ArrayList<Put> puts = new ArrayList<>(rowCounts[index]);
242 for (int i = 0; i < rowCounts[index]; i++) {
243 byte[] k = Bytes.toBytes(i);
244 Put put = new Put(k);
245 put.addColumn(familyNames[index], q, k);
246 puts.add(put);
248 table.put(puts);
250 if (rowCount < rowCounts[index]) {
251 rowCount = rowCounts[index];
255 // get the initial layout (should just be one region)
256 List<HRegionLocation> m = locator.getAllRegionLocations();
257 LOG.info("Initial regions (" + m.size() + "): " + m);
258 assertTrue(m.size() == 1);
260 // Verify row count
261 Scan scan = new Scan();
262 int rows;
263 try (ResultScanner scanner = table.getScanner(scan)) {
264 rows = count(scanner);
266 assertEquals(rowCount, rows);
268 // Have an outstanding scan going on to make sure we can scan over splits.
269 scan = new Scan();
270 try (ResultScanner scanner = table.getScanner(scan)) {
271 // Scan first row so we are into first region before split happens.
272 scanner.next();
274 // Split the table
275 if (async) {
276 if (splitPoint != null) {
277 ADMIN.split(tableName, splitPoint);
278 } else {
279 ADMIN.split(tableName);
281 final AtomicInteger count = new AtomicInteger(0);
282 Thread t = new Thread("CheckForSplit") {
283 @Override
284 public void run() {
285 for (int i = 0; i < 45; i++) {
286 try {
287 sleep(1000);
288 } catch (InterruptedException e) {
289 continue;
291 // check again
292 List<HRegionLocation> regions = null;
293 try {
294 regions = locator.getAllRegionLocations();
295 } catch (IOException e) {
296 LOG.warn("get location failed", e);
298 if (regions == null) {
299 continue;
301 count.set(regions.size());
302 if (count.get() >= 2) {
303 LOG.info("Found: " + regions);
304 break;
306 LOG.debug("Cycle waiting on split");
308 LOG.debug("CheckForSplit thread exited, current region count: " + count.get());
311 t.setPriority(Thread.NORM_PRIORITY - 2);
312 t.start();
313 t.join();
314 } else {
315 // Sync split region, no need to create a thread to check
316 ADMIN.splitRegionAsync(m.get(0).getRegion().getRegionName(), splitPoint).get();
318 // Verify row count
319 rows = 1 + count(scanner); // We counted one row above.
321 assertEquals(rowCount, rows);
323 List<HRegionLocation> regions = null;
324 try {
325 regions = locator.getAllRegionLocations();
326 } catch (IOException e) {
327 e.printStackTrace();
329 assertEquals(2, regions.size());
330 if (splitPoint != null) {
331 // make sure the split point matches our explicit configuration
332 assertEquals(Bytes.toString(splitPoint),
333 Bytes.toString(regions.get(0).getRegion().getEndKey()));
334 assertEquals(Bytes.toString(splitPoint),
335 Bytes.toString(regions.get(1).getRegion().getStartKey()));
336 LOG.debug("Properly split on " + Bytes.toString(splitPoint));
337 } else {
338 if (familyNames.length > 1) {
339 int splitKey = Bytes.toInt(regions.get(0).getRegion().getEndKey());
340 // check if splitKey is based on the largest column family
341 // in terms of it store size
342 int deltaForLargestFamily = Math.abs(rowCount / 2 - splitKey);
343 LOG.debug("SplitKey=" + splitKey + "&deltaForLargestFamily=" + deltaForLargestFamily +
344 ", r=" + regions.get(0).getRegion());
345 for (int index = 0; index < familyNames.length; index++) {
346 int delta = Math.abs(rowCounts[index] / 2 - splitKey);
347 if (delta < deltaForLargestFamily) {
348 assertTrue("Delta " + delta + " for family " + index + " should be at least " +
349 "deltaForLargestFamily " + deltaForLargestFamily, false);
354 TEST_UTIL.deleteTable(tableName);
358 @Test
359 public void testSplitAndMergeWithReplicaTable() throws Exception {
360 // The test tries to directly split replica regions and directly merge replica regions. These
361 // are not allowed. The test validates that. Then the test does a valid split/merge of allowed
362 // regions.
363 // Set up a table with 3 regions and replication set to 3
364 TableName tableName = TableName.valueOf(name.getMethodName());
365 byte[] cf = Bytes.toBytes("f");
366 TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(3)
367 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf)).build();
368 byte[][] splitRows = new byte[2][];
369 splitRows[0] = new byte[] { (byte) '4' };
370 splitRows[1] = new byte[] { (byte) '7' };
371 TEST_UTIL.getAdmin().createTable(desc, splitRows);
372 List<HRegion> oldRegions;
373 do {
374 oldRegions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
375 Thread.sleep(10);
376 } while (oldRegions.size() != 9); // 3 regions * 3 replicas
377 // write some data to the table
378 Table ht = TEST_UTIL.getConnection().getTable(tableName);
379 List<Put> puts = new ArrayList<>();
380 byte[] qualifier = Bytes.toBytes("c");
381 Put put = new Put(new byte[] { (byte) '1' });
382 put.addColumn(cf, qualifier, Bytes.toBytes("100"));
383 puts.add(put);
384 put = new Put(new byte[] { (byte) '6' });
385 put.addColumn(cf, qualifier, Bytes.toBytes("100"));
386 puts.add(put);
387 put = new Put(new byte[] { (byte) '8' });
388 put.addColumn(cf, qualifier, Bytes.toBytes("100"));
389 puts.add(put);
390 ht.put(puts);
391 ht.close();
392 List<Pair<RegionInfo, ServerName>> regions =
393 MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), tableName);
394 boolean gotException = false;
395 // the element at index 1 would be a replica (since the metareader gives us ordered
396 // regions). Try splitting that region via the split API . Should fail
397 try {
398 FutureUtils.get(
399 TEST_UTIL.getAdmin().splitRegionAsync(regions.get(1).getFirst().getRegionName()));
400 } catch (IllegalArgumentException ex) {
401 gotException = true;
403 assertTrue(gotException);
404 gotException = false;
405 // the element at index 1 would be a replica (since the metareader gives us ordered
406 // regions). Try splitting that region via a different split API (the difference is
407 // this API goes direct to the regionserver skipping any checks in the admin). Should fail
408 try {
409 FutureUtils.get(TEST_UTIL.getAdmin().splitRegionAsync(
410 regions.get(1).getFirst().getEncodedNameAsBytes(), new byte[] { (byte) '1' }));
411 } catch (IllegalArgumentException ex) {
412 gotException = true;
414 assertTrue(gotException);
416 gotException = false;
417 // testing Sync split operation
418 try {
419 FutureUtils.get(TEST_UTIL.getAdmin()
420 .splitRegionAsync(regions.get(1).getFirst().getRegionName(), new byte[] { (byte) '1' }));
421 } catch (IllegalArgumentException ex) {
422 gotException = true;
424 assertTrue(gotException);
426 gotException = false;
427 // Try merging a replica with another. Should fail.
428 try {
429 FutureUtils.get(TEST_UTIL.getAdmin().mergeRegionsAsync(
430 regions.get(1).getFirst().getEncodedNameAsBytes(),
431 regions.get(2).getFirst().getEncodedNameAsBytes(),
432 true));
433 } catch (IllegalArgumentException m) {
434 gotException = true;
436 assertTrue(gotException);
437 // Try going to the master directly (that will skip the check in admin)
438 try {
439 byte[][] nameofRegionsToMerge = new byte[2][];
440 nameofRegionsToMerge[0] = regions.get(1).getFirst().getEncodedNameAsBytes();
441 nameofRegionsToMerge[1] = regions.get(2).getFirst().getEncodedNameAsBytes();
442 MergeTableRegionsRequest request = RequestConverter.buildMergeTableRegionsRequest(
443 nameofRegionsToMerge, true, HConstants.NO_NONCE, HConstants.NO_NONCE);
444 TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterRpcServices().mergeTableRegions(null,
445 request);
446 } catch (org.apache.hbase.thirdparty.com.google.protobuf.ServiceException m) {
447 Throwable t = m.getCause();
448 do {
449 if (t instanceof MergeRegionException) {
450 gotException = true;
451 break;
453 t = t.getCause();
454 } while (t != null);
456 assertTrue(gotException);
459 @Test(expected = IllegalArgumentException.class)
460 public void testInvalidColumnDescriptor() throws IOException {
461 ColumnFamilyDescriptorBuilder.of("/cfamily/name");
465 * Test DFS replication for column families, where one CF has default replication(3) and the other
466 * is set to 1.
468 @Test
469 public void testHFileReplication() throws Exception {
470 final TableName tableName = TableName.valueOf(this.name.getMethodName());
471 String fn1 = "rep1";
472 String fn = "defaultRep";
473 TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
474 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fn))
475 .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(fn1))
476 .setDFSReplication((short) 1).build())
477 .build();
478 Table table = TEST_UTIL.createTable(htd, null);
479 TEST_UTIL.waitTableAvailable(tableName);
480 Put p = new Put(Bytes.toBytes("defaultRep_rk"));
481 byte[] q1 = Bytes.toBytes("q1");
482 byte[] v1 = Bytes.toBytes("v1");
483 p.addColumn(Bytes.toBytes(fn), q1, v1);
484 List<Put> puts = new ArrayList<>(2);
485 puts.add(p);
486 p = new Put(Bytes.toBytes("rep1_rk"));
487 p.addColumn(Bytes.toBytes(fn1), q1, v1);
488 puts.add(p);
489 try {
490 table.put(puts);
491 ADMIN.flush(tableName);
493 List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(tableName);
494 for (HRegion r : regions) {
495 HStore store = r.getStore(Bytes.toBytes(fn));
496 for (HStoreFile sf : store.getStorefiles()) {
497 assertTrue(sf.toString().contains(fn));
498 assertTrue("Column family " + fn + " should have 3 copies",
499 FSUtils.getDefaultReplication(TEST_UTIL.getTestFileSystem(),
500 sf.getPath()) == (sf.getFileInfo().getFileStatus().getReplication()));
503 store = r.getStore(Bytes.toBytes(fn1));
504 for (HStoreFile sf : store.getStorefiles()) {
505 assertTrue(sf.toString().contains(fn1));
506 assertTrue("Column family " + fn1 + " should have only 1 copy",
507 1 == sf.getFileInfo().getFileStatus().getReplication());
510 } finally {
511 if (ADMIN.isTableEnabled(tableName)) {
512 ADMIN.disableTable(tableName);
513 ADMIN.deleteTable(tableName);
518 @Test
519 public void testMergeRegions() throws Exception {
520 final TableName tableName = TableName.valueOf(name.getMethodName());
521 TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
522 .setColumnFamily(ColumnFamilyDescriptorBuilder.of("d")).build();
523 byte[][] splitRows = new byte[2][];
524 splitRows[0] = new byte[] { (byte) '3' };
525 splitRows[1] = new byte[] { (byte) '6' };
526 try {
527 TEST_UTIL.createTable(td, splitRows);
528 TEST_UTIL.waitTableAvailable(tableName);
530 List<RegionInfo> tableRegions;
531 RegionInfo regionA;
532 RegionInfo regionB;
534 // merge with full name
535 tableRegions = ADMIN.getRegions(tableName);
536 assertEquals(3, ADMIN.getRegions(tableName).size());
537 regionA = tableRegions.get(0);
538 regionB = tableRegions.get(1);
539 // TODO convert this to version that is synchronous (See HBASE-16668)
540 ADMIN.mergeRegionsAsync(regionA.getRegionName(), regionB.getRegionName(), false).get(60,
541 TimeUnit.SECONDS);
543 assertEquals(2, ADMIN.getRegions(tableName).size());
545 // merge with encoded name
546 tableRegions = ADMIN.getRegions(tableName);
547 regionA = tableRegions.get(0);
548 regionB = tableRegions.get(1);
549 // TODO convert this to version that is synchronous (See HBASE-16668)
550 ADMIN
551 .mergeRegionsAsync(regionA.getEncodedNameAsBytes(), regionB.getEncodedNameAsBytes(), false)
552 .get(60, TimeUnit.SECONDS);
554 assertEquals(1, ADMIN.getRegions(tableName).size());
555 } finally {
556 ADMIN.disableTable(tableName);
557 ADMIN.deleteTable(tableName);
561 @Test
562 public void testMergeRegionsInvalidRegionCount()
563 throws IOException, InterruptedException, ExecutionException {
564 TableName tableName = TableName.valueOf(name.getMethodName());
565 TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
566 .setColumnFamily(ColumnFamilyDescriptorBuilder.of("d")).build();
567 byte[][] splitRows = new byte[2][];
568 splitRows[0] = new byte[] { (byte) '3' };
569 splitRows[1] = new byte[] { (byte) '6' };
570 try {
571 TEST_UTIL.createTable(td, splitRows);
572 TEST_UTIL.waitTableAvailable(tableName);
574 List<RegionInfo> tableRegions = ADMIN.getRegions(tableName);
575 // 0
576 try {
577 FutureUtils.get(ADMIN.mergeRegionsAsync(new byte[0][0], false));
578 fail();
579 } catch (IllegalArgumentException e) {
580 // expected
582 // 1
583 try {
584 FutureUtils.get(ADMIN
585 .mergeRegionsAsync(new byte[][] { tableRegions.get(0).getEncodedNameAsBytes() }, false));
586 fail();
587 } catch (IllegalArgumentException e) {
588 // expected
590 } finally {
591 ADMIN.disableTable(tableName);
592 ADMIN.deleteTable(tableName);
596 @Test
597 public void testSplitShouldNotHappenIfSplitIsDisabledForTable() throws Exception {
598 final TableName tableName = TableName.valueOf(name.getMethodName());
599 TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
600 .setRegionSplitPolicyClassName(DisabledRegionSplitPolicy.class.getName())
601 .setColumnFamily(ColumnFamilyDescriptorBuilder.of("f")).build();
602 Table table = TEST_UTIL.createTable(htd, null);
603 for (int i = 0; i < 10; i++) {
604 Put p = new Put(Bytes.toBytes("row" + i));
605 byte[] q1 = Bytes.toBytes("q1");
606 byte[] v1 = Bytes.toBytes("v1");
607 p.addColumn(Bytes.toBytes("f"), q1, v1);
608 table.put(p);
610 ADMIN.flush(tableName);
611 try {
612 ADMIN.split(tableName, Bytes.toBytes("row5"));
613 Threads.sleep(10000);
614 } catch (Exception e) {
615 // Nothing to do.
617 // Split should not happen.
618 List<RegionInfo> allRegions =
619 MetaTableAccessor.getTableRegions(ADMIN.getConnection(), tableName, true);
620 assertEquals(1, allRegions.size());