You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/05/02 09:05:28 UTC
[hbase] 16/25: HBASE-22238 Fix TestRpcControllerFactory
This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git
commit e9bd715d4e8a17ecb292d33dd4b48735d88ec756
Author: zhangduo <zh...@apache.org>
AuthorDate: Sun Apr 14 21:30:34 2019 +0800
HBASE-22238 Fix TestRpcControllerFactory
---
.../hbase/client/TestRpcControllerFactory.java | 171 ++++++++++-----------
1 file changed, 77 insertions(+), 94 deletions(-)
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
index 2d60733..bdda4e8 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Scan.ReadType;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ProtobufCoprocessorService;
import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController;
@@ -52,12 +53,12 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ConcurrentHashMulti
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Multiset;
-@Category({MediumTests.class, ClientTests.class})
+@Category({ MediumTests.class, ClientTests.class })
public class TestRpcControllerFactory {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestRpcControllerFactory.class);
+ HBaseClassTestRule.forClass(TestRpcControllerFactory.class);
public static class StaticRpcControllerFactory extends RpcControllerFactory {
@@ -85,7 +86,6 @@ public class TestRpcControllerFactory {
private static Multiset<Integer> GROUPED_PRIORITY = ConcurrentHashMultiset.create();
private static AtomicInteger INT_PRIORITY = new AtomicInteger();
- private static AtomicInteger TABLE_PRIORITY = new AtomicInteger();
public CountingRpcController(HBaseRpcController delegate) {
super(delegate);
@@ -93,24 +93,8 @@ public class TestRpcControllerFactory {
@Override
public void setPriority(int priority) {
- int oldPriority = getPriority();
- super.setPriority(priority);
- int newPriority = getPriority();
- if (newPriority != oldPriority) {
- INT_PRIORITY.incrementAndGet();
- GROUPED_PRIORITY.add(priority);
- }
- }
-
- @Override
- public void setPriority(TableName tn) {
- super.setPriority(tn);
- // ignore counts for system tables - it could change and we really only want to check on what
- // the client should change
- if (tn != null && !tn.isSystemTable()) {
- TABLE_PRIORITY.incrementAndGet();
- }
-
+ INT_PRIORITY.incrementAndGet();
+ GROUPED_PRIORITY.add(priority);
}
}
@@ -120,7 +104,7 @@ public class TestRpcControllerFactory {
public TestName name = new TestName();
@BeforeClass
- public static void setup() throws Exception {
+ public static void setUp() throws Exception {
// load an endpoint so we have an endpoint to test - it doesn't matter which one, but
// this is already in tests, so we can just use it.
Configuration conf = UTIL.getConfiguration();
@@ -131,7 +115,7 @@ public class TestRpcControllerFactory {
}
@AfterClass
- public static void teardown() throws Exception {
+ public static void tearDown() throws Exception {
UTIL.shutdownMiniCluster();
}
@@ -154,84 +138,83 @@ public class TestRpcControllerFactory {
// change one of the connection properties so we get a new Connection with our configuration
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1);
- Connection connection = ConnectionFactory.createConnection(conf);
- Table table = connection.getTable(tableName);
- byte[] row = Bytes.toBytes("row");
- Put p = new Put(row);
- p.addColumn(fam1, fam1, Bytes.toBytes("val0"));
- table.put(p);
-
- Integer counter = 1;
- counter = verifyCount(counter);
-
- Delete d = new Delete(row);
- d.addColumn(fam1, fam1);
- table.delete(d);
- counter = verifyCount(counter);
-
- Put p2 = new Put(row);
- p2.addColumn(fam1, Bytes.toBytes("qual"), Bytes.toBytes("val1"));
- table.batch(Lists.newArrayList(p, p2), null);
- // this only goes to a single server, so we don't need to change the count here
- counter = verifyCount(counter);
-
- Append append = new Append(row);
- append.addColumn(fam1, fam1, Bytes.toBytes("val2"));
- table.append(append);
- counter = verifyCount(counter);
-
- // and check the major lookup calls as well
- Get g = new Get(row);
- table.get(g);
- counter = verifyCount(counter);
-
- ResultScanner scan = table.getScanner(fam1);
- scan.next();
- scan.close();
- counter = verifyCount(counter + 1);
-
- Get g2 = new Get(row);
- table.get(Lists.newArrayList(g, g2));
- // same server, so same as above for not changing count
- counter = verifyCount(counter);
-
- // make sure all the scanner types are covered
- Scan scanInfo = new Scan(row);
- // regular small
- scanInfo.setSmall(true);
- counter = doScan(table, scanInfo, counter);
-
- // reversed, small
- scanInfo.setReversed(true);
- counter = doScan(table, scanInfo, counter);
-
- // reversed, regular
- scanInfo.setSmall(false);
- counter = doScan(table, scanInfo, counter + 1);
-
- // make sure we have no priority count
- verifyPriorityGroupCount(HConstants.ADMIN_QOS, 0);
- // lets set a custom priority on a get
- Get get = new Get(row);
- get.setPriority(HConstants.ADMIN_QOS);
- table.get(get);
- verifyPriorityGroupCount(HConstants.ADMIN_QOS, 1);
-
- table.close();
- connection.close();
+ try (Connection connection = ConnectionFactory.createConnection(conf);
+ Table table = connection.getTable(tableName)) {
+ byte[] row = Bytes.toBytes("row");
+ Put p = new Put(row);
+ p.addColumn(fam1, fam1, Bytes.toBytes("val0"));
+ table.put(p);
+
+ Integer counter = 1;
+ counter = verifyCount(counter);
+
+ Delete d = new Delete(row);
+ d.addColumn(fam1, fam1);
+ table.delete(d);
+ counter = verifyCount(counter);
+
+ Put p2 = new Put(row);
+ p2.addColumn(fam1, Bytes.toBytes("qual"), Bytes.toBytes("val1"));
+ table.batch(Lists.newArrayList(p, p2), null);
+ // this only goes to a single server, so we don't need to change the count here
+ counter = verifyCount(counter);
+
+ Append append = new Append(row);
+ append.addColumn(fam1, fam1, Bytes.toBytes("val2"));
+ table.append(append);
+ counter = verifyCount(counter);
+
+ // and check the major lookup calls as well
+ Get g = new Get(row);
+ table.get(g);
+ counter = verifyCount(counter);
+
+ ResultScanner scan = table.getScanner(fam1);
+ scan.next();
+ scan.close();
+ counter = verifyCount(counter + 1);
+
+ Get g2 = new Get(row);
+ table.get(Lists.newArrayList(g, g2));
+ // same server, so same as above for not changing count
+ counter = verifyCount(counter);
+
+ // make sure all the scanner types are covered
+ Scan scanInfo = new Scan().withStartRow(row);
+ // regular small
+ scanInfo.setReadType(ReadType.PREAD);
+ counter = doScan(table, scanInfo, counter);
+
+ // reversed, small
+ scanInfo.setReversed(true);
+ counter = doScan(table, scanInfo, counter);
+
+ // reversed, regular
+ scanInfo.setReadType(ReadType.STREAM);
+ counter = doScan(table, scanInfo, counter + 1);
+
+ // make sure we have no priority count
+ verifyPriorityGroupCount(HConstants.ADMIN_QOS, 0);
+ // lets set a custom priority on a get
+ Get get = new Get(row);
+ get.setPriority(HConstants.ADMIN_QOS);
+ table.get(get);
+ // we will reset the controller for setting the call timeout so it will lead to an extra
+ // setPriority
+ verifyPriorityGroupCount(HConstants.ADMIN_QOS, 2);
+ }
}
int doScan(Table table, Scan scan, int expectedCount) throws IOException {
- ResultScanner results = table.getScanner(scan);
- results.next();
- results.close();
+ try (ResultScanner results = table.getScanner(scan)) {
+ results.next();
+ }
return verifyCount(expectedCount);
}
int verifyCount(Integer counter) {
- assertTrue(CountingRpcController.TABLE_PRIORITY.get() >= counter);
- assertEquals(0, CountingRpcController.INT_PRIORITY.get());
- return CountingRpcController.TABLE_PRIORITY.get() + 1;
+ assertTrue(CountingRpcController.INT_PRIORITY.get() >= counter);
+ return CountingRpcController.GROUPED_PRIORITY.count(HConstants.NORMAL_QOS) + 1;
}
void verifyPriorityGroupCount(int priorityLevel, int count) {