HBASE-19510 TestDistributedLogSplitting is flakey for AsyncFSWAL

This commit is contained in:
zhangduo 2017-12-14 15:59:41 +08:00
parent cb4bbea0f1
commit a1f8821b85
3 changed files with 208 additions and 187 deletions

View File

@ -51,11 +51,8 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@ -76,21 +73,16 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKeyImpl;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.After;
@ -99,111 +91,94 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@Category({MasterTests.class, LargeTests.class})
@SuppressWarnings("deprecation")
public class TestDistributedLogSplitting {
/**
* Base class for testing distributed log splitting.
*/
public abstract class AbstractTestDLS {
private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
static {
// Uncomment the following line if more verbosity is needed for
// debugging (see HBASE-12285 for details).
//Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
// test ThreeRSAbort fails under hadoop2 (2.0.2-alpha) if shortcircuit-read (scr) is on. this
// turns it off for this test. TODO: Figure out why scr breaks recovery.
System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
}
// Start a cluster with 2 masters and 5 regionservers
private static final int NUM_MASTERS = 2;
private static final int NUM_RS = 5;
private static byte[] COLUMN_FAMILY = Bytes.toBytes("family");
@Rule
public TestName testName = new TestName();
TableName tableName;
// Start a cluster with 2 masters and 6 regionservers
static final int NUM_MASTERS = 2;
static final int NUM_RS = 5;
static byte[] COLUMN_FAMILY = Bytes.toBytes("family");
MiniHBaseCluster cluster;
HMaster master;
Configuration conf;
static Configuration originalConf;
static HBaseTestingUtility TEST_UTIL;
static MiniZooKeeperCluster zkCluster;
private TableName tableName;
private MiniHBaseCluster cluster;
private HMaster master;
private Configuration conf;
@Rule
public TestName name = new TestName();
@BeforeClass
public static void setup() throws Exception {
TEST_UTIL = new HBaseTestingUtility(HBaseConfiguration.create());
zkCluster = TEST_UTIL.startMiniZKCluster();
originalConf = TEST_UTIL.getConfiguration();
// Uncomment the following line if more verbosity is needed for
// debugging (see HBASE-12285 for details).
// Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(3);
}
@AfterClass
public static void tearDown() throws IOException {
TEST_UTIL.shutdownMiniZKCluster();
TEST_UTIL.shutdownMiniDFSCluster();
TEST_UTIL.shutdownMiniHBaseCluster();
public static void tearDown() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
private void startCluster(int num_rs) throws Exception {
protected abstract String getWalProvider();
private void startCluster(int numRS) throws Exception {
SplitLogCounters.resetCounters();
LOG.info("Starting cluster");
conf.getLong("hbase.splitlog.max.resubmit", 0);
conf.setLong("hbase.splitlog.max.resubmit", 0);
// Make the failure test faster
conf.setInt("zookeeper.recovery.retry", 0);
conf.setInt(HConstants.REGIONSERVER_INFO_PORT, -1);
conf.setFloat(HConstants.LOAD_BALANCER_SLOP_KEY, (float) 100.0); // no load balancing
conf.setInt("hbase.regionserver.wal.max.splitters", 3);
conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.setZkCluster(zkCluster);
TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, num_rs);
conf.set("hbase.wal.provider", getWalProvider());
TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, numRS);
cluster = TEST_UTIL.getHBaseCluster();
LOG.info("Waiting for active/ready master");
cluster.waitForActiveAndReadyMaster();
master = cluster.getMaster();
while (cluster.getLiveRegionServerThreads().size() < num_rs) {
Threads.sleep(10);
}
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return cluster.getLiveRegionServerThreads().size() >= numRS;
}
});
}
@Before
public void before() throws Exception {
// refresh configuration
conf = HBaseConfiguration.create(originalConf);
conf = TEST_UTIL.getConfiguration();
tableName = TableName.valueOf(testName.getMethodName());
}
@After
public void after() throws Exception {
try {
if (TEST_UTIL.getHBaseCluster() != null) {
for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
mt.getMaster().abort("closing...", null);
}
}
TEST_UTIL.shutdownMiniHBaseCluster();
} finally {
TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
}
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testRecoveredEdits() throws Exception {
conf.setLong("hbase.regionserver.hlog.blocksize", 30 * 1024); // create more than one wal
startCluster(NUM_RS);
final int NUM_LOG_LINES = 10000;
final SplitLogManager slm = master.getMasterWalManager().getSplitLogManager();
int numLogLines = 10000;
SplitLogManager slm = master.getMasterWalManager().getSplitLogManager();
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
@ -214,8 +189,8 @@ public class TestDistributedLogSplitting {
Path rootdir = FSUtils.getRootDir(conf);
int numRegions = 50;
Table t = installTable(new ZKWatcher(conf, "table-creation", null), numRegions);
try {
try (ZKWatcher zkw = new ZKWatcher(conf, "table-creation", null);
Table t = installTable(zkw, numRegions)) {
TableName table = t.getName();
List<RegionInfo> regions = null;
HRegionServer hrs = null;
@ -223,10 +198,12 @@ public class TestDistributedLogSplitting {
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
// At least one RS will have >= to average number of regions.
if (regions.size() >= numRegions/NUM_RS) break;
if (regions.size() >= numRegions / NUM_RS) {
break;
}
}
final Path logDir = new Path(rootdir, AbstractFSWALProvider.getWALDirectoryName(hrs
.getServerName().toString()));
Path logDir = new Path(rootdir,
AbstractFSWALProvider.getWALDirectoryName(hrs.getServerName().toString()));
LOG.info("#regions = " + regions.size());
Iterator<RegionInfo> it = regions.iterator();
@ -238,15 +215,16 @@ public class TestDistributedLogSplitting {
}
}
makeWAL(hrs, regions, NUM_LOG_LINES, 100);
makeWAL(hrs, regions, numLogLines, 100);
slm.splitLogDistributed(logDir);
int count = 0;
for (RegionInfo hri : regions) {
Path tdir = FSUtils.getTableDir(rootdir, table);
Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(
HRegion.getRegionDir(tdir, hri.getEncodedName()));
@SuppressWarnings("deprecation")
Path editsdir = WALSplitter
.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName()));
LOG.debug("checking edits dir " + editsdir);
FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
@Override
@ -258,8 +236,8 @@ public class TestDistributedLogSplitting {
}
});
assertTrue(
"edits dir should have more than a single file in it. instead has " + files.length,
files.length > 1);
"edits dir should have more than a single file in it. instead has " + files.length,
files.length > 1);
for (int i = 0; i < files.length; i++) {
int c = countWAL(files[i].getPath(), fs, conf);
count += c;
@ -269,9 +247,7 @@ public class TestDistributedLogSplitting {
// check that the log file is moved
assertFalse(fs.exists(logDir));
assertEquals(NUM_LOG_LINES, count);
} finally {
if (t != null) t.close();
assertEquals(numLogLines, count);
}
}
@ -280,18 +256,17 @@ public class TestDistributedLogSplitting {
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
startCluster(NUM_RS);
final int NUM_REGIONS_TO_CREATE = 40;
final int NUM_LOG_LINES = 1000;
int numRegionsToCreate = 40;
int numLogLines = 1000;
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
final ZKWatcher zkw = new ZKWatcher(conf, "table-creation", null);
Table ht = installTable(zkw, NUM_REGIONS_TO_CREATE);
try {
try (ZKWatcher zkw = new ZKWatcher(conf, "table-creation", null);
Table ht = installTable(zkw, numRegionsToCreate);) {
HRegionServer hrs = findRSToKill(false);
List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeWAL(hrs, regions, NUM_LOG_LINES, 100);
makeWAL(hrs, regions, numLogLines, 100);
// abort master
abortMaster(cluster);
@ -309,71 +284,68 @@ public class TestDistributedLogSplitting {
});
Thread.sleep(2000);
LOG.info("Current Open Regions:"
+ HBaseTestingUtility.getAllOnlineRegions(cluster).size());
LOG.info("Current Open Regions:" + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
// wait for abort completes
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
>= (NUM_REGIONS_TO_CREATE + 1));
return (HBaseTestingUtility.getAllOnlineRegions(cluster)
.size() >= (numRegionsToCreate + 1));
}
});
LOG.info("Current Open Regions After Master Node Starts Up:"
+ HBaseTestingUtility.getAllOnlineRegions(cluster).size());
LOG.info("Current Open Regions After Master Node Starts Up:" +
HBaseTestingUtility.getAllOnlineRegions(cluster).size());
assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
} finally {
if (ht != null) ht.close();
if (zkw != null) zkw.close();
assertEquals(numLogLines, TEST_UTIL.countRows(ht));
}
}
/**
* The original intention of this test was to force an abort of a region
* server and to make sure that the failure path in the region servers is
* properly evaluated. But it is difficult to ensure that the region server
* doesn't finish the log splitting before it aborts. Also now, there is
* this code path where the master will preempt the region server when master
* detects that the region server has aborted.
* The original intention of this test was to force an abort of a region server and to make sure
* that the failure path in the region servers is properly evaluated. But it is difficult to
* ensure that the region server doesn't finish the log splitting before it aborts. Also now,
* there is this code path where the master will preempt the region server when master detects
* that the region server has aborted.
* @throws Exception
*/
// Was marked flaky before Distributed Log Replay cleanup.
@Test (timeout=300000)
@Test(timeout = 300000)
public void testWorkerAbort() throws Exception {
LOG.info("testWorkerAbort");
startCluster(3);
final int NUM_LOG_LINES = 10000;
final SplitLogManager slm = master.getMasterWalManager().getSplitLogManager();
int numLogLines = 10000;
SplitLogManager slm = master.getMasterWalManager().getSplitLogManager();
FileSystem fs = master.getMasterFileSystem().getFileSystem();
final List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
HRegionServer hrs = findRSToKill(false);
Path rootdir = FSUtils.getRootDir(conf);
final Path logDir = new Path(rootdir,
AbstractFSWALProvider.getWALDirectoryName(hrs.getServerName().toString()));
AbstractFSWALProvider.getWALDirectoryName(hrs.getServerName().toString()));
Table t = installTable(new ZKWatcher(conf, "table-creation", null), 40);
try {
makeWAL(hrs, ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()), NUM_LOG_LINES, 100);
try (ZKWatcher zkw = new ZKWatcher(conf, "table-creation", null);
Table t = installTable(zkw, 40)) {
makeWAL(hrs, ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()), numLogLines, 100);
new Thread() {
@Override
public void run() {
waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
try {
waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
} catch (InterruptedException e) {
}
for (RegionServerThread rst : rsts) {
rst.getRegionServer().abort("testing");
break;
}
}
}.start();
// slm.splitLogDistributed(logDir);
FileStatus[] logfiles = fs.listStatus(logDir);
TaskBatch batch = new TaskBatch();
slm.enqueueSplitTask(logfiles[0].getPath().toString(), batch);
//waitForCounter but for one of the 2 counters
// waitForCounter but for one of the 2 counters
long curt = System.currentTimeMillis();
long waitTime = 80000;
long endt = curt + waitTime;
@ -381,7 +353,7 @@ public class TestDistributedLogSplitting {
if ((tot_wkr_task_resigned.sum() + tot_wkr_task_err.sum() +
tot_wkr_final_transition_failed.sum() + tot_wkr_task_done.sum() +
tot_wkr_preempt_task.sum()) == 0) {
Thread.yield();
Thread.sleep(100);
curt = System.currentTimeMillis();
} else {
assertTrue(1 <= (tot_wkr_task_resigned.sum() + tot_wkr_task_err.sum() +
@ -390,29 +362,23 @@ public class TestDistributedLogSplitting {
return;
}
}
fail("none of the following counters went up in " + waitTime +
" milliseconds - " +
fail("none of the following counters went up in " + waitTime + " milliseconds - " +
"tot_wkr_task_resigned, tot_wkr_task_err, " +
"tot_wkr_final_transition_failed, tot_wkr_task_done, " +
"tot_wkr_preempt_task");
} finally {
if (t != null) t.close();
"tot_wkr_final_transition_failed, tot_wkr_task_done, " + "tot_wkr_preempt_task");
}
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testThreeRSAbort() throws Exception {
LOG.info("testThreeRSAbort");
final int NUM_REGIONS_TO_CREATE = 40;
final int NUM_ROWS_PER_REGION = 100;
int numRegionsToCreate = 40;
int numRowsPerRegion = 100;
startCluster(NUM_RS); // NUM_RS=6.
final ZKWatcher zkw = new ZKWatcher(conf, "distributed log splitting test", null);
Table table = installTable(zkw, NUM_REGIONS_TO_CREATE);
try {
populateDataInTable(NUM_ROWS_PER_REGION);
try (ZKWatcher zkw = new ZKWatcher(conf, "distributed log splitting test", null);
Table table = installTable(zkw, numRegionsToCreate)) {
populateDataInTable(numRowsPerRegion);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
assertEquals(NUM_RS, rsts.size());
@ -420,22 +386,24 @@ public class TestDistributedLogSplitting {
cluster.killRegionServer(rsts.get(1).getRegionServer().getServerName());
cluster.killRegionServer(rsts.get(2).getRegionServer().getServerName());
long start = EnvironmentEdgeManager.currentTime();
while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
if (EnvironmentEdgeManager.currentTime() - start > 60000) {
fail("Timed out waiting for server aborts.");
TEST_UTIL.waitFor(60000, new Waiter.ExplainingPredicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return cluster.getLiveRegionServerThreads().size() <= NUM_RS - 3;
}
Thread.sleep(200);
}
@Override
public String explainFailure() throws Exception {
return "Timed out waiting for server aborts.";
}
});
TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION, TEST_UTIL.countRows(table));
} finally {
if (table != null) table.close();
if (zkw != null) zkw.close();
assertEquals(numRegionsToCreate * numRowsPerRegion, TEST_UTIL.countRows(table));
}
}
@Test(timeout=30000)
@Test(timeout = 30000)
public void testDelayedDeleteOnFailure() throws Exception {
LOG.info("testDelayedDeleteOnFailure");
startCluster(1);
@ -511,12 +479,13 @@ public class TestDistributedLogSplitting {
Table ht = installTable(zkw, 10);
try {
FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path tableDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), TableName.valueOf(name.getMethodName()));
Path tableDir =
FSUtils.getTableDir(FSUtils.getRootDir(conf), TableName.valueOf(name.getMethodName()));
List<Path> regionDirs = FSUtils.getRegionDirs(fs, tableDir);
long newSeqId = WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0), 1L, 1000L);
WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0) , 1L, 1000L);
WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0), 1L, 1000L);
assertEquals(newSeqId + 2000,
WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0), 3L, 1000L));
WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0), 3L, 1000L));
Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regionDirs.get(0));
FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
@ -538,13 +507,13 @@ public class TestDistributedLogSplitting {
}
}
Table installTable(ZKWatcher zkw, int nrs) throws Exception {
private Table installTable(ZKWatcher zkw, int nrs) throws Exception {
return installTable(zkw, nrs, 0);
}
Table installTable(ZKWatcher zkw, int nrs, int existingRegions) throws Exception {
private Table installTable(ZKWatcher zkw, int nrs, int existingRegions) throws Exception {
// Create a table with regions
byte [] family = Bytes.toBytes("family");
byte[] family = Bytes.toBytes("family");
LOG.info("Creating table with " + nrs + " regions");
Table table = TEST_UTIL.createMultiRegionTable(tableName, family, nrs);
int numRegions = -1;
@ -588,8 +557,8 @@ public class TestDistributedLogSplitting {
if (hri.getTable().isSystemTable()) {
continue;
}
LOG.debug("adding data to rs = " + rst.getName() +
" region = "+ hri.getRegionNameAsString());
LOG.debug(
"adding data to rs = " + rst.getName() + " region = " + hri.getRegionNameAsString());
Region region = hrs.getOnlineRegion(hri.getRegionName());
assertTrue(region != null);
putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), COLUMN_FAMILY);
@ -609,8 +578,8 @@ public class TestDistributedLogSplitting {
if (hri.getTable().isSystemTable()) {
continue;
}
LOG.debug("adding data to rs = " + mt.getName() +
" region = "+ hri.getRegionNameAsString());
LOG.debug(
"adding data to rs = " + mt.getName() + " region = " + hri.getRegionNameAsString());
Region region = hrs.getOnlineRegion(hri.getRegionName());
assertTrue(region != null);
putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), COLUMN_FAMILY);
@ -623,20 +592,18 @@ public class TestDistributedLogSplitting {
makeWAL(hrs, regions, num_edits, edit_size, true);
}
public void makeWAL(HRegionServer hrs, List<RegionInfo> regions,
int num_edits, int edit_size, boolean cleanShutdown) throws IOException {
public void makeWAL(HRegionServer hrs, List<RegionInfo> regions, int numEdits, int editSize,
boolean cleanShutdown) throws IOException {
// remove root and meta region
regions.remove(RegionInfoBuilder.FIRST_META_REGIONINFO);
for(Iterator<RegionInfo> iter = regions.iterator(); iter.hasNext(); ) {
for (Iterator<RegionInfo> iter = regions.iterator(); iter.hasNext();) {
RegionInfo regionInfo = iter.next();
if(regionInfo.getTable().isSystemTable()) {
if (regionInfo.getTable().isSystemTable()) {
iter.remove();
}
}
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor(COLUMN_FAMILY));
byte[] value = new byte[edit_size];
byte[] value = new byte[editSize];
List<RegionInfo> hris = new ArrayList<>();
for (RegionInfo region : regions) {
@ -646,19 +613,19 @@ public class TestDistributedLogSplitting {
hris.add(region);
}
LOG.info("Creating wal edits across " + hris.size() + " regions.");
for (int i = 0; i < edit_size; i++) {
for (int i = 0; i < editSize; i++) {
value[i] = (byte) ('a' + (i % 26));
}
int n = hris.size();
int[] counts = new int[n];
// sync every ~30k to line up with desired wal rolls
final int syncEvery = 30 * 1024 / edit_size;
final int syncEvery = 30 * 1024 / editSize;
MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
if (n > 0) {
for (int i = 0; i < num_edits; i += 1) {
for (int i = 0; i < numEdits; i += 1) {
WALEdit e = new WALEdit();
RegionInfo curRegionInfo = hris.get(i % n);
final WAL log = hrs.getWAL(curRegionInfo);
WAL log = hrs.getWAL(curRegionInfo);
byte[] startRow = curRegionInfo.getStartKey();
if (startRow == null || startRow.length == 0) {
startRow = new byte[] { 0, 0, 0, 0, 1 };
@ -668,9 +635,9 @@ public class TestDistributedLogSplitting {
// HBaseTestingUtility.createMultiRegions use 5 bytes key
byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
e.add(new KeyValue(row, COLUMN_FAMILY, qualifier, System.currentTimeMillis(), value));
log.append(curRegionInfo,
new WALKeyImpl(curRegionInfo.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc), e, true);
log.append(curRegionInfo, new WALKeyImpl(curRegionInfo.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc),
e, true);
if (0 == i % syncEvery) {
log.sync();
}
@ -680,12 +647,12 @@ public class TestDistributedLogSplitting {
// done as two passes because the regions might share logs. shutdown is idempotent, but sync
// will cause errors if done after.
for (RegionInfo info : hris) {
final WAL log = hrs.getWAL(info);
WAL log = hrs.getWAL(info);
log.sync();
}
if (cleanShutdown) {
for (RegionInfo info : hris) {
final WAL log = hrs.getWAL(info);
WAL log = hrs.getWAL(info);
log.shutdown();
}
}
@ -695,24 +662,15 @@ public class TestDistributedLogSplitting {
return;
}
private int countWAL(Path log, FileSystem fs, Configuration conf)
throws IOException {
private int countWAL(Path log, FileSystem fs, Configuration conf) throws IOException {
int count = 0;
WAL.Reader in = WALFactory.createReader(fs, log, conf);
try {
try (WAL.Reader in = WALFactory.createReader(fs, log, conf)) {
WAL.Entry e;
while ((e = in.next()) != null) {
if (!WALEdit.isMetaEditFamily(e.getEdit().getCells().get(0))) {
count++;
}
}
} finally {
try {
in.close();
} catch (IOException exception) {
LOG.warn("Problem closing wal: " + exception.getMessage());
LOG.debug("exception details.", exception);
}
}
return count;
}
@ -721,32 +679,31 @@ public class TestDistributedLogSplitting {
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
}
private void putData(Region region, byte[] startRow, int numRows, byte [] qf,
byte [] ...families)
private void putData(Region region, byte[] startRow, int numRows, byte[] qf, byte[]... families)
throws IOException {
for(int i = 0; i < numRows; i++) {
for (int i = 0; i < numRows; i++) {
Put put = new Put(Bytes.add(startRow, Bytes.toBytes(i)));
for(byte [] family : families) {
for (byte[] family : families) {
put.addColumn(family, qf, null);
}
region.put(put);
}
}
private void waitForCounter(LongAdder ctr, long oldval, long newval,
long timems) {
private void waitForCounter(LongAdder ctr, long oldval, long newval, long timems)
throws InterruptedException {
long curt = System.currentTimeMillis();
long endt = curt + timems;
while (curt < endt) {
if (ctr.sum() == oldval) {
Thread.yield();
Thread.sleep(100);
curt = System.currentTimeMillis();
} else {
assertEquals(newval, ctr.sum());
return;
}
}
assertTrue(false);
fail();
}
private void abortMaster(MiniHBaseCluster cluster) throws InterruptedException {
@ -769,7 +726,7 @@ public class TestDistributedLogSplitting {
List<RegionInfo> regions = null;
HRegionServer hrs = null;
for (RegionServerThread rst: rsts) {
for (RegionServerThread rst : rsts) {
hrs = rst.getRegionServer();
while (rst.isAlive() && !hrs.isOnline()) {
Thread.sleep(100);
@ -794,14 +751,14 @@ public class TestDistributedLogSplitting {
if (isCarryingMeta && hasMetaRegion) {
// clients ask for a RS with META
if (!foundTableRegion) {
final HRegionServer destRS = hrs;
HRegionServer destRS = hrs;
// the RS doesn't have regions of the specified table so we need move one to this RS
List<RegionInfo> tableRegions = TEST_UTIL.getAdmin().getRegions(tableName);
final RegionInfo hri = tableRegions.get(0);
RegionInfo hri = tableRegions.get(0);
TEST_UTIL.getAdmin().move(hri.getEncodedNameAsBytes(),
Bytes.toBytes(destRS.getServerName().getServerName()));
Bytes.toBytes(destRS.getServerName().getServerName()));
// wait for region move completes
final RegionStates regionStates =
RegionStates regionStates =
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
TEST_UTIL.waitFor(45000, 200, new Waiter.Predicate<Exception>() {
@Override
@ -815,7 +772,9 @@ public class TestDistributedLogSplitting {
} else if (hasMetaRegion || isCarryingMeta) {
continue;
}
if (foundTableRegion) break;
if (foundTableRegion) {
break;
}
}
return hrs;

View File

@ -0,0 +1,31 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, LargeTests.class })
public class TestDLSAsyncFSWAL extends AbstractTestDLS {
@Override
protected String getWalProvider() {
return "asyncfs";
}
}

View File

@ -0,0 +1,31 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, LargeTests.class })
public class TestDLSFSHLog extends AbstractTestDLS {
@Override
protected String getWalProvider() {
return "filesystem";
}
}