HBASE-14969 Add throughput controller for flush; ADDENDUM

This commit is contained in:
stack 2016-02-01 08:08:07 -08:00
parent 2cc48e039d
commit fc5e698c0a
1 changed files with 6 additions and 3 deletions

View File

@ -168,9 +168,6 @@ public class TestFlushWithThroughputController {
conf.setInt(PressureAwareFlushThroughputController.HBASE_HSTORE_FLUSH_THROUGHPUT_TUNE_PERIOD,
3000);
TEST_UTIL.startMiniCluster(1);
assertEquals(10L * 1024 * 1024,
((PressureAwareThroughputController) TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
.getFlushThroughputController()).getMaxThroughput(), EPSILON);
Connection conn = ConnectionFactory.createConnection(conf);
try {
HTableDescriptor htd = new HTableDescriptor(tableName);
@ -181,6 +178,12 @@ public class TestFlushWithThroughputController {
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
PressureAwareFlushThroughputController throughputController =
(PressureAwareFlushThroughputController) regionServer.getFlushThroughputController();
for (Region region : regionServer.getOnlineRegions()) {
region.flush(true);
}
assertEquals(0.0, regionServer.getFlushPressure(), EPSILON);
Thread.sleep(5000);
assertEquals(10L * 1024 * 1024, throughputController.getMaxThroughput(), EPSILON);
Table table = conn.getTable(tableName);
Random rand = new Random();
for (int i = 0; i < 10; i++) {