Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-22238 Fix TestRpcControllerFactory #147

Merged
merged 1 commit into from
Apr 16, 2019
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
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;
Expand All @@ -52,12 +53,12 @@
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 {

Expand Down Expand Up @@ -85,32 +86,15 @@ public static class CountingRpcController extends DelegatingHBaseRpcController {

private static Multiset<Integer> GROUPED_PRIORITY = ConcurrentHashMultiset.create();
private static AtomicInteger INT_PRIORITY = new AtomicInteger();
private static AtomicInteger TABLE_PRIORITY = new AtomicInteger();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For what purpose, we need to remove the table priroirty counter ? don't understand here ...

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We do not call the setPriority(TableName) method any more in async client, we will decide the priority outside the RpcController and then set with the setPriority(int) method. This is why we need to change the UT itself.


public CountingRpcController(HBaseRpcController delegate) {
super(delegate);
}

@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);
}
}

Expand All @@ -120,7 +104,7 @@ public void setPriority(TableName tn) {
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();
Expand All @@ -131,7 +115,7 @@ public static void setup() throws Exception {
}

@AfterClass
public static void teardown() throws Exception {
public static void tearDown() throws Exception {
UTIL.shutdownMiniCluster();
}

Expand All @@ -154,84 +138,83 @@ public void testCountController() throws Exception {
// 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) {
Expand Down