HBASE-26364 TestThriftServer is failing 100% in our flaky test job (#3759)
Signed-off-by: Xiaolin Ha <haxiaolin@apache.org>
Signed-off-by: Duo Zhang <zhangduo@apache.org>
(cherry picked from commit 4454c2f909
)
This commit is contained in:
parent
533f53587a
commit
7bbbd3dafc
|
@ -774,44 +774,47 @@ public class TestThriftServer {
|
|||
String col = "c";
|
||||
// create a table which will throw exceptions for requests
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
HTableDescriptor tableDesc = new HTableDescriptor(tableName);
|
||||
tableDesc.addCoprocessor(ErrorThrowingGetObserver.class.getName());
|
||||
tableDesc.addFamily(new HColumnDescriptor(family));
|
||||
try {
|
||||
HTableDescriptor tableDesc = new HTableDescriptor(tableName);
|
||||
tableDesc.addCoprocessor(ErrorThrowingGetObserver.class.getName());
|
||||
tableDesc.addFamily(new HColumnDescriptor(family));
|
||||
|
||||
Table table = UTIL.createTable(tableDesc, null);
|
||||
long now = System.currentTimeMillis();
|
||||
table.put(new Put(Bytes.toBytes(rowkey))
|
||||
Table table = UTIL.createTable(tableDesc, null);
|
||||
long now = System.currentTimeMillis();
|
||||
table.put(new Put(Bytes.toBytes(rowkey))
|
||||
.addColumn(Bytes.toBytes(family), Bytes.toBytes(col), now, Bytes.toBytes("val1")));
|
||||
|
||||
Configuration conf = UTIL.getConfiguration();
|
||||
ThriftMetrics metrics = getMetrics(conf);
|
||||
ThriftHBaseServiceHandler hbaseHandler =
|
||||
new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
|
||||
UserProvider.instantiate(UTIL.getConfiguration()));
|
||||
Hbase.Iface handler = HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, conf);
|
||||
Configuration conf = UTIL.getConfiguration();
|
||||
ThriftMetrics metrics = getMetrics(conf);
|
||||
ThriftHBaseServiceHandler hbaseHandler =
|
||||
new ThriftHBaseServiceHandler(UTIL.getConfiguration(), UserProvider.instantiate(UTIL.getConfiguration()));
|
||||
Hbase.Iface handler = HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, conf);
|
||||
|
||||
ByteBuffer tTableName = asByteBuffer(tableName.getNameAsString());
|
||||
ByteBuffer tTableName = asByteBuffer(tableName.getNameAsString());
|
||||
|
||||
// check metrics increment with a successful get
|
||||
long preGetCounter = metricsHelper.checkCounterExists("getRow_num_ops", metrics.getSource()) ?
|
||||
// check metrics increment with a successful get
|
||||
long preGetCounter = metricsHelper.checkCounterExists("getRow_num_ops", metrics.getSource()) ?
|
||||
metricsHelper.getCounter("getRow_num_ops", metrics.getSource()) :
|
||||
0;
|
||||
List<TRowResult> tRowResult = handler.getRow(tTableName, asByteBuffer(rowkey), null);
|
||||
assertEquals(1, tRowResult.size());
|
||||
TRowResult tResult = tRowResult.get(0);
|
||||
List<TRowResult> tRowResult = handler.getRow(tTableName, asByteBuffer(rowkey), null);
|
||||
assertEquals(1, tRowResult.size());
|
||||
TRowResult tResult = tRowResult.get(0);
|
||||
|
||||
TCell expectedColumnValue = new TCell(asByteBuffer("val1"), now);
|
||||
TCell expectedColumnValue = new TCell(asByteBuffer("val1"), now);
|
||||
|
||||
assertArrayEquals(Bytes.toBytes(rowkey), tResult.getRow());
|
||||
Collection<TCell> returnedColumnValues = tResult.getColumns().values();
|
||||
assertEquals(1, returnedColumnValues.size());
|
||||
assertEquals(expectedColumnValue, returnedColumnValues.iterator().next());
|
||||
assertArrayEquals(Bytes.toBytes(rowkey), tResult.getRow());
|
||||
Collection<TCell> returnedColumnValues = tResult.getColumns().values();
|
||||
assertEquals(1, returnedColumnValues.size());
|
||||
assertEquals(expectedColumnValue, returnedColumnValues.iterator().next());
|
||||
|
||||
metricsHelper.assertCounter("getRow_num_ops", preGetCounter + 1, metrics.getSource());
|
||||
metricsHelper.assertCounter("getRow_num_ops", preGetCounter + 1, metrics.getSource());
|
||||
|
||||
// check metrics increment when the get throws each exception type
|
||||
for (ErrorThrowingGetObserver.ErrorType type : ErrorThrowingGetObserver.ErrorType.values()) {
|
||||
testExceptionType(handler, metrics, tTableName, rowkey, type);
|
||||
// check metrics increment when the get throws each exception type
|
||||
for (ErrorThrowingGetObserver.ErrorType type : ErrorThrowingGetObserver.ErrorType.values()) {
|
||||
testExceptionType(handler, metrics, tTableName, rowkey, type);
|
||||
}
|
||||
} finally {
|
||||
UTIL.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue