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
;
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
{
67 public static final HBaseClassTestRule CLASS_RULE
= HBaseClassTestRule
.forClass(TestAdmin1
.class);
69 private static final Logger LOG
= LoggerFactory
.getLogger(TestAdmin1
.class);
72 public void testSplitFlushCompactUnknownTable() throws InterruptedException
{
73 final TableName unknowntable
= TableName
.valueOf(name
.getMethodName());
74 Exception exception
= null;
76 ADMIN
.compact(unknowntable
);
77 } catch (IOException e
) {
80 assertTrue(exception
instanceof TableNotFoundException
);
84 ADMIN
.flush(unknowntable
);
85 } catch (IOException e
) {
88 assertTrue(exception
instanceof TableNotFoundException
);
92 ADMIN
.split(unknowntable
);
93 } catch (IOException e
) {
96 assertTrue(exception
instanceof TableNotFoundException
);
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
);
108 Put p
= new Put(Bytes
.toBytes("row1"));
109 p
.addColumn(Bytes
.toBytes("fam1"), Bytes
.toBytes("fam1"), Bytes
.toBytes("fam1"));
111 ts
= ADMIN
.getLastMajorCompactionTimestamp(tableName
);
112 // no files written -> no data
115 ADMIN
.flush(tableName
);
116 ts
= ADMIN
.getLastMajorCompactionTimestamp(tableName
);
117 // still 0, we flushed a file, but no major compaction happened
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"));
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);
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
);
146 ADMIN
.flush(tableName
);
147 ts
= ADMIN
.getLastMajorCompactionTimestamp(tableName
);
148 assertEquals(ts
, ts1
);
152 @Test(expected
= IllegalArgumentException
.class)
153 public void testColumnValidName() {
154 ColumnFamilyDescriptorBuilder
.of("\\test\\abc");
158 public void testTableExist() throws IOException
{
159 final TableName table
= TableName
.valueOf(name
.getMethodName());
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.
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
;
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
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
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
{
217 while (scanner
.next() != null) {
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
)) {
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
);
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);
261 Scan scan
= new Scan();
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.
270 try (ResultScanner scanner
= table
.getScanner(scan
)) {
271 // Scan first row so we are into first region before split happens.
276 if (splitPoint
!= null) {
277 ADMIN
.split(tableName
, splitPoint
);
279 ADMIN
.split(tableName
);
281 final AtomicInteger count
= new AtomicInteger(0);
282 Thread t
= new Thread("CheckForSplit") {
285 for (int i
= 0; i
< 45; i
++) {
288 } catch (InterruptedException e
) {
292 List
<HRegionLocation
> regions
= null;
294 regions
= locator
.getAllRegionLocations();
295 } catch (IOException e
) {
296 LOG
.warn("get location failed", e
);
298 if (regions
== null) {
301 count
.set(regions
.size());
302 if (count
.get() >= 2) {
303 LOG
.info("Found: " + regions
);
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);
315 // Sync split region, no need to create a thread to check
316 ADMIN
.splitRegionAsync(m
.get(0).getRegion().getRegionName(), splitPoint
).get();
319 rows
= 1 + count(scanner
); // We counted one row above.
321 assertEquals(rowCount
, rows
);
323 List
<HRegionLocation
> regions
= null;
325 regions
= locator
.getAllRegionLocations();
326 } catch (IOException e
) {
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
));
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
);
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
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
;
374 oldRegions
= TEST_UTIL
.getHBaseCluster().getRegions(tableName
);
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"));
384 put
= new Put(new byte[] { (byte) '6' });
385 put
.addColumn(cf
, qualifier
, Bytes
.toBytes("100"));
387 put
= new Put(new byte[] { (byte) '8' });
388 put
.addColumn(cf
, qualifier
, Bytes
.toBytes("100"));
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
399 TEST_UTIL
.getAdmin().splitRegionAsync(regions
.get(1).getFirst().getRegionName()));
400 } catch (IllegalArgumentException ex
) {
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
409 FutureUtils
.get(TEST_UTIL
.getAdmin().splitRegionAsync(
410 regions
.get(1).getFirst().getEncodedNameAsBytes(), new byte[] { (byte) '1' }));
411 } catch (IllegalArgumentException ex
) {
414 assertTrue(gotException
);
416 gotException
= false;
417 // testing Sync split operation
419 FutureUtils
.get(TEST_UTIL
.getAdmin()
420 .splitRegionAsync(regions
.get(1).getFirst().getRegionName(), new byte[] { (byte) '1' }));
421 } catch (IllegalArgumentException ex
) {
424 assertTrue(gotException
);
426 gotException
= false;
427 // Try merging a replica with another. Should fail.
429 FutureUtils
.get(TEST_UTIL
.getAdmin().mergeRegionsAsync(
430 regions
.get(1).getFirst().getEncodedNameAsBytes(),
431 regions
.get(2).getFirst().getEncodedNameAsBytes(),
433 } catch (IllegalArgumentException m
) {
436 assertTrue(gotException
);
437 // Try going to the master directly (that will skip the check in admin)
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,
446 } catch (org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.ServiceException m
) {
447 Throwable t
= m
.getCause();
449 if (t
instanceof MergeRegionException
) {
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
469 public void testHFileReplication() throws Exception
{
470 final TableName tableName
= TableName
.valueOf(this.name
.getMethodName());
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())
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);
486 p
= new Put(Bytes
.toBytes("rep1_rk"));
487 p
.addColumn(Bytes
.toBytes(fn1
), q1
, v1
);
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());
511 if (ADMIN
.isTableEnabled(tableName
)) {
512 ADMIN
.disableTable(tableName
);
513 ADMIN
.deleteTable(tableName
);
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' };
527 TEST_UTIL
.createTable(td
, splitRows
);
528 TEST_UTIL
.waitTableAvailable(tableName
);
530 List
<RegionInfo
> tableRegions
;
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,
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)
551 .mergeRegionsAsync(regionA
.getEncodedNameAsBytes(), regionB
.getEncodedNameAsBytes(), false)
552 .get(60, TimeUnit
.SECONDS
);
554 assertEquals(1, ADMIN
.getRegions(tableName
).size());
556 ADMIN
.disableTable(tableName
);
557 ADMIN
.deleteTable(tableName
);
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' };
571 TEST_UTIL
.createTable(td
, splitRows
);
572 TEST_UTIL
.waitTableAvailable(tableName
);
574 List
<RegionInfo
> tableRegions
= ADMIN
.getRegions(tableName
);
577 FutureUtils
.get(ADMIN
.mergeRegionsAsync(new byte[0][0], false));
579 } catch (IllegalArgumentException e
) {
584 FutureUtils
.get(ADMIN
585 .mergeRegionsAsync(new byte[][] { tableRegions
.get(0).getEncodedNameAsBytes() }, false));
587 } catch (IllegalArgumentException e
) {
591 ADMIN
.disableTable(tableName
);
592 ADMIN
.deleteTable(tableName
);
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
);
610 ADMIN
.flush(tableName
);
612 ADMIN
.split(tableName
, Bytes
.toBytes("row5"));
613 Threads
.sleep(10000);
614 } catch (Exception e
) {
617 // Split should not happen.
618 List
<RegionInfo
> allRegions
=
619 MetaTableAccessor
.getTableRegions(ADMIN
.getConnection(), tableName
, true);
620 assertEquals(1, allRegions
.size());