Skip to content

Commit 3f0a27b

Browse files
committed
HBASE-22238 Fix TestRpcControllerFactory
1 parent f47f6df commit 3f0a27b

File tree

1 file changed

+77
-94
lines changed

1 file changed

+77
-94
lines changed

hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java

Lines changed: 77 additions & 94 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import org.apache.hadoop.hbase.HBaseTestingUtility;
3333
import org.apache.hadoop.hbase.HConstants;
3434
import org.apache.hadoop.hbase.TableName;
35+
import org.apache.hadoop.hbase.client.Scan.ReadType;
3536
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
3637
import org.apache.hadoop.hbase.coprocessor.ProtobufCoprocessorService;
3738
import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController;
@@ -52,12 +53,12 @@
5253
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
5354
import org.apache.hbase.thirdparty.com.google.common.collect.Multiset;
5455

55-
@Category({MediumTests.class, ClientTests.class})
56+
@Category({ MediumTests.class, ClientTests.class })
5657
public class TestRpcControllerFactory {
5758

5859
@ClassRule
5960
public static final HBaseClassTestRule CLASS_RULE =
60-
HBaseClassTestRule.forClass(TestRpcControllerFactory.class);
61+
HBaseClassTestRule.forClass(TestRpcControllerFactory.class);
6162

6263
public static class StaticRpcControllerFactory extends RpcControllerFactory {
6364

@@ -85,32 +86,15 @@ public static class CountingRpcController extends DelegatingHBaseRpcController {
8586

8687
private static Multiset<Integer> GROUPED_PRIORITY = ConcurrentHashMultiset.create();
8788
private static AtomicInteger INT_PRIORITY = new AtomicInteger();
88-
private static AtomicInteger TABLE_PRIORITY = new AtomicInteger();
8989

9090
public CountingRpcController(HBaseRpcController delegate) {
9191
super(delegate);
9292
}
9393

9494
@Override
9595
public void setPriority(int priority) {
96-
int oldPriority = getPriority();
97-
super.setPriority(priority);
98-
int newPriority = getPriority();
99-
if (newPriority != oldPriority) {
100-
INT_PRIORITY.incrementAndGet();
101-
GROUPED_PRIORITY.add(priority);
102-
}
103-
}
104-
105-
@Override
106-
public void setPriority(TableName tn) {
107-
super.setPriority(tn);
108-
// ignore counts for system tables - it could change and we really only want to check on what
109-
// the client should change
110-
if (tn != null && !tn.isSystemTable()) {
111-
TABLE_PRIORITY.incrementAndGet();
112-
}
113-
96+
INT_PRIORITY.incrementAndGet();
97+
GROUPED_PRIORITY.add(priority);
11498
}
11599
}
116100

@@ -120,7 +104,7 @@ public void setPriority(TableName tn) {
120104
public TestName name = new TestName();
121105

122106
@BeforeClass
123-
public static void setup() throws Exception {
107+
public static void setUp() throws Exception {
124108
// load an endpoint so we have an endpoint to test - it doesn't matter which one, but
125109
// this is already in tests, so we can just use it.
126110
Configuration conf = UTIL.getConfiguration();
@@ -131,7 +115,7 @@ public static void setup() throws Exception {
131115
}
132116

133117
@AfterClass
134-
public static void teardown() throws Exception {
118+
public static void tearDown() throws Exception {
135119
UTIL.shutdownMiniCluster();
136120
}
137121

@@ -154,84 +138,83 @@ public void testCountController() throws Exception {
154138
// change one of the connection properties so we get a new Connection with our configuration
155139
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1);
156140

157-
Connection connection = ConnectionFactory.createConnection(conf);
158-
Table table = connection.getTable(tableName);
159-
byte[] row = Bytes.toBytes("row");
160-
Put p = new Put(row);
161-
p.addColumn(fam1, fam1, Bytes.toBytes("val0"));
162-
table.put(p);
163-
164-
Integer counter = 1;
165-
counter = verifyCount(counter);
166-
167-
Delete d = new Delete(row);
168-
d.addColumn(fam1, fam1);
169-
table.delete(d);
170-
counter = verifyCount(counter);
171-
172-
Put p2 = new Put(row);
173-
p2.addColumn(fam1, Bytes.toBytes("qual"), Bytes.toBytes("val1"));
174-
table.batch(Lists.newArrayList(p, p2), null);
175-
// this only goes to a single server, so we don't need to change the count here
176-
counter = verifyCount(counter);
177-
178-
Append append = new Append(row);
179-
append.addColumn(fam1, fam1, Bytes.toBytes("val2"));
180-
table.append(append);
181-
counter = verifyCount(counter);
182-
183-
// and check the major lookup calls as well
184-
Get g = new Get(row);
185-
table.get(g);
186-
counter = verifyCount(counter);
187-
188-
ResultScanner scan = table.getScanner(fam1);
189-
scan.next();
190-
scan.close();
191-
counter = verifyCount(counter + 1);
192-
193-
Get g2 = new Get(row);
194-
table.get(Lists.newArrayList(g, g2));
195-
// same server, so same as above for not changing count
196-
counter = verifyCount(counter);
197-
198-
// make sure all the scanner types are covered
199-
Scan scanInfo = new Scan(row);
200-
// regular small
201-
scanInfo.setSmall(true);
202-
counter = doScan(table, scanInfo, counter);
203-
204-
// reversed, small
205-
scanInfo.setReversed(true);
206-
counter = doScan(table, scanInfo, counter);
207-
208-
// reversed, regular
209-
scanInfo.setSmall(false);
210-
counter = doScan(table, scanInfo, counter + 1);
211-
212-
// make sure we have no priority count
213-
verifyPriorityGroupCount(HConstants.ADMIN_QOS, 0);
214-
// lets set a custom priority on a get
215-
Get get = new Get(row);
216-
get.setPriority(HConstants.ADMIN_QOS);
217-
table.get(get);
218-
verifyPriorityGroupCount(HConstants.ADMIN_QOS, 1);
219-
220-
table.close();
221-
connection.close();
141+
try (Connection connection = ConnectionFactory.createConnection(conf);
142+
Table table = connection.getTable(tableName)) {
143+
byte[] row = Bytes.toBytes("row");
144+
Put p = new Put(row);
145+
p.addColumn(fam1, fam1, Bytes.toBytes("val0"));
146+
table.put(p);
147+
148+
Integer counter = 1;
149+
counter = verifyCount(counter);
150+
151+
Delete d = new Delete(row);
152+
d.addColumn(fam1, fam1);
153+
table.delete(d);
154+
counter = verifyCount(counter);
155+
156+
Put p2 = new Put(row);
157+
p2.addColumn(fam1, Bytes.toBytes("qual"), Bytes.toBytes("val1"));
158+
table.batch(Lists.newArrayList(p, p2), null);
159+
// this only goes to a single server, so we don't need to change the count here
160+
counter = verifyCount(counter);
161+
162+
Append append = new Append(row);
163+
append.addColumn(fam1, fam1, Bytes.toBytes("val2"));
164+
table.append(append);
165+
counter = verifyCount(counter);
166+
167+
// and check the major lookup calls as well
168+
Get g = new Get(row);
169+
table.get(g);
170+
counter = verifyCount(counter);
171+
172+
ResultScanner scan = table.getScanner(fam1);
173+
scan.next();
174+
scan.close();
175+
counter = verifyCount(counter + 1);
176+
177+
Get g2 = new Get(row);
178+
table.get(Lists.newArrayList(g, g2));
179+
// same server, so same as above for not changing count
180+
counter = verifyCount(counter);
181+
182+
// make sure all the scanner types are covered
183+
Scan scanInfo = new Scan().withStartRow(row);
184+
// regular small
185+
scanInfo.setReadType(ReadType.PREAD);
186+
counter = doScan(table, scanInfo, counter);
187+
188+
// reversed, small
189+
scanInfo.setReversed(true);
190+
counter = doScan(table, scanInfo, counter);
191+
192+
// reversed, regular
193+
scanInfo.setReadType(ReadType.STREAM);
194+
counter = doScan(table, scanInfo, counter + 1);
195+
196+
// make sure we have no priority count
197+
verifyPriorityGroupCount(HConstants.ADMIN_QOS, 0);
198+
// lets set a custom priority on a get
199+
Get get = new Get(row);
200+
get.setPriority(HConstants.ADMIN_QOS);
201+
table.get(get);
202+
// we will reset the controller for setting the call timeout so it will lead to an extra
203+
// setPriority
204+
verifyPriorityGroupCount(HConstants.ADMIN_QOS, 2);
205+
}
222206
}
223207

224208
int doScan(Table table, Scan scan, int expectedCount) throws IOException {
225-
ResultScanner results = table.getScanner(scan);
226-
results.next();
227-
results.close();
209+
try (ResultScanner results = table.getScanner(scan)) {
210+
results.next();
211+
}
228212
return verifyCount(expectedCount);
229213
}
230214

231215
int verifyCount(Integer counter) {
232-
assertTrue(CountingRpcController.TABLE_PRIORITY.get() >= counter);
233-
assertEquals(0, CountingRpcController.INT_PRIORITY.get());
234-
return CountingRpcController.TABLE_PRIORITY.get() + 1;
216+
assertTrue(CountingRpcController.INT_PRIORITY.get() >= counter);
217+
return CountingRpcController.GROUPED_PRIORITY.count(HConstants.NORMAL_QOS) + 1;
235218
}
236219

237220
void verifyPriorityGroupCount(int priorityLevel, int count) {

0 commit comments

Comments
 (0)