HBASE-22552 Rewrite TestEndToEndSplitTransaction.testCanSplitJustAfterASplit
This commit is contained in:
parent
8db9c84cd0
commit
03436e68bc
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -28,9 +29,7 @@ import java.util.Map;
|
|||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ChoreService;
|
||||
|
@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.Stoppable;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.CompactionState;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
|
@ -60,13 +58,9 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||
import org.apache.hadoop.hbase.util.RetryCounter;
|
||||
import org.apache.hadoop.hbase.util.StoppableImplementation;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
|
@ -76,12 +70,16 @@ import org.junit.rules.TestName;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
|
||||
import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
|
||||
|
||||
@Category(LargeTests.class)
|
||||
public class TestEndToEndSplitTransaction {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestEndToEndSplitTransaction.class);
|
||||
HBaseClassTestRule.forClass(TestEndToEndSplitTransaction.class);
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestEndToEndSplitTransaction.class);
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
@ -93,7 +91,7 @@ public class TestEndToEndSplitTransaction {
|
|||
@BeforeClass
|
||||
public static void beforeAllTests() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
|
||||
TEST_UTIL.startMiniCluster();
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -101,64 +99,56 @@ public class TestEndToEndSplitTransaction {
|
|||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
/**
|
||||
* This is the test for : HBASE-20940 This test will split the region and try to open an reference
|
||||
* over store file. Once store file has any reference, it makes sure that region can't be split
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testCanSplitJustAfterASplit() throws Exception {
|
||||
LOG.info("Starting testCanSplitJustAfterASplit");
|
||||
byte[] fam = Bytes.toBytes("cf_split");
|
||||
|
||||
CompactSplit compactSplit =
|
||||
TEST_UTIL.getMiniHBaseCluster().getRegionServer(0).getCompactSplitThread();
|
||||
TableName tableName = TableName.valueOf("CanSplitTable");
|
||||
Table source = TEST_UTIL.getConnection().getTable(tableName);
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
// set a large min compaction file count to avoid compaction just after splitting.
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build();
|
||||
Map<String, StoreFileReader> scanner = Maps.newHashMap();
|
||||
|
||||
try {
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build();
|
||||
|
||||
admin.createTable(htd);
|
||||
TEST_UTIL.loadTable(source, fam);
|
||||
List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
|
||||
regions.get(0).forceSplit(null);
|
||||
compactSplit.setCompactionsEnabled(false);
|
||||
TEST_UTIL.getHBaseCluster().getRegions(tableName).get(0).forceSplit(null);
|
||||
admin.split(tableName);
|
||||
TEST_UTIL.waitFor(60000, () -> TEST_UTIL.getHBaseCluster().getRegions(tableName).size() == 2);
|
||||
|
||||
while (regions.size() <= 1) {
|
||||
regions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
|
||||
regions.stream()
|
||||
.forEach(r -> r.getStores().get(0).getStorefiles().stream()
|
||||
.filter(
|
||||
s -> s.isReference() && !scanner.containsKey(r.getRegionInfo().getEncodedName()))
|
||||
.forEach(sf -> {
|
||||
StoreFileReader reader = ((HStoreFile) sf).getReader();
|
||||
reader.getStoreFileScanner(true, false, false, 0, 0, false);
|
||||
scanner.put(r.getRegionInfo().getEncodedName(), reader);
|
||||
LOG.info("Got reference to file = " + sf.getPath() + ",for region = "
|
||||
+ r.getRegionInfo().getEncodedName());
|
||||
}));
|
||||
List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
|
||||
regions.stream()
|
||||
.forEach(r -> r.getStores().get(0).getStorefiles().stream()
|
||||
.filter(s -> s.isReference() && !scanner.containsKey(r.getRegionInfo().getEncodedName()))
|
||||
.forEach(sf -> {
|
||||
StoreFileReader reader = ((HStoreFile) sf).getReader();
|
||||
reader.getStoreFileScanner(true, false, false, 0, 0, false);
|
||||
scanner.put(r.getRegionInfo().getEncodedName(), reader);
|
||||
LOG.info("Got reference to file = " + sf.getPath() + ",for region = " +
|
||||
r.getRegionInfo().getEncodedName());
|
||||
}));
|
||||
assertTrue("Regions did not split properly", regions.size() > 1);
|
||||
assertTrue("Could not get reference any of the store file", scanner.size() > 1);
|
||||
compactSplit.setCompactionsEnabled(true);
|
||||
for (HRegion region : regions) {
|
||||
region.compact(true);
|
||||
}
|
||||
|
||||
Assert.assertTrue("Regions did not split properly", regions.size() > 1);
|
||||
Assert.assertTrue("Could not get reference any of the store file", scanner.size() > 1);
|
||||
|
||||
RetryCounter retrier = new RetryCounter(30, 1, TimeUnit.SECONDS);
|
||||
while (CompactionState.NONE != admin.getCompactionState(tableName) && retrier.shouldRetry()) {
|
||||
retrier.sleepUntilNextRetry();
|
||||
}
|
||||
|
||||
Assert.assertEquals("Compaction did not complete in 30 secs", CompactionState.NONE,
|
||||
admin.getCompactionState(tableName));
|
||||
|
||||
regions.stream()
|
||||
.filter(region -> scanner.containsKey(region.getRegionInfo().getEncodedName()))
|
||||
.forEach(r -> Assert.assertTrue("Contains an open file reference which can be split",
|
||||
!r.getStores().get(0).canSplit()));
|
||||
.filter(region -> scanner.containsKey(region.getRegionInfo().getEncodedName()))
|
||||
.forEach(r -> assertFalse("Contains an open file reference which can be split",
|
||||
r.getStores().get(0).canSplit()));
|
||||
} finally {
|
||||
scanner.values().stream().forEach(s -> {
|
||||
scanner.values().forEach(s -> {
|
||||
try {
|
||||
s.close(true);
|
||||
} catch (IOException ioe) {
|
||||
|
@ -166,8 +156,9 @@ public class TestEndToEndSplitTransaction {
|
|||
}
|
||||
});
|
||||
scanner.clear();
|
||||
if (source != null) {
|
||||
source.close();
|
||||
Closeables.close(source, true);
|
||||
if (!compactSplit.isCompactionsEnabled()) {
|
||||
compactSplit.setCompactionsEnabled(true);
|
||||
}
|
||||
TEST_UTIL.deleteTableIfAny(tableName);
|
||||
}
|
||||
|
@ -182,8 +173,8 @@ public class TestEndToEndSplitTransaction {
|
|||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
final byte[] FAMILY = Bytes.toBytes("family");
|
||||
|
||||
//SplitTransaction will update the meta table by offlining the parent region, and adding info
|
||||
//for daughters.
|
||||
// SplitTransaction will update the meta table by offlining the parent region, and adding info
|
||||
// for daughters.
|
||||
Table table = TEST_UTIL.createTable(tableName, FAMILY);
|
||||
|
||||
Stoppable stopper = new StoppableImplementation();
|
||||
|
@ -194,7 +185,7 @@ public class TestEndToEndSplitTransaction {
|
|||
choreService.scheduleChore(regionChecker);
|
||||
regionSplitter.start();
|
||||
|
||||
//wait until the splitter is finished
|
||||
// wait until the splitter is finished
|
||||
regionSplitter.join();
|
||||
stopper.stop(null);
|
||||
|
||||
|
@ -206,7 +197,7 @@ public class TestEndToEndSplitTransaction {
|
|||
throw new AssertionError("regionSplitter", regionSplitter.ex);
|
||||
}
|
||||
|
||||
//one final check
|
||||
// one final check
|
||||
regionChecker.verify();
|
||||
}
|
||||
|
||||
|
@ -222,7 +213,7 @@ public class TestEndToEndSplitTransaction {
|
|||
RegionSplitter(Table table) throws IOException {
|
||||
this.table = table;
|
||||
this.tableName = table.getName();
|
||||
this.family = table.getTableDescriptor().getFamiliesKeys().iterator().next();
|
||||
this.family = table.getDescriptor().getColumnFamilies()[0].getName();
|
||||
admin = TEST_UTIL.getAdmin();
|
||||
rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
|
||||
connection = TEST_UTIL.getConnection();
|
||||
|
@ -276,7 +267,7 @@ public class TestEndToEndSplitTransaction {
|
|||
|
||||
void addData(int start) throws IOException {
|
||||
List<Put> puts = new ArrayList<>();
|
||||
for (int i=start; i< start + 100; i++) {
|
||||
for (int i = start; i < start + 100; i++) {
|
||||
Put put = new Put(Bytes.toBytes(i));
|
||||
put.addColumn(family, family, Bytes.toBytes(i));
|
||||
puts.add(put);
|
||||
|
@ -306,26 +297,26 @@ public class TestEndToEndSplitTransaction {
|
|||
void verifyRegionsUsingMetaTableAccessor() throws Exception {
|
||||
List<RegionInfo> regionList = MetaTableAccessor.getTableRegions(connection, tableName, true);
|
||||
verifyTableRegions(regionList.stream()
|
||||
.collect(Collectors.toCollection(() -> new TreeSet<>(RegionInfo.COMPARATOR))));
|
||||
.collect(Collectors.toCollection(() -> new TreeSet<>(RegionInfo.COMPARATOR))));
|
||||
regionList = MetaTableAccessor.getAllRegions(connection, true);
|
||||
verifyTableRegions(regionList.stream()
|
||||
.collect(Collectors.toCollection(() -> new TreeSet<>(RegionInfo.COMPARATOR))));
|
||||
.collect(Collectors.toCollection(() -> new TreeSet<>(RegionInfo.COMPARATOR))));
|
||||
}
|
||||
|
||||
/** verify region boundaries obtained from HTable.getStartEndKeys() */
|
||||
void verifyRegionsUsingHTable() throws IOException {
|
||||
Table table = null;
|
||||
try {
|
||||
//HTable.getStartEndKeys()
|
||||
// HTable.getStartEndKeys()
|
||||
table = connection.getTable(tableName);
|
||||
|
||||
try(RegionLocator rl = connection.getRegionLocator(tableName)) {
|
||||
try (RegionLocator rl = connection.getRegionLocator(tableName)) {
|
||||
Pair<byte[][], byte[][]> keys = rl.getStartEndKeys();
|
||||
verifyStartEndKeys(keys);
|
||||
|
||||
Set<RegionInfo> regions = new TreeSet<>(RegionInfo.COMPARATOR);
|
||||
for (HRegionLocation loc : rl.getAllRegionLocations()) {
|
||||
regions.add(loc.getRegionInfo());
|
||||
regions.add(loc.getRegion());
|
||||
}
|
||||
verifyTableRegions(regions);
|
||||
}
|
||||
|
@ -346,7 +337,7 @@ public class TestEndToEndSplitTransaction {
|
|||
byte[][] startKeys = new byte[regions.size()][];
|
||||
byte[][] endKeys = new byte[regions.size()][];
|
||||
|
||||
int i=0;
|
||||
int i = 0;
|
||||
for (RegionInfo region : regions) {
|
||||
startKeys[i] = region.getStartKey();
|
||||
endKeys[i] = region.getEndKey();
|
||||
|
@ -363,20 +354,20 @@ public class TestEndToEndSplitTransaction {
|
|||
assertEquals(startKeys.length, endKeys.length);
|
||||
assertTrue("Found 0 regions for the table", startKeys.length > 0);
|
||||
|
||||
assertArrayEquals("Start key for the first region is not byte[0]",
|
||||
HConstants.EMPTY_START_ROW, startKeys[0]);
|
||||
assertArrayEquals("Start key for the first region is not byte[0]", HConstants.EMPTY_START_ROW,
|
||||
startKeys[0]);
|
||||
byte[] prevEndKey = HConstants.EMPTY_START_ROW;
|
||||
|
||||
// ensure that we do not have any gaps
|
||||
for (int i=0; i<startKeys.length; i++) {
|
||||
for (int i = 0; i < startKeys.length; i++) {
|
||||
assertArrayEquals(
|
||||
"Hole in hbase:meta is detected. prevEndKey=" + Bytes.toStringBinary(prevEndKey)
|
||||
+ " ,regionStartKey=" + Bytes.toStringBinary(startKeys[i]), prevEndKey,
|
||||
startKeys[i]);
|
||||
"Hole in hbase:meta is detected. prevEndKey=" + Bytes.toStringBinary(prevEndKey) +
|
||||
" ,regionStartKey=" + Bytes.toStringBinary(startKeys[i]),
|
||||
prevEndKey, startKeys[i]);
|
||||
prevEndKey = endKeys[i];
|
||||
}
|
||||
assertArrayEquals("End key for the last region is not byte[0]", HConstants.EMPTY_END_ROW,
|
||||
endKeys[endKeys.length - 1]);
|
||||
endKeys[endKeys.length - 1]);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -428,10 +419,9 @@ public class TestEndToEndSplitTransaction {
|
|||
* Blocks until the region split is complete in hbase:meta and region server opens the daughters
|
||||
*/
|
||||
public static void blockUntilRegionSplit(Configuration conf, long timeout,
|
||||
final byte[] regionName, boolean waitForDaughters)
|
||||
throws IOException, InterruptedException {
|
||||
final byte[] regionName, boolean waitForDaughters) throws IOException, InterruptedException {
|
||||
long start = System.currentTimeMillis();
|
||||
log("blocking until region is split:" + Bytes.toStringBinary(regionName));
|
||||
log("blocking until region is split:" + Bytes.toStringBinary(regionName));
|
||||
RegionInfo daughterA = null, daughterB = null;
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Table metaTable = conn.getTable(TableName.META_TABLE_NAME)) {
|
||||
|
@ -459,7 +449,7 @@ public class TestEndToEndSplitTransaction {
|
|||
Bytes.toString(regionName) + ", region=" + region);
|
||||
}
|
||||
|
||||
//if we are here, this means the region split is complete or timed out
|
||||
// if we are here, this means the region split is complete or timed out
|
||||
if (waitForDaughters) {
|
||||
long rem = timeout - (System.currentTimeMillis() - start);
|
||||
blockUntilRegionIsInMeta(conn, rem, daughterA);
|
||||
|
@ -504,7 +494,7 @@ public class TestEndToEndSplitTransaction {
|
|||
long start = System.currentTimeMillis();
|
||||
while (System.currentTimeMillis() - start < timeout) {
|
||||
HRegionLocation loc = MetaTableAccessor.getRegionLocation(conn, hri);
|
||||
if (loc != null && !loc.getRegionInfo().isOffline()) {
|
||||
if (loc != null && !loc.getRegion().isOffline()) {
|
||||
log("found region in META: " + hri.getRegionNameAsString());
|
||||
break;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue