HBASE-22456 Polish TestSplitTransitionOnCluster
This commit is contained in:
parent
8e47c8e739
commit
f1a8aa4fd7
|
@ -31,7 +31,9 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -53,7 +55,6 @@ import org.apache.hadoop.hbase.UnknownRegionException;
|
|||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
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.Consistency;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
|
||||
|
@ -91,7 +92,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsck;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
import org.apache.hadoop.hbase.util.RetryCounter;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
||||
|
@ -298,7 +298,7 @@ public class TestSplitTransactionOnCluster {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSplitRollbackOnRegionClosing() throws IOException, InterruptedException {
|
||||
public void testSplitRollbackOnRegionClosing() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
|
||||
// Create table then get the single region for our new table.
|
||||
|
@ -340,7 +340,7 @@ public class TestSplitTransactionOnCluster {
|
|||
}
|
||||
regionStates.updateRegionState(hri, State.OPEN);
|
||||
// Now try splitting and it should work.
|
||||
split(hri, server, regionCount);
|
||||
admin.splitRegionAsync(hri.getRegionName()).get(2, TimeUnit.MINUTES);
|
||||
// Get daughters
|
||||
checkAndGetDaughters(tableName);
|
||||
// OK, so split happened after we cleared the blocking node.
|
||||
|
@ -352,13 +352,11 @@ public class TestSplitTransactionOnCluster {
|
|||
}
|
||||
|
||||
/**
|
||||
* Test that if daughter split on us, we won't do the shutdown handler fixup
|
||||
* just because we can't find the immediate daughter of an offlined parent.
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
* Test that if daughter split on us, we won't do the shutdown handler fixup just because we can't
|
||||
* find the immediate daughter of an offlined parent.
|
||||
*/
|
||||
@Test
|
||||
public void testShutdownFixupWhenDaughterHasSplit()throws IOException, InterruptedException {
|
||||
public void testShutdownFixupWhenDaughterHasSplit() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
|
||||
// Create table then get the single region for our new table.
|
||||
|
@ -378,42 +376,27 @@ public class TestSplitTransactionOnCluster {
|
|||
// Get region pre-split.
|
||||
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
|
||||
printOutRegions(server, "Initial regions: ");
|
||||
int regionCount = cluster.getRegions(hri.getTable()).size();
|
||||
// Now split.
|
||||
split(hri, server, regionCount);
|
||||
admin.splitRegionAsync(hri.getRegionName()).get(2, TimeUnit.MINUTES);
|
||||
// Get daughters
|
||||
List<HRegion> daughters = checkAndGetDaughters(tableName);
|
||||
HRegion daughterRegion = daughters.get(0);
|
||||
// Now split one of the daughters.
|
||||
regionCount = cluster.getRegions(hri.getTable()).size();
|
||||
RegionInfo daughter = daughters.get(0).getRegionInfo();
|
||||
RegionInfo daughter = daughterRegion.getRegionInfo();
|
||||
LOG.info("Daughter we are going to split: " + daughter);
|
||||
// Compact first to ensure we have cleaned up references -- else the split
|
||||
// will fail.
|
||||
this.admin.compactRegion(daughter.getRegionName());
|
||||
RetryCounter retrier = new RetryCounter(30, 1, TimeUnit.SECONDS);
|
||||
while (CompactionState.NONE != admin.getCompactionStateForRegion(daughter.getRegionName())
|
||||
&& retrier.shouldRetry()) {
|
||||
retrier.sleepUntilNextRetry();
|
||||
}
|
||||
daughters = cluster.getRegions(tableName);
|
||||
HRegion daughterRegion = null;
|
||||
for (HRegion r : daughters) {
|
||||
if (RegionInfo.COMPARATOR.compare(r.getRegionInfo(), daughter) == 0) {
|
||||
daughterRegion = r;
|
||||
// Archiving the compacted references file
|
||||
r.getStores().get(0).closeAndArchiveCompactedFiles();
|
||||
LOG.info("Found matching HRI: " + daughterRegion);
|
||||
daughterRegion.compact(true);
|
||||
daughterRegion.getStores().get(0).closeAndArchiveCompactedFiles();
|
||||
for (int i = 0; i < 100; i++) {
|
||||
if (!daughterRegion.hasReferences()) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(daughterRegion != null);
|
||||
for (int i = 0; i < 100; i++) {
|
||||
if (!daughterRegion.hasReferences()) break;
|
||||
Threads.sleep(100);
|
||||
}
|
||||
assertFalse("Waiting for reference to be compacted", daughterRegion.hasReferences());
|
||||
LOG.info("Daughter hri before split (has been compacted): " + daughter);
|
||||
split(daughter, server, regionCount);
|
||||
admin.splitRegionAsync(daughter.getRegionName()).get(2, TimeUnit.MINUTES);
|
||||
// Get list of daughters
|
||||
daughters = cluster.getRegions(tableName);
|
||||
for (HRegion d: daughters) {
|
||||
|
@ -510,38 +493,32 @@ public class TestSplitTransactionOnCluster {
|
|||
}
|
||||
|
||||
/**
|
||||
* Verifies HBASE-5806. Here the case is that splitting is completed but before the
|
||||
* CJ could remove the parent region the master is killed and restarted.
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
* @throws NodeExistsException
|
||||
* @throws KeeperException
|
||||
* Verifies HBASE-5806. Here the case is that splitting is completed but before the CJ could
|
||||
* remove the parent region the master is killed and restarted.
|
||||
*/
|
||||
@Test
|
||||
public void testMasterRestartAtRegionSplitPendingCatalogJanitor()
|
||||
throws IOException, InterruptedException, NodeExistsException,
|
||||
KeeperException, ServiceException {
|
||||
throws IOException, InterruptedException, NodeExistsException, KeeperException,
|
||||
ServiceException, ExecutionException, TimeoutException {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
|
||||
// Create table then get the single region for our new table.
|
||||
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
|
||||
List<HRegion> regions = cluster.getRegions(tableName);
|
||||
RegionInfo hri = getAndCheckSingleTableRegion(regions);
|
||||
try (Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY)) {
|
||||
List<HRegion> regions = cluster.getRegions(tableName);
|
||||
RegionInfo hri = getAndCheckSingleTableRegion(regions);
|
||||
|
||||
int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
|
||||
int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
|
||||
|
||||
// Turn off balancer so it doesn't cut in and mess up our placements.
|
||||
this.admin.balancerSwitch(false, true);
|
||||
// Turn off the meta scanner so it don't remove parent on us.
|
||||
cluster.getMaster().setCatalogJanitorEnabled(false);
|
||||
try {
|
||||
// Turn off balancer so it doesn't cut in and mess up our placements.
|
||||
this.admin.balancerSwitch(false, true);
|
||||
// Turn off the meta scanner so it don't remove parent on us.
|
||||
cluster.getMaster().setCatalogJanitorEnabled(false);
|
||||
// Add a bit of load up into the table so splittable.
|
||||
TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY, false);
|
||||
// Get region pre-split.
|
||||
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
|
||||
printOutRegions(server, "Initial regions: ");
|
||||
// Call split.
|
||||
this.admin.splitRegionAsync(hri.getRegionName());
|
||||
this.admin.splitRegionAsync(hri.getRegionName()).get(2, TimeUnit.MINUTES);
|
||||
List<HRegion> daughters = checkAndGetDaughters(tableName);
|
||||
|
||||
// Before cleanup, get a new master.
|
||||
|
@ -549,12 +526,6 @@ public class TestSplitTransactionOnCluster {
|
|||
// Now call compact on the daughters and clean up any references.
|
||||
for (HRegion daughter : daughters) {
|
||||
daughter.compact(true);
|
||||
RetryCounter retrier = new RetryCounter(30, 1, TimeUnit.SECONDS);
|
||||
while (CompactionState.NONE != admin
|
||||
.getCompactionStateForRegion(daughter.getRegionInfo().getRegionName())
|
||||
&& retrier.shouldRetry()) {
|
||||
retrier.sleepUntilNextRetry();
|
||||
}
|
||||
daughter.getStores().get(0).closeAndArchiveCompactedFiles();
|
||||
assertFalse(daughter.hasReferences());
|
||||
}
|
||||
|
@ -563,7 +534,7 @@ public class TestSplitTransactionOnCluster {
|
|||
// is reading from. Compacted-away files are picked up later by the compacted
|
||||
// file discharger process. It runs infrequently. Make it run so CatalogJanitor
|
||||
// doens't find any references.
|
||||
for (RegionServerThread rst: cluster.getRegionServerThreads()) {
|
||||
for (RegionServerThread rst : cluster.getRegionServerThreads()) {
|
||||
boolean oldSetting = rst.getRegionServer().compactedFileDischarger.setUseExecutor(false);
|
||||
rst.getRegionServer().compactedFileDischarger.run();
|
||||
rst.getRegionServer().compactedFileDischarger.setUseExecutor(oldSetting);
|
||||
|
@ -578,7 +549,6 @@ public class TestSplitTransactionOnCluster {
|
|||
} finally {
|
||||
TESTING_UTIL.getAdmin().balancerSwitch(true, false);
|
||||
cluster.getMaster().setCatalogJanitorEnabled(true);
|
||||
t.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -810,24 +780,24 @@ public class TestSplitTransactionOnCluster {
|
|||
}
|
||||
Thread.sleep(100);
|
||||
}
|
||||
return(null);
|
||||
return null;
|
||||
}
|
||||
|
||||
private List<HRegion> checkAndGetDaughters(TableName tableName)
|
||||
throws InterruptedException {
|
||||
private List<HRegion> checkAndGetDaughters(TableName tableName) throws InterruptedException {
|
||||
List<HRegion> daughters = null;
|
||||
// try up to 10s
|
||||
for (int i=0; i<100; i++) {
|
||||
for (int i = 0; i < 100; i++) {
|
||||
daughters = cluster.getRegions(tableName);
|
||||
if (daughters.size() >= 2) break;
|
||||
if (daughters.size() >= 2) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
}
|
||||
assertTrue(daughters.size() >= 2);
|
||||
return daughters;
|
||||
}
|
||||
|
||||
private HMaster abortAndWaitForMaster()
|
||||
throws IOException, InterruptedException {
|
||||
private HMaster abortAndWaitForMaster() throws IOException, InterruptedException {
|
||||
cluster.abortMaster(0);
|
||||
cluster.waitOnMaster(0);
|
||||
HMaster master = cluster.startMaster().getMaster();
|
||||
|
@ -835,17 +805,6 @@ public class TestSplitTransactionOnCluster {
|
|||
return master;
|
||||
}
|
||||
|
||||
private void split(final RegionInfo hri, final HRegionServer server, final int regionCount)
|
||||
throws IOException, InterruptedException {
|
||||
admin.splitRegionAsync(hri.getRegionName());
|
||||
for (int i = 0; cluster.getRegions(hri.getTable()).size() <= regionCount && i < 60; i++) {
|
||||
LOG.debug("Waiting on region " + hri.getRegionNameAsString() + " to split");
|
||||
Thread.sleep(2000);
|
||||
}
|
||||
assertFalse("Waited too long for split",
|
||||
cluster.getRegions(hri.getTable()).size() <= regionCount);
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure single table region is not on same server as the single hbase:meta table
|
||||
* region.
|
||||
|
|
Loading…
Reference in New Issue