Skip to content

Commit

Permalink
HBASE-22552 Rewrite TestEndToEndSplitTransaction.testCanSplitJustAfte…
Browse files Browse the repository at this point in the history
…rASplit
  • Loading branch information
Apache9 committed Jun 10, 2019
1 parent 8db9c84 commit 03436e6
Showing 1 changed file with 62 additions and 72 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

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;
Expand All @@ -28,9 +29,7 @@
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;
Expand All @@ -45,7 +44,6 @@
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;
Expand All @@ -60,13 +58,9 @@
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;
Expand All @@ -76,12 +70,16 @@
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();
Expand All @@ -93,81 +91,74 @@ 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
public static void afterAllTests() throws Exception {
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());
}));
}

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();
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.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) {
LOG.error("Failed while closing store file", ioe);
}
});
scanner.clear();
if (source != null) {
source.close();
Closeables.close(source, true);
if (!compactSplit.isCompactionsEnabled()) {
compactSplit.setCompactionsEnabled(true);
}
TEST_UTIL.deleteTableIfAny(tableName);
}
Expand All @@ -182,8 +173,8 @@ public void testFromClientSideWhileSplitting() throws Throwable {
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();
Expand All @@ -194,7 +185,7 @@ public void testFromClientSideWhileSplitting() throws Throwable {
choreService.scheduleChore(regionChecker);
regionSplitter.start();

//wait until the splitter is finished
// wait until the splitter is finished
regionSplitter.join();
stopper.stop(null);

Expand All @@ -206,7 +197,7 @@ public void testFromClientSideWhileSplitting() throws Throwable {
throw new AssertionError("regionSplitter", regionSplitter.ex);
}

//one final check
// one final check
regionChecker.verify();
}

Expand All @@ -222,7 +213,7 @@ static class RegionSplitter extends Thread {
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();
Expand Down Expand Up @@ -276,7 +267,7 @@ public void run() {

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);
Expand Down Expand Up @@ -306,26 +297,26 @@ static class RegionChecker extends ScheduledChore {
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);
}
Expand All @@ -346,7 +337,7 @@ void verifyTableRegions(Set<RegionInfo> regions) {
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();
Expand All @@ -363,20 +354,20 @@ void verifyStartEndKeys(Pair<byte[][], byte[][]> keys) {
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
Expand Down Expand Up @@ -428,10 +419,9 @@ public static void compactAndBlockUntilDone(Admin admin, HRegionServer rs, byte[
* 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)) {
Expand Down Expand Up @@ -459,7 +449,7 @@ public static void blockUntilRegionSplit(Configuration conf, long timeout,
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);
Expand Down Expand Up @@ -504,7 +494,7 @@ public static void blockUntilRegionIsInMeta(Connection conn, long timeout, Regio
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;
}
Expand Down

0 comments on commit 03436e6

Please sign in to comment.