HBASE-23937 : Support Online LargeLogs similar to SlowLogs APIs (#1346)
Signed-off-by: Bharath Vissapragada <bharathv@apache.org>
This commit is contained in:
parent
fb17ecdd07
commit
0dcbf80583
|
@ -2273,19 +2273,19 @@ public interface Admin extends Abortable, Closeable {
|
|||
boolean isSnapshotCleanupEnabled() throws IOException;
|
||||
|
||||
/**
|
||||
* Retrieves online slow RPC logs from the provided list of
|
||||
* Retrieves online slow/large RPC logs from the provided list of
|
||||
* RegionServers
|
||||
*
|
||||
* @param serverNames Server names to get slowlog responses from
|
||||
* @param slowLogQueryFilter filter to be used if provided
|
||||
* @param logQueryFilter filter to be used if provided (determines slow / large RPC logs)
|
||||
* @return online slowlog response list
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
List<SlowLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
|
||||
final SlowLogQueryFilter slowLogQueryFilter) throws IOException;
|
||||
List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
|
||||
final LogQueryFilter logQueryFilter) throws IOException;
|
||||
|
||||
/**
|
||||
* Clears online slow RPC logs from the provided list of
|
||||
* Clears online slow/large RPC logs from the provided list of
|
||||
* RegionServers
|
||||
*
|
||||
* @param serverNames Set of Server names to clean slowlog responses from
|
||||
|
|
|
@ -965,9 +965,9 @@ class AdminOverAsyncAdmin implements Admin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<SlowLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
|
||||
final SlowLogQueryFilter slowLogQueryFilter) throws IOException {
|
||||
return get(admin.getSlowLogResponses(serverNames, slowLogQueryFilter));
|
||||
public List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
|
||||
final LogQueryFilter logQueryFilter) throws IOException {
|
||||
return get(admin.getSlowLogResponses(serverNames, logQueryFilter));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1515,11 +1515,11 @@ public interface AsyncAdmin {
|
|||
* RegionServers
|
||||
*
|
||||
* @param serverNames Server names to get slowlog responses from
|
||||
* @param slowLogQueryFilter filter to be used if provided
|
||||
* @param logQueryFilter filter to be used if provided
|
||||
* @return Online slowlog response list. The return value wrapped by a {@link CompletableFuture}
|
||||
*/
|
||||
CompletableFuture<List<SlowLogRecord>> getSlowLogResponses(final Set<ServerName> serverNames,
|
||||
final SlowLogQueryFilter slowLogQueryFilter);
|
||||
CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(final Set<ServerName> serverNames,
|
||||
final LogQueryFilter logQueryFilter);
|
||||
|
||||
/**
|
||||
* Clears online slow RPC logs from the provided list of
|
||||
|
|
|
@ -843,9 +843,9 @@ class AsyncHBaseAdmin implements AsyncAdmin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<List<SlowLogRecord>> getSlowLogResponses(
|
||||
final Set<ServerName> serverNames, final SlowLogQueryFilter slowLogQueryFilter) {
|
||||
return wrap(rawAdmin.getSlowLogResponses(serverNames, slowLogQueryFilter));
|
||||
public CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(
|
||||
final Set<ServerName> serverNames, final LogQueryFilter logQueryFilter) {
|
||||
return wrap(rawAdmin.getSlowLogResponses(serverNames, logQueryFilter));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,16 +25,23 @@ import org.apache.commons.lang3.builder.ToStringBuilder;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* SlowLog Query Filter with all filter and limit parameters
|
||||
* Slow/Large Log Query Filter with all filter and limit parameters
|
||||
* Used by Admin API: getSlowLogResponses
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SlowLogQueryFilter {
|
||||
public class LogQueryFilter {
|
||||
|
||||
private String regionName;
|
||||
private String clientAddress;
|
||||
private String tableName;
|
||||
private String userName;
|
||||
private int limit = 10;
|
||||
private Type type = Type.SLOW_LOG;
|
||||
|
||||
public enum Type {
|
||||
SLOW_LOG,
|
||||
LARGE_LOG
|
||||
}
|
||||
|
||||
public String getRegionName() {
|
||||
return regionName;
|
||||
|
@ -76,6 +83,14 @@ public class SlowLogQueryFilter {
|
|||
this.limit = limit;
|
||||
}
|
||||
|
||||
public Type getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public void setType(Type type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
|
@ -86,7 +101,7 @@ public class SlowLogQueryFilter {
|
|||
return false;
|
||||
}
|
||||
|
||||
SlowLogQueryFilter that = (SlowLogQueryFilter) o;
|
||||
LogQueryFilter that = (LogQueryFilter) o;
|
||||
|
||||
return new EqualsBuilder()
|
||||
.append(limit, that.limit)
|
||||
|
@ -94,6 +109,7 @@ public class SlowLogQueryFilter {
|
|||
.append(clientAddress, that.clientAddress)
|
||||
.append(tableName, that.tableName)
|
||||
.append(userName, that.userName)
|
||||
.append(type, that.type)
|
||||
.isEquals();
|
||||
}
|
||||
|
||||
|
@ -105,6 +121,7 @@ public class SlowLogQueryFilter {
|
|||
.append(tableName)
|
||||
.append(userName)
|
||||
.append(limit)
|
||||
.append(type)
|
||||
.toHashCode();
|
||||
}
|
||||
|
||||
|
@ -116,7 +133,7 @@ public class SlowLogQueryFilter {
|
|||
.append("tableName", tableName)
|
||||
.append("userName", userName)
|
||||
.append("limit", limit)
|
||||
.append("type", type)
|
||||
.toString();
|
||||
}
|
||||
|
||||
}
|
|
@ -30,16 +30,17 @@ import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
|
|||
import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
|
||||
|
||||
/**
|
||||
* SlowLog payload for hbase-client, to be used by Admin API get_slow_responses
|
||||
* Slow/Large Log payload for hbase-client, to be used by Admin API get_slow_responses and
|
||||
* get_large_responses
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
final public class SlowLogRecord {
|
||||
final public class OnlineLogRecord {
|
||||
|
||||
// used to convert object to pretty printed format
|
||||
// used by toJsonPrettyPrint()
|
||||
private static final Gson GSON = GsonUtil.createGson()
|
||||
.setPrettyPrinting()
|
||||
.registerTypeAdapter(SlowLogRecord.class, (JsonSerializer<SlowLogRecord>)
|
||||
.registerTypeAdapter(OnlineLogRecord.class, (JsonSerializer<OnlineLogRecord>)
|
||||
(slowLogPayload, type, jsonSerializationContext) -> {
|
||||
Gson gson = new Gson();
|
||||
JsonObject jsonObj = (JsonObject) gson.toJsonTree(slowLogPayload);
|
||||
|
@ -128,7 +129,7 @@ final public class SlowLogRecord {
|
|||
return multiServiceCalls;
|
||||
}
|
||||
|
||||
private SlowLogRecord(final long startTime, final int processingTime, final int queueTime,
|
||||
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,
|
||||
|
@ -149,7 +150,7 @@ final public class SlowLogRecord {
|
|||
this.multiServiceCalls = multiServiceCalls;
|
||||
}
|
||||
|
||||
public static class SlowLogRecordBuilder {
|
||||
public static class OnlineLogRecordBuilder {
|
||||
private long startTime;
|
||||
private int processingTime;
|
||||
private int queueTime;
|
||||
|
@ -165,78 +166,78 @@ final public class SlowLogRecord {
|
|||
private int multiMutationsCount;
|
||||
private int multiServiceCalls;
|
||||
|
||||
public SlowLogRecordBuilder setStartTime(long startTime) {
|
||||
public OnlineLogRecordBuilder setStartTime(long startTime) {
|
||||
this.startTime = startTime;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setProcessingTime(int processingTime) {
|
||||
public OnlineLogRecordBuilder setProcessingTime(int processingTime) {
|
||||
this.processingTime = processingTime;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setQueueTime(int queueTime) {
|
||||
public OnlineLogRecordBuilder setQueueTime(int queueTime) {
|
||||
this.queueTime = queueTime;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setResponseSize(long responseSize) {
|
||||
public OnlineLogRecordBuilder setResponseSize(long responseSize) {
|
||||
this.responseSize = responseSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setClientAddress(String clientAddress) {
|
||||
public OnlineLogRecordBuilder setClientAddress(String clientAddress) {
|
||||
this.clientAddress = clientAddress;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setServerClass(String serverClass) {
|
||||
public OnlineLogRecordBuilder setServerClass(String serverClass) {
|
||||
this.serverClass = serverClass;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setMethodName(String methodName) {
|
||||
public OnlineLogRecordBuilder setMethodName(String methodName) {
|
||||
this.methodName = methodName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setCallDetails(String callDetails) {
|
||||
public OnlineLogRecordBuilder setCallDetails(String callDetails) {
|
||||
this.callDetails = callDetails;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setParam(String param) {
|
||||
public OnlineLogRecordBuilder setParam(String param) {
|
||||
this.param = param;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setRegionName(String regionName) {
|
||||
public OnlineLogRecordBuilder setRegionName(String regionName) {
|
||||
this.regionName = regionName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setUserName(String userName) {
|
||||
public OnlineLogRecordBuilder setUserName(String userName) {
|
||||
this.userName = userName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setMultiGetsCount(int multiGetsCount) {
|
||||
public OnlineLogRecordBuilder setMultiGetsCount(int multiGetsCount) {
|
||||
this.multiGetsCount = multiGetsCount;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setMultiMutationsCount(int multiMutationsCount) {
|
||||
public OnlineLogRecordBuilder setMultiMutationsCount(int multiMutationsCount) {
|
||||
this.multiMutationsCount = multiMutationsCount;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecordBuilder setMultiServiceCalls(int multiServiceCalls) {
|
||||
public OnlineLogRecordBuilder setMultiServiceCalls(int multiServiceCalls) {
|
||||
this.multiServiceCalls = multiServiceCalls;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SlowLogRecord build() {
|
||||
return new SlowLogRecord(startTime, processingTime, queueTime, responseSize,
|
||||
public OnlineLogRecord build() {
|
||||
return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize,
|
||||
clientAddress, serverClass, methodName, callDetails, param, regionName,
|
||||
userName, multiGetsCount, multiMutationsCount, multiServiceCalls);
|
||||
}
|
||||
|
@ -252,7 +253,7 @@ final public class SlowLogRecord {
|
|||
return false;
|
||||
}
|
||||
|
||||
SlowLogRecord that = (SlowLogRecord) o;
|
||||
OnlineLogRecord that = (OnlineLogRecord) o;
|
||||
|
||||
return new EqualsBuilder()
|
||||
.append(startTime, that.startTime)
|
|
@ -3951,31 +3951,52 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<List<SlowLogRecord>> getSlowLogResponses(
|
||||
public CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(
|
||||
@Nullable final Set<ServerName> serverNames,
|
||||
final SlowLogQueryFilter slowLogQueryFilter) {
|
||||
final LogQueryFilter logQueryFilter) {
|
||||
if (CollectionUtils.isEmpty(serverNames)) {
|
||||
return CompletableFuture.completedFuture(Collections.emptyList());
|
||||
}
|
||||
if (logQueryFilter.getType() == null
|
||||
|| logQueryFilter.getType() == LogQueryFilter.Type.SLOW_LOG) {
|
||||
return CompletableFuture.supplyAsync(() -> serverNames.stream()
|
||||
.map((ServerName serverName) ->
|
||||
getSlowLogResponseFromServer(serverName, slowLogQueryFilter))
|
||||
getSlowLogResponseFromServer(serverName, logQueryFilter))
|
||||
.map(CompletableFuture::join)
|
||||
.flatMap(List::stream)
|
||||
.collect(Collectors.toList()));
|
||||
} else {
|
||||
return CompletableFuture.supplyAsync(() -> serverNames.stream()
|
||||
.map((ServerName serverName) ->
|
||||
getLargeLogResponseFromServer(serverName, logQueryFilter))
|
||||
.map(CompletableFuture::join)
|
||||
.flatMap(List::stream)
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
private CompletableFuture<List<SlowLogRecord>> getSlowLogResponseFromServer(
|
||||
final ServerName serverName, final SlowLogQueryFilter slowLogQueryFilter) {
|
||||
return this.<List<SlowLogRecord>>newAdminCaller()
|
||||
private CompletableFuture<List<OnlineLogRecord>> getSlowLogResponseFromServer(
|
||||
final ServerName serverName, final LogQueryFilter logQueryFilter) {
|
||||
return this.<List<OnlineLogRecord>>newAdminCaller()
|
||||
.action((controller, stub) -> this
|
||||
.adminCall(
|
||||
controller, stub, RequestConverter.buildSlowLogResponseRequest(slowLogQueryFilter),
|
||||
controller, stub, RequestConverter.buildSlowLogResponseRequest(logQueryFilter),
|
||||
AdminService.Interface::getSlowLogResponses,
|
||||
ProtobufUtil::toSlowLogPayloads))
|
||||
.serverName(serverName).call();
|
||||
}
|
||||
|
||||
private CompletableFuture<List<OnlineLogRecord>> getLargeLogResponseFromServer(
|
||||
final ServerName serverName, final LogQueryFilter logQueryFilter) {
|
||||
return this.<List<OnlineLogRecord>>newAdminCaller()
|
||||
.action((controller, stub) -> this
|
||||
.adminCall(
|
||||
controller, stub, RequestConverter.buildSlowLogResponseRequest(logQueryFilter),
|
||||
AdminService.Interface::getLargeLogResponses,
|
||||
ProtobufUtil::toSlowLogPayloads))
|
||||
.serverName(serverName).call();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<List<Boolean>> clearSlowLogResponses(
|
||||
@Nullable Set<ServerName> serverNames) {
|
||||
|
|
|
@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.client.Durability;
|
|||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.OnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
|
@ -85,7 +86,6 @@ import org.apache.hadoop.hbase.client.RegionStatesCount;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.SlowLogParams;
|
||||
import org.apache.hadoop.hbase.client.SlowLogRecord;
|
||||
import org.apache.hadoop.hbase.client.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.client.SnapshotType;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -3522,14 +3522,14 @@ public final class ProtobufUtil {
|
|||
/**
|
||||
* Convert Protobuf class
|
||||
* {@link org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload}
|
||||
* To client SlowLog Payload class {@link SlowLogRecord}
|
||||
* To client SlowLog Payload class {@link OnlineLogRecord}
|
||||
*
|
||||
* @param slowLogPayload SlowLog Payload protobuf instance
|
||||
* @return SlowLog Payload for client usecase
|
||||
*/
|
||||
private static SlowLogRecord getSlowLogRecord(
|
||||
private static OnlineLogRecord getSlowLogRecord(
|
||||
final TooSlowLog.SlowLogPayload slowLogPayload) {
|
||||
SlowLogRecord clientSlowLogRecord = new SlowLogRecord.SlowLogRecordBuilder()
|
||||
OnlineLogRecord onlineLogRecord = new OnlineLogRecord.OnlineLogRecordBuilder()
|
||||
.setCallDetails(slowLogPayload.getCallDetails())
|
||||
.setClientAddress(slowLogPayload.getClientAddress())
|
||||
.setMethodName(slowLogPayload.getMethodName())
|
||||
|
@ -3545,20 +3545,20 @@ public final class ProtobufUtil {
|
|||
.setStartTime(slowLogPayload.getStartTime())
|
||||
.setUserName(slowLogPayload.getUserName())
|
||||
.build();
|
||||
return clientSlowLogRecord;
|
||||
return onlineLogRecord;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert AdminProtos#SlowLogResponses to list of {@link SlowLogRecord}
|
||||
* Convert AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
|
||||
*
|
||||
* @param slowLogResponses slowlog response protobuf instance
|
||||
* @return list of SlowLog payloads for client usecase
|
||||
*/
|
||||
public static List<SlowLogRecord> toSlowLogPayloads(
|
||||
public static List<OnlineLogRecord> toSlowLogPayloads(
|
||||
final AdminProtos.SlowLogResponses slowLogResponses) {
|
||||
List<SlowLogRecord> slowLogRecords = slowLogResponses.getSlowLogPayloadsList()
|
||||
List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
|
||||
.stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
|
||||
return slowLogRecords;
|
||||
return onlineLogRecords;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.client.Row;
|
||||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.SlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.LogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
|
||||
|
@ -1754,32 +1754,32 @@ public final class RequestConverter {
|
|||
/**
|
||||
* Create a protocol buffer {@link SlowLogResponseRequest}
|
||||
*
|
||||
* @param slowLogQueryFilter filter to use if provided
|
||||
* @param logQueryFilter filter to use if provided
|
||||
* @return a protocol buffer SlowLogResponseRequest
|
||||
*/
|
||||
public static SlowLogResponseRequest buildSlowLogResponseRequest(
|
||||
final SlowLogQueryFilter slowLogQueryFilter) {
|
||||
final LogQueryFilter logQueryFilter) {
|
||||
SlowLogResponseRequest.Builder builder = SlowLogResponseRequest.newBuilder();
|
||||
if (slowLogQueryFilter == null) {
|
||||
if (logQueryFilter == null) {
|
||||
return builder.build();
|
||||
}
|
||||
final String clientAddress = slowLogQueryFilter.getClientAddress();
|
||||
final String clientAddress = logQueryFilter.getClientAddress();
|
||||
if (StringUtils.isNotEmpty(clientAddress)) {
|
||||
builder.setClientAddress(clientAddress);
|
||||
}
|
||||
final String regionName = slowLogQueryFilter.getRegionName();
|
||||
final String regionName = logQueryFilter.getRegionName();
|
||||
if (StringUtils.isNotEmpty(regionName)) {
|
||||
builder.setRegionName(regionName);
|
||||
}
|
||||
final String tableName = slowLogQueryFilter.getTableName();
|
||||
final String tableName = logQueryFilter.getTableName();
|
||||
if (StringUtils.isNotEmpty(tableName)) {
|
||||
builder.setTableName(tableName);
|
||||
}
|
||||
final String userName = slowLogQueryFilter.getUserName();
|
||||
final String userName = logQueryFilter.getUserName();
|
||||
if (StringUtils.isNotEmpty(userName)) {
|
||||
builder.setUserName(userName);
|
||||
}
|
||||
return builder.setLimit(slowLogQueryFilter.getLimit()).build();
|
||||
return builder.setLimit(logQueryFilter.getLimit()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -369,6 +369,9 @@ service AdminService {
|
|||
rpc GetSlowLogResponses(SlowLogResponseRequest)
|
||||
returns(SlowLogResponses);
|
||||
|
||||
rpc GetLargeLogResponses(SlowLogResponseRequest)
|
||||
returns(SlowLogResponses);
|
||||
|
||||
rpc ClearSlowLogsResponses(ClearSlowLogResponseRequest)
|
||||
returns(ClearSlowLogResponses);
|
||||
}
|
||||
|
|
|
@ -42,4 +42,15 @@ message SlowLogPayload {
|
|||
optional int32 multi_gets = 12 [default = 0];
|
||||
optional int32 multi_mutations = 13 [default = 0];
|
||||
optional int32 multi_service_calls = 14 [default = 0];
|
||||
required Type type = 15;
|
||||
|
||||
// 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
|
||||
enum Type {
|
||||
SLOW_LOG = 0;
|
||||
LARGE_LOG = 1;
|
||||
ALL = 2;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -427,12 +427,13 @@ public abstract class RpcServer implements RpcServerInterface,
|
|||
tooLarge, tooSlow,
|
||||
status.getClient(), startTime, processingTime, qTime,
|
||||
responseSize, userName);
|
||||
if (tooSlow && this.slowLogRecorder != null) {
|
||||
if (this.slowLogRecorder != null) {
|
||||
// send logs to ring buffer owned by slowLogRecorder
|
||||
final String className = server == null ? StringUtils.EMPTY :
|
||||
server.getClass().getSimpleName();
|
||||
this.slowLogRecorder.addSlowLogPayload(
|
||||
new RpcLogDetails(call, status.getClient(), responseSize, className));
|
||||
new RpcLogDetails(call, status.getClient(), responseSize, className, tooSlow,
|
||||
tooLarge));
|
||||
}
|
||||
}
|
||||
return new Pair<>(result, controller.cellScanner());
|
||||
|
|
|
@ -3872,6 +3872,22 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
return slowLogResponses;
|
||||
}
|
||||
|
||||
@Override
|
||||
@QosPriority(priority = HConstants.ADMIN_QOS)
|
||||
public SlowLogResponses getLargeLogResponses(final RpcController controller,
|
||||
final SlowLogResponseRequest request) {
|
||||
final SlowLogRecorder slowLogRecorder =
|
||||
this.regionServer.getSlowLogRecorder();
|
||||
final List<SlowLogPayload> slowLogPayloads;
|
||||
slowLogPayloads = slowLogRecorder != null
|
||||
? slowLogRecorder.getLargeLogPayloads(request)
|
||||
: Collections.emptyList();
|
||||
SlowLogResponses slowLogResponses = SlowLogResponses.newBuilder()
|
||||
.addAllSlowLogPayloads(slowLogPayloads)
|
||||
.build();
|
||||
return slowLogResponses;
|
||||
}
|
||||
|
||||
@Override
|
||||
@QosPriority(priority = HConstants.ADMIN_QOS)
|
||||
public ClearSlowLogResponses clearSlowLogsResponses(final RpcController controller,
|
||||
|
|
|
@ -50,13 +50,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPaylo
|
|||
* Event Handler run by disruptor ringbuffer consumer
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class SlowLogEventHandler implements EventHandler<RingBufferEnvelope> {
|
||||
class LogEventHandler implements EventHandler<RingBufferEnvelope> {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(SlowLogEventHandler.class);
|
||||
private static final Logger LOG = LoggerFactory.getLogger(LogEventHandler.class);
|
||||
|
||||
private final Queue<SlowLogPayload> queue;
|
||||
|
||||
SlowLogEventHandler(int eventCount) {
|
||||
LogEventHandler(int eventCount) {
|
||||
EvictingQueue<SlowLogPayload> evictingQueue = EvictingQueue.create(eventCount);
|
||||
queue = Queues.synchronizedQueue(evictingQueue);
|
||||
}
|
||||
|
@ -78,6 +78,10 @@ class SlowLogEventHandler implements EventHandler<RingBufferEnvelope> {
|
|||
final String clientAddress = rpcCallDetails.getClientAddress();
|
||||
final long responseSize = rpcCallDetails.getResponseSize();
|
||||
final String className = rpcCallDetails.getClassName();
|
||||
final SlowLogPayload.Type type = getLogType(rpcCallDetails);
|
||||
if (type == null) {
|
||||
return;
|
||||
}
|
||||
Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
|
||||
Message param = rpcCall.getParam();
|
||||
long receiveTime = rpcCall.getReceiveTime();
|
||||
|
@ -122,11 +126,31 @@ class SlowLogEventHandler implements EventHandler<RingBufferEnvelope> {
|
|||
.setResponseSize(responseSize)
|
||||
.setServerClass(className)
|
||||
.setStartTime(startTime)
|
||||
.setType(type)
|
||||
.setUserName(userName)
|
||||
.build();
|
||||
queue.add(slowLogPayload);
|
||||
}
|
||||
|
||||
private SlowLogPayload.Type getLogType(RpcLogDetails rpcCallDetails) {
|
||||
final boolean isSlowLog = rpcCallDetails.isSlowLog();
|
||||
final boolean isLargeLog = rpcCallDetails.isLargeLog();
|
||||
final SlowLogPayload.Type type;
|
||||
if (!isSlowLog && !isLargeLog) {
|
||||
LOG.error("slowLog and largeLog both are false. Ignoring the event. rpcCallDetails: {}",
|
||||
rpcCallDetails);
|
||||
return null;
|
||||
}
|
||||
if (isSlowLog && isLargeLog) {
|
||||
type = SlowLogPayload.Type.ALL;
|
||||
} else if (isSlowLog) {
|
||||
type = SlowLogPayload.Type.SLOW_LOG;
|
||||
} else {
|
||||
type = SlowLogPayload.Type.LARGE_LOG;
|
||||
}
|
||||
return type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleans up slow log payloads
|
||||
*
|
||||
|
@ -148,17 +172,44 @@ class SlowLogEventHandler implements EventHandler<RingBufferEnvelope> {
|
|||
*/
|
||||
List<SlowLogPayload> getSlowLogPayloads(final AdminProtos.SlowLogResponseRequest request) {
|
||||
List<SlowLogPayload> slowLogPayloadList =
|
||||
Arrays.stream(queue.toArray(new SlowLogPayload[0])).collect(Collectors.toList());
|
||||
Arrays.stream(queue.toArray(new SlowLogPayload[0]))
|
||||
.filter(e -> e.getType() == SlowLogPayload.Type.ALL
|
||||
|| e.getType() == SlowLogPayload.Type.SLOW_LOG)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// latest slow logs first, operator is interested in latest records from in-memory buffer
|
||||
Collections.reverse(slowLogPayloadList);
|
||||
|
||||
if (isFilterProvided(request)) {
|
||||
slowLogPayloadList = filterSlowLogs(request, slowLogPayloadList);
|
||||
return getFilteredLogs(request, slowLogPayloadList);
|
||||
}
|
||||
int limit = request.getLimit() >= slowLogPayloadList.size() ? slowLogPayloadList.size()
|
||||
|
||||
/**
|
||||
* Retrieve list of large log payloads
|
||||
*
|
||||
* @param request large log request parameters
|
||||
* @return list of large log payloads
|
||||
*/
|
||||
List<SlowLogPayload> getLargeLogPayloads(final AdminProtos.SlowLogResponseRequest request) {
|
||||
List<SlowLogPayload> slowLogPayloadList =
|
||||
Arrays.stream(queue.toArray(new SlowLogPayload[0]))
|
||||
.filter(e -> e.getType() == SlowLogPayload.Type.ALL
|
||||
|| e.getType() == SlowLogPayload.Type.LARGE_LOG)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// latest large logs first, operator is interested in latest records from in-memory buffer
|
||||
Collections.reverse(slowLogPayloadList);
|
||||
|
||||
return getFilteredLogs(request, slowLogPayloadList);
|
||||
}
|
||||
|
||||
private List<SlowLogPayload> getFilteredLogs(AdminProtos.SlowLogResponseRequest request,
|
||||
List<SlowLogPayload> logPayloadList) {
|
||||
if (isFilterProvided(request)) {
|
||||
logPayloadList = filterLogs(request, logPayloadList);
|
||||
}
|
||||
int limit = request.getLimit() >= logPayloadList.size() ? logPayloadList.size()
|
||||
: request.getLimit();
|
||||
return slowLogPayloadList.subList(0, limit);
|
||||
return logPayloadList.subList(0, limit);
|
||||
}
|
||||
|
||||
private boolean isFilterProvided(AdminProtos.SlowLogResponseRequest request) {
|
||||
|
@ -174,7 +225,7 @@ class SlowLogEventHandler implements EventHandler<RingBufferEnvelope> {
|
|||
return StringUtils.isNotEmpty(request.getRegionName());
|
||||
}
|
||||
|
||||
private List<SlowLogPayload> filterSlowLogs(AdminProtos.SlowLogResponseRequest request,
|
||||
private List<SlowLogPayload> filterLogs(AdminProtos.SlowLogResponseRequest request,
|
||||
List<SlowLogPayload> slowLogPayloadList) {
|
||||
List<SlowLogPayload> filteredSlowLogPayloads = new ArrayList<>();
|
||||
for (SlowLogPayload slowLogPayload : slowLogPayloadList) {
|
|
@ -29,17 +29,21 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
@InterfaceAudience.Private
|
||||
public class RpcLogDetails {
|
||||
|
||||
private RpcCall rpcCall;
|
||||
private String clientAddress;
|
||||
private long responseSize;
|
||||
private String className;
|
||||
private final RpcCall rpcCall;
|
||||
private final String clientAddress;
|
||||
private final long responseSize;
|
||||
private final String className;
|
||||
private final boolean isSlowLog;
|
||||
private final boolean isLargeLog;
|
||||
|
||||
public RpcLogDetails(RpcCall rpcCall, String clientAddress, long responseSize,
|
||||
String className) {
|
||||
String className, boolean isSlowLog, boolean isLargeLog) {
|
||||
this.rpcCall = rpcCall;
|
||||
this.clientAddress = clientAddress;
|
||||
this.responseSize = responseSize;
|
||||
this.className = className;
|
||||
this.isSlowLog = isSlowLog;
|
||||
this.isLargeLog = isLargeLog;
|
||||
}
|
||||
|
||||
public RpcCall getRpcCall() {
|
||||
|
@ -58,6 +62,14 @@ public class RpcLogDetails {
|
|||
return className;
|
||||
}
|
||||
|
||||
public boolean isSlowLog() {
|
||||
return isSlowLog;
|
||||
}
|
||||
|
||||
public boolean isLargeLog() {
|
||||
return isLargeLog;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new ToStringBuilder(this)
|
||||
|
@ -65,6 +77,8 @@ public class RpcLogDetails {
|
|||
.append("clientAddress", clientAddress)
|
||||
.append("responseSize", responseSize)
|
||||
.append("className", className)
|
||||
.append("isSlowLog", isSlowLog)
|
||||
.append("isLargeLog", isLargeLog)
|
||||
.toString();
|
||||
}
|
||||
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
|
||||
|
||||
/**
|
||||
* Online SlowLog Provider Service that keeps slow RPC logs in the ring buffer.
|
||||
* Online Slow/Large Log Provider Service that keeps slow/large RPC logs in the ring buffer.
|
||||
* The service uses LMAX Disruptor to save slow records which are then consumed by
|
||||
* a queue and based on the ring buffer size, the available records are then fetched
|
||||
* from the queue in thread-safe manner.
|
||||
|
@ -49,9 +49,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPaylo
|
|||
public class SlowLogRecorder {
|
||||
|
||||
private final Disruptor<RingBufferEnvelope> disruptor;
|
||||
private final SlowLogEventHandler slowLogEventHandler;
|
||||
private final LogEventHandler logEventHandler;
|
||||
private final int eventCount;
|
||||
private final boolean isOnlineSlowLogProviderEnabled;
|
||||
private final boolean isOnlineLogProviderEnabled;
|
||||
|
||||
private static final String SLOW_LOG_RING_BUFFER_SIZE =
|
||||
"hbase.regionserver.slowlog.ringbuffer.size";
|
||||
|
@ -60,12 +60,12 @@ public class SlowLogRecorder {
|
|||
* Initialize disruptor with configurable ringbuffer size
|
||||
*/
|
||||
public SlowLogRecorder(Configuration conf) {
|
||||
isOnlineSlowLogProviderEnabled = conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
|
||||
isOnlineLogProviderEnabled = conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
|
||||
HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
|
||||
|
||||
if (!isOnlineSlowLogProviderEnabled) {
|
||||
if (!isOnlineLogProviderEnabled) {
|
||||
this.disruptor = null;
|
||||
this.slowLogEventHandler = null;
|
||||
this.logEventHandler = null;
|
||||
this.eventCount = 0;
|
||||
return;
|
||||
}
|
||||
|
@ -86,8 +86,8 @@ public class SlowLogRecorder {
|
|||
this.disruptor.setDefaultExceptionHandler(new DisruptorExceptionHandler());
|
||||
|
||||
// initialize ringbuffer event handler
|
||||
this.slowLogEventHandler = new SlowLogEventHandler(this.eventCount);
|
||||
this.disruptor.handleEventsWith(new SlowLogEventHandler[]{this.slowLogEventHandler});
|
||||
this.logEventHandler = new LogEventHandler(this.eventCount);
|
||||
this.disruptor.handleEventsWith(new LogEventHandler[]{this.logEventHandler});
|
||||
this.disruptor.start();
|
||||
}
|
||||
|
||||
|
@ -113,7 +113,18 @@ public class SlowLogRecorder {
|
|||
* @return online slow logs from ringbuffer
|
||||
*/
|
||||
public List<SlowLogPayload> getSlowLogPayloads(AdminProtos.SlowLogResponseRequest request) {
|
||||
return isOnlineSlowLogProviderEnabled ? this.slowLogEventHandler.getSlowLogPayloads(request)
|
||||
return isOnlineLogProviderEnabled ? this.logEventHandler.getSlowLogPayloads(request)
|
||||
: Collections.emptyList();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve online large logs from ringbuffer
|
||||
*
|
||||
* @param request large log request parameters
|
||||
* @return online large logs from ringbuffer
|
||||
*/
|
||||
public List<SlowLogPayload> getLargeLogPayloads(AdminProtos.SlowLogResponseRequest request) {
|
||||
return isOnlineLogProviderEnabled ? this.logEventHandler.getLargeLogPayloads(request)
|
||||
: Collections.emptyList();
|
||||
}
|
||||
|
||||
|
@ -125,10 +136,10 @@ public class SlowLogRecorder {
|
|||
* clean up slow logs
|
||||
*/
|
||||
public boolean clearSlowLogPayloads() {
|
||||
if (!isOnlineSlowLogProviderEnabled) {
|
||||
if (!isOnlineLogProviderEnabled) {
|
||||
return true;
|
||||
}
|
||||
return this.slowLogEventHandler.clearSlowLogs();
|
||||
return this.logEventHandler.clearSlowLogs();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -138,7 +149,7 @@ public class SlowLogRecorder {
|
|||
* consumers
|
||||
*/
|
||||
public void addSlowLogPayload(RpcLogDetails rpcLogDetails) {
|
||||
if (!isOnlineSlowLogProviderEnabled) {
|
||||
if (!isOnlineLogProviderEnabled) {
|
||||
return;
|
||||
}
|
||||
RingBuffer<RingBufferEnvelope> ringBuffer = this.disruptor.getRingBuffer();
|
||||
|
|
|
@ -856,12 +856,12 @@ public class TestAdmin2 extends TestAdminBase {
|
|||
}
|
||||
Assert.assertEquals(countFailedClearSlowResponse, 0);
|
||||
|
||||
SlowLogQueryFilter slowLogQueryFilter = new SlowLogQueryFilter();
|
||||
List<SlowLogRecord> slowLogRecords = ADMIN.getSlowLogResponses(new HashSet<>(serverNames),
|
||||
slowLogQueryFilter);
|
||||
LogQueryFilter logQueryFilter = new LogQueryFilter();
|
||||
List<OnlineLogRecord> onlineLogRecords = ADMIN.getSlowLogResponses(new HashSet<>(serverNames),
|
||||
logQueryFilter);
|
||||
|
||||
// after cleanup of slowlog responses, total count of slowlog payloads should be 0
|
||||
Assert.assertEquals(slowLogRecords.size(), 0);
|
||||
Assert.assertEquals(onlineLogRecords.size(), 0);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -684,6 +684,12 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SlowLogResponses getLargeLogResponses(RpcController controller,
|
||||
SlowLogResponseRequest request) throws ServiceException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClearSlowLogResponses clearSlowLogsResponses(RpcController controller,
|
||||
ClearSlowLogResponseRequest request) throws ServiceException {
|
||||
|
|
|
@ -195,6 +195,19 @@ public class TestSlowLogRecorder {
|
|||
})
|
||||
);
|
||||
|
||||
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||
() -> {
|
||||
List<SlowLogPayload> slowLogPayloadsList = slowLogRecorder.getLargeLogPayloads(request);
|
||||
// confirm ringbuffer is full
|
||||
// and ordered events
|
||||
return slowLogPayloadsList.size() == 8
|
||||
&& confirmPayloadParams(0, 14, slowLogPayloadsList)
|
||||
&& confirmPayloadParams(1, 13, slowLogPayloadsList)
|
||||
&& confirmPayloadParams(2, 12, slowLogPayloadsList)
|
||||
&& confirmPayloadParams(3, 11, slowLogPayloadsList);
|
||||
})
|
||||
);
|
||||
|
||||
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||
() -> {
|
||||
boolean isRingBufferCleaned = slowLogRecorder.clearSlowLogPayloads();
|
||||
|
@ -388,11 +401,100 @@ public class TestSlowLogRecorder {
|
|||
|
||||
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(
|
||||
5000, () -> slowLogRecorder.getSlowLogPayloads(request).size() > 10000));
|
||||
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(
|
||||
5000, () -> slowLogRecorder.getLargeLogPayloads(request).size() > 10000));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSlowLargeLogEvents() throws Exception {
|
||||
Configuration conf = applySlowLogRecorderConf(28);
|
||||
slowLogRecorder = new SlowLogRecorder(conf);
|
||||
AdminProtos.SlowLogResponseRequest request =
|
||||
AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(14 * 11).build();
|
||||
|
||||
Assert.assertEquals(slowLogRecorder.getSlowLogPayloads(request).size(), 0);
|
||||
LOG.debug("Initially ringbuffer of Slow Log records is empty");
|
||||
|
||||
boolean isSlowLog;
|
||||
boolean isLargeLog;
|
||||
for (int i = 0; i < 14 * 11; i++) {
|
||||
if (i % 2 == 0) {
|
||||
isSlowLog = true;
|
||||
isLargeLog = false;
|
||||
} else {
|
||||
isSlowLog = false;
|
||||
isLargeLog = true;
|
||||
}
|
||||
RpcLogDetails rpcLogDetails =
|
||||
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1),
|
||||
isSlowLog, isLargeLog);
|
||||
slowLogRecorder.addSlowLogPayload(rpcLogDetails);
|
||||
}
|
||||
LOG.debug("Added 14 * 11 records, ringbuffer should only provide latest 14 records");
|
||||
|
||||
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||
() -> slowLogRecorder.getSlowLogPayloads(request).size() == 14));
|
||||
|
||||
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||
() -> {
|
||||
List<SlowLogPayload> slowLogPayloads = slowLogRecorder.getSlowLogPayloads(request);
|
||||
|
||||
// confirm strict order of slow log payloads
|
||||
return slowLogPayloads.size() == 14
|
||||
&& confirmPayloadParams(0, 153, slowLogPayloads)
|
||||
&& confirmPayloadParams(1, 151, slowLogPayloads)
|
||||
&& confirmPayloadParams(2, 149, slowLogPayloads)
|
||||
&& confirmPayloadParams(3, 147, slowLogPayloads)
|
||||
&& confirmPayloadParams(4, 145, slowLogPayloads)
|
||||
&& confirmPayloadParams(5, 143, slowLogPayloads)
|
||||
&& confirmPayloadParams(6, 141, slowLogPayloads)
|
||||
&& confirmPayloadParams(7, 139, slowLogPayloads)
|
||||
&& confirmPayloadParams(8, 137, slowLogPayloads)
|
||||
&& confirmPayloadParams(9, 135, slowLogPayloads)
|
||||
&& confirmPayloadParams(10, 133, slowLogPayloads)
|
||||
&& confirmPayloadParams(11, 131, slowLogPayloads)
|
||||
&& confirmPayloadParams(12, 129, slowLogPayloads)
|
||||
&& confirmPayloadParams(13, 127, slowLogPayloads);
|
||||
})
|
||||
);
|
||||
|
||||
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||
() -> slowLogRecorder.getLargeLogPayloads(request).size() == 14));
|
||||
|
||||
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||
() -> {
|
||||
List<SlowLogPayload> largeLogPayloads = slowLogRecorder.getLargeLogPayloads(request);
|
||||
|
||||
// confirm strict order of slow log payloads
|
||||
return largeLogPayloads.size() == 14
|
||||
&& confirmPayloadParams(0, 154, largeLogPayloads)
|
||||
&& confirmPayloadParams(1, 152, largeLogPayloads)
|
||||
&& confirmPayloadParams(2, 150, largeLogPayloads)
|
||||
&& confirmPayloadParams(3, 148, largeLogPayloads)
|
||||
&& confirmPayloadParams(4, 146, largeLogPayloads)
|
||||
&& confirmPayloadParams(5, 144, largeLogPayloads)
|
||||
&& confirmPayloadParams(6, 142, largeLogPayloads)
|
||||
&& confirmPayloadParams(7, 140, largeLogPayloads)
|
||||
&& confirmPayloadParams(8, 138, largeLogPayloads)
|
||||
&& confirmPayloadParams(9, 136, largeLogPayloads)
|
||||
&& confirmPayloadParams(10, 134, largeLogPayloads)
|
||||
&& confirmPayloadParams(11, 132, largeLogPayloads)
|
||||
&& confirmPayloadParams(12, 130, largeLogPayloads)
|
||||
&& confirmPayloadParams(13, 128, largeLogPayloads);
|
||||
})
|
||||
);
|
||||
|
||||
}
|
||||
|
||||
private RpcLogDetails getRpcLogDetails(String userName, String clientAddress,
|
||||
String className) {
|
||||
return new RpcLogDetails(getRpcCall(userName), clientAddress, 0, className);
|
||||
return new RpcLogDetails(getRpcCall(userName), clientAddress, 0, className, true, true);
|
||||
}
|
||||
|
||||
private RpcLogDetails getRpcLogDetails(String userName, String clientAddress,
|
||||
String className, boolean isSlowLog, boolean isLargeLog) {
|
||||
return new RpcLogDetails(getRpcCall(userName), clientAddress, 0, className, isSlowLog,
|
||||
isLargeLog);
|
||||
}
|
||||
|
||||
private RpcCall getRpcCall(String userName) {
|
||||
|
|
|
@ -48,12 +48,12 @@ import org.apache.hadoop.hbase.client.CompactType;
|
|||
import org.apache.hadoop.hbase.client.CompactionState;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.OnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.SlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.SlowLogRecord;
|
||||
import org.apache.hadoop.hbase.client.LogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -886,8 +886,8 @@ public class VerifyingRSGroupAdmin implements Admin, Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<SlowLogRecord> getSlowLogResponses(Set<ServerName> serverNames,
|
||||
SlowLogQueryFilter slowLogQueryFilter) throws IOException {
|
||||
public List<OnlineLogRecord> getSlowLogResponses(Set<ServerName> serverNames,
|
||||
LogQueryFilter logQueryFilter) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -1482,6 +1482,7 @@ module Hbase
|
|||
server_names = getServerNames(server_names_list, false)
|
||||
end
|
||||
filter_params = get_filter_params(args)
|
||||
filter_params.setType(org.apache.hadoop.hbase.client.LogQueryFilter::Type::SLOW_LOG)
|
||||
slow_log_responses = @admin.getSlowLogResponses(java.util.HashSet.new(server_names),
|
||||
filter_params)
|
||||
slow_log_responses_arr = []
|
||||
|
@ -1493,7 +1494,7 @@ module Hbase
|
|||
end
|
||||
|
||||
def get_filter_params(args)
|
||||
filter_params = org.apache.hadoop.hbase.client.SlowLogQueryFilter.new
|
||||
filter_params = org.apache.hadoop.hbase.client.LogQueryFilter.new
|
||||
if args.key? 'REGION_NAME'
|
||||
region_name = args['REGION_NAME']
|
||||
filter_params.setRegionName(region_name)
|
||||
|
@ -1517,6 +1518,31 @@ module Hbase
|
|||
filter_params
|
||||
end
|
||||
|
||||
#----------------------------------------------------------------------------------------------
|
||||
# Retrieve LargeLog Responses from RegionServers
|
||||
def get_largelog_responses(server_names, args)
|
||||
unless server_names.is_a?(Array) || server_names.is_a?(String)
|
||||
raise(ArgumentError,
|
||||
"#{server_names.class} of #{server_names.inspect} is not of Array/String type")
|
||||
end
|
||||
if server_names == '*'
|
||||
server_names = getServerNames([], true)
|
||||
else
|
||||
server_names_list = to_server_names(server_names)
|
||||
server_names = getServerNames(server_names_list, false)
|
||||
end
|
||||
filter_params = get_filter_params(args)
|
||||
filter_params.setType(org.apache.hadoop.hbase.client.LogQueryFilter::Type::LARGE_LOG)
|
||||
large_log_responses = @admin.getSlowLogResponses(java.util.HashSet.new(server_names),
|
||||
filter_params)
|
||||
large_log_responses_arr = []
|
||||
for large_log_response in large_log_responses
|
||||
large_log_responses_arr << large_log_response.toJsonPrettyPrint
|
||||
end
|
||||
puts 'Retrieved LargeLog Responses from RegionServers'
|
||||
puts large_log_responses_arr
|
||||
end
|
||||
|
||||
#----------------------------------------------------------------------------------------------
|
||||
# Clears SlowLog Responses from RegionServers
|
||||
def clear_slowlog_responses(server_names)
|
||||
|
|
|
@ -339,6 +339,7 @@ Shell.load_command_group(
|
|||
compaction_switch
|
||||
flush
|
||||
get_slowlog_responses
|
||||
get_largelog_responses
|
||||
major_compact
|
||||
move
|
||||
split
|
||||
|
|
|
@ -0,0 +1,78 @@
|
|||
#
|
||||
#
|
||||
# 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.
|
||||
|
||||
# Retrieve latest large log responses maintained in memory by RegionServers
|
||||
|
||||
module Shell
|
||||
module Commands
|
||||
# Retrieve latest large log responses
|
||||
class GetLargelogResponses < Command
|
||||
def help
|
||||
<<-EOF
|
||||
Retrieve latest LargeLog Responses maintained by each or specific RegionServers.
|
||||
Specify '*' to include all RS otherwise array of server names for specific
|
||||
RS. A server name is the host, port plus startcode of a RegionServer.
|
||||
e.g.: host187.example.com,60020,1289493121758 (find servername in
|
||||
master ui or when you do detailed status in shell)
|
||||
|
||||
Provide optional filter parameters as Hash.
|
||||
Default Limit of each server for providing no of large log records is 10. User can specify
|
||||
more limit by 'LIMIT' param in case more than 10 records should be retrieved.
|
||||
|
||||
Examples:
|
||||
|
||||
hbase> get_largelog_responses '*' => get largelog responses from all RS
|
||||
hbase> get_largelog_responses '*', {'LIMIT' => 50} => get largelog responses from all RS
|
||||
with 50 records limit (default limit: 10)
|
||||
hbase> get_largelog_responses ['SERVER_NAME1', 'SERVER_NAME2'] => get largelog responses from SERVER_NAME1,
|
||||
SERVER_NAME2
|
||||
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1'}
|
||||
=> get largelog responses only related to meta
|
||||
region
|
||||
hbase> get_largelog_responses '*', {'TABLE_NAME' => 't1'} => get largelog responses only related to t1 table
|
||||
hbase> get_largelog_responses '*', {'CLIENT_IP' => '192.162.1.40:60225', 'LIMIT' => 100}
|
||||
=> get largelog responses with given client
|
||||
IP address and get 100 records limit
|
||||
(default limit: 10)
|
||||
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1'}
|
||||
=> get largelog responses with given region name
|
||||
or table name
|
||||
hbase> get_largelog_responses '*', {'USER' => 'user_name', 'CLIENT_IP' => '192.162.1.40:60225'}
|
||||
=> get largelog responses that match either
|
||||
provided client IP address or user name
|
||||
|
||||
Sometimes output can be long pretty printed json for user to scroll in
|
||||
a single screen and hence user might prefer
|
||||
redirecting output of get_largelog_responses to a file.
|
||||
|
||||
Example:
|
||||
|
||||
echo "get_largelog_responses '*'" | hbase shell > xyz.out 2>&1
|
||||
|
||||
EOF
|
||||
end
|
||||
|
||||
def command(server_names, args = {})
|
||||
unless args.is_a? Hash
|
||||
raise 'Filter parameters are not Hash'
|
||||
end
|
||||
|
||||
admin.get_largelog_responses(server_names, args)
|
||||
end
|
||||
end
|
||||
end
|
||||
end
|
|
@ -65,8 +65,8 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.SlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.SlowLogRecord;
|
||||
import org.apache.hadoop.hbase.client.LogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.OnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
|
@ -81,14 +81,14 @@ import org.apache.hadoop.hbase.thrift2.generated.THRegionLocation;
|
|||
import org.apache.hadoop.hbase.thrift2.generated.TIOError;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TIllegalArgument;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TOnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TPut;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TResult;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TScan;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TServerName;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TSlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TSlowLogRecord;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableName;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TThriftServerType;
|
||||
|
@ -830,15 +830,15 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements TH
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<TSlowLogRecord> getSlowLogResponses(Set<TServerName> tServerNames,
|
||||
TSlowLogQueryFilter tSlowLogQueryFilter) throws TIOError, TException {
|
||||
public List<TOnlineLogRecord> getSlowLogResponses(Set<TServerName> tServerNames,
|
||||
TLogQueryFilter tLogQueryFilter) throws TIOError, TException {
|
||||
try {
|
||||
Set<ServerName> serverNames = ThriftUtilities.getServerNamesFromThrift(tServerNames);
|
||||
SlowLogQueryFilter slowLogQueryFilter =
|
||||
ThriftUtilities.getSlowLogQueryFromThrift(tSlowLogQueryFilter);
|
||||
List<SlowLogRecord> slowLogRecords =
|
||||
connectionCache.getAdmin().getSlowLogResponses(serverNames, slowLogQueryFilter);
|
||||
return ThriftUtilities.getSlowLogRecordsFromHBase(slowLogRecords);
|
||||
LogQueryFilter logQueryFilter =
|
||||
ThriftUtilities.getSlowLogQueryFromThrift(tLogQueryFilter);
|
||||
List<OnlineLogRecord> onlineLogRecords =
|
||||
connectionCache.getAdmin().getSlowLogResponses(serverNames, logQueryFilter);
|
||||
return ThriftUtilities.getSlowLogRecordsFromHBase(onlineLogRecords);
|
||||
} catch (IOException e) {
|
||||
throw getTIOError(e);
|
||||
}
|
||||
|
|
|
@ -53,7 +53,9 @@ import org.apache.hadoop.hbase.client.Delete;
|
|||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.LogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.OnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.client.OperationWithAttributes;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
|
@ -61,8 +63,6 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Scan.ReadType;
|
||||
import org.apache.hadoop.hbase.client.SlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.SlowLogRecord;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -94,16 +94,17 @@ import org.apache.hadoop.hbase.thrift2.generated.THRegionInfo;
|
|||
import org.apache.hadoop.hbase.thrift2.generated.THRegionLocation;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TKeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TLogType;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TMutation;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TOnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TPut;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TReadType;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TResult;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TScan;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TServerName;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TSlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TSlowLogRecord;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableName;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTimeRange;
|
||||
|
@ -1504,61 +1505,102 @@ public final class ThriftUtilities {
|
|||
.collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
public static TSlowLogQueryFilter getSlowLogQueryFromHBase(
|
||||
SlowLogQueryFilter slowLogQueryFilter) {
|
||||
TSlowLogQueryFilter tSlowLogQueryFilter = new TSlowLogQueryFilter();
|
||||
tSlowLogQueryFilter.setRegionName(slowLogQueryFilter.getRegionName());
|
||||
tSlowLogQueryFilter.setClientAddress(slowLogQueryFilter.getClientAddress());
|
||||
tSlowLogQueryFilter.setTableName(slowLogQueryFilter.getTableName());
|
||||
tSlowLogQueryFilter.setUserName(slowLogQueryFilter.getUserName());
|
||||
tSlowLogQueryFilter.setLimit(slowLogQueryFilter.getLimit());
|
||||
return tSlowLogQueryFilter;
|
||||
public static TLogQueryFilter getSlowLogQueryFromHBase(
|
||||
LogQueryFilter logQueryFilter) {
|
||||
TLogQueryFilter tLogQueryFilter = new TLogQueryFilter();
|
||||
tLogQueryFilter.setRegionName(logQueryFilter.getRegionName());
|
||||
tLogQueryFilter.setClientAddress(logQueryFilter.getClientAddress());
|
||||
tLogQueryFilter.setTableName(logQueryFilter.getTableName());
|
||||
tLogQueryFilter.setUserName(logQueryFilter.getUserName());
|
||||
tLogQueryFilter.setLimit(logQueryFilter.getLimit());
|
||||
TLogType tLogType = gettLogTypeFromHBase(logQueryFilter);
|
||||
tLogQueryFilter.setLogType(tLogType);
|
||||
return tLogQueryFilter;
|
||||
}
|
||||
|
||||
public static SlowLogQueryFilter getSlowLogQueryFromThrift(
|
||||
TSlowLogQueryFilter tSlowLogQueryFilter) {
|
||||
SlowLogQueryFilter slowLogQueryFilter = new SlowLogQueryFilter();
|
||||
slowLogQueryFilter.setRegionName(tSlowLogQueryFilter.getRegionName());
|
||||
slowLogQueryFilter.setClientAddress(tSlowLogQueryFilter.getClientAddress());
|
||||
slowLogQueryFilter.setTableName(tSlowLogQueryFilter.getTableName());
|
||||
slowLogQueryFilter.setUserName(tSlowLogQueryFilter.getUserName());
|
||||
slowLogQueryFilter.setLimit(tSlowLogQueryFilter.getLimit());
|
||||
return slowLogQueryFilter;
|
||||
private static TLogType gettLogTypeFromHBase(final LogQueryFilter logQueryFilter) {
|
||||
TLogType tLogType;
|
||||
switch (logQueryFilter.getType()) {
|
||||
case SLOW_LOG: {
|
||||
tLogType = TLogType.SLOW_LOG;
|
||||
break;
|
||||
}
|
||||
case LARGE_LOG: {
|
||||
tLogType = TLogType.LARGE_LOG;
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
tLogType = TLogType.SLOW_LOG;
|
||||
}
|
||||
}
|
||||
return tLogType;
|
||||
}
|
||||
|
||||
public static List<TSlowLogRecord> getSlowLogRecordsFromHBase(
|
||||
List<SlowLogRecord> slowLogRecords) {
|
||||
if (CollectionUtils.isEmpty(slowLogRecords)) {
|
||||
public static LogQueryFilter getSlowLogQueryFromThrift(
|
||||
TLogQueryFilter tLogQueryFilter) {
|
||||
LogQueryFilter logQueryFilter = new LogQueryFilter();
|
||||
logQueryFilter.setRegionName(tLogQueryFilter.getRegionName());
|
||||
logQueryFilter.setClientAddress(tLogQueryFilter.getClientAddress());
|
||||
logQueryFilter.setTableName(tLogQueryFilter.getTableName());
|
||||
logQueryFilter.setUserName(tLogQueryFilter.getUserName());
|
||||
logQueryFilter.setLimit(tLogQueryFilter.getLimit());
|
||||
LogQueryFilter.Type type = getLogTypeFromThrift(tLogQueryFilter);
|
||||
logQueryFilter.setType(type);
|
||||
return logQueryFilter;
|
||||
}
|
||||
|
||||
private static LogQueryFilter.Type getLogTypeFromThrift(
|
||||
final TLogQueryFilter tSlowLogQueryFilter) {
|
||||
LogQueryFilter.Type type;
|
||||
switch (tSlowLogQueryFilter.getLogType()) {
|
||||
case SLOW_LOG: {
|
||||
type = LogQueryFilter.Type.SLOW_LOG;
|
||||
break;
|
||||
}
|
||||
case LARGE_LOG: {
|
||||
type = LogQueryFilter.Type.LARGE_LOG;
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
type = LogQueryFilter.Type.SLOW_LOG;
|
||||
}
|
||||
}
|
||||
return type;
|
||||
}
|
||||
|
||||
public static List<TOnlineLogRecord> getSlowLogRecordsFromHBase(
|
||||
List<OnlineLogRecord> onlineLogRecords) {
|
||||
if (CollectionUtils.isEmpty(onlineLogRecords)) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return slowLogRecords.stream()
|
||||
return onlineLogRecords.stream()
|
||||
.map(slowLogRecord -> {
|
||||
TSlowLogRecord tSlowLogRecord = new TSlowLogRecord();
|
||||
tSlowLogRecord.setCallDetails(slowLogRecord.getCallDetails());
|
||||
tSlowLogRecord.setClientAddress(slowLogRecord.getClientAddress());
|
||||
tSlowLogRecord.setMethodName(slowLogRecord.getMethodName());
|
||||
tSlowLogRecord.setMultiGetsCount(slowLogRecord.getMultiGetsCount());
|
||||
tSlowLogRecord.setMultiMutationsCount(slowLogRecord.getMultiMutationsCount());
|
||||
tSlowLogRecord.setMultiServiceCalls(slowLogRecord.getMultiServiceCalls());
|
||||
tSlowLogRecord.setParam(slowLogRecord.getParam());
|
||||
tSlowLogRecord.setProcessingTime(slowLogRecord.getProcessingTime());
|
||||
tSlowLogRecord.setQueueTime(slowLogRecord.getQueueTime());
|
||||
tSlowLogRecord.setRegionName(slowLogRecord.getRegionName());
|
||||
tSlowLogRecord.setResponseSize(slowLogRecord.getResponseSize());
|
||||
tSlowLogRecord.setServerClass(slowLogRecord.getServerClass());
|
||||
tSlowLogRecord.setStartTime(slowLogRecord.getStartTime());
|
||||
tSlowLogRecord.setUserName(slowLogRecord.getUserName());
|
||||
return tSlowLogRecord;
|
||||
TOnlineLogRecord tOnlineLogRecord = new TOnlineLogRecord();
|
||||
tOnlineLogRecord.setCallDetails(slowLogRecord.getCallDetails());
|
||||
tOnlineLogRecord.setClientAddress(slowLogRecord.getClientAddress());
|
||||
tOnlineLogRecord.setMethodName(slowLogRecord.getMethodName());
|
||||
tOnlineLogRecord.setMultiGetsCount(slowLogRecord.getMultiGetsCount());
|
||||
tOnlineLogRecord.setMultiMutationsCount(slowLogRecord.getMultiMutationsCount());
|
||||
tOnlineLogRecord.setMultiServiceCalls(slowLogRecord.getMultiServiceCalls());
|
||||
tOnlineLogRecord.setParam(slowLogRecord.getParam());
|
||||
tOnlineLogRecord.setProcessingTime(slowLogRecord.getProcessingTime());
|
||||
tOnlineLogRecord.setQueueTime(slowLogRecord.getQueueTime());
|
||||
tOnlineLogRecord.setRegionName(slowLogRecord.getRegionName());
|
||||
tOnlineLogRecord.setResponseSize(slowLogRecord.getResponseSize());
|
||||
tOnlineLogRecord.setServerClass(slowLogRecord.getServerClass());
|
||||
tOnlineLogRecord.setStartTime(slowLogRecord.getStartTime());
|
||||
tOnlineLogRecord.setUserName(slowLogRecord.getUserName());
|
||||
return tOnlineLogRecord;
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static List<SlowLogRecord> getSlowLogRecordsFromThrift(
|
||||
List<TSlowLogRecord> tSlowLogRecords) {
|
||||
if (CollectionUtils.isEmpty(tSlowLogRecords)) {
|
||||
public static List<OnlineLogRecord> getSlowLogRecordsFromThrift(
|
||||
List<TOnlineLogRecord> tOnlineLogRecords) {
|
||||
if (CollectionUtils.isEmpty(tOnlineLogRecords)) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return tSlowLogRecords.stream()
|
||||
.map(tSlowLogRecord -> new SlowLogRecord.SlowLogRecordBuilder()
|
||||
return tOnlineLogRecords.stream()
|
||||
.map(tSlowLogRecord -> new OnlineLogRecord.OnlineLogRecordBuilder()
|
||||
.setCallDetails(tSlowLogRecord.getCallDetails())
|
||||
.setClientAddress(tSlowLogRecord.getClientAddress())
|
||||
.setMethodName(tSlowLogRecord.getMethodName())
|
||||
|
|
|
@ -43,9 +43,9 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|||
import org.apache.hadoop.hbase.client.CompactType;
|
||||
import org.apache.hadoop.hbase.client.CompactionState;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.LogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.OnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.SlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.client.SlowLogRecord;
|
||||
import org.apache.hadoop.hbase.client.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.client.SnapshotType;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -67,10 +67,10 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
|
|||
import org.apache.hadoop.hbase.thrift2.ThriftUtilities;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TOnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TServerName;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TSlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TSlowLogRecord;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableName;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -1166,15 +1166,15 @@ public class ThriftAdmin implements Admin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<SlowLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
|
||||
final SlowLogQueryFilter slowLogQueryFilter) throws IOException {
|
||||
public List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
|
||||
final LogQueryFilter logQueryFilter) throws IOException {
|
||||
Set<TServerName> tServerNames = ThriftUtilities.getServerNamesFromHBase(serverNames);
|
||||
TSlowLogQueryFilter tSlowLogQueryFilter =
|
||||
ThriftUtilities.getSlowLogQueryFromHBase(slowLogQueryFilter);
|
||||
TLogQueryFilter tLogQueryFilter =
|
||||
ThriftUtilities.getSlowLogQueryFromHBase(logQueryFilter);
|
||||
try {
|
||||
List<TSlowLogRecord> tSlowLogRecords =
|
||||
client.getSlowLogResponses(tServerNames, tSlowLogQueryFilter);
|
||||
return ThriftUtilities.getSlowLogRecordsFromThrift(tSlowLogRecords);
|
||||
List<TOnlineLogRecord> tOnlineLogRecords =
|
||||
client.getSlowLogResponses(tServerNames, tLogQueryFilter);
|
||||
return ThriftUtilities.getSlowLogRecordsFromThrift(tOnlineLogRecords);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
@ -1256,4 +1256,5 @@ public class ThriftAdmin implements Admin {
|
|||
getConfiguredNamespacesAndTablesInRSGroup(String groupName) throws IOException {
|
||||
throw new NotImplementedException("setRSGroup not supported in ThriftAdmin");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TAuthorization implements org.apache.thrift.TBase<TAuthorization, TAuthorization._Fields>, java.io.Serializable, Cloneable, Comparable<TAuthorization> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.regionserver.BloomType
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TBloomFilterType implements org.apache.thrift.TEnum {
|
||||
/**
|
||||
* Bloomfilters disabled
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TCellVisibility implements org.apache.thrift.TBase<TCellVisibility, TCellVisibility._Fields>, java.io.Serializable, Cloneable, Comparable<TCellVisibility> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility");
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* in a HBase table by column family and optionally
|
||||
* a column qualifier and timestamp
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.ColumnFamilyDescriptor
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TColumnFamilyDescriptor implements org.apache.thrift.TBase<TColumnFamilyDescriptor, TColumnFamilyDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnFamilyDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnFamilyDescriptor");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* Represents a single cell and the amount to increment it by
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TColumnIncrement implements org.apache.thrift.TBase<TColumnIncrement, TColumnIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnIncrement> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* Represents a single cell and its value.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColumnValue._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnValue> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.CompareOperator.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TCompareOperator implements org.apache.thrift.TEnum {
|
||||
LESS(0),
|
||||
LESS_OR_EQUAL(1),
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.io.compress.Algorithm
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TCompressionAlgorithm implements org.apache.thrift.TEnum {
|
||||
LZO(0),
|
||||
GZ(1),
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* - STRONG means reads only from primary region
|
||||
* - TIMELINE means reads might return values from secondary region replicas
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TConsistency implements org.apache.thrift.TEnum {
|
||||
STRONG(1),
|
||||
TIMELINE(2);
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TDataBlockEncoding implements org.apache.thrift.TEnum {
|
||||
/**
|
||||
* Disable data block encoding.
|
||||
|
|
|
@ -33,7 +33,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields>, java.io.Serializable, Cloneable, Comparable<TDelete> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete");
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* - DELETE_COLUMN means exactly one version will be removed,
|
||||
* - DELETE_COLUMNS means previous versions will also be removed.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TDeleteType implements org.apache.thrift.TEnum {
|
||||
DELETE_COLUMN(0),
|
||||
DELETE_COLUMNS(1),
|
||||
|
|
|
@ -14,7 +14,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* - SYNC_WAL means write the Mutation to the WAL synchronously,
|
||||
* - FSYNC_WAL means Write the Mutation to the WAL synchronously and force the entries to disk.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TDurability implements org.apache.thrift.TEnum {
|
||||
USE_DEFAULT(0),
|
||||
SKIP_WAL(1),
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* If you specify a time range and a timestamp the range is ignored.
|
||||
* Timestamps on TColumns are ignored.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.io.Serializable, Cloneable, Comparable<TGet> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class THBaseService {
|
||||
|
||||
public interface Iface {
|
||||
|
@ -521,12 +521,12 @@ public class THBaseService {
|
|||
*
|
||||
* @param serverNames @param serverNames Server names to get slowlog responses from
|
||||
*
|
||||
* @param slowLogQueryFilter @param slowLogQueryFilter filter to be used if provided
|
||||
* @param logQueryFilter @param logQueryFilter filter to be used if provided
|
||||
*/
|
||||
public java.util.List<TSlowLogRecord> getSlowLogResponses(java.util.Set<TServerName> serverNames, TSlowLogQueryFilter slowLogQueryFilter) throws TIOError, org.apache.thrift.TException;
|
||||
public java.util.List<TOnlineLogRecord> getSlowLogResponses(java.util.Set<TServerName> serverNames, TLogQueryFilter logQueryFilter) throws TIOError, org.apache.thrift.TException;
|
||||
|
||||
/**
|
||||
* Clears online slow RPC logs from the provided list of
|
||||
* Clears online slow/large RPC logs from the provided list of
|
||||
* RegionServers
|
||||
*
|
||||
* @return List of booleans representing if online slowlog response buffer is cleaned
|
||||
|
@ -637,7 +637,7 @@ public class THBaseService {
|
|||
|
||||
public void getClusterId(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
|
||||
|
||||
public void getSlowLogResponses(java.util.Set<TServerName> serverNames, TSlowLogQueryFilter slowLogQueryFilter, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TSlowLogRecord>> resultHandler) throws org.apache.thrift.TException;
|
||||
public void getSlowLogResponses(java.util.Set<TServerName> serverNames, TLogQueryFilter logQueryFilter, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TOnlineLogRecord>> resultHandler) throws org.apache.thrift.TException;
|
||||
|
||||
public void clearSlowLogResponses(java.util.Set<TServerName> serverNames, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.Boolean>> resultHandler) throws org.apache.thrift.TException;
|
||||
|
||||
|
@ -1897,21 +1897,21 @@ public class THBaseService {
|
|||
throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterId failed: unknown result");
|
||||
}
|
||||
|
||||
public java.util.List<TSlowLogRecord> getSlowLogResponses(java.util.Set<TServerName> serverNames, TSlowLogQueryFilter slowLogQueryFilter) throws TIOError, org.apache.thrift.TException
|
||||
public java.util.List<TOnlineLogRecord> getSlowLogResponses(java.util.Set<TServerName> serverNames, TLogQueryFilter logQueryFilter) throws TIOError, org.apache.thrift.TException
|
||||
{
|
||||
send_getSlowLogResponses(serverNames, slowLogQueryFilter);
|
||||
send_getSlowLogResponses(serverNames, logQueryFilter);
|
||||
return recv_getSlowLogResponses();
|
||||
}
|
||||
|
||||
public void send_getSlowLogResponses(java.util.Set<TServerName> serverNames, TSlowLogQueryFilter slowLogQueryFilter) throws org.apache.thrift.TException
|
||||
public void send_getSlowLogResponses(java.util.Set<TServerName> serverNames, TLogQueryFilter logQueryFilter) throws org.apache.thrift.TException
|
||||
{
|
||||
getSlowLogResponses_args args = new getSlowLogResponses_args();
|
||||
args.setServerNames(serverNames);
|
||||
args.setSlowLogQueryFilter(slowLogQueryFilter);
|
||||
args.setLogQueryFilter(logQueryFilter);
|
||||
sendBase("getSlowLogResponses", args);
|
||||
}
|
||||
|
||||
public java.util.List<TSlowLogRecord> recv_getSlowLogResponses() throws TIOError, org.apache.thrift.TException
|
||||
public java.util.List<TOnlineLogRecord> recv_getSlowLogResponses() throws TIOError, org.apache.thrift.TException
|
||||
{
|
||||
getSlowLogResponses_result result = new getSlowLogResponses_result();
|
||||
receiveBase(result, "getSlowLogResponses");
|
||||
|
@ -3615,32 +3615,32 @@ public class THBaseService {
|
|||
}
|
||||
}
|
||||
|
||||
public void getSlowLogResponses(java.util.Set<TServerName> serverNames, TSlowLogQueryFilter slowLogQueryFilter, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TSlowLogRecord>> resultHandler) throws org.apache.thrift.TException {
|
||||
public void getSlowLogResponses(java.util.Set<TServerName> serverNames, TLogQueryFilter logQueryFilter, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TOnlineLogRecord>> resultHandler) throws org.apache.thrift.TException {
|
||||
checkReady();
|
||||
getSlowLogResponses_call method_call = new getSlowLogResponses_call(serverNames, slowLogQueryFilter, resultHandler, this, ___protocolFactory, ___transport);
|
||||
getSlowLogResponses_call method_call = new getSlowLogResponses_call(serverNames, logQueryFilter, resultHandler, this, ___protocolFactory, ___transport);
|
||||
this.___currentMethod = method_call;
|
||||
___manager.call(method_call);
|
||||
}
|
||||
|
||||
public static class getSlowLogResponses_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<TSlowLogRecord>> {
|
||||
public static class getSlowLogResponses_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<TOnlineLogRecord>> {
|
||||
private java.util.Set<TServerName> serverNames;
|
||||
private TSlowLogQueryFilter slowLogQueryFilter;
|
||||
public getSlowLogResponses_call(java.util.Set<TServerName> serverNames, TSlowLogQueryFilter slowLogQueryFilter, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TSlowLogRecord>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
|
||||
private TLogQueryFilter logQueryFilter;
|
||||
public getSlowLogResponses_call(java.util.Set<TServerName> serverNames, TLogQueryFilter logQueryFilter, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TOnlineLogRecord>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
|
||||
super(client, protocolFactory, transport, resultHandler, false);
|
||||
this.serverNames = serverNames;
|
||||
this.slowLogQueryFilter = slowLogQueryFilter;
|
||||
this.logQueryFilter = logQueryFilter;
|
||||
}
|
||||
|
||||
public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
|
||||
prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getSlowLogResponses", org.apache.thrift.protocol.TMessageType.CALL, 0));
|
||||
getSlowLogResponses_args args = new getSlowLogResponses_args();
|
||||
args.setServerNames(serverNames);
|
||||
args.setSlowLogQueryFilter(slowLogQueryFilter);
|
||||
args.setLogQueryFilter(logQueryFilter);
|
||||
args.write(prot);
|
||||
prot.writeMessageEnd();
|
||||
}
|
||||
|
||||
public java.util.List<TSlowLogRecord> getResult() throws TIOError, org.apache.thrift.TException {
|
||||
public java.util.List<TOnlineLogRecord> getResult() throws TIOError, org.apache.thrift.TException {
|
||||
if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
|
||||
throw new java.lang.IllegalStateException("Method call not finished!");
|
||||
}
|
||||
|
@ -5167,7 +5167,7 @@ public class THBaseService {
|
|||
public getSlowLogResponses_result getResult(I iface, getSlowLogResponses_args args) throws org.apache.thrift.TException {
|
||||
getSlowLogResponses_result result = new getSlowLogResponses_result();
|
||||
try {
|
||||
result.success = iface.getSlowLogResponses(args.serverNames, args.slowLogQueryFilter);
|
||||
result.success = iface.getSlowLogResponses(args.serverNames, args.logQueryFilter);
|
||||
} catch (TIOError io) {
|
||||
result.io = io;
|
||||
}
|
||||
|
@ -8383,7 +8383,7 @@ public class THBaseService {
|
|||
}
|
||||
}
|
||||
|
||||
public static class getSlowLogResponses<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getSlowLogResponses_args, java.util.List<TSlowLogRecord>> {
|
||||
public static class getSlowLogResponses<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getSlowLogResponses_args, java.util.List<TOnlineLogRecord>> {
|
||||
public getSlowLogResponses() {
|
||||
super("getSlowLogResponses");
|
||||
}
|
||||
|
@ -8392,10 +8392,10 @@ public class THBaseService {
|
|||
return new getSlowLogResponses_args();
|
||||
}
|
||||
|
||||
public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TSlowLogRecord>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
|
||||
public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TOnlineLogRecord>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
|
||||
final org.apache.thrift.AsyncProcessFunction fcall = this;
|
||||
return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<TSlowLogRecord>>() {
|
||||
public void onComplete(java.util.List<TSlowLogRecord> o) {
|
||||
return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<TOnlineLogRecord>>() {
|
||||
public void onComplete(java.util.List<TOnlineLogRecord> o) {
|
||||
getSlowLogResponses_result result = new getSlowLogResponses_result();
|
||||
result.success = o;
|
||||
try {
|
||||
|
@ -8443,8 +8443,8 @@ public class THBaseService {
|
|||
return false;
|
||||
}
|
||||
|
||||
public void start(I iface, getSlowLogResponses_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TSlowLogRecord>> resultHandler) throws org.apache.thrift.TException {
|
||||
iface.getSlowLogResponses(args.serverNames, args.slowLogQueryFilter,resultHandler);
|
||||
public void start(I iface, getSlowLogResponses_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TOnlineLogRecord>> resultHandler) throws org.apache.thrift.TException {
|
||||
iface.getSlowLogResponses(args.serverNames, args.logQueryFilter,resultHandler);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -53224,7 +53224,7 @@ public class THBaseService {
|
|||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getSlowLogResponses_args");
|
||||
|
||||
private static final org.apache.thrift.protocol.TField SERVER_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("serverNames", org.apache.thrift.protocol.TType.SET, (short)1);
|
||||
private static final org.apache.thrift.protocol.TField SLOW_LOG_QUERY_FILTER_FIELD_DESC = new org.apache.thrift.protocol.TField("slowLogQueryFilter", org.apache.thrift.protocol.TType.STRUCT, (short)2);
|
||||
private static final org.apache.thrift.protocol.TField LOG_QUERY_FILTER_FIELD_DESC = new org.apache.thrift.protocol.TField("logQueryFilter", org.apache.thrift.protocol.TType.STRUCT, (short)2);
|
||||
|
||||
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getSlowLogResponses_argsStandardSchemeFactory();
|
||||
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getSlowLogResponses_argsTupleSchemeFactory();
|
||||
|
@ -53234,9 +53234,9 @@ public class THBaseService {
|
|||
*/
|
||||
public @org.apache.thrift.annotation.Nullable java.util.Set<TServerName> serverNames; // required
|
||||
/**
|
||||
* @param slowLogQueryFilter filter to be used if provided
|
||||
* @param logQueryFilter filter to be used if provided
|
||||
*/
|
||||
public @org.apache.thrift.annotation.Nullable TSlowLogQueryFilter slowLogQueryFilter; // required
|
||||
public @org.apache.thrift.annotation.Nullable TLogQueryFilter logQueryFilter; // required
|
||||
|
||||
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
|
||||
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
|
||||
|
@ -53245,9 +53245,9 @@ public class THBaseService {
|
|||
*/
|
||||
SERVER_NAMES((short)1, "serverNames"),
|
||||
/**
|
||||
* @param slowLogQueryFilter filter to be used if provided
|
||||
* @param logQueryFilter filter to be used if provided
|
||||
*/
|
||||
SLOW_LOG_QUERY_FILTER((short)2, "slowLogQueryFilter");
|
||||
LOG_QUERY_FILTER((short)2, "logQueryFilter");
|
||||
|
||||
private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
|
||||
|
||||
|
@ -53265,8 +53265,8 @@ public class THBaseService {
|
|||
switch(fieldId) {
|
||||
case 1: // SERVER_NAMES
|
||||
return SERVER_NAMES;
|
||||
case 2: // SLOW_LOG_QUERY_FILTER
|
||||
return SLOW_LOG_QUERY_FILTER;
|
||||
case 2: // LOG_QUERY_FILTER
|
||||
return LOG_QUERY_FILTER;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
|
@ -53314,8 +53314,8 @@ public class THBaseService {
|
|||
tmpMap.put(_Fields.SERVER_NAMES, new org.apache.thrift.meta_data.FieldMetaData("serverNames", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET,
|
||||
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TServerName.class))));
|
||||
tmpMap.put(_Fields.SLOW_LOG_QUERY_FILTER, new org.apache.thrift.meta_data.FieldMetaData("slowLogQueryFilter", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSlowLogQueryFilter.class)));
|
||||
tmpMap.put(_Fields.LOG_QUERY_FILTER, new org.apache.thrift.meta_data.FieldMetaData("logQueryFilter", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TLogQueryFilter.class)));
|
||||
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getSlowLogResponses_args.class, metaDataMap);
|
||||
}
|
||||
|
@ -53325,11 +53325,11 @@ public class THBaseService {
|
|||
|
||||
public getSlowLogResponses_args(
|
||||
java.util.Set<TServerName> serverNames,
|
||||
TSlowLogQueryFilter slowLogQueryFilter)
|
||||
TLogQueryFilter logQueryFilter)
|
||||
{
|
||||
this();
|
||||
this.serverNames = serverNames;
|
||||
this.slowLogQueryFilter = slowLogQueryFilter;
|
||||
this.logQueryFilter = logQueryFilter;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -53343,8 +53343,8 @@ public class THBaseService {
|
|||
}
|
||||
this.serverNames = __this__serverNames;
|
||||
}
|
||||
if (other.isSetSlowLogQueryFilter()) {
|
||||
this.slowLogQueryFilter = new TSlowLogQueryFilter(other.slowLogQueryFilter);
|
||||
if (other.isSetLogQueryFilter()) {
|
||||
this.logQueryFilter = new TLogQueryFilter(other.logQueryFilter);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -53355,7 +53355,7 @@ public class THBaseService {
|
|||
@Override
|
||||
public void clear() {
|
||||
this.serverNames = null;
|
||||
this.slowLogQueryFilter = null;
|
||||
this.logQueryFilter = null;
|
||||
}
|
||||
|
||||
public int getServerNamesSize() {
|
||||
|
@ -53406,33 +53406,33 @@ public class THBaseService {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param slowLogQueryFilter filter to be used if provided
|
||||
* @param logQueryFilter filter to be used if provided
|
||||
*/
|
||||
@org.apache.thrift.annotation.Nullable
|
||||
public TSlowLogQueryFilter getSlowLogQueryFilter() {
|
||||
return this.slowLogQueryFilter;
|
||||
public TLogQueryFilter getLogQueryFilter() {
|
||||
return this.logQueryFilter;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param slowLogQueryFilter filter to be used if provided
|
||||
* @param logQueryFilter filter to be used if provided
|
||||
*/
|
||||
public getSlowLogResponses_args setSlowLogQueryFilter(@org.apache.thrift.annotation.Nullable TSlowLogQueryFilter slowLogQueryFilter) {
|
||||
this.slowLogQueryFilter = slowLogQueryFilter;
|
||||
public getSlowLogResponses_args setLogQueryFilter(@org.apache.thrift.annotation.Nullable TLogQueryFilter logQueryFilter) {
|
||||
this.logQueryFilter = logQueryFilter;
|
||||
return this;
|
||||
}
|
||||
|
||||
public void unsetSlowLogQueryFilter() {
|
||||
this.slowLogQueryFilter = null;
|
||||
public void unsetLogQueryFilter() {
|
||||
this.logQueryFilter = null;
|
||||
}
|
||||
|
||||
/** Returns true if field slowLogQueryFilter is set (has been assigned a value) and false otherwise */
|
||||
public boolean isSetSlowLogQueryFilter() {
|
||||
return this.slowLogQueryFilter != null;
|
||||
/** Returns true if field logQueryFilter is set (has been assigned a value) and false otherwise */
|
||||
public boolean isSetLogQueryFilter() {
|
||||
return this.logQueryFilter != null;
|
||||
}
|
||||
|
||||
public void setSlowLogQueryFilterIsSet(boolean value) {
|
||||
public void setLogQueryFilterIsSet(boolean value) {
|
||||
if (!value) {
|
||||
this.slowLogQueryFilter = null;
|
||||
this.logQueryFilter = null;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -53446,11 +53446,11 @@ public class THBaseService {
|
|||
}
|
||||
break;
|
||||
|
||||
case SLOW_LOG_QUERY_FILTER:
|
||||
case LOG_QUERY_FILTER:
|
||||
if (value == null) {
|
||||
unsetSlowLogQueryFilter();
|
||||
unsetLogQueryFilter();
|
||||
} else {
|
||||
setSlowLogQueryFilter((TSlowLogQueryFilter)value);
|
||||
setLogQueryFilter((TLogQueryFilter)value);
|
||||
}
|
||||
break;
|
||||
|
||||
|
@ -53463,8 +53463,8 @@ public class THBaseService {
|
|||
case SERVER_NAMES:
|
||||
return getServerNames();
|
||||
|
||||
case SLOW_LOG_QUERY_FILTER:
|
||||
return getSlowLogQueryFilter();
|
||||
case LOG_QUERY_FILTER:
|
||||
return getLogQueryFilter();
|
||||
|
||||
}
|
||||
throw new java.lang.IllegalStateException();
|
||||
|
@ -53479,8 +53479,8 @@ public class THBaseService {
|
|||
switch (field) {
|
||||
case SERVER_NAMES:
|
||||
return isSetServerNames();
|
||||
case SLOW_LOG_QUERY_FILTER:
|
||||
return isSetSlowLogQueryFilter();
|
||||
case LOG_QUERY_FILTER:
|
||||
return isSetLogQueryFilter();
|
||||
}
|
||||
throw new java.lang.IllegalStateException();
|
||||
}
|
||||
|
@ -53509,12 +53509,12 @@ public class THBaseService {
|
|||
return false;
|
||||
}
|
||||
|
||||
boolean this_present_slowLogQueryFilter = true && this.isSetSlowLogQueryFilter();
|
||||
boolean that_present_slowLogQueryFilter = true && that.isSetSlowLogQueryFilter();
|
||||
if (this_present_slowLogQueryFilter || that_present_slowLogQueryFilter) {
|
||||
if (!(this_present_slowLogQueryFilter && that_present_slowLogQueryFilter))
|
||||
boolean this_present_logQueryFilter = true && this.isSetLogQueryFilter();
|
||||
boolean that_present_logQueryFilter = true && that.isSetLogQueryFilter();
|
||||
if (this_present_logQueryFilter || that_present_logQueryFilter) {
|
||||
if (!(this_present_logQueryFilter && that_present_logQueryFilter))
|
||||
return false;
|
||||
if (!this.slowLogQueryFilter.equals(that.slowLogQueryFilter))
|
||||
if (!this.logQueryFilter.equals(that.logQueryFilter))
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -53529,9 +53529,9 @@ public class THBaseService {
|
|||
if (isSetServerNames())
|
||||
hashCode = hashCode * 8191 + serverNames.hashCode();
|
||||
|
||||
hashCode = hashCode * 8191 + ((isSetSlowLogQueryFilter()) ? 131071 : 524287);
|
||||
if (isSetSlowLogQueryFilter())
|
||||
hashCode = hashCode * 8191 + slowLogQueryFilter.hashCode();
|
||||
hashCode = hashCode * 8191 + ((isSetLogQueryFilter()) ? 131071 : 524287);
|
||||
if (isSetLogQueryFilter())
|
||||
hashCode = hashCode * 8191 + logQueryFilter.hashCode();
|
||||
|
||||
return hashCode;
|
||||
}
|
||||
|
@ -53554,12 +53554,12 @@ public class THBaseService {
|
|||
return lastComparison;
|
||||
}
|
||||
}
|
||||
lastComparison = java.lang.Boolean.valueOf(isSetSlowLogQueryFilter()).compareTo(other.isSetSlowLogQueryFilter());
|
||||
lastComparison = java.lang.Boolean.valueOf(isSetLogQueryFilter()).compareTo(other.isSetLogQueryFilter());
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
if (isSetSlowLogQueryFilter()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.slowLogQueryFilter, other.slowLogQueryFilter);
|
||||
if (isSetLogQueryFilter()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.logQueryFilter, other.logQueryFilter);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
|
@ -53593,11 +53593,11 @@ public class THBaseService {
|
|||
}
|
||||
first = false;
|
||||
if (!first) sb.append(", ");
|
||||
sb.append("slowLogQueryFilter:");
|
||||
if (this.slowLogQueryFilter == null) {
|
||||
sb.append("logQueryFilter:");
|
||||
if (this.logQueryFilter == null) {
|
||||
sb.append("null");
|
||||
} else {
|
||||
sb.append(this.slowLogQueryFilter);
|
||||
sb.append(this.logQueryFilter);
|
||||
}
|
||||
first = false;
|
||||
sb.append(")");
|
||||
|
@ -53607,8 +53607,8 @@ public class THBaseService {
|
|||
public void validate() throws org.apache.thrift.TException {
|
||||
// check for required fields
|
||||
// check for sub-struct validity
|
||||
if (slowLogQueryFilter != null) {
|
||||
slowLogQueryFilter.validate();
|
||||
if (logQueryFilter != null) {
|
||||
logQueryFilter.validate();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -53665,11 +53665,11 @@ public class THBaseService {
|
|||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
case 2: // SLOW_LOG_QUERY_FILTER
|
||||
case 2: // LOG_QUERY_FILTER
|
||||
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
|
||||
struct.slowLogQueryFilter = new TSlowLogQueryFilter();
|
||||
struct.slowLogQueryFilter.read(iprot);
|
||||
struct.setSlowLogQueryFilterIsSet(true);
|
||||
struct.logQueryFilter = new TLogQueryFilter();
|
||||
struct.logQueryFilter.read(iprot);
|
||||
struct.setLogQueryFilterIsSet(true);
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
|
@ -53701,9 +53701,9 @@ public class THBaseService {
|
|||
}
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
if (struct.slowLogQueryFilter != null) {
|
||||
oprot.writeFieldBegin(SLOW_LOG_QUERY_FILTER_FIELD_DESC);
|
||||
struct.slowLogQueryFilter.write(oprot);
|
||||
if (struct.logQueryFilter != null) {
|
||||
oprot.writeFieldBegin(LOG_QUERY_FILTER_FIELD_DESC);
|
||||
struct.logQueryFilter.write(oprot);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
oprot.writeFieldStop();
|
||||
|
@ -53727,7 +53727,7 @@ public class THBaseService {
|
|||
if (struct.isSetServerNames()) {
|
||||
optionals.set(0);
|
||||
}
|
||||
if (struct.isSetSlowLogQueryFilter()) {
|
||||
if (struct.isSetLogQueryFilter()) {
|
||||
optionals.set(1);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 2);
|
||||
|
@ -53740,8 +53740,8 @@ public class THBaseService {
|
|||
}
|
||||
}
|
||||
}
|
||||
if (struct.isSetSlowLogQueryFilter()) {
|
||||
struct.slowLogQueryFilter.write(oprot);
|
||||
if (struct.isSetLogQueryFilter()) {
|
||||
struct.logQueryFilter.write(oprot);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -53764,9 +53764,9 @@ public class THBaseService {
|
|||
struct.setServerNamesIsSet(true);
|
||||
}
|
||||
if (incoming.get(1)) {
|
||||
struct.slowLogQueryFilter = new TSlowLogQueryFilter();
|
||||
struct.slowLogQueryFilter.read(iprot);
|
||||
struct.setSlowLogQueryFilterIsSet(true);
|
||||
struct.logQueryFilter = new TLogQueryFilter();
|
||||
struct.logQueryFilter.read(iprot);
|
||||
struct.setLogQueryFilterIsSet(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -53785,7 +53785,7 @@ public class THBaseService {
|
|||
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getSlowLogResponses_resultStandardSchemeFactory();
|
||||
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getSlowLogResponses_resultTupleSchemeFactory();
|
||||
|
||||
public @org.apache.thrift.annotation.Nullable java.util.List<TSlowLogRecord> success; // required
|
||||
public @org.apache.thrift.annotation.Nullable java.util.List<TOnlineLogRecord> success; // required
|
||||
public @org.apache.thrift.annotation.Nullable TIOError io; // required
|
||||
|
||||
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
|
||||
|
@ -53857,7 +53857,7 @@ public class THBaseService {
|
|||
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
|
||||
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSlowLogRecord.class))));
|
||||
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOnlineLogRecord.class))));
|
||||
tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
|
||||
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
|
||||
|
@ -53868,7 +53868,7 @@ public class THBaseService {
|
|||
}
|
||||
|
||||
public getSlowLogResponses_result(
|
||||
java.util.List<TSlowLogRecord> success,
|
||||
java.util.List<TOnlineLogRecord> success,
|
||||
TIOError io)
|
||||
{
|
||||
this();
|
||||
|
@ -53881,9 +53881,9 @@ public class THBaseService {
|
|||
*/
|
||||
public getSlowLogResponses_result(getSlowLogResponses_result other) {
|
||||
if (other.isSetSuccess()) {
|
||||
java.util.List<TSlowLogRecord> __this__success = new java.util.ArrayList<TSlowLogRecord>(other.success.size());
|
||||
for (TSlowLogRecord other_element : other.success) {
|
||||
__this__success.add(new TSlowLogRecord(other_element));
|
||||
java.util.List<TOnlineLogRecord> __this__success = new java.util.ArrayList<TOnlineLogRecord>(other.success.size());
|
||||
for (TOnlineLogRecord other_element : other.success) {
|
||||
__this__success.add(new TOnlineLogRecord(other_element));
|
||||
}
|
||||
this.success = __this__success;
|
||||
}
|
||||
|
@ -53907,23 +53907,23 @@ public class THBaseService {
|
|||
}
|
||||
|
||||
@org.apache.thrift.annotation.Nullable
|
||||
public java.util.Iterator<TSlowLogRecord> getSuccessIterator() {
|
||||
public java.util.Iterator<TOnlineLogRecord> getSuccessIterator() {
|
||||
return (this.success == null) ? null : this.success.iterator();
|
||||
}
|
||||
|
||||
public void addToSuccess(TSlowLogRecord elem) {
|
||||
public void addToSuccess(TOnlineLogRecord elem) {
|
||||
if (this.success == null) {
|
||||
this.success = new java.util.ArrayList<TSlowLogRecord>();
|
||||
this.success = new java.util.ArrayList<TOnlineLogRecord>();
|
||||
}
|
||||
this.success.add(elem);
|
||||
}
|
||||
|
||||
@org.apache.thrift.annotation.Nullable
|
||||
public java.util.List<TSlowLogRecord> getSuccess() {
|
||||
public java.util.List<TOnlineLogRecord> getSuccess() {
|
||||
return this.success;
|
||||
}
|
||||
|
||||
public getSlowLogResponses_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<TSlowLogRecord> success) {
|
||||
public getSlowLogResponses_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<TOnlineLogRecord> success) {
|
||||
this.success = success;
|
||||
return this;
|
||||
}
|
||||
|
@ -53974,7 +53974,7 @@ public class THBaseService {
|
|||
if (value == null) {
|
||||
unsetSuccess();
|
||||
} else {
|
||||
setSuccess((java.util.List<TSlowLogRecord>)value);
|
||||
setSuccess((java.util.List<TOnlineLogRecord>)value);
|
||||
}
|
||||
break;
|
||||
|
||||
|
@ -54179,11 +54179,11 @@ public class THBaseService {
|
|||
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
|
||||
{
|
||||
org.apache.thrift.protocol.TList _list358 = iprot.readListBegin();
|
||||
struct.success = new java.util.ArrayList<TSlowLogRecord>(_list358.size);
|
||||
@org.apache.thrift.annotation.Nullable TSlowLogRecord _elem359;
|
||||
struct.success = new java.util.ArrayList<TOnlineLogRecord>(_list358.size);
|
||||
@org.apache.thrift.annotation.Nullable TOnlineLogRecord _elem359;
|
||||
for (int _i360 = 0; _i360 < _list358.size; ++_i360)
|
||||
{
|
||||
_elem359 = new TSlowLogRecord();
|
||||
_elem359 = new TOnlineLogRecord();
|
||||
_elem359.read(iprot);
|
||||
struct.success.add(_elem359);
|
||||
}
|
||||
|
@ -54222,7 +54222,7 @@ public class THBaseService {
|
|||
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
|
||||
{
|
||||
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
|
||||
for (TSlowLogRecord _iter361 : struct.success)
|
||||
for (TOnlineLogRecord _iter361 : struct.success)
|
||||
{
|
||||
_iter361.write(oprot);
|
||||
}
|
||||
|
@ -54263,7 +54263,7 @@ public class THBaseService {
|
|||
if (struct.isSetSuccess()) {
|
||||
{
|
||||
oprot.writeI32(struct.success.size());
|
||||
for (TSlowLogRecord _iter362 : struct.success)
|
||||
for (TOnlineLogRecord _iter362 : struct.success)
|
||||
{
|
||||
_iter362.write(oprot);
|
||||
}
|
||||
|
@ -54281,11 +54281,11 @@ public class THBaseService {
|
|||
if (incoming.get(0)) {
|
||||
{
|
||||
org.apache.thrift.protocol.TList _list363 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
|
||||
struct.success = new java.util.ArrayList<TSlowLogRecord>(_list363.size);
|
||||
@org.apache.thrift.annotation.Nullable TSlowLogRecord _elem364;
|
||||
struct.success = new java.util.ArrayList<TOnlineLogRecord>(_list363.size);
|
||||
@org.apache.thrift.annotation.Nullable TOnlineLogRecord _elem364;
|
||||
for (int _i365 = 0; _i365 < _list363.size; ++_i365)
|
||||
{
|
||||
_elem364 = new TSlowLogRecord();
|
||||
_elem364 = new TOnlineLogRecord();
|
||||
_elem364.read(iprot);
|
||||
struct.success.add(_elem364);
|
||||
}
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class THRegionInfo implements org.apache.thrift.TBase<THRegionInfo, THRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionInfo> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionInfo");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class THRegionLocation implements org.apache.thrift.TBase<THRegionLocation, THRegionLocation._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionLocation> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionLocation");
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* to the HBase master or a HBase region server. Also used to return
|
||||
* more general HBase error conditions.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TIOError extends org.apache.thrift.TException implements org.apache.thrift.TBase<TIOError, TIOError._Fields>, java.io.Serializable, Cloneable, Comparable<TIOError> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* A TIllegalArgument exception indicates an illegal or invalid
|
||||
* argument was passed into a procedure.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TIllegalArgument extends org.apache.thrift.TException implements org.apache.thrift.TBase<TIllegalArgument, TIllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<TIllegalArgument> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument");
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.KeepDeletedCells
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TKeepDeletedCells implements org.apache.thrift.TEnum {
|
||||
/**
|
||||
* Deleted Cells are not retained.
|
||||
|
|
|
@ -9,26 +9,32 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
/**
|
||||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.SlowLogQueryFilter
|
||||
* org.apache.hadoop.hbase.client.LogQueryFilter
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQueryFilter, TSlowLogQueryFilter._Fields>, java.io.Serializable, Cloneable, Comparable<TSlowLogQueryFilter> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSlowLogQueryFilter");
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TLogQueryFilter implements org.apache.thrift.TBase<TLogQueryFilter, TLogQueryFilter._Fields>, java.io.Serializable, Cloneable, Comparable<TLogQueryFilter> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TLogQueryFilter");
|
||||
|
||||
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)1);
|
||||
private static final org.apache.thrift.protocol.TField CLIENT_ADDRESS_FIELD_DESC = new org.apache.thrift.protocol.TField("clientAddress", org.apache.thrift.protocol.TType.STRING, (short)2);
|
||||
private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)3);
|
||||
private static final org.apache.thrift.protocol.TField USER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("userName", org.apache.thrift.protocol.TType.STRING, (short)4);
|
||||
private static final org.apache.thrift.protocol.TField LIMIT_FIELD_DESC = new org.apache.thrift.protocol.TField("limit", org.apache.thrift.protocol.TType.I32, (short)5);
|
||||
private static final org.apache.thrift.protocol.TField LOG_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("logType", org.apache.thrift.protocol.TType.I32, (short)6);
|
||||
|
||||
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TSlowLogQueryFilterStandardSchemeFactory();
|
||||
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TSlowLogQueryFilterTupleSchemeFactory();
|
||||
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TLogQueryFilterStandardSchemeFactory();
|
||||
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TLogQueryFilterTupleSchemeFactory();
|
||||
|
||||
public @org.apache.thrift.annotation.Nullable java.lang.String regionName; // optional
|
||||
public @org.apache.thrift.annotation.Nullable java.lang.String clientAddress; // optional
|
||||
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // optional
|
||||
public @org.apache.thrift.annotation.Nullable java.lang.String userName; // optional
|
||||
public int limit; // optional
|
||||
/**
|
||||
*
|
||||
* @see TLogType
|
||||
*/
|
||||
public @org.apache.thrift.annotation.Nullable TLogType logType; // 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 {
|
||||
|
@ -36,7 +42,12 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
CLIENT_ADDRESS((short)2, "clientAddress"),
|
||||
TABLE_NAME((short)3, "tableName"),
|
||||
USER_NAME((short)4, "userName"),
|
||||
LIMIT((short)5, "limit");
|
||||
LIMIT((short)5, "limit"),
|
||||
/**
|
||||
*
|
||||
* @see TLogType
|
||||
*/
|
||||
LOG_TYPE((short)6, "logType");
|
||||
|
||||
private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
|
||||
|
||||
|
@ -62,6 +73,8 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return USER_NAME;
|
||||
case 5: // LIMIT
|
||||
return LIMIT;
|
||||
case 6: // LOG_TYPE
|
||||
return LOG_TYPE;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
|
@ -105,7 +118,7 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
// isset id assignments
|
||||
private static final int __LIMIT_ISSET_ID = 0;
|
||||
private byte __isset_bitfield = 0;
|
||||
private static final _Fields optionals[] = {_Fields.REGION_NAME,_Fields.CLIENT_ADDRESS,_Fields.TABLE_NAME,_Fields.USER_NAME,_Fields.LIMIT};
|
||||
private static final _Fields optionals[] = {_Fields.REGION_NAME,_Fields.CLIENT_ADDRESS,_Fields.TABLE_NAME,_Fields.USER_NAME,_Fields.LIMIT,_Fields.LOG_TYPE};
|
||||
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);
|
||||
|
@ -119,19 +132,23 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
|
||||
tmpMap.put(_Fields.LIMIT, new org.apache.thrift.meta_data.FieldMetaData("limit", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
|
||||
tmpMap.put(_Fields.LOG_TYPE, new org.apache.thrift.meta_data.FieldMetaData("logType", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TLogType.class)));
|
||||
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSlowLogQueryFilter.class, metaDataMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TLogQueryFilter.class, metaDataMap);
|
||||
}
|
||||
|
||||
public TSlowLogQueryFilter() {
|
||||
public TLogQueryFilter() {
|
||||
this.limit = 10;
|
||||
|
||||
this.logType = org.apache.hadoop.hbase.thrift2.generated.TLogType.SLOW_LOG;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs a deep copy on <i>other</i>.
|
||||
*/
|
||||
public TSlowLogQueryFilter(TSlowLogQueryFilter other) {
|
||||
public TLogQueryFilter(TLogQueryFilter other) {
|
||||
__isset_bitfield = other.__isset_bitfield;
|
||||
if (other.isSetRegionName()) {
|
||||
this.regionName = other.regionName;
|
||||
|
@ -146,10 +163,13 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
this.userName = other.userName;
|
||||
}
|
||||
this.limit = other.limit;
|
||||
if (other.isSetLogType()) {
|
||||
this.logType = other.logType;
|
||||
}
|
||||
}
|
||||
|
||||
public TSlowLogQueryFilter deepCopy() {
|
||||
return new TSlowLogQueryFilter(this);
|
||||
public TLogQueryFilter deepCopy() {
|
||||
return new TLogQueryFilter(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -160,6 +180,8 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
this.userName = null;
|
||||
this.limit = 10;
|
||||
|
||||
this.logType = org.apache.hadoop.hbase.thrift2.generated.TLogType.SLOW_LOG;
|
||||
|
||||
}
|
||||
|
||||
@org.apache.thrift.annotation.Nullable
|
||||
|
@ -167,7 +189,7 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return this.regionName;
|
||||
}
|
||||
|
||||
public TSlowLogQueryFilter setRegionName(@org.apache.thrift.annotation.Nullable java.lang.String regionName) {
|
||||
public TLogQueryFilter setRegionName(@org.apache.thrift.annotation.Nullable java.lang.String regionName) {
|
||||
this.regionName = regionName;
|
||||
return this;
|
||||
}
|
||||
|
@ -192,7 +214,7 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return this.clientAddress;
|
||||
}
|
||||
|
||||
public TSlowLogQueryFilter setClientAddress(@org.apache.thrift.annotation.Nullable java.lang.String clientAddress) {
|
||||
public TLogQueryFilter setClientAddress(@org.apache.thrift.annotation.Nullable java.lang.String clientAddress) {
|
||||
this.clientAddress = clientAddress;
|
||||
return this;
|
||||
}
|
||||
|
@ -217,7 +239,7 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return this.tableName;
|
||||
}
|
||||
|
||||
public TSlowLogQueryFilter setTableName(@org.apache.thrift.annotation.Nullable java.lang.String tableName) {
|
||||
public TLogQueryFilter setTableName(@org.apache.thrift.annotation.Nullable java.lang.String tableName) {
|
||||
this.tableName = tableName;
|
||||
return this;
|
||||
}
|
||||
|
@ -242,7 +264,7 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return this.userName;
|
||||
}
|
||||
|
||||
public TSlowLogQueryFilter setUserName(@org.apache.thrift.annotation.Nullable java.lang.String userName) {
|
||||
public TLogQueryFilter setUserName(@org.apache.thrift.annotation.Nullable java.lang.String userName) {
|
||||
this.userName = userName;
|
||||
return this;
|
||||
}
|
||||
|
@ -266,7 +288,7 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return this.limit;
|
||||
}
|
||||
|
||||
public TSlowLogQueryFilter setLimit(int limit) {
|
||||
public TLogQueryFilter setLimit(int limit) {
|
||||
this.limit = limit;
|
||||
setLimitIsSet(true);
|
||||
return this;
|
||||
|
@ -285,6 +307,39 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
__isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __LIMIT_ISSET_ID, value);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @see TLogType
|
||||
*/
|
||||
@org.apache.thrift.annotation.Nullable
|
||||
public TLogType getLogType() {
|
||||
return this.logType;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @see TLogType
|
||||
*/
|
||||
public TLogQueryFilter setLogType(@org.apache.thrift.annotation.Nullable TLogType logType) {
|
||||
this.logType = logType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public void unsetLogType() {
|
||||
this.logType = null;
|
||||
}
|
||||
|
||||
/** Returns true if field logType is set (has been assigned a value) and false otherwise */
|
||||
public boolean isSetLogType() {
|
||||
return this.logType != null;
|
||||
}
|
||||
|
||||
public void setLogTypeIsSet(boolean value) {
|
||||
if (!value) {
|
||||
this.logType = null;
|
||||
}
|
||||
}
|
||||
|
||||
public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
|
||||
switch (field) {
|
||||
case REGION_NAME:
|
||||
|
@ -327,6 +382,14 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
}
|
||||
break;
|
||||
|
||||
case LOG_TYPE:
|
||||
if (value == null) {
|
||||
unsetLogType();
|
||||
} else {
|
||||
setLogType((TLogType)value);
|
||||
}
|
||||
break;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -348,6 +411,9 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
case LIMIT:
|
||||
return getLimit();
|
||||
|
||||
case LOG_TYPE:
|
||||
return getLogType();
|
||||
|
||||
}
|
||||
throw new java.lang.IllegalStateException();
|
||||
}
|
||||
|
@ -369,6 +435,8 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return isSetUserName();
|
||||
case LIMIT:
|
||||
return isSetLimit();
|
||||
case LOG_TYPE:
|
||||
return isSetLogType();
|
||||
}
|
||||
throw new java.lang.IllegalStateException();
|
||||
}
|
||||
|
@ -377,12 +445,12 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
public boolean equals(java.lang.Object that) {
|
||||
if (that == null)
|
||||
return false;
|
||||
if (that instanceof TSlowLogQueryFilter)
|
||||
return this.equals((TSlowLogQueryFilter)that);
|
||||
if (that instanceof TLogQueryFilter)
|
||||
return this.equals((TLogQueryFilter)that);
|
||||
return false;
|
||||
}
|
||||
|
||||
public boolean equals(TSlowLogQueryFilter that) {
|
||||
public boolean equals(TLogQueryFilter that) {
|
||||
if (that == null)
|
||||
return false;
|
||||
if (this == that)
|
||||
|
@ -433,6 +501,15 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return false;
|
||||
}
|
||||
|
||||
boolean this_present_logType = true && this.isSetLogType();
|
||||
boolean that_present_logType = true && that.isSetLogType();
|
||||
if (this_present_logType || that_present_logType) {
|
||||
if (!(this_present_logType && that_present_logType))
|
||||
return false;
|
||||
if (!this.logType.equals(that.logType))
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -460,11 +537,15 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
if (isSetLimit())
|
||||
hashCode = hashCode * 8191 + limit;
|
||||
|
||||
hashCode = hashCode * 8191 + ((isSetLogType()) ? 131071 : 524287);
|
||||
if (isSetLogType())
|
||||
hashCode = hashCode * 8191 + logType.getValue();
|
||||
|
||||
return hashCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(TSlowLogQueryFilter other) {
|
||||
public int compareTo(TLogQueryFilter other) {
|
||||
if (!getClass().equals(other.getClass())) {
|
||||
return getClass().getName().compareTo(other.getClass().getName());
|
||||
}
|
||||
|
@ -521,6 +602,16 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
return lastComparison;
|
||||
}
|
||||
}
|
||||
lastComparison = java.lang.Boolean.valueOf(isSetLogType()).compareTo(other.isSetLogType());
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
if (isSetLogType()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.logType, other.logType);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -539,7 +630,7 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
|
||||
@Override
|
||||
public java.lang.String toString() {
|
||||
java.lang.StringBuilder sb = new java.lang.StringBuilder("TSlowLogQueryFilter(");
|
||||
java.lang.StringBuilder sb = new java.lang.StringBuilder("TLogQueryFilter(");
|
||||
boolean first = true;
|
||||
|
||||
if (isSetRegionName()) {
|
||||
|
@ -587,6 +678,16 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
sb.append(this.limit);
|
||||
first = false;
|
||||
}
|
||||
if (isSetLogType()) {
|
||||
if (!first) sb.append(", ");
|
||||
sb.append("logType:");
|
||||
if (this.logType == null) {
|
||||
sb.append("null");
|
||||
} else {
|
||||
sb.append(this.logType);
|
||||
}
|
||||
first = false;
|
||||
}
|
||||
sb.append(")");
|
||||
return sb.toString();
|
||||
}
|
||||
|
@ -614,15 +715,15 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
}
|
||||
}
|
||||
|
||||
private static class TSlowLogQueryFilterStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
|
||||
public TSlowLogQueryFilterStandardScheme getScheme() {
|
||||
return new TSlowLogQueryFilterStandardScheme();
|
||||
private static class TLogQueryFilterStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
|
||||
public TLogQueryFilterStandardScheme getScheme() {
|
||||
return new TLogQueryFilterStandardScheme();
|
||||
}
|
||||
}
|
||||
|
||||
private static class TSlowLogQueryFilterStandardScheme extends org.apache.thrift.scheme.StandardScheme<TSlowLogQueryFilter> {
|
||||
private static class TLogQueryFilterStandardScheme extends org.apache.thrift.scheme.StandardScheme<TLogQueryFilter> {
|
||||
|
||||
public void read(org.apache.thrift.protocol.TProtocol iprot, TSlowLogQueryFilter struct) throws org.apache.thrift.TException {
|
||||
public void read(org.apache.thrift.protocol.TProtocol iprot, TLogQueryFilter struct) throws org.apache.thrift.TException {
|
||||
org.apache.thrift.protocol.TField schemeField;
|
||||
iprot.readStructBegin();
|
||||
while (true)
|
||||
|
@ -672,6 +773,14 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
case 6: // LOG_TYPE
|
||||
if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
|
||||
struct.logType = org.apache.hadoop.hbase.thrift2.generated.TLogType.findByValue(iprot.readI32());
|
||||
struct.setLogTypeIsSet(true);
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
|
@ -683,7 +792,7 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
struct.validate();
|
||||
}
|
||||
|
||||
public void write(org.apache.thrift.protocol.TProtocol oprot, TSlowLogQueryFilter struct) throws org.apache.thrift.TException {
|
||||
public void write(org.apache.thrift.protocol.TProtocol oprot, TLogQueryFilter struct) throws org.apache.thrift.TException {
|
||||
struct.validate();
|
||||
|
||||
oprot.writeStructBegin(STRUCT_DESC);
|
||||
|
@ -720,22 +829,29 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
oprot.writeI32(struct.limit);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
if (struct.logType != null) {
|
||||
if (struct.isSetLogType()) {
|
||||
oprot.writeFieldBegin(LOG_TYPE_FIELD_DESC);
|
||||
oprot.writeI32(struct.logType.getValue());
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
}
|
||||
oprot.writeFieldStop();
|
||||
oprot.writeStructEnd();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class TSlowLogQueryFilterTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
|
||||
public TSlowLogQueryFilterTupleScheme getScheme() {
|
||||
return new TSlowLogQueryFilterTupleScheme();
|
||||
private static class TLogQueryFilterTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
|
||||
public TLogQueryFilterTupleScheme getScheme() {
|
||||
return new TLogQueryFilterTupleScheme();
|
||||
}
|
||||
}
|
||||
|
||||
private static class TSlowLogQueryFilterTupleScheme extends org.apache.thrift.scheme.TupleScheme<TSlowLogQueryFilter> {
|
||||
private static class TLogQueryFilterTupleScheme extends org.apache.thrift.scheme.TupleScheme<TLogQueryFilter> {
|
||||
|
||||
@Override
|
||||
public void write(org.apache.thrift.protocol.TProtocol prot, TSlowLogQueryFilter struct) throws org.apache.thrift.TException {
|
||||
public void write(org.apache.thrift.protocol.TProtocol prot, TLogQueryFilter struct) throws org.apache.thrift.TException {
|
||||
org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
|
||||
java.util.BitSet optionals = new java.util.BitSet();
|
||||
if (struct.isSetRegionName()) {
|
||||
|
@ -753,7 +869,10 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
if (struct.isSetLimit()) {
|
||||
optionals.set(4);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 5);
|
||||
if (struct.isSetLogType()) {
|
||||
optionals.set(5);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 6);
|
||||
if (struct.isSetRegionName()) {
|
||||
oprot.writeString(struct.regionName);
|
||||
}
|
||||
|
@ -769,12 +888,15 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
if (struct.isSetLimit()) {
|
||||
oprot.writeI32(struct.limit);
|
||||
}
|
||||
if (struct.isSetLogType()) {
|
||||
oprot.writeI32(struct.logType.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void read(org.apache.thrift.protocol.TProtocol prot, TSlowLogQueryFilter struct) throws org.apache.thrift.TException {
|
||||
public void read(org.apache.thrift.protocol.TProtocol prot, TLogQueryFilter struct) throws org.apache.thrift.TException {
|
||||
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
|
||||
java.util.BitSet incoming = iprot.readBitSet(5);
|
||||
java.util.BitSet incoming = iprot.readBitSet(6);
|
||||
if (incoming.get(0)) {
|
||||
struct.regionName = iprot.readString();
|
||||
struct.setRegionNameIsSet(true);
|
||||
|
@ -795,6 +917,10 @@ public class TSlowLogQueryFilter implements org.apache.thrift.TBase<TSlowLogQuer
|
|||
struct.limit = iprot.readI32();
|
||||
struct.setLimitIsSet(true);
|
||||
}
|
||||
if (incoming.get(5)) {
|
||||
struct.logType = org.apache.hadoop.hbase.thrift2.generated.TLogType.findByValue(iprot.readI32());
|
||||
struct.setLogTypeIsSet(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,43 @@
|
|||
/**
|
||||
* Autogenerated by Thrift Compiler (0.13.0)
|
||||
*
|
||||
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
|
||||
* @generated
|
||||
*/
|
||||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TLogType implements org.apache.thrift.TEnum {
|
||||
SLOW_LOG(1),
|
||||
LARGE_LOG(2);
|
||||
|
||||
private final int value;
|
||||
|
||||
private TLogType(int value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the integer value of this enum value, as defined in the Thrift IDL.
|
||||
*/
|
||||
public int getValue() {
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Find a the enum type by its integer value, as defined in the Thrift IDL.
|
||||
* @return null if the value is not found.
|
||||
*/
|
||||
@org.apache.thrift.annotation.Nullable
|
||||
public static TLogType findByValue(int value) {
|
||||
switch (value) {
|
||||
case 1:
|
||||
return SLOW_LOG;
|
||||
case 2:
|
||||
return LARGE_LOG;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* Atomic mutation for the specified row. It can be either Put or Delete.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TMutation extends org.apache.thrift.TUnion<TMutation, TMutation._Fields> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMutation");
|
||||
private static final org.apache.thrift.protocol.TField PUT_FIELD_DESC = new org.apache.thrift.protocol.TField("put", org.apache.thrift.protocol.TType.STRUCT, (short)1);
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.NamespaceDescriptor
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TNamespaceDescriptor implements org.apache.thrift.TBase<TNamespaceDescriptor, TNamespaceDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TNamespaceDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TNamespaceDescriptor");
|
||||
|
||||
|
|
|
@ -9,11 +9,11 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
/**
|
||||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.SlowLogRecord
|
||||
* org.apache.hadoop.hbase.client.OnlineLogRecordrd
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, TSlowLogRecord._Fields>, java.io.Serializable, Cloneable, Comparable<TSlowLogRecord> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSlowLogRecord");
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
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");
|
||||
|
||||
private static final org.apache.thrift.protocol.TField START_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("startTime", org.apache.thrift.protocol.TType.I64, (short)1);
|
||||
private static final org.apache.thrift.protocol.TField PROCESSING_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("processingTime", org.apache.thrift.protocol.TType.I32, (short)2);
|
||||
|
@ -30,8 +30,8 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TSlowLogRecordStandardSchemeFactory();
|
||||
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TSlowLogRecordTupleSchemeFactory();
|
||||
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();
|
||||
|
||||
public long startTime; // required
|
||||
public int processingTime; // required
|
||||
|
@ -189,13 +189,13 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
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)));
|
||||
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSlowLogRecord.class, metaDataMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOnlineLogRecord.class, metaDataMap);
|
||||
}
|
||||
|
||||
public TSlowLogRecord() {
|
||||
public TOnlineLogRecord() {
|
||||
}
|
||||
|
||||
public TSlowLogRecord(
|
||||
public TOnlineLogRecord(
|
||||
long startTime,
|
||||
int processingTime,
|
||||
int queueTime,
|
||||
|
@ -236,7 +236,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
/**
|
||||
* Performs a deep copy on <i>other</i>.
|
||||
*/
|
||||
public TSlowLogRecord(TSlowLogRecord other) {
|
||||
public TOnlineLogRecord(TOnlineLogRecord other) {
|
||||
__isset_bitfield = other.__isset_bitfield;
|
||||
this.startTime = other.startTime;
|
||||
this.processingTime = other.processingTime;
|
||||
|
@ -268,8 +268,8 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
}
|
||||
}
|
||||
|
||||
public TSlowLogRecord deepCopy() {
|
||||
return new TSlowLogRecord(this);
|
||||
public TOnlineLogRecord deepCopy() {
|
||||
return new TOnlineLogRecord(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -301,7 +301,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.startTime;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setStartTime(long startTime) {
|
||||
public TOnlineLogRecord setStartTime(long startTime) {
|
||||
this.startTime = startTime;
|
||||
setStartTimeIsSet(true);
|
||||
return this;
|
||||
|
@ -324,7 +324,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.processingTime;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setProcessingTime(int processingTime) {
|
||||
public TOnlineLogRecord setProcessingTime(int processingTime) {
|
||||
this.processingTime = processingTime;
|
||||
setProcessingTimeIsSet(true);
|
||||
return this;
|
||||
|
@ -347,7 +347,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.queueTime;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setQueueTime(int queueTime) {
|
||||
public TOnlineLogRecord setQueueTime(int queueTime) {
|
||||
this.queueTime = queueTime;
|
||||
setQueueTimeIsSet(true);
|
||||
return this;
|
||||
|
@ -370,7 +370,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.responseSize;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setResponseSize(long responseSize) {
|
||||
public TOnlineLogRecord setResponseSize(long responseSize) {
|
||||
this.responseSize = responseSize;
|
||||
setResponseSizeIsSet(true);
|
||||
return this;
|
||||
|
@ -394,7 +394,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.clientAddress;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setClientAddress(@org.apache.thrift.annotation.Nullable java.lang.String clientAddress) {
|
||||
public TOnlineLogRecord setClientAddress(@org.apache.thrift.annotation.Nullable java.lang.String clientAddress) {
|
||||
this.clientAddress = clientAddress;
|
||||
return this;
|
||||
}
|
||||
|
@ -419,7 +419,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.serverClass;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setServerClass(@org.apache.thrift.annotation.Nullable java.lang.String serverClass) {
|
||||
public TOnlineLogRecord setServerClass(@org.apache.thrift.annotation.Nullable java.lang.String serverClass) {
|
||||
this.serverClass = serverClass;
|
||||
return this;
|
||||
}
|
||||
|
@ -444,7 +444,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.methodName;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setMethodName(@org.apache.thrift.annotation.Nullable java.lang.String methodName) {
|
||||
public TOnlineLogRecord setMethodName(@org.apache.thrift.annotation.Nullable java.lang.String methodName) {
|
||||
this.methodName = methodName;
|
||||
return this;
|
||||
}
|
||||
|
@ -469,7 +469,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.callDetails;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setCallDetails(@org.apache.thrift.annotation.Nullable java.lang.String callDetails) {
|
||||
public TOnlineLogRecord setCallDetails(@org.apache.thrift.annotation.Nullable java.lang.String callDetails) {
|
||||
this.callDetails = callDetails;
|
||||
return this;
|
||||
}
|
||||
|
@ -494,7 +494,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.param;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setParam(@org.apache.thrift.annotation.Nullable java.lang.String param) {
|
||||
public TOnlineLogRecord setParam(@org.apache.thrift.annotation.Nullable java.lang.String param) {
|
||||
this.param = param;
|
||||
return this;
|
||||
}
|
||||
|
@ -519,7 +519,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.userName;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setUserName(@org.apache.thrift.annotation.Nullable java.lang.String userName) {
|
||||
public TOnlineLogRecord setUserName(@org.apache.thrift.annotation.Nullable java.lang.String userName) {
|
||||
this.userName = userName;
|
||||
return this;
|
||||
}
|
||||
|
@ -543,7 +543,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.multiGetsCount;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setMultiGetsCount(int multiGetsCount) {
|
||||
public TOnlineLogRecord setMultiGetsCount(int multiGetsCount) {
|
||||
this.multiGetsCount = multiGetsCount;
|
||||
setMultiGetsCountIsSet(true);
|
||||
return this;
|
||||
|
@ -566,7 +566,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.multiMutationsCount;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setMultiMutationsCount(int multiMutationsCount) {
|
||||
public TOnlineLogRecord setMultiMutationsCount(int multiMutationsCount) {
|
||||
this.multiMutationsCount = multiMutationsCount;
|
||||
setMultiMutationsCountIsSet(true);
|
||||
return this;
|
||||
|
@ -589,7 +589,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.multiServiceCalls;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setMultiServiceCalls(int multiServiceCalls) {
|
||||
public TOnlineLogRecord setMultiServiceCalls(int multiServiceCalls) {
|
||||
this.multiServiceCalls = multiServiceCalls;
|
||||
setMultiServiceCallsIsSet(true);
|
||||
return this;
|
||||
|
@ -613,7 +613,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
return this.regionName;
|
||||
}
|
||||
|
||||
public TSlowLogRecord setRegionName(@org.apache.thrift.annotation.Nullable java.lang.String regionName) {
|
||||
public TOnlineLogRecord setRegionName(@org.apache.thrift.annotation.Nullable java.lang.String regionName) {
|
||||
this.regionName = regionName;
|
||||
return this;
|
||||
}
|
||||
|
@ -842,12 +842,12 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
public boolean equals(java.lang.Object that) {
|
||||
if (that == null)
|
||||
return false;
|
||||
if (that instanceof TSlowLogRecord)
|
||||
return this.equals((TSlowLogRecord)that);
|
||||
if (that instanceof TOnlineLogRecord)
|
||||
return this.equals((TOnlineLogRecord)that);
|
||||
return false;
|
||||
}
|
||||
|
||||
public boolean equals(TSlowLogRecord that) {
|
||||
public boolean equals(TOnlineLogRecord that) {
|
||||
if (that == null)
|
||||
return false;
|
||||
if (this == that)
|
||||
|
@ -1032,7 +1032,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(TSlowLogRecord other) {
|
||||
public int compareTo(TOnlineLogRecord other) {
|
||||
if (!getClass().equals(other.getClass())) {
|
||||
return getClass().getName().compareTo(other.getClass().getName());
|
||||
}
|
||||
|
@ -1197,7 +1197,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
|
||||
@Override
|
||||
public java.lang.String toString() {
|
||||
java.lang.StringBuilder sb = new java.lang.StringBuilder("TSlowLogRecord(");
|
||||
java.lang.StringBuilder sb = new java.lang.StringBuilder("TOnlineLogRecord(");
|
||||
boolean first = true;
|
||||
|
||||
sb.append("startTime:");
|
||||
|
@ -1337,15 +1337,15 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
}
|
||||
}
|
||||
|
||||
private static class TSlowLogRecordStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
|
||||
public TSlowLogRecordStandardScheme getScheme() {
|
||||
return new TSlowLogRecordStandardScheme();
|
||||
private static class TOnlineLogRecordStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
|
||||
public TOnlineLogRecordStandardScheme getScheme() {
|
||||
return new TOnlineLogRecordStandardScheme();
|
||||
}
|
||||
}
|
||||
|
||||
private static class TSlowLogRecordStandardScheme extends org.apache.thrift.scheme.StandardScheme<TSlowLogRecord> {
|
||||
private static class TOnlineLogRecordStandardScheme extends org.apache.thrift.scheme.StandardScheme<TOnlineLogRecord> {
|
||||
|
||||
public void read(org.apache.thrift.protocol.TProtocol iprot, TSlowLogRecord struct) throws org.apache.thrift.TException {
|
||||
public void read(org.apache.thrift.protocol.TProtocol iprot, TOnlineLogRecord struct) throws org.apache.thrift.TException {
|
||||
org.apache.thrift.protocol.TField schemeField;
|
||||
iprot.readStructBegin();
|
||||
while (true)
|
||||
|
@ -1499,7 +1499,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
struct.validate();
|
||||
}
|
||||
|
||||
public void write(org.apache.thrift.protocol.TProtocol oprot, TSlowLogRecord struct) throws org.apache.thrift.TException {
|
||||
public void write(org.apache.thrift.protocol.TProtocol oprot, TOnlineLogRecord struct) throws org.apache.thrift.TException {
|
||||
struct.validate();
|
||||
|
||||
oprot.writeStructBegin(STRUCT_DESC);
|
||||
|
@ -1567,16 +1567,16 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
|
||||
}
|
||||
|
||||
private static class TSlowLogRecordTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
|
||||
public TSlowLogRecordTupleScheme getScheme() {
|
||||
return new TSlowLogRecordTupleScheme();
|
||||
private static class TOnlineLogRecordTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
|
||||
public TOnlineLogRecordTupleScheme getScheme() {
|
||||
return new TOnlineLogRecordTupleScheme();
|
||||
}
|
||||
}
|
||||
|
||||
private static class TSlowLogRecordTupleScheme extends org.apache.thrift.scheme.TupleScheme<TSlowLogRecord> {
|
||||
private static class TOnlineLogRecordTupleScheme extends org.apache.thrift.scheme.TupleScheme<TOnlineLogRecord> {
|
||||
|
||||
@Override
|
||||
public void write(org.apache.thrift.protocol.TProtocol prot, TSlowLogRecord struct) throws org.apache.thrift.TException {
|
||||
public void write(org.apache.thrift.protocol.TProtocol prot, TOnlineLogRecord struct) throws org.apache.thrift.TException {
|
||||
org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
|
||||
oprot.writeI64(struct.startTime);
|
||||
oprot.writeI32(struct.processingTime);
|
||||
|
@ -1602,7 +1602,7 @@ public class TSlowLogRecord implements org.apache.thrift.TBase<TSlowLogRecord, T
|
|||
}
|
||||
|
||||
@Override
|
||||
public void read(org.apache.thrift.protocol.TProtocol prot, TSlowLogRecord struct) throws org.apache.thrift.TException {
|
||||
public void read(org.apache.thrift.protocol.TProtocol prot, TOnlineLogRecord struct) throws org.apache.thrift.TException {
|
||||
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
|
||||
struct.startTime = iprot.readI64();
|
||||
struct.setStartTimeIsSet(true);
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TPut implements org.apache.thrift.TBase<TPut, TPut._Fields>, java.io.Serializable, Cloneable, Comparable<TPut> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPut");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TReadType implements org.apache.thrift.TEnum {
|
||||
DEFAULT(1),
|
||||
STREAM(2),
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* if no Result is found, row and columnValues will not be set.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields>, java.io.Serializable, Cloneable, Comparable<TResult> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TResult");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* A TRowMutations object is used to apply a number of Mutations to a single row.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRowMutations._Fields>, java.io.Serializable, Cloneable, Comparable<TRowMutations> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowMutations");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Any timestamps in the columns are ignored but the colFamTimeRangeMap included, use timeRange to select by timestamp.
|
||||
* Max versions defaults to 1.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TServerName implements org.apache.thrift.TBase<TServerName, TServerName._Fields>, java.io.Serializable, Cloneable, Comparable<TServerName> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TServerName");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.TableDescriptor
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TTableDescriptor implements org.apache.thrift.TBase<TTableDescriptor, TTableDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TTableDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableDescriptor");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.TableName
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TTableName implements org.apache.thrift.TBase<TTableName, TTableName._Fields>, java.io.Serializable, Cloneable, Comparable<TTableName> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableName");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* Specify type of thrift server: thrift and thrift2
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public enum TThriftServerType implements org.apache.thrift.TEnum {
|
||||
ONE(1),
|
||||
TWO(2);
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-21")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
|
||||
public class TTimeRange implements org.apache.thrift.TBase<TTimeRange, TTimeRange._Fields>, java.io.Serializable, Cloneable, Comparable<TTimeRange> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTimeRange");
|
||||
|
||||
|
|
|
@ -454,24 +454,30 @@ struct TNamespaceDescriptor {
|
|||
2: optional map<string, string> configuration
|
||||
}
|
||||
|
||||
enum TLogType {
|
||||
SLOW_LOG = 1,
|
||||
LARGE_LOG = 2
|
||||
}
|
||||
|
||||
/**
|
||||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.SlowLogQueryFilter
|
||||
* org.apache.hadoop.hbase.client.LogQueryFilter
|
||||
*/
|
||||
struct TSlowLogQueryFilter {
|
||||
struct TLogQueryFilter {
|
||||
1: optional string regionName
|
||||
2: optional string clientAddress
|
||||
3: optional string tableName
|
||||
4: optional string userName
|
||||
5: optional i32 limit = 10
|
||||
6: optional TLogType logType = 1
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.SlowLogRecord
|
||||
* org.apache.hadoop.hbase.client.OnlineLogRecordrd
|
||||
*/
|
||||
struct TSlowLogRecord {
|
||||
struct TOnlineLogRecord {
|
||||
1: required i64 startTime
|
||||
2: required i32 processingTime
|
||||
3: required i32 queueTime
|
||||
|
@ -1099,15 +1105,15 @@ service THBaseService {
|
|||
* @return online slowlog response list
|
||||
* @throws TIOError if a remote or network exception occurs
|
||||
*/
|
||||
list<TSlowLogRecord> getSlowLogResponses(
|
||||
list<TOnlineLogRecord> getSlowLogResponses(
|
||||
/** @param serverNames Server names to get slowlog responses from */
|
||||
1: set<TServerName> serverNames
|
||||
/** @param slowLogQueryFilter filter to be used if provided */
|
||||
2: TSlowLogQueryFilter slowLogQueryFilter
|
||||
/** @param logQueryFilter filter to be used if provided */
|
||||
2: TLogQueryFilter logQueryFilter
|
||||
) throws (1: TIOError io)
|
||||
|
||||
/**
|
||||
* Clears online slow RPC logs from the provided list of
|
||||
* Clears online slow/large RPC logs from the provided list of
|
||||
* RegionServers
|
||||
*
|
||||
* @return List of booleans representing if online slowlog response buffer is cleaned
|
||||
|
|
|
@ -97,16 +97,16 @@ import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
|
|||
import org.apache.hadoop.hbase.thrift2.generated.TIOError;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TIllegalArgument;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TMutation;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TOnlineLogRecord;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TPut;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TReadType;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TResult;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TScan;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TServerName;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TSlowLogQueryFilter;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TSlowLogRecord;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableName;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TThriftServerType;
|
||||
|
@ -1765,11 +1765,11 @@ public class TestThriftHBaseServiceHandler {
|
|||
List<Boolean> clearedResponses =
|
||||
thriftHBaseServiceHandler.clearSlowLogResponses(tServerNames);
|
||||
clearedResponses.forEach(Assert::assertTrue);
|
||||
TSlowLogQueryFilter tSlowLogQueryFilter = new TSlowLogQueryFilter();
|
||||
tSlowLogQueryFilter.setLimit(15);
|
||||
List<TSlowLogRecord> tSlowLogRecords =
|
||||
thriftHBaseServiceHandler.getSlowLogResponses(tServerNames, tSlowLogQueryFilter);
|
||||
assertEquals(tSlowLogRecords.size(), 0);
|
||||
TLogQueryFilter tLogQueryFilter = new TLogQueryFilter();
|
||||
tLogQueryFilter.setLimit(15);
|
||||
List<TOnlineLogRecord> tLogRecords =
|
||||
thriftHBaseServiceHandler.getSlowLogResponses(tServerNames, tLogQueryFilter);
|
||||
assertEquals(tLogRecords.size(), 0);
|
||||
}
|
||||
|
||||
public static class DelayingRegionObserver implements RegionCoprocessor, RegionObserver {
|
||||
|
|
Loading…
Reference in New Issue