You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by vj...@apache.org on 2020/04/01 14:27:19 UTC

[hbase] branch branch-2 updated: HBASE-23937 : Support Online LargeLogs similar to SlowLogs APIs (#1346)

This is an automated email from the ASF dual-hosted git repository.

vjasani pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 3433c7a  HBASE-23937 : Support Online LargeLogs similar to SlowLogs APIs (#1346)
3433c7a is described below

commit 3433c7a2db6ab1c832ba9dd43129764d337da183
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Wed Apr 1 19:52:50 2020 +0530

    HBASE-23937 : Support Online LargeLogs similar to SlowLogs APIs (#1346)
    
    Signed-off-by: Bharath Vissapragada <bh...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java |  10 +-
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |   4 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |   6 +-
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |  36 ++--
 ...SlowLogQueryFilter.java => LogQueryFilter.java} |  25 ++-
 .../{SlowLogRecord.java => OnlineLogRecord.java}   |  45 ++---
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  47 +++--
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |  16 +-
 .../hbase/shaded/protobuf/RequestConverter.java    |  18 +-
 .../src/main/protobuf/Admin.proto                  |   3 +
 .../src/main/protobuf/TooSlowLog.proto             |  11 ++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java     |   5 +-
 .../hadoop/hbase/regionserver/RSRpcServices.java   |  18 +-
 ...owLogEventHandler.java => LogEventHandler.java} |  67 ++++++-
 .../hbase/regionserver/slowlog/RpcLogDetails.java  |  24 ++-
 .../regionserver/slowlog/SlowLogRecorder.java      |  35 ++--
 .../org/apache/hadoop/hbase/client/TestAdmin2.java |   8 +-
 .../hadoop/hbase/master/MockRegionServer.java      |   6 +
 .../regionserver/slowlog/TestSlowLogRecorder.java  | 104 +++++++++-
 hbase-shell/src/main/ruby/hbase/admin.rb           |  28 ++-
 hbase-shell/src/main/ruby/shell.rb                 |   1 +
 .../ruby/shell/commands/get_largelog_responses.rb  |  78 ++++++++
 .../hbase/thrift2/ThriftHBaseServiceHandler.java   |  22 +--
 .../hadoop/hbase/thrift2/ThriftUtilities.java      |  95 ++++-----
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |  23 +--
 .../hadoop/hbase/thrift2/generated/TAppend.java    |   2 +-
 .../hbase/thrift2/generated/TAuthorization.java    |   2 +-
 .../hbase/thrift2/generated/TBloomFilterType.java  |   2 +-
 .../hbase/thrift2/generated/TCellVisibility.java   |   2 +-
 .../hadoop/hbase/thrift2/generated/TColumn.java    |   2 +-
 .../thrift2/generated/TColumnFamilyDescriptor.java |   2 +-
 .../hbase/thrift2/generated/TColumnIncrement.java  |   2 +-
 .../hbase/thrift2/generated/TColumnValue.java      |   2 +-
 .../hadoop/hbase/thrift2/generated/TCompareOp.java |   2 +-
 .../thrift2/generated/TCompressionAlgorithm.java   |   2 +-
 .../hbase/thrift2/generated/TConsistency.java      |   2 +-
 .../thrift2/generated/TDataBlockEncoding.java      |   2 +-
 .../hadoop/hbase/thrift2/generated/TDelete.java    |   2 +-
 .../hbase/thrift2/generated/TDeleteType.java       |   2 +-
 .../hbase/thrift2/generated/TDurability.java       |   2 +-
 .../hadoop/hbase/thrift2/generated/TGet.java       |   2 +-
 .../hbase/thrift2/generated/THBaseService.java     | 216 ++++++++++-----------
 .../hbase/thrift2/generated/THRegionInfo.java      |   2 +-
 .../hbase/thrift2/generated/THRegionLocation.java  |   2 +-
 .../hadoop/hbase/thrift2/generated/TIOError.java   |   2 +-
 .../hbase/thrift2/generated/TIllegalArgument.java  |   2 +-
 .../hadoop/hbase/thrift2/generated/TIncrement.java |   2 +-
 .../hbase/thrift2/generated/TKeepDeletedCells.java |   2 +-
 ...lowLogQueryFilter.java => TLogQueryFilter.java} | 200 +++++++++++++++----
 .../generated/{TReadType.java => TLogType.java}    |  19 +-
 .../hadoop/hbase/thrift2/generated/TMutation.java  |   2 +-
 .../thrift2/generated/TNamespaceDescriptor.java    |   2 +-
 .../{TSlowLogRecord.java => TOnlineLogRecord.java} |  86 ++++----
 .../hadoop/hbase/thrift2/generated/TPut.java       |   2 +-
 .../hadoop/hbase/thrift2/generated/TReadType.java  |   2 +-
 .../hadoop/hbase/thrift2/generated/TResult.java    |   2 +-
 .../hbase/thrift2/generated/TRowMutations.java     |   2 +-
 .../hadoop/hbase/thrift2/generated/TScan.java      |   2 +-
 .../hbase/thrift2/generated/TServerName.java       |   2 +-
 .../hbase/thrift2/generated/TTableDescriptor.java  |   2 +-
 .../hadoop/hbase/thrift2/generated/TTableName.java |   2 +-
 .../hbase/thrift2/generated/TThriftServerType.java |   2 +-
 .../hadoop/hbase/thrift2/generated/TTimeRange.java |   2 +-
 .../org/apache/hadoop/hbase/thrift2/hbase.thrift   |  22 ++-
 .../thrift2/TestThriftHBaseServiceHandler.java     |  14 +-
 65 files changed, 929 insertions(+), 431 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 4576d7d..66e62b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -3136,19 +3136,19 @@ public interface Admin extends Abortable, Closeable {
 
 
   /**
-   * 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
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 9e61a38..b69fcc6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -1492,8 +1492,8 @@ public interface AsyncAdmin {
    * @param slowLogQueryFilter 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 slowLogQueryFilter);
 
   /**
    * Clears online slow RPC logs from the provided list of
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index d5c9d09..55e0877 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -838,9 +838,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
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 7ef5d4d..4b1a738 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -4361,14 +4361,14 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public List<SlowLogRecord> getSlowLogResponses(@Nullable final Set<ServerName> serverNames,
-      final SlowLogQueryFilter slowLogQueryFilter) throws IOException {
+  public List<OnlineLogRecord> getSlowLogResponses(@Nullable final Set<ServerName> serverNames,
+      final LogQueryFilter logQueryFilter) throws IOException {
     if (CollectionUtils.isEmpty(serverNames)) {
       return Collections.emptyList();
     }
     return serverNames.stream().map(serverName -> {
         try {
-          return getSlowLogResponseFromServer(serverName, slowLogQueryFilter);
+          return getSlowLogResponseFromServer(serverName, logQueryFilter);
         } catch (IOException e) {
           throw new RuntimeException(e);
         }
@@ -4376,21 +4376,29 @@ public class HBaseAdmin implements Admin {
     ).flatMap(List::stream).collect(Collectors.toList());
   }
 
-  private List<SlowLogRecord> getSlowLogResponseFromServer(final ServerName serverName,
-      final SlowLogQueryFilter slowLogQueryFilter) throws IOException {
-    return getSlowLogResponsesFromServer(this.connection.getAdmin(serverName), slowLogQueryFilter);
+  private List<OnlineLogRecord> getSlowLogResponseFromServer(final ServerName serverName,
+      final LogQueryFilter logQueryFilter) throws IOException {
+    return getSlowLogResponsesFromServer(this.connection.getAdmin(serverName), logQueryFilter);
   }
 
-  private List<SlowLogRecord> getSlowLogResponsesFromServer(AdminService.BlockingInterface admin,
-      SlowLogQueryFilter slowLogQueryFilter) throws IOException {
-    return executeCallable(new RpcRetryingCallable<List<SlowLogRecord>>() {
+  private List<OnlineLogRecord> getSlowLogResponsesFromServer(AdminService.BlockingInterface admin,
+      LogQueryFilter logQueryFilter) throws IOException {
+    return executeCallable(new RpcRetryingCallable<List<OnlineLogRecord>>() {
       @Override
-      protected List<SlowLogRecord> rpcCall(int callTimeout) throws Exception {
+      protected List<OnlineLogRecord> rpcCall(int callTimeout) throws Exception {
         HBaseRpcController controller = rpcControllerFactory.newController();
-        AdminProtos.SlowLogResponses slowLogResponses =
-          admin.getSlowLogResponses(controller,
-            RequestConverter.buildSlowLogResponseRequest(slowLogQueryFilter));
-        return ProtobufUtil.toSlowLogPayloads(slowLogResponses);
+        if (logQueryFilter.getType() == null
+            || logQueryFilter.getType() == LogQueryFilter.Type.SLOW_LOG) {
+          AdminProtos.SlowLogResponses slowLogResponses =
+            admin.getSlowLogResponses(controller,
+              RequestConverter.buildSlowLogResponseRequest(logQueryFilter));
+          return ProtobufUtil.toSlowLogPayloads(slowLogResponses);
+        } else {
+          AdminProtos.SlowLogResponses slowLogResponses =
+            admin.getLargeLogResponses(controller,
+              RequestConverter.buildSlowLogResponseRequest(logQueryFilter));
+          return ProtobufUtil.toSlowLogPayloads(slowLogResponses);
+        }
       }
     });
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogQueryFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogQueryFilter.java
similarity index 86%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogQueryFilter.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogQueryFilter.java
index aa56a8a..8734f7b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogQueryFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogQueryFilter.java
@@ -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();
   }
-
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogRecord.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java
similarity index 83%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogRecord.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java
index 9593618..8af0013 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogRecord.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java
@@ -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)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index ddaf786..d98b02e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -3889,26 +3889,47 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<SlowLogRecord>> getSlowLogResponses(
-    @Nullable final Set<ServerName> serverNames,
-    final SlowLogQueryFilter slowLogQueryFilter) {
+  public CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(
+      @Nullable final Set<ServerName> serverNames,
+      final LogQueryFilter logQueryFilter) {
     if (CollectionUtils.isEmpty(serverNames)) {
       return CompletableFuture.completedFuture(Collections.emptyList());
     }
-    return CompletableFuture.supplyAsync(() -> serverNames.stream()
-      .map((ServerName serverName) ->
-        getSlowLogResponseFromServer(serverName, slowLogQueryFilter))
-      .map(CompletableFuture::join)
-      .flatMap(List::stream)
-      .collect(Collectors.toList()));
+    if (logQueryFilter.getType() == null
+      || logQueryFilter.getType() == LogQueryFilter.Type.SLOW_LOG) {
+      return CompletableFuture.supplyAsync(() -> serverNames.stream()
+        .map((ServerName serverName) ->
+          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<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();
   }
 
-  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();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index dc4091b..1c3a217 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -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;
@@ -3478,14 +3478,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())
@@ -3501,18 +3501,18 @@ 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> slowLogRecords = slowLogResponses.getSlowLogPayloadsList()
       .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
     return slowLogRecords;
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index f293bcf..63ef6f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -52,7 +52,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;
@@ -1930,32 +1930,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();
   }
 
   /**
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 34c9806..8dd5516 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -369,6 +369,9 @@ service AdminService {
   rpc GetSlowLogResponses(SlowLogResponseRequest)
     returns(SlowLogResponses);
 
+  rpc GetLargeLogResponses(SlowLogResponseRequest)
+    returns(SlowLogResponses);
+
   rpc ClearSlowLogsResponses(ClearSlowLogResponseRequest)
     returns(ClearSlowLogResponses);
 }
diff --git a/hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto b/hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto
index 26dabde..36ed9d5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto
@@ -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;
+  }
+
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 6f3089c..5d5b1c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -425,12 +425,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());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index ab1aea3..bc62757 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -3789,7 +3789,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   @Override
   @QosPriority(priority = HConstants.ADMIN_QOS)
   public SlowLogResponses getSlowLogResponses(final RpcController controller,
-    final SlowLogResponseRequest request) {
+      final SlowLogResponseRequest request) {
     final SlowLogRecorder slowLogRecorder =
       this.regionServer.getSlowLogRecorder();
     final List<SlowLogPayload> slowLogPayloads;
@@ -3804,6 +3804,22 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
   @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,
     final ClearSlowLogResponseRequest request) {
     final SlowLogRecorder slowLogRecorder =
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
similarity index 75%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
index 24e8460..508f086 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
@@ -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);
 
+    return getFilteredLogs(request, slowLogPayloadList);
+  }
+
+  /**
+   * 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)) {
-      slowLogPayloadList = filterSlowLogs(request, slowLogPayloadList);
+      logPayloadList = filterLogs(request, logPayloadList);
     }
-    int limit = request.getLimit() >= slowLogPayloadList.size() ? slowLogPayloadList.size()
+    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) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/RpcLogDetails.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/RpcLogDetails.java
index e7ab7d4..7d5558c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/RpcLogDetails.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/RpcLogDetails.java
@@ -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();
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogRecorder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogRecorder.java
index d750642..a69b0ad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogRecorder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogRecorder.java
@@ -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();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 89329c5..e24aafe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -887,12 +887,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);
   }
 
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index dbd1d9d..aaeae01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -688,6 +688,12 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   }
 
   @Override
+  public SlowLogResponses getLargeLogResponses(RpcController controller,
+      SlowLogResponseRequest request) throws ServiceException {
+    return null;
+  }
+
+  @Override
   public ClearSlowLogResponses clearSlowLogsResponses(RpcController controller,
       ClearSlowLogResponseRequest request) throws ServiceException {
     return null;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
index 72afdd4..bdd5c89 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
@@ -197,6 +197,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();
 
         LOG.debug("cleared the ringbuffer of Online Slow Log records");
@@ -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) {
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index c3b4a8e..79cc7da 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -1447,6 +1447,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 = []
@@ -1458,7 +1459,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)
@@ -1483,6 +1484,31 @@ module Hbase
     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)
       unless server_names.nil? || server_names.is_a?(Array) || server_names.is_a?(String)
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 6b441d6..9697c73 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -339,6 +339,7 @@ Shell.load_command_group(
     compaction_switch
     flush
     get_slowlog_responses
+    get_largelog_responses
     major_compact
     move
     split
diff --git a/hbase-shell/src/main/ruby/shell/commands/get_largelog_responses.rb b/hbase-shell/src/main/ruby/shell/commands/get_largelog_responses.rb
new file mode 100644
index 0000000..3bdf614
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/get_largelog_responses.rb
@@ -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
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
index 37ee9a1..4f8ce52 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
@@ -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;
@@ -826,15 +826,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);
     }
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
index 7ce75a85..56a753c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
@@ -54,15 +54,15 @@ 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.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;
@@ -1499,61 +1500,61 @@ public 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 slowLogQueryFilter) {
+    TLogQueryFilter tLogQueryFilter = new TLogQueryFilter();
+    tLogQueryFilter.setRegionName(slowLogQueryFilter.getRegionName());
+    tLogQueryFilter.setClientAddress(slowLogQueryFilter.getClientAddress());
+    tLogQueryFilter.setTableName(slowLogQueryFilter.getTableName());
+    tLogQueryFilter.setUserName(slowLogQueryFilter.getUserName());
+    tLogQueryFilter.setLimit(slowLogQueryFilter.getLimit());
+    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;
+  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());
+    return logQueryFilter;
   }
 
-  public static List<TSlowLogRecord> getSlowLogRecordsFromHBase(
-    List<SlowLogRecord> slowLogRecords) {
-    if (CollectionUtils.isEmpty(slowLogRecords)) {
+  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())
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index befdaa3..facdc62 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -45,9 +45,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;
@@ -1398,15 +1398,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);
     }
@@ -1427,4 +1427,5 @@ public class ThriftAdmin implements Admin {
   public Future<Void> splitRegionAsync(byte[] regionName) throws IOException {
     return splitRegionAsync(regionName, null);
   }
+
 }
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
index d51c490..dbc1765 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
index 5b21e71..a954df5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
index 38299e4..d8739d9 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
@@ -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-23")
+@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
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
index c6c9df0..641fdac 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
index 443dfbf..9bd9d24 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
index 7008872..124887f 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
index 934ce44..4ea28f6 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
index f09c4a5..feeeabe 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java
index f94db11..467aeeb 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * Thrift wrapper around
  * org.apache.hadoop.hbase.filter.CompareFilter$CompareOp.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-23")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
 public enum TCompareOp implements org.apache.thrift.TEnum {
   LESS(0),
   LESS_OR_EQUAL(1),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
index 3fe4dda..05ed3f1 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
@@ -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-23")
+@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),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java
index c38a98c..d4c1f92 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java
@@ -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-23")
+@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);
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
index cc47bf9..c28b607 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
@@ -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-23")
+@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.
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
index 2ac2c48..86d4458 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java
index 67cd2a8..7b8384d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java
@@ -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-23")
+@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),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
index 93bd821..3949251 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
@@ -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-23")
+@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),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
index 0457e1c..551dceb 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
index 95063f2..67e43a8 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
@@ -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-23")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-04-01")
 public class THBaseService {
 
   public interface Iface {
@@ -516,12 +516,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
@@ -630,7 +630,7 @@ public class THBaseService {
 
     public void getThriftServerType(org.apache.thrift.async.AsyncMethodCallback<TThriftServerType> 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;
 
@@ -1868,21 +1868,21 @@ public class THBaseService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getThriftServerType 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");
@@ -3557,32 +3557,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!");
         }
@@ -5083,7 +5083,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;
         }
@@ -8237,7 +8237,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");
       }
@@ -8246,10 +8246,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 {
@@ -8297,8 +8297,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);
       }
     }
 
@@ -52457,7 +52457,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();
@@ -52467,9 +52467,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 {
@@ -52478,9 +52478,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>();
 
@@ -52498,8 +52498,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;
         }
@@ -52547,8 +52547,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);
     }
@@ -52558,11 +52558,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;
     }
 
     /**
@@ -52576,8 +52576,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);
       }
     }
 
@@ -52588,7 +52588,7 @@ public class THBaseService {
     @Override
     public void clear() {
       this.serverNames = null;
-      this.slowLogQueryFilter = null;
+      this.logQueryFilter = null;
     }
 
     public int getServerNamesSize() {
@@ -52639,33 +52639,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;
       }
     }
 
@@ -52679,11 +52679,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;
 
@@ -52696,8 +52696,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();
@@ -52712,8 +52712,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();
     }
@@ -52742,12 +52742,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;
       }
 
@@ -52762,9 +52762,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;
     }
@@ -52787,12 +52787,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;
         }
@@ -52826,11 +52826,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(")");
@@ -52840,8 +52840,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();
       }
     }
 
@@ -52898,11 +52898,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);
               }
@@ -52934,9 +52934,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();
@@ -52960,7 +52960,7 @@ public class THBaseService {
         if (struct.isSetServerNames()) {
           optionals.set(0);
         }
-        if (struct.isSetSlowLogQueryFilter()) {
+        if (struct.isSetLogQueryFilter()) {
           optionals.set(1);
         }
         oprot.writeBitSet(optionals, 2);
@@ -52973,8 +52973,8 @@ public class THBaseService {
             }
           }
         }
-        if (struct.isSetSlowLogQueryFilter()) {
-          struct.slowLogQueryFilter.write(oprot);
+        if (struct.isSetLogQueryFilter()) {
+          struct.logQueryFilter.write(oprot);
         }
       }
 
@@ -52997,9 +52997,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);
         }
       }
     }
@@ -53018,7 +53018,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. */
@@ -53090,7 +53090,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);
@@ -53101,7 +53101,7 @@ public class THBaseService {
     }
 
     public getSlowLogResponses_result(
-      java.util.List<TSlowLogRecord> success,
+      java.util.List<TOnlineLogRecord> success,
       TIOError io)
     {
       this();
@@ -53114,9 +53114,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;
       }
@@ -53140,23 +53140,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;
     }
@@ -53207,7 +53207,7 @@ public class THBaseService {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((java.util.List<TSlowLogRecord>)value);
+          setSuccess((java.util.List<TOnlineLogRecord>)value);
         }
         break;
 
@@ -53412,11 +53412,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);
                   }
@@ -53455,7 +53455,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);
             }
@@ -53496,7 +53496,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);
             }
@@ -53514,11 +53514,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);
             }
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
index c0e2f98..956cfcd 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
index 19a4d87..cf21d8c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
index d701176..d7f5b17 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
index cc41a76..d79e242 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
index 79ba078..a6758f8 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
index 6dddc0d..2ae03f7 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
@@ -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-23")
+@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.
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TSlowLogQueryFilter.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TLogQueryFilter.java
similarity index 79%
rename from hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TSlowLogQueryFilter.java
rename to hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TLogQueryFilter.java
index b60dbf9..f394993 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TSlowLogQueryFilter.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TLogQueryFilter.java
@@ -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-23")
-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);
+      }
     }
   }
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TLogType.java
similarity index 70%
copy from hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java
copy to hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TLogType.java
index 747e62d..e840b29 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TLogType.java
@@ -7,15 +7,14 @@
 package org.apache.hadoop.hbase.thrift2.generated;
 
 
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2020-03-23")
-public enum TReadType implements org.apache.thrift.TEnum {
-  DEFAULT(1),
-  STREAM(2),
-  PREAD(3);
+@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 TReadType(int value) {
+  private TLogType(int value) {
     this.value = value;
   }
 
@@ -31,14 +30,12 @@ public enum TReadType implements org.apache.thrift.TEnum {
    * @return null if the value is not found.
    */
   @org.apache.thrift.annotation.Nullable
-  public static TReadType findByValue(int value) { 
+  public static TLogType findByValue(int value) { 
     switch (value) {
       case 1:
-        return DEFAULT;
+        return SLOW_LOG;
       case 2:
-        return STREAM;
-      case 3:
-        return PREAD;
+        return LARGE_LOG;
       default:
         return null;
     }
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java
index f4dae9b..105cf58 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java
@@ -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-23")
+@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);
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
index c2ecad6..e227bde 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TSlowLogRecord.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java
similarity index 94%
rename from hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TSlowLogRecord.java
rename to hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java
index 5943a36..47b2eb0 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TSlowLogRecord.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java
@@ -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-23")
-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);
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
index 78038bb..9765b92 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java
index 747e62d..82d62fe 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java
@@ -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-23")
+@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),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
index 6113f05..7c6a30d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
index 542982d..d128849 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
index 0540b2c..6178aef 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
index b87a46b..672372b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
index acde7b0..4276fb2 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
index ccc559c..c78025d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TThriftServerType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TThriftServerType.java
index baa3723..470fb9c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TThriftServerType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TThriftServerType.java
@@ -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-23")
+@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);
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
index 21c3cd4..a4ae0a5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
@@ -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-23")
+@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");
 
diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
index 6c600d0..41e4650 100644
--- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
+++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
@@ -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
@@ -1094,15 +1100,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
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
index 2e4ce10..a0618dc 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
@@ -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;
@@ -1757,11 +1757,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 {