HBASE-27534 Determine too large requests by response block size rather than just cell size (#5007)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
parent
0cfe043e85
commit
fa6b5d17c2
|
@ -59,6 +59,7 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
private final int processingTime;
|
||||
private final int queueTime;
|
||||
private final long responseSize;
|
||||
private final long blockBytesScanned;
|
||||
private final String clientAddress;
|
||||
private final String serverClass;
|
||||
private final String methodName;
|
||||
|
@ -88,6 +89,13 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
return responseSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the amount of block bytes scanned to retrieve the response cells.
|
||||
*/
|
||||
public long getBlockBytesScanned() {
|
||||
return blockBytesScanned;
|
||||
}
|
||||
|
||||
public String getClientAddress() {
|
||||
return clientAddress;
|
||||
}
|
||||
|
@ -129,14 +137,15 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
}
|
||||
|
||||
private OnlineLogRecord(final long startTime, final int processingTime, final int queueTime,
|
||||
final long responseSize, final String clientAddress, final String serverClass,
|
||||
final String methodName, final String callDetails, final String param, final String regionName,
|
||||
final String userName, final int multiGetsCount, final int multiMutationsCount,
|
||||
final int multiServiceCalls) {
|
||||
final long responseSize, final long blockBytesScanned, final String clientAddress,
|
||||
final String serverClass, final String methodName, final String callDetails, final String param,
|
||||
final String regionName, final String userName, final int multiGetsCount,
|
||||
final int multiMutationsCount, final int multiServiceCalls) {
|
||||
this.startTime = startTime;
|
||||
this.processingTime = processingTime;
|
||||
this.queueTime = queueTime;
|
||||
this.responseSize = responseSize;
|
||||
this.blockBytesScanned = blockBytesScanned;
|
||||
this.clientAddress = clientAddress;
|
||||
this.serverClass = serverClass;
|
||||
this.methodName = methodName;
|
||||
|
@ -154,6 +163,7 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
private int processingTime;
|
||||
private int queueTime;
|
||||
private long responseSize;
|
||||
private long blockBytesScanned;
|
||||
private String clientAddress;
|
||||
private String serverClass;
|
||||
private String methodName;
|
||||
|
@ -185,6 +195,14 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the amount of block bytes scanned to retrieve the response cells.
|
||||
*/
|
||||
public OnlineLogRecordBuilder setBlockBytesScanned(long blockBytesScanned) {
|
||||
this.blockBytesScanned = blockBytesScanned;
|
||||
return this;
|
||||
}
|
||||
|
||||
public OnlineLogRecordBuilder setClientAddress(String clientAddress) {
|
||||
this.clientAddress = clientAddress;
|
||||
return this;
|
||||
|
@ -236,9 +254,9 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
}
|
||||
|
||||
public OnlineLogRecord build() {
|
||||
return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize, clientAddress,
|
||||
serverClass, methodName, callDetails, param, regionName, userName, multiGetsCount,
|
||||
multiMutationsCount, multiServiceCalls);
|
||||
return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize,
|
||||
blockBytesScanned, clientAddress, serverClass, methodName, callDetails, param, regionName,
|
||||
userName, multiGetsCount, multiMutationsCount, multiServiceCalls);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -256,7 +274,8 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
|
||||
return new EqualsBuilder().append(startTime, that.startTime)
|
||||
.append(processingTime, that.processingTime).append(queueTime, that.queueTime)
|
||||
.append(responseSize, that.responseSize).append(multiGetsCount, that.multiGetsCount)
|
||||
.append(responseSize, that.responseSize).append(blockBytesScanned, that.blockBytesScanned)
|
||||
.append(multiGetsCount, that.multiGetsCount)
|
||||
.append(multiMutationsCount, that.multiMutationsCount)
|
||||
.append(multiServiceCalls, that.multiServiceCalls).append(clientAddress, that.clientAddress)
|
||||
.append(serverClass, that.serverClass).append(methodName, that.methodName)
|
||||
|
@ -267,9 +286,9 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder(17, 37).append(startTime).append(processingTime).append(queueTime)
|
||||
.append(responseSize).append(clientAddress).append(serverClass).append(methodName)
|
||||
.append(callDetails).append(param).append(regionName).append(userName).append(multiGetsCount)
|
||||
.append(multiMutationsCount).append(multiServiceCalls).toHashCode();
|
||||
.append(responseSize).append(blockBytesScanned).append(clientAddress).append(serverClass)
|
||||
.append(methodName).append(callDetails).append(param).append(regionName).append(userName)
|
||||
.append(multiGetsCount).append(multiMutationsCount).append(multiServiceCalls).toHashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -281,11 +300,11 @@ final public class OnlineLogRecord extends LogEntry {
|
|||
public String toString() {
|
||||
return new ToStringBuilder(this).append("startTime", startTime)
|
||||
.append("processingTime", processingTime).append("queueTime", queueTime)
|
||||
.append("responseSize", responseSize).append("clientAddress", clientAddress)
|
||||
.append("serverClass", serverClass).append("methodName", methodName)
|
||||
.append("callDetails", callDetails).append("param", param).append("regionName", regionName)
|
||||
.append("userName", userName).append("multiGetsCount", multiGetsCount)
|
||||
.append("multiMutationsCount", multiMutationsCount)
|
||||
.append("responseSize", responseSize).append("blockBytesScanned", blockBytesScanned)
|
||||
.append("clientAddress", clientAddress).append("serverClass", serverClass)
|
||||
.append("methodName", methodName).append("callDetails", callDetails).append("param", param)
|
||||
.append("regionName", regionName).append("userName", userName)
|
||||
.append("multiGetsCount", multiGetsCount).append("multiMutationsCount", multiMutationsCount)
|
||||
.append("multiServiceCalls", multiServiceCalls).toString();
|
||||
}
|
||||
|
||||
|
|
|
@ -3375,6 +3375,7 @@ public final class ProtobufUtil {
|
|||
.setParam(slowLogPayload.getParam()).setProcessingTime(slowLogPayload.getProcessingTime())
|
||||
.setQueueTime(slowLogPayload.getQueueTime()).setRegionName(slowLogPayload.getRegionName())
|
||||
.setResponseSize(slowLogPayload.getResponseSize())
|
||||
.setBlockBytesScanned(slowLogPayload.getBlockBytesScanned())
|
||||
.setServerClass(slowLogPayload.getServerClass()).setStartTime(slowLogPayload.getStartTime())
|
||||
.setUserName(slowLogPayload.getUserName()).build();
|
||||
return onlineLogRecord;
|
||||
|
|
|
@ -87,6 +87,8 @@ public class SlowLogTableAccessor {
|
|||
Bytes.toBytes(slowLogPayload.getRegionName()))
|
||||
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("response_size"),
|
||||
Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize())))
|
||||
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("block_bytes_scanned"),
|
||||
Bytes.toBytes(Long.toString(slowLogPayload.getBlockBytesScanned())))
|
||||
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("server_class"),
|
||||
Bytes.toBytes(slowLogPayload.getServerClass()))
|
||||
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),
|
||||
|
|
|
@ -44,6 +44,8 @@ message SlowLogPayload {
|
|||
optional int32 multi_service_calls = 14 [default = 0];
|
||||
required Type type = 15;
|
||||
|
||||
optional int64 block_bytes_scanned = 16;
|
||||
|
||||
// SLOW_LOG is RPC call slow in nature whereas LARGE_LOG is RPC call quite large.
|
||||
// Majority of times, slow logs are also large logs and hence, ALL is combination of
|
||||
// both
|
||||
|
|
|
@ -428,6 +428,7 @@ public abstract class RpcServer implements RpcServerInterface, ConfigurationObse
|
|||
// Use the raw request call size for now.
|
||||
long requestSize = call.getSize();
|
||||
long responseSize = result.getSerializedSize();
|
||||
long responseBlockSize = call.getResponseBlockSize();
|
||||
if (call.isClientCellBlockSupported()) {
|
||||
// Include the payload size in HBaseRpcController
|
||||
responseSize += call.getResponseCellSize();
|
||||
|
@ -441,20 +442,21 @@ public abstract class RpcServer implements RpcServerInterface, ConfigurationObse
|
|||
// log any RPC responses that are slower than the configured warn
|
||||
// response time or larger than configured warning size
|
||||
boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
|
||||
boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
|
||||
boolean tooLarge = (warnResponseSize > -1
|
||||
&& (responseSize > warnResponseSize || responseBlockSize > warnResponseSize));
|
||||
if (tooSlow || tooLarge) {
|
||||
final String userName = call.getRequestUserName().orElse(StringUtils.EMPTY);
|
||||
// when tagging, we let TooLarge trump TooSmall to keep output simple
|
||||
// note that large responses will often also be slow.
|
||||
logResponse(param, md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
|
||||
tooLarge, tooSlow, status.getClient(), startTime, processingTime, qTime, responseSize,
|
||||
userName);
|
||||
responseBlockSize, userName);
|
||||
if (this.namedQueueRecorder != null && this.isOnlineLogProviderEnabled) {
|
||||
// send logs to ring buffer owned by slowLogRecorder
|
||||
final String className =
|
||||
server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();
|
||||
this.namedQueueRecorder.addRecord(new RpcLogDetails(call, param, status.getClient(),
|
||||
responseSize, className, tooSlow, tooLarge));
|
||||
responseSize, responseBlockSize, className, tooSlow, tooLarge));
|
||||
}
|
||||
}
|
||||
return new Pair<>(result, controller.cellScanner());
|
||||
|
@ -482,22 +484,23 @@ public abstract class RpcServer implements RpcServerInterface, ConfigurationObse
|
|||
|
||||
/**
|
||||
* Logs an RPC response to the LOG file, producing valid JSON objects for client Operations.
|
||||
* @param param The parameters received in the call.
|
||||
* @param methodName The name of the method invoked
|
||||
* @param call The string representation of the call
|
||||
* @param tooLarge To indicate if the event is tooLarge
|
||||
* @param tooSlow To indicate if the event is tooSlow
|
||||
* @param clientAddress The address of the client who made this call.
|
||||
* @param startTime The time that the call was initiated, in ms.
|
||||
* @param processingTime The duration that the call took to run, in ms.
|
||||
* @param qTime The duration that the call spent on the queue prior to being initiated,
|
||||
* in ms.
|
||||
* @param responseSize The size in bytes of the response buffer.
|
||||
* @param userName UserName of the current RPC Call
|
||||
* @param param The parameters received in the call.
|
||||
* @param methodName The name of the method invoked
|
||||
* @param call The string representation of the call
|
||||
* @param tooLarge To indicate if the event is tooLarge
|
||||
* @param tooSlow To indicate if the event is tooSlow
|
||||
* @param clientAddress The address of the client who made this call.
|
||||
* @param startTime The time that the call was initiated, in ms.
|
||||
* @param processingTime The duration that the call took to run, in ms.
|
||||
* @param qTime The duration that the call spent on the queue prior to being
|
||||
* initiated, in ms.
|
||||
* @param responseSize The size in bytes of the response buffer.
|
||||
* @param blockBytesScanned The size of block bytes scanned to retrieve the response.
|
||||
* @param userName UserName of the current RPC Call
|
||||
*/
|
||||
void logResponse(Message param, String methodName, String call, boolean tooLarge, boolean tooSlow,
|
||||
String clientAddress, long startTime, int processingTime, int qTime, long responseSize,
|
||||
String userName) {
|
||||
long blockBytesScanned, String userName) {
|
||||
final String className = server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();
|
||||
// base information that is reported regardless of type of call
|
||||
Map<String, Object> responseInfo = new HashMap<>();
|
||||
|
@ -505,6 +508,7 @@ public abstract class RpcServer implements RpcServerInterface, ConfigurationObse
|
|||
responseInfo.put("processingtimems", processingTime);
|
||||
responseInfo.put("queuetimems", qTime);
|
||||
responseInfo.put("responsesize", responseSize);
|
||||
responseInfo.put("blockbytesscanned", blockBytesScanned);
|
||||
responseInfo.put("client", clientAddress);
|
||||
responseInfo.put("class", className);
|
||||
responseInfo.put("method", methodName);
|
||||
|
|
|
@ -35,17 +35,19 @@ public class RpcLogDetails extends NamedQueuePayload {
|
|||
private final Message param;
|
||||
private final String clientAddress;
|
||||
private final long responseSize;
|
||||
private final long blockBytesScanned;
|
||||
private final String className;
|
||||
private final boolean isSlowLog;
|
||||
private final boolean isLargeLog;
|
||||
|
||||
public RpcLogDetails(RpcCall rpcCall, Message param, String clientAddress, long responseSize,
|
||||
String className, boolean isSlowLog, boolean isLargeLog) {
|
||||
long blockBytesScanned, String className, boolean isSlowLog, boolean isLargeLog) {
|
||||
super(SLOW_LOG_EVENT);
|
||||
this.rpcCall = rpcCall;
|
||||
this.param = param;
|
||||
this.clientAddress = clientAddress;
|
||||
this.responseSize = responseSize;
|
||||
this.blockBytesScanned = blockBytesScanned;
|
||||
this.className = className;
|
||||
this.isSlowLog = isSlowLog;
|
||||
this.isLargeLog = isLargeLog;
|
||||
|
@ -63,6 +65,10 @@ public class RpcLogDetails extends NamedQueuePayload {
|
|||
return responseSize;
|
||||
}
|
||||
|
||||
public long getBlockBytesScanned() {
|
||||
return blockBytesScanned;
|
||||
}
|
||||
|
||||
public String getClassName() {
|
||||
return className;
|
||||
}
|
||||
|
|
|
@ -116,6 +116,7 @@ public class SlowLogQueueService implements NamedQueueService {
|
|||
final RpcCall rpcCall = rpcLogDetails.getRpcCall();
|
||||
final String clientAddress = rpcLogDetails.getClientAddress();
|
||||
final long responseSize = rpcLogDetails.getResponseSize();
|
||||
final long blockBytesScanned = rpcLogDetails.getBlockBytesScanned();
|
||||
final String className = rpcLogDetails.getClassName();
|
||||
final TooSlowLog.SlowLogPayload.Type type = getLogType(rpcLogDetails);
|
||||
if (type == null) {
|
||||
|
@ -158,8 +159,9 @@ public class SlowLogQueueService implements NamedQueueService {
|
|||
.setParam(slowLogParams != null ? slowLogParams.getParams() : StringUtils.EMPTY)
|
||||
.setProcessingTime(processingTime).setQueueTime(qTime)
|
||||
.setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : StringUtils.EMPTY)
|
||||
.setResponseSize(responseSize).setServerClass(className).setStartTime(startTime).setType(type)
|
||||
.setUserName(userName).build();
|
||||
.setResponseSize(responseSize).setBlockBytesScanned(blockBytesScanned)
|
||||
.setServerClass(className).setStartTime(startTime).setType(type).setUserName(userName)
|
||||
.build();
|
||||
slowLogQueue.add(slowLogPayload);
|
||||
if (isSlowLogTableEnabled) {
|
||||
if (!slowLogPayload.getRegionName().startsWith("hbase:slowlog")) {
|
||||
|
|
|
@ -97,6 +97,7 @@
|
|||
<th>Processing Time</th>
|
||||
<th>Queue Time</th>
|
||||
<th>Response Size</th>
|
||||
<th>Block Bytes Scanned</th>
|
||||
<th>Client Address</th>
|
||||
<th>Server Class</th>
|
||||
<th>Method Name</th>
|
||||
|
@ -115,6 +116,7 @@
|
|||
<td><%=r.getProcessingTime()%>ms</td>
|
||||
<td><%=r.getQueueTime()%>ms</td>
|
||||
<td><%=StringUtils.byteDesc(r.getResponseSize())%></td>
|
||||
<td><%=StringUtils.byteDesc(r.getBlockBytesScanned())%></td>
|
||||
<td><%=r.getClientAddress()%></td>
|
||||
<td><%=r.getServerClass()%></td>
|
||||
<td><%=r.getMethodName()%></td>
|
||||
|
@ -138,6 +140,7 @@
|
|||
<th>Processing Time</th>
|
||||
<th>Queue Time</th>
|
||||
<th>Response Size</th>
|
||||
<th>Block Bytes Scanned</th>
|
||||
<th>Client Address</th>
|
||||
<th>Server Class</th>
|
||||
<th>Method Name</th>
|
||||
|
@ -156,6 +159,7 @@
|
|||
<td><%=r.getProcessingTime()%>ms</td>
|
||||
<td><%=r.getQueueTime()%>ms</td>
|
||||
<td><%=StringUtils.byteDesc(r.getResponseSize())%></td>
|
||||
<td><%=StringUtils.byteDesc(r.getBlockBytesScanned())%></td>
|
||||
<td><%=r.getClientAddress()%></td>
|
||||
<td><%=r.getServerClass()%></td>
|
||||
<td><%=r.getMethodName()%></td>
|
||||
|
|
|
@ -529,13 +529,14 @@ public class TestNamedQueueRecorder {
|
|||
|
||||
static RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className) {
|
||||
RpcCall rpcCall = getRpcCall(userName);
|
||||
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, className, true, true);
|
||||
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, true,
|
||||
true);
|
||||
}
|
||||
|
||||
private RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className,
|
||||
boolean isSlowLog, boolean isLargeLog) {
|
||||
RpcCall rpcCall = getRpcCall(userName);
|
||||
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, className, isSlowLog,
|
||||
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, isSlowLog,
|
||||
isLargeLog);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,107 @@
|
|||
/*
|
||||
* 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.namequeues;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.LogEntry;
|
||||
import org.apache.hadoop.hbase.client.OnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.ServerType;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
@Category({ RegionServerTests.class, MediumTests.class })
|
||||
public class TestTooLargeLog {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestTooLargeLog.class);
|
||||
|
||||
protected final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
|
||||
protected static Admin ADMIN;
|
||||
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.ipc.warn.response.size", 100);
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
ADMIN = TEST_UTIL.getAdmin();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that we can trigger based on blocks scanned, and also that we properly pass the block
|
||||
* bytes scanned value through to the client.
|
||||
*/
|
||||
@Test
|
||||
public void testLogLargeBlockBytesScanned() throws IOException, InterruptedException {
|
||||
byte[] family = Bytes.toBytes("0");
|
||||
Table table = TEST_UTIL.createTable(TableName.valueOf("testLogLargeBlockBytesScanned"), family);
|
||||
TEST_UTIL.loadTable(table, family);
|
||||
TEST_UTIL.flush(table.getName());
|
||||
|
||||
Set<ServerName> server =
|
||||
Collections.singleton(TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName());
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
admin.clearSlowLogResponses(server);
|
||||
|
||||
Scan scan = new Scan();
|
||||
scan.setCaching(1);
|
||||
|
||||
try (ResultScanner scanner = table.getScanner(scan)) {
|
||||
scanner.next();
|
||||
}
|
||||
|
||||
List<LogEntry> entries =
|
||||
admin.getLogEntries(server, "LARGE_LOG", ServerType.REGION_SERVER, 1, Collections.emptyMap());
|
||||
|
||||
assertEquals(1, entries.size());
|
||||
|
||||
OnlineLogRecord record = (OnlineLogRecord) entries.get(0);
|
||||
System.out.println(record.toJsonPrettyPrint());
|
||||
|
||||
assertTrue("expected " + record.getBlockBytesScanned() + " to be >= 100",
|
||||
record.getBlockBytesScanned() >= 100);
|
||||
assertTrue("expected " + record.getResponseSize() + " to be < 100",
|
||||
record.getResponseSize() < 100);
|
||||
|
||||
}
|
||||
}
|
|
@ -1646,6 +1646,7 @@ public final class ThriftUtilities {
|
|||
tOnlineLogRecord.setQueueTime(slowLogRecord.getQueueTime());
|
||||
tOnlineLogRecord.setRegionName(slowLogRecord.getRegionName());
|
||||
tOnlineLogRecord.setResponseSize(slowLogRecord.getResponseSize());
|
||||
tOnlineLogRecord.setBlockBytesScanned(slowLogRecord.getBlockBytesScanned());
|
||||
tOnlineLogRecord.setServerClass(slowLogRecord.getServerClass());
|
||||
tOnlineLogRecord.setStartTime(slowLogRecord.getStartTime());
|
||||
tOnlineLogRecord.setUserName(slowLogRecord.getUserName());
|
||||
|
@ -1669,6 +1670,7 @@ public final class ThriftUtilities {
|
|||
.setParam(tSlowLogRecord.getParam()).setProcessingTime(tSlowLogRecord.getProcessingTime())
|
||||
.setQueueTime(tSlowLogRecord.getQueueTime()).setRegionName(tSlowLogRecord.getRegionName())
|
||||
.setResponseSize(tSlowLogRecord.getResponseSize())
|
||||
.setBlockBytesScanned(tSlowLogRecord.getBlockBytesScanned())
|
||||
.setServerClass(tSlowLogRecord.getServerClass()).setStartTime(tSlowLogRecord.getStartTime())
|
||||
.setUserName(tSlowLogRecord.getUserName()).build())
|
||||
.collect(Collectors.toList());
|
||||
|
|
|
@ -9,9 +9,9 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
/**
|
||||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.OnlineLogRecordrd
|
||||
* org.apache.hadoop.hbase.client.OnlineLogRecord
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2021-07-19")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2023-02-04")
|
||||
public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecord, TOnlineLogRecord._Fields>, java.io.Serializable, Cloneable, Comparable<TOnlineLogRecord> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOnlineLogRecord");
|
||||
|
||||
|
@ -29,6 +29,7 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
private static final org.apache.thrift.protocol.TField MULTI_MUTATIONS_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("multiMutationsCount", org.apache.thrift.protocol.TType.I32, (short)12);
|
||||
private static final org.apache.thrift.protocol.TField MULTI_SERVICE_CALLS_FIELD_DESC = new org.apache.thrift.protocol.TField("multiServiceCalls", org.apache.thrift.protocol.TType.I32, (short)13);
|
||||
private static final org.apache.thrift.protocol.TField REGION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("regionName", org.apache.thrift.protocol.TType.STRING, (short)14);
|
||||
private static final org.apache.thrift.protocol.TField BLOCK_BYTES_SCANNED_FIELD_DESC = new org.apache.thrift.protocol.TField("blockBytesScanned", org.apache.thrift.protocol.TType.I64, (short)15);
|
||||
|
||||
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TOnlineLogRecordStandardSchemeFactory();
|
||||
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TOnlineLogRecordTupleSchemeFactory();
|
||||
|
@ -47,6 +48,7 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
public int multiMutationsCount; // required
|
||||
public int multiServiceCalls; // required
|
||||
public @org.apache.thrift.annotation.Nullable java.lang.String regionName; // optional
|
||||
public long blockBytesScanned; // optional
|
||||
|
||||
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
|
||||
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
|
||||
|
@ -63,7 +65,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
MULTI_GETS_COUNT((short)11, "multiGetsCount"),
|
||||
MULTI_MUTATIONS_COUNT((short)12, "multiMutationsCount"),
|
||||
MULTI_SERVICE_CALLS((short)13, "multiServiceCalls"),
|
||||
REGION_NAME((short)14, "regionName");
|
||||
REGION_NAME((short)14, "regionName"),
|
||||
BLOCK_BYTES_SCANNED((short)15, "blockBytesScanned");
|
||||
|
||||
private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
|
||||
|
||||
|
@ -107,6 +110,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
return MULTI_SERVICE_CALLS;
|
||||
case 14: // REGION_NAME
|
||||
return REGION_NAME;
|
||||
case 15: // BLOCK_BYTES_SCANNED
|
||||
return BLOCK_BYTES_SCANNED;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
|
@ -155,8 +160,9 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
private static final int __MULTIGETSCOUNT_ISSET_ID = 4;
|
||||
private static final int __MULTIMUTATIONSCOUNT_ISSET_ID = 5;
|
||||
private static final int __MULTISERVICECALLS_ISSET_ID = 6;
|
||||
private static final int __BLOCKBYTESSCANNED_ISSET_ID = 7;
|
||||
private byte __isset_bitfield = 0;
|
||||
private static final _Fields optionals[] = {_Fields.REGION_NAME};
|
||||
private static final _Fields optionals[] = {_Fields.REGION_NAME,_Fields.BLOCK_BYTES_SCANNED};
|
||||
public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
|
||||
static {
|
||||
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
|
@ -188,6 +194,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
|
||||
tmpMap.put(_Fields.REGION_NAME, new org.apache.thrift.meta_data.FieldMetaData("regionName", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
|
||||
tmpMap.put(_Fields.BLOCK_BYTES_SCANNED, new org.apache.thrift.meta_data.FieldMetaData("blockBytesScanned", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
|
||||
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOnlineLogRecord.class, metaDataMap);
|
||||
}
|
||||
|
@ -266,6 +274,7 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
if (other.isSetRegionName()) {
|
||||
this.regionName = other.regionName;
|
||||
}
|
||||
this.blockBytesScanned = other.blockBytesScanned;
|
||||
}
|
||||
|
||||
public TOnlineLogRecord deepCopy() {
|
||||
|
@ -295,6 +304,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
setMultiServiceCallsIsSet(false);
|
||||
this.multiServiceCalls = 0;
|
||||
this.regionName = null;
|
||||
setBlockBytesScannedIsSet(false);
|
||||
this.blockBytesScanned = 0;
|
||||
}
|
||||
|
||||
public long getStartTime() {
|
||||
|
@ -633,6 +644,29 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
}
|
||||
}
|
||||
|
||||
public long getBlockBytesScanned() {
|
||||
return this.blockBytesScanned;
|
||||
}
|
||||
|
||||
public TOnlineLogRecord setBlockBytesScanned(long blockBytesScanned) {
|
||||
this.blockBytesScanned = blockBytesScanned;
|
||||
setBlockBytesScannedIsSet(true);
|
||||
return this;
|
||||
}
|
||||
|
||||
public void unsetBlockBytesScanned() {
|
||||
__isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __BLOCKBYTESSCANNED_ISSET_ID);
|
||||
}
|
||||
|
||||
/** Returns true if field blockBytesScanned is set (has been assigned a value) and false otherwise */
|
||||
public boolean isSetBlockBytesScanned() {
|
||||
return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __BLOCKBYTESSCANNED_ISSET_ID);
|
||||
}
|
||||
|
||||
public void setBlockBytesScannedIsSet(boolean value) {
|
||||
__isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __BLOCKBYTESSCANNED_ISSET_ID, value);
|
||||
}
|
||||
|
||||
public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
|
||||
switch (field) {
|
||||
case START_TIME:
|
||||
|
@ -747,6 +781,14 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
}
|
||||
break;
|
||||
|
||||
case BLOCK_BYTES_SCANNED:
|
||||
if (value == null) {
|
||||
unsetBlockBytesScanned();
|
||||
} else {
|
||||
setBlockBytesScanned((java.lang.Long)value);
|
||||
}
|
||||
break;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -795,6 +837,9 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
case REGION_NAME:
|
||||
return getRegionName();
|
||||
|
||||
case BLOCK_BYTES_SCANNED:
|
||||
return getBlockBytesScanned();
|
||||
|
||||
}
|
||||
throw new java.lang.IllegalStateException();
|
||||
}
|
||||
|
@ -834,6 +879,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
return isSetMultiServiceCalls();
|
||||
case REGION_NAME:
|
||||
return isSetRegionName();
|
||||
case BLOCK_BYTES_SCANNED:
|
||||
return isSetBlockBytesScanned();
|
||||
}
|
||||
throw new java.lang.IllegalStateException();
|
||||
}
|
||||
|
@ -977,6 +1024,15 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
return false;
|
||||
}
|
||||
|
||||
boolean this_present_blockBytesScanned = true && this.isSetBlockBytesScanned();
|
||||
boolean that_present_blockBytesScanned = true && that.isSetBlockBytesScanned();
|
||||
if (this_present_blockBytesScanned || that_present_blockBytesScanned) {
|
||||
if (!(this_present_blockBytesScanned && that_present_blockBytesScanned))
|
||||
return false;
|
||||
if (this.blockBytesScanned != that.blockBytesScanned)
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -1026,6 +1082,10 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
if (isSetRegionName())
|
||||
hashCode = hashCode * 8191 + regionName.hashCode();
|
||||
|
||||
hashCode = hashCode * 8191 + ((isSetBlockBytesScanned()) ? 131071 : 524287);
|
||||
if (isSetBlockBytesScanned())
|
||||
hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(blockBytesScanned);
|
||||
|
||||
return hashCode;
|
||||
}
|
||||
|
||||
|
@ -1177,6 +1237,16 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
return lastComparison;
|
||||
}
|
||||
}
|
||||
lastComparison = java.lang.Boolean.compare(isSetBlockBytesScanned(), other.isSetBlockBytesScanned());
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
if (isSetBlockBytesScanned()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.blockBytesScanned, other.blockBytesScanned);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -1283,6 +1353,12 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
}
|
||||
first = false;
|
||||
}
|
||||
if (isSetBlockBytesScanned()) {
|
||||
if (!first) sb.append(", ");
|
||||
sb.append("blockBytesScanned:");
|
||||
sb.append(this.blockBytesScanned);
|
||||
first = false;
|
||||
}
|
||||
sb.append(")");
|
||||
return sb.toString();
|
||||
}
|
||||
|
@ -1465,6 +1541,14 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
case 15: // BLOCK_BYTES_SCANNED
|
||||
if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
|
||||
struct.blockBytesScanned = iprot.readI64();
|
||||
struct.setBlockBytesScannedIsSet(true);
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
|
@ -1559,6 +1643,11 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
oprot.writeFieldEnd();
|
||||
}
|
||||
}
|
||||
if (struct.isSetBlockBytesScanned()) {
|
||||
oprot.writeFieldBegin(BLOCK_BYTES_SCANNED_FIELD_DESC);
|
||||
oprot.writeI64(struct.blockBytesScanned);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
oprot.writeFieldStop();
|
||||
oprot.writeStructEnd();
|
||||
}
|
||||
|
@ -1593,10 +1682,16 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
if (struct.isSetRegionName()) {
|
||||
optionals.set(0);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 1);
|
||||
if (struct.isSetBlockBytesScanned()) {
|
||||
optionals.set(1);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 2);
|
||||
if (struct.isSetRegionName()) {
|
||||
oprot.writeString(struct.regionName);
|
||||
}
|
||||
if (struct.isSetBlockBytesScanned()) {
|
||||
oprot.writeI64(struct.blockBytesScanned);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1628,11 +1723,15 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
|
|||
struct.setMultiMutationsCountIsSet(true);
|
||||
struct.multiServiceCalls = iprot.readI32();
|
||||
struct.setMultiServiceCallsIsSet(true);
|
||||
java.util.BitSet incoming = iprot.readBitSet(1);
|
||||
java.util.BitSet incoming = iprot.readBitSet(2);
|
||||
if (incoming.get(0)) {
|
||||
struct.regionName = iprot.readString();
|
||||
struct.setRegionNameIsSet(true);
|
||||
}
|
||||
if (incoming.get(1)) {
|
||||
struct.blockBytesScanned = iprot.readI64();
|
||||
struct.setBlockBytesScannedIsSet(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -481,7 +481,7 @@ struct TLogQueryFilter {
|
|||
|
||||
/**
|
||||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.OnlineLogRecordrd
|
||||
* org.apache.hadoop.hbase.client.OnlineLogRecord
|
||||
*/
|
||||
struct TOnlineLogRecord {
|
||||
1: required i64 startTime
|
||||
|
@ -498,6 +498,7 @@ struct TOnlineLogRecord {
|
|||
12: required i32 multiMutationsCount
|
||||
13: required i32 multiServiceCalls
|
||||
14: optional string regionName
|
||||
15: optional i64 blockBytesScanned
|
||||
}
|
||||
|
||||
//
|
||||
|
|
Loading…
Reference in New Issue