HBASE-12840 Improve unit test coverage of the client pushback mechanism
This commit is contained in:
parent
092c91eb0f
commit
03e17168c3
|
@ -0,0 +1,62 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestDelayingRunner {
|
||||
|
||||
private static final TableName DUMMY_TABLE =
|
||||
TableName.valueOf("DUMMY_TABLE");
|
||||
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes();
|
||||
private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes();
|
||||
private static HRegionInfo hri1 =
|
||||
new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1);
|
||||
|
||||
@SuppressWarnings({ "rawtypes", "unchecked" })
|
||||
@Test
|
||||
public void testDelayingRunner() throws Exception{
|
||||
MultiAction<Row> ma = new MultiAction<Row>();
|
||||
ma.add(hri1.getRegionName(), new Action<Row>(new Put(DUMMY_BYTES_1), 0));
|
||||
final AtomicLong endTime = new AtomicLong();
|
||||
final long sleepTime = 1000;
|
||||
DelayingRunner runner = new DelayingRunner(sleepTime, ma.actions.entrySet().iterator().next());
|
||||
runner.setRunner(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
endTime.set(EnvironmentEdgeManager.currentTime());
|
||||
}
|
||||
});
|
||||
long startTime = EnvironmentEdgeManager.currentTime();
|
||||
runner.run();
|
||||
long delay = endTime.get() - startTime;
|
||||
assertTrue("DelayingRunner did not delay long enough", delay >= sleepTime);
|
||||
assertFalse("DelayingRunner delayed too long", delay > sleepTime + sleepTime*0.2);
|
||||
}
|
||||
|
||||
}
|
|
@ -17,22 +17,34 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
|
||||
import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy;
|
||||
import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Test that we can actually send and use region metrics to slowdown client writes
|
||||
|
@ -53,6 +65,9 @@ public class TestClientPushback {
|
|||
Configuration conf = UTIL.getConfiguration();
|
||||
// enable backpressure
|
||||
conf.setBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE, true);
|
||||
// use the exponential backoff policy
|
||||
conf.setClass(ClientBackoffPolicy.BACKOFF_POLICY_CLASS, ExponentialClientBackoffPolicy.class,
|
||||
ClientBackoffPolicy.class);
|
||||
// turn the memstore size way down so we don't need to write a lot to see changes in memstore
|
||||
// load
|
||||
conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSizeBytes);
|
||||
|
@ -68,7 +83,7 @@ public class TestClientPushback {
|
|||
UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout=60000)
|
||||
public void testClientTracksServerPushback() throws Exception{
|
||||
Configuration conf = UTIL.getConfiguration();
|
||||
TableName tablename = TableName.valueOf(tableName);
|
||||
|
@ -91,6 +106,10 @@ public class TestClientPushback {
|
|||
|
||||
// get the stats for the region hosting our table
|
||||
ClusterConnection connection = table.connection;
|
||||
ClientBackoffPolicy backoffPolicy = connection.getBackoffPolicy();
|
||||
assertTrue("Backoff policy is not correctly configured",
|
||||
backoffPolicy instanceof ExponentialClientBackoffPolicy);
|
||||
|
||||
ServerStatisticTracker stats = connection.getStatisticsTracker();
|
||||
assertNotNull( "No stats configured for the client!", stats);
|
||||
// get the names so we can query the stats
|
||||
|
@ -100,6 +119,35 @@ public class TestClientPushback {
|
|||
// check to see we found some load on the memstore
|
||||
ServerStatistics serverStats = stats.getServerStatsForTesting(server);
|
||||
ServerStatistics.RegionStatistics regionStats = serverStats.getStatsForRegion(regionName);
|
||||
assertEquals(load, regionStats.getMemstoreLoadPercent());
|
||||
assertEquals("We did not find some load on the memstore", load,
|
||||
regionStats.getMemstoreLoadPercent());
|
||||
|
||||
// check that the load reported produces a nonzero delay
|
||||
long backoffTime = backoffPolicy.getBackoffTime(server, regionName, serverStats);
|
||||
assertNotEquals("Reported load does not produce a backoff", backoffTime, 0);
|
||||
LOG.debug("Backoff calculated for " + region.getRegionNameAsString() + " @ " + server +
|
||||
" is " + backoffTime);
|
||||
|
||||
// Reach into the connection and submit work directly to AsyncProcess so we can
|
||||
// monitor how long the submission was delayed via a callback
|
||||
List<Row> ops = new ArrayList<Row>(1);
|
||||
ops.add(p);
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
final AtomicLong endTime = new AtomicLong();
|
||||
long startTime = EnvironmentEdgeManager.currentTime();
|
||||
table.ap.submit(tablename, ops, true, new Batch.Callback<Result>() {
|
||||
@Override
|
||||
public void update(byte[] region, byte[] row, Result result) {
|
||||
endTime.set(EnvironmentEdgeManager.currentTime());
|
||||
latch.countDown();
|
||||
}
|
||||
}, true);
|
||||
// Currently the ExponentialClientBackoffPolicy under these test conditions
|
||||
// produces a backoffTime of 151 milliseconds. This is long enough so the
|
||||
// wait and related checks below are reasonable. Revisit if the backoff
|
||||
// time reported by above debug logging has significantly deviated.
|
||||
latch.await(backoffTime * 2, TimeUnit.MILLISECONDS);
|
||||
assertNotEquals("AsyncProcess did not submit the work time", endTime.get(), 0);
|
||||
assertTrue("AsyncProcess did not delay long enough", endTime.get() - startTime >= backoffTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue