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/10/16 06:51:36 UTC

[hbase] branch branch-1 updated: HBASE-23935 : Backport HBASE-22978, HBASE-24528, HBASE-24718 to branch-1 (#2312)

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

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


The following commit(s) were added to refs/heads/branch-1 by this push:
     new fb25a7d  HBASE-23935 : Backport HBASE-22978, HBASE-24528, HBASE-24718 to branch-1 (#2312)
fb25a7d is described below

commit fb25a7d53077227a4505beaea8c9e4c96d9d7360
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Fri Oct 16 12:21:17 2020 +0530

    HBASE-23935 : Backport HBASE-22978, HBASE-24528, HBASE-24718 to branch-1 (#2312)
    
    * HBASE-22978: Online slow response log (with sub-tasks: Large logs, Replicate logs to HDFS, Filter support in query)
    * HBASE-24718: Generic NamedQueue framework for recent in-memory history
    * HBASE-24528: Improve balancer decision observability
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
    Signed-off-by: Reid Chan <re...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java |   29 +
 .../hadoop/hbase/client/BalancerDecision.java      |  160 +
 .../hadoop/hbase/client/ConnectionManager.java     |    7 +
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |   92 +
 .../org/apache/hadoop/hbase/client/LogEntry.java   |   39 +
 .../hadoop/hbase/client/OnlineLogRecord.java       |  329 ++
 .../org/apache/hadoop/hbase/client/ServerType.java |   35 +
 .../apache/hadoop/hbase/client/SlowLogParams.java  |   89 +
 .../apache/hadoop/hbase/protobuf/ProtobufUtil.java |  180 +
 .../hadoop/hbase/protobuf/RequestConverter.java    |   79 +
 .../hadoop/hbase/slowlog/SlowLogTableAccessor.java |  154 +
 .../java/org/apache/hadoop/hbase/HConstants.java   |   16 +
 hbase-common/src/main/resources/hbase-default.xml  |   56 +
 hbase-protocol/pom.xml                             |    2 +
 .../hbase/protobuf/generated/AdminProtos.java      | 3427 +++++++++++++++++++-
 .../hbase/protobuf/generated/HBaseProtos.java      | 1285 +++++++-
 .../hbase/protobuf/generated/MasterProtos.java     | 2071 +++++++++---
 .../hbase/protobuf/generated/RecentLogs.java       | 1229 +++++++
 .../hbase/protobuf/generated/TooSlowLog.java       | 2465 ++++++++++++++
 hbase-protocol/src/main/protobuf/Admin.proto       |   54 +
 hbase-protocol/src/main/protobuf/HBase.proto       |   10 +
 hbase-protocol/src/main/protobuf/Master.proto      |   26 +
 hbase-protocol/src/main/protobuf/RecentLogs.proto  |   37 +
 hbase-protocol/src/main/protobuf/TooSlowLog.proto  |   54 +
 .../org/apache/hadoop/hbase/ipc/RpcServer.java     |   57 +-
 .../hadoop/hbase/ipc/RpcServerInterface.java       |    9 +
 .../org/apache/hadoop/hbase/master/HMaster.java    |    4 +
 .../hadoop/hbase/master/MasterRpcServices.java     |   51 +
 .../hbase/master/balancer/BaseLoadBalancer.java    |   11 +
 .../master/balancer/StochasticLoadBalancer.java    |   53 +-
 .../hbase/master/slowlog/SlowLogMasterService.java |   73 +
 .../hbase/namequeues/BalancerDecisionDetails.java  |   51 +
 .../namequeues/DisruptorExceptionHandler.java      |   50 +
 .../hadoop/hbase/namequeues/LogEventHandler.java   |  128 +
 .../hadoop/hbase/namequeues/LogHandlerUtils.java   |  107 +
 .../hadoop/hbase/namequeues/NamedQueuePayload.java |   72 +
 .../hbase/namequeues/NamedQueueRecorder.java       |  164 +
 .../hadoop/hbase/namequeues/NamedQueueService.java |   69 +
 .../hbase/namequeues/RingBufferEnvelope.java       |   56 +
 .../hadoop/hbase/namequeues/RpcLogDetails.java     |  117 +
 .../hbase/namequeues/SlowLogPersistentService.java |   94 +
 .../hbase/namequeues/SlowLogTableOpsChore.java     |   63 +
 .../impl/BalancerDecisionQueueService.java         |  144 +
 .../hbase/namequeues/impl/SlowLogQueueService.java |  268 ++
 .../hbase/namequeues/queue/EvictingQueue.java      |   85 +
 .../namequeues/request/NamedQueueGetRequest.java   |   77 +
 .../namequeues/response/NamedQueueGetResponse.java |   72 +
 .../hadoop/hbase/regionserver/HRegionServer.java   |   50 +
 .../hadoop/hbase/regionserver/RSRpcServices.java   |   83 +-
 .../hadoop/hbase/master/MockRegionServer.java      |   13 +
 .../master/balancer/TestBalancerDecision.java      |  102 +
 .../hbase/namequeues/TestNamedQueueRecorder.java   |  716 ++++
 .../hbase/namequeues/TestSlowLogAccessor.java      |  237 ++
 .../replication/regionserver/TestReplicator.java   |   13 +
 hbase-shell/src/main/ruby/hbase/admin.rb           |  108 +
 hbase-shell/src/main/ruby/shell.rb                 |    4 +
 .../ruby/shell/commands/clear_slowlog_responses.rb |   47 +
 .../ruby/shell/commands/get_balancer_decisions.rb  |   47 +
 .../ruby/shell/commands/get_largelog_responses.rb  |   98 +
 .../ruby/shell/commands/get_slowlog_responses.rb   |   98 +
 src/main/asciidoc/_chapters/hbase-default.adoc     |   72 +
 src/main/asciidoc/_chapters/ops_mgt.adoc           |  181 ++
 .../slow_log_responses_from_systable.adoc          |  118 +
 63 files changed, 15251 insertions(+), 536 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 c32b391..81d9257 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
@@ -22,6 +22,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Future;
 import java.util.regex.Pattern;
 
@@ -1681,5 +1682,33 @@ public interface Admin extends Abortable, Closeable {
    */
   boolean isSnapshotCleanupEnabled() throws IOException;
 
+  /**
+   * Clears online slow/large RPC logs from the provided list of
+   * RegionServers
+   *
+   * @param serverNames Set of Server names to clean slowlog responses from
+   * @return List of booleans representing if online slowlog response buffer is cleaned
+   *   from each RegionServer
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<Boolean> clearSlowLogResponses(final Set<ServerName> serverNames)
+    throws IOException;
 
+
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param serverNames servers to retrieve records from, useful in case of records maintained
+   *   by RegionServer as we can select specific server. In case of servertype=MASTER, logs will
+   *   only come from the currently active master.
+   * @param logType string representing type of log records
+   * @param serverType enum for server type: HMaster or RegionServer
+   * @param limit put a limit to list of records that server should send in response
+   * @param filterParams additional filter params
+   * @return Log entries representing online records from servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
+    ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException;
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecision.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecision.java
new file mode 100644
index 0000000..45e0261
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecision.java
@@ -0,0 +1,160 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.lang.reflect.Type;
+import java.util.List;
+
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.GsonUtil;
+
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializationContext;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+
+/**
+ * History of balancer decisions taken for region movements.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+final public class BalancerDecision extends LogEntry {
+
+  private final String initialFunctionCosts;
+  private final String finalFunctionCosts;
+  private final double initTotalCost;
+  private final double computedTotalCost;
+  private final long computedSteps;
+  private final List<String> regionPlans;
+
+  // used to convert object to pretty printed format
+  // used by toJsonPrettyPrint()
+  private static final Gson GSON = GsonUtil.createGson()
+    .setPrettyPrinting()
+    .registerTypeAdapter(BalancerDecision.class, new JsonSerializer<BalancerDecision>() {
+      @Override
+      public JsonElement serialize(BalancerDecision balancerDecision, Type type,
+        JsonSerializationContext jsonSerializationContext) {
+        Gson gson = new Gson();
+        return gson.toJsonTree(balancerDecision);
+      }
+    }).create();
+
+  private BalancerDecision(String initialFunctionCosts, String finalFunctionCosts,
+    double initTotalCost, double computedTotalCost, List<String> regionPlans,
+    long computedSteps) {
+    this.initialFunctionCosts = initialFunctionCosts;
+    this.finalFunctionCosts = finalFunctionCosts;
+    this.initTotalCost = initTotalCost;
+    this.computedTotalCost = computedTotalCost;
+    this.regionPlans = regionPlans;
+    this.computedSteps = computedSteps;
+  }
+
+  public String getInitialFunctionCosts() {
+    return initialFunctionCosts;
+  }
+
+  public String getFinalFunctionCosts() {
+    return finalFunctionCosts;
+  }
+
+  public double getInitTotalCost() {
+    return initTotalCost;
+  }
+
+  public double getComputedTotalCost() {
+    return computedTotalCost;
+  }
+
+  public List<String> getRegionPlans() {
+    return regionPlans;
+  }
+
+  public long getComputedSteps() {
+    return computedSteps;
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this)
+      .append("initialFunctionCosts", initialFunctionCosts)
+      .append("finalFunctionCosts", finalFunctionCosts)
+      .append("initTotalCost", initTotalCost)
+      .append("computedTotalCost", computedTotalCost)
+      .append("computedSteps", computedSteps)
+      .append("regionPlans", regionPlans)
+      .toString();
+  }
+
+  @Override
+  public String toJsonPrettyPrint() {
+    return GSON.toJson(this);
+  }
+
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public static class Builder {
+    private String initialFunctionCosts;
+    private String finalFunctionCosts;
+    private double initTotalCost;
+    private double computedTotalCost;
+    private long computedSteps;
+    private List<String> regionPlans;
+
+    public Builder setInitialFunctionCosts(String initialFunctionCosts) {
+      this.initialFunctionCosts = initialFunctionCosts;
+      return this;
+    }
+
+    public Builder setFinalFunctionCosts(String finalFunctionCosts) {
+      this.finalFunctionCosts = finalFunctionCosts;
+      return this;
+    }
+
+    public Builder setInitTotalCost(double initTotalCost) {
+      this.initTotalCost = initTotalCost;
+      return this;
+    }
+
+    public Builder setComputedTotalCost(double computedTotalCost) {
+      this.computedTotalCost = computedTotalCost;
+      return this;
+    }
+
+    public Builder setRegionPlans(List<String> regionPlans) {
+      this.regionPlans = regionPlans;
+      return this;
+    }
+
+    public Builder setComputedSteps(long computedSteps) {
+      this.computedSteps = computedSteps;
+      return this;
+    }
+
+    public BalancerDecision build() {
+      return new BalancerDecision(initialFunctionCosts, finalFunctionCosts,
+        initTotalCost, computedTotalCost, regionPlans, computedSteps);
+    }
+  }
+
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
index 82b364e..5986df1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
@@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
@@ -2146,6 +2147,12 @@ class ConnectionManager {
         }
 
         @Override
+        public HBaseProtos.LogEntry getLogEntries(RpcController controller,
+            HBaseProtos.LogRequest request) throws ServiceException {
+          return stub.getLogEntries(controller, request);
+        }
+
+        @Override
         public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller,
             ListNamespaceDescriptorsRequest request) throws ServiceException {
           return stub.listNamespaceDescriptors(controller, request);
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 d8ebcc9..915d53a 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
@@ -26,11 +26,13 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -79,6 +81,7 @@ import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
@@ -175,6 +178,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
@@ -5089,4 +5093,92 @@ public class HBaseAdmin implements Admin {
 
   }
 
+  private List<LogEntry> getSlowLogResponses(
+      final Map<String, Object> filterParams, final Set<ServerName> serverNames, final int limit,
+      final String logType) {
+    if (CollectionUtils.isEmpty(serverNames)) {
+      return Collections.emptyList();
+    }
+    List<LogEntry> logRecords = new ArrayList<>();
+    for (ServerName serverName : serverNames) {
+      try {
+        logRecords.addAll(getSlowLogs(serverName, filterParams, limit, logType));
+      } catch (ServiceException | IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    return logRecords;
+  }
+
+  private List<LogEntry> getSlowLogs(ServerName serverName, Map<String, Object> filterParams,
+      int limit, String logType) throws IOException, ServiceException {
+    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
+    HBaseRpcController controller = rpcControllerFactory.newController();
+    HBaseProtos.LogRequest logRequest =
+      RequestConverter.buildSlowLogResponseRequest(filterParams, limit, logType);
+    HBaseProtos.LogEntry logEntry = admin.getLogEntries(controller, logRequest);
+    return ProtobufUtil.toSlowLogPayloads(logEntry);
+  }
+
+  @Override
+  public List<Boolean> clearSlowLogResponses(Set<ServerName> serverNames)
+    throws IOException {
+    if (CollectionUtils.isEmpty(serverNames)) {
+      return Collections.emptyList();
+    }
+    List<Boolean> logsCleared = new ArrayList<>();
+    for (ServerName serverName : serverNames) {
+      try {
+        logsCleared.add(clearSlowLogsResponses(serverName));
+      } catch (ServiceException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    return logsCleared;
+  }
+
+  private Boolean clearSlowLogsResponses(final ServerName serverName)
+      throws IOException, ServiceException {
+    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
+    HBaseRpcController controller = rpcControllerFactory.newController();
+    AdminProtos.ClearSlowLogResponses clearSlowLogResponses =
+      admin.clearSlowLogsResponses(controller,
+        RequestConverter.buildClearSlowLogResponseRequest());
+    return ProtobufUtil.toClearSlowLogPayload(clearSlowLogResponses);
+  }
+
+  @Override
+  public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
+    ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException {
+    if (logType == null || serverType == null) {
+      throw new IllegalArgumentException("logType and/or serverType cannot be empty");
+    }
+    if (logType.equals("SLOW_LOG") || logType.equals("LARGE_LOG")) {
+      if (ServerType.MASTER.equals(serverType)) {
+        throw new IllegalArgumentException("Slow/Large logs are not maintained by HMaster");
+      }
+      return getSlowLogResponses(filterParams, serverNames, limit, logType);
+    } else if (logType.equals("BALANCER_DECISION")) {
+      if (ServerType.REGION_SERVER.equals(serverType)) {
+        throw new IllegalArgumentException(
+          "Balancer Decision logs are not maintained by HRegionServer");
+      }
+      return getBalancerDecisions(limit);
+    }
+    return Collections.emptyList();
+  }
+
+  private List<LogEntry> getBalancerDecisions(final int limit) throws IOException {
+    return executeCallable(new MasterCallable<List<LogEntry>>(getConnection()) {
+      @Override
+      public List<LogEntry> call(int callTimeout) throws Exception {
+        HBaseRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        HBaseProtos.LogEntry logEntry =
+          master.getLogEntries(controller, ProtobufUtil.toBalancerDecisionRequest(limit));
+        return ProtobufUtil.toBalancerDecisionResponse(logEntry);
+      }
+    });
+  }
+
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogEntry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogEntry.java
new file mode 100644
index 0000000..908f2dc
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogEntry.java
@@ -0,0 +1,39 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Abstract response class representing online logs response from ring-buffer use-cases
+ * e.g slow/large RPC logs, balancer decision logs
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class LogEntry {
+
+  /**
+   * Based on response sent by server, provide pretty printed Json representation in string
+   * @return Pretty printed Json representation
+   */
+  public abstract String toJsonPrettyPrint();
+
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java
new file mode 100644
index 0000000..c12db50
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java
@@ -0,0 +1,329 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.lang.reflect.Type;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.GsonUtil;
+
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
+import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializationContext;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+
+/**
+ * Slow/Large Log payload for hbase-client, to be used by Admin API get_slow_responses and
+ * get_large_responses
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+final public class OnlineLogRecord extends LogEntry {
+  // used to convert object to pretty printed format
+  // used by toJsonPrettyPrint()
+  private static final Gson GSON = GsonUtil.createGson().setPrettyPrinting()
+    .registerTypeAdapter(OnlineLogRecord.class, new JsonSerializer<OnlineLogRecord>() {
+      @Override
+      public JsonElement serialize(OnlineLogRecord slowLogPayload, Type type,
+        JsonSerializationContext jsonSerializationContext) {
+        Gson gson = new Gson();
+        JsonObject jsonObj = (JsonObject) gson.toJsonTree(slowLogPayload);
+        if (slowLogPayload.getMultiGetsCount() == 0) {
+          jsonObj.remove("multiGetsCount");
+        }
+        if (slowLogPayload.getMultiMutationsCount() == 0) {
+          jsonObj.remove("multiMutationsCount");
+        }
+        if (slowLogPayload.getMultiServiceCalls() == 0) {
+          jsonObj.remove("multiServiceCalls");
+        }
+        return jsonObj;
+      }
+    }).create();
+
+  private final long startTime;
+  private final int processingTime;
+  private final int queueTime;
+  private final long responseSize;
+  private final String clientAddress;
+  private final String serverClass;
+  private final String methodName;
+  private final String callDetails;
+  private final String param;
+  // we don't want to serialize region name, it is just for the filter purpose
+  // hence avoiding deserialization
+  private final transient String regionName;
+  private final String userName;
+  private final int multiGetsCount;
+  private final int multiMutationsCount;
+  private final int multiServiceCalls;
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public int getProcessingTime() {
+    return processingTime;
+  }
+
+  public int getQueueTime() {
+    return queueTime;
+  }
+
+  public long getResponseSize() {
+    return responseSize;
+  }
+
+  public String getClientAddress() {
+    return clientAddress;
+  }
+
+  public String getServerClass() {
+    return serverClass;
+  }
+
+  public String getMethodName() {
+    return methodName;
+  }
+
+  public String getCallDetails() {
+    return callDetails;
+  }
+
+  public String getParam() {
+    return param;
+  }
+
+  public String getRegionName() {
+    return regionName;
+  }
+
+  public String getUserName() {
+    return userName;
+  }
+
+  public int getMultiGetsCount() {
+    return multiGetsCount;
+  }
+
+  public int getMultiMutationsCount() {
+    return multiMutationsCount;
+  }
+
+  public int getMultiServiceCalls() {
+    return multiServiceCalls;
+  }
+
+  private OnlineLogRecord(final long startTime, final int processingTime, final int queueTime,
+    final long responseSize, final String clientAddress, final String serverClass,
+    final String methodName, final String callDetails, final String param,
+    final String regionName, final String userName, final int multiGetsCount,
+    final int multiMutationsCount, final int multiServiceCalls) {
+    this.startTime = startTime;
+    this.processingTime = processingTime;
+    this.queueTime = queueTime;
+    this.responseSize = responseSize;
+    this.clientAddress = clientAddress;
+    this.serverClass = serverClass;
+    this.methodName = methodName;
+    this.callDetails = callDetails;
+    this.param = param;
+    this.regionName = regionName;
+    this.userName = userName;
+    this.multiGetsCount = multiGetsCount;
+    this.multiMutationsCount = multiMutationsCount;
+    this.multiServiceCalls = multiServiceCalls;
+  }
+
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public static class OnlineLogRecordBuilder {
+    private long startTime;
+    private int processingTime;
+    private int queueTime;
+    private long responseSize;
+    private String clientAddress;
+    private String serverClass;
+    private String methodName;
+    private String callDetails;
+    private String param;
+    private String regionName;
+    private String userName;
+    private int multiGetsCount;
+    private int multiMutationsCount;
+    private int multiServiceCalls;
+
+    public OnlineLogRecordBuilder setStartTime(long startTime) {
+      this.startTime = startTime;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setProcessingTime(int processingTime) {
+      this.processingTime = processingTime;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setQueueTime(int queueTime) {
+      this.queueTime = queueTime;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setResponseSize(long responseSize) {
+      this.responseSize = responseSize;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setClientAddress(String clientAddress) {
+      this.clientAddress = clientAddress;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setServerClass(String serverClass) {
+      this.serverClass = serverClass;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setMethodName(String methodName) {
+      this.methodName = methodName;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setCallDetails(String callDetails) {
+      this.callDetails = callDetails;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setParam(String param) {
+      this.param = param;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setRegionName(String regionName) {
+      this.regionName = regionName;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setUserName(String userName) {
+      this.userName = userName;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setMultiGetsCount(int multiGetsCount) {
+      this.multiGetsCount = multiGetsCount;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setMultiMutationsCount(int multiMutationsCount) {
+      this.multiMutationsCount = multiMutationsCount;
+      return this;
+    }
+
+    public OnlineLogRecordBuilder setMultiServiceCalls(int multiServiceCalls) {
+      this.multiServiceCalls = multiServiceCalls;
+      return this;
+    }
+
+    public OnlineLogRecord build() {
+      return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize,
+        clientAddress, serverClass, methodName, callDetails, param, regionName,
+        userName, multiGetsCount, multiMutationsCount, multiServiceCalls);
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    OnlineLogRecord that = (OnlineLogRecord) o;
+
+    return new EqualsBuilder()
+      .append(startTime, that.startTime)
+      .append(processingTime, that.processingTime)
+      .append(queueTime, that.queueTime)
+      .append(responseSize, that.responseSize)
+      .append(multiGetsCount, that.multiGetsCount)
+      .append(multiMutationsCount, that.multiMutationsCount)
+      .append(multiServiceCalls, that.multiServiceCalls)
+      .append(clientAddress, that.clientAddress)
+      .append(serverClass, that.serverClass)
+      .append(methodName, that.methodName)
+      .append(callDetails, that.callDetails)
+      .append(param, that.param)
+      .append(regionName, that.regionName)
+      .append(userName, that.userName)
+      .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+      .append(startTime)
+      .append(processingTime)
+      .append(queueTime)
+      .append(responseSize)
+      .append(clientAddress)
+      .append(serverClass)
+      .append(methodName)
+      .append(callDetails)
+      .append(param)
+      .append(regionName)
+      .append(userName)
+      .append(multiGetsCount)
+      .append(multiMutationsCount)
+      .append(multiServiceCalls)
+      .toHashCode();
+  }
+
+  public String toJsonPrettyPrint() {
+    return GSON.toJson(this);
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this)
+      .append("startTime", startTime)
+      .append("processingTime", processingTime)
+      .append("queueTime", queueTime)
+      .append("responseSize", responseSize)
+      .append("clientAddress", clientAddress)
+      .append("serverClass", serverClass)
+      .append("methodName", methodName)
+      .append("callDetails", callDetails)
+      .append("param", param)
+      .append("regionName", regionName)
+      .append("userName", userName)
+      .append("multiGetsCount", multiGetsCount)
+      .append("multiMutationsCount", multiMutationsCount)
+      .append("multiServiceCalls", multiServiceCalls)
+      .toString();
+  }
+
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
new file mode 100644
index 0000000..e70d348
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
@@ -0,0 +1,35 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Select server type i.e destination for RPC request associated with ring buffer.
+ * e.g slow/large log records are maintained by HRegionServer, whereas balancer decisions
+ * are maintained by HMaster.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public enum ServerType {
+  MASTER,
+  REGION_SERVER
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
new file mode 100644
index 0000000..30206c4
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
@@ -0,0 +1,89 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * SlowLog params object that contains detailed info as params and region name : to be used
+ * for filter purpose
+ */
+@InterfaceAudience.Private
+public class SlowLogParams {
+
+  private final String regionName;
+  private final String params;
+
+  public SlowLogParams(String regionName, String params) {
+    this.regionName = regionName;
+    this.params = params;
+  }
+
+  public SlowLogParams(String params) {
+    this.regionName = StringUtils.EMPTY;
+    this.params = params;
+  }
+
+  public String getRegionName() {
+    return regionName;
+  }
+
+  public String getParams() {
+    return params;
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this)
+      .append("regionName", regionName)
+      .append("params", params)
+      .toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    SlowLogParams that = (SlowLogParams) o;
+
+    return new EqualsBuilder()
+      .append(regionName, that.regionName)
+      .append(params, that.params)
+      .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+      .append(regionName)
+      .append(params)
+      .toHashCode();
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 240133a..6a1da08 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.BalancerDecision;
 import org.apache.hadoop.hbase.client.ClientUtil;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Cursor;
@@ -84,12 +85,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.LogEntry;
 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.RegionReplicaUtil;
 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.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -101,6 +105,7 @@ import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -114,6 +119,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRespon
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
@@ -150,9 +156,11 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableReques
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
 import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RecentLogs;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.TableProtos;
+import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
@@ -177,6 +185,7 @@ import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.util.DynamicClassLoader;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
@@ -3035,6 +3044,62 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Return SlowLogParams to maintain recent online slowlog responses
+   *
+   * @param message Message object {@link Message}
+   * @return SlowLogParams with regionName(for filter queries) and params
+   */
+  public static SlowLogParams getSlowLogParams(Message message) {
+    if (message == null) {
+      return null;
+    }
+    if (message instanceof ScanRequest) {
+      ScanRequest scanRequest = (ScanRequest) message;
+      String regionName = getStringForByteString(scanRequest.getRegion().getValue());
+      String params = TextFormat.shortDebugString(message);
+      return new SlowLogParams(regionName, params);
+    } else if (message instanceof MutationProto) {
+      MutationProto mutationProto = (MutationProto) message;
+      String params = "type= " + mutationProto.getMutateType().toString();
+      return new SlowLogParams(params);
+    } else if (message instanceof GetRequest) {
+      GetRequest getRequest = (GetRequest) message;
+      String regionName = getStringForByteString(getRequest.getRegion().getValue());
+      String params =
+        "region= " + regionName + ", row= " + getStringForByteString(getRequest.getGet().getRow());
+      return new SlowLogParams(regionName, params);
+    } else if (message instanceof ClientProtos.MultiRequest) {
+      ClientProtos.MultiRequest multiRequest = (ClientProtos.MultiRequest) message;
+      int actionsCount = 0;
+      for (ClientProtos.RegionAction regionAction : multiRequest.getRegionActionList()) {
+        actionsCount += regionAction.getActionCount();
+      }
+      ClientProtos.RegionAction actions = multiRequest.getRegionActionList().get(0);
+      String row = actions.getActionCount() <= 0 ? "" :
+        getStringForByteString(actions.getAction(0).hasGet() ?
+          actions.getAction(0).getGet().getRow() :
+          actions.getAction(0).getMutation().getRow());
+      String regionName = getStringForByteString(actions.getRegion().getValue());
+      String params =
+        "region= " + regionName + ", for " + actionsCount + " action(s) and 1st row key=" + row;
+      return new SlowLogParams(regionName, params);
+    } else if (message instanceof ClientProtos.MutateRequest) {
+      ClientProtos.MutateRequest mutateRequest = (ClientProtos.MutateRequest) message;
+      String regionName = getStringForByteString(mutateRequest.getRegion().getValue());
+      String params = "region= " + regionName;
+      return new SlowLogParams(regionName, params);
+    } else if (message instanceof CoprocessorServiceRequest) {
+      CoprocessorServiceRequest coprocessorServiceRequest = (CoprocessorServiceRequest) message;
+      String params =
+        "coprocessorService= " + coprocessorServiceRequest.getCall().getServiceName() + ":"
+          + coprocessorServiceRequest.getCall().getMethodName();
+      return new SlowLogParams(params);
+    }
+    String params = message.getClass().toString();
+    return new SlowLogParams(params);
+  }
+
+  /**
    * Print out some subset of a MutationProto rather than all of it and its data
    * @param proto Protobuf to print out
    * @return Short String of mutation proto
@@ -3664,6 +3729,121 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Convert Protobuf class
+   * {@link TooSlowLog.SlowLogPayload}
+   * To client SlowLog Payload class {@link OnlineLogRecord}
+   *
+   * @param slowLogPayload SlowLog Payload protobuf instance
+   * @return SlowLog Payload for client usecase
+   */
+  private static LogEntry getSlowLogRecord(
+      final TooSlowLog.SlowLogPayload slowLogPayload) {
+    OnlineLogRecord onlineLogRecord = new OnlineLogRecord.OnlineLogRecordBuilder()
+      .setCallDetails(slowLogPayload.getCallDetails())
+      .setClientAddress(slowLogPayload.getClientAddress())
+      .setMethodName(slowLogPayload.getMethodName())
+      .setMultiGetsCount(slowLogPayload.getMultiGets())
+      .setMultiMutationsCount(slowLogPayload.getMultiMutations())
+      .setMultiServiceCalls(slowLogPayload.getMultiServiceCalls())
+      .setParam(slowLogPayload.getParam())
+      .setProcessingTime(slowLogPayload.getProcessingTime())
+      .setQueueTime(slowLogPayload.getQueueTime())
+      .setRegionName(slowLogPayload.getRegionName())
+      .setResponseSize(slowLogPayload.getResponseSize())
+      .setServerClass(slowLogPayload.getServerClass())
+      .setStartTime(slowLogPayload.getStartTime())
+      .setUserName(slowLogPayload.getUserName())
+      .build();
+    return onlineLogRecord;
+  }
+
+  /**
+   * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
+   *
+   * @param logEntry slowlog response protobuf instance
+   * @return list of SlowLog payloads for client usecase
+   */
+  public static List<LogEntry> toSlowLogPayloads(
+      final HBaseProtos.LogEntry logEntry) {
+    try {
+      final String logClassName = logEntry.getLogClassName();
+      Class<?> logClass = Class.forName(logClassName).asSubclass(Message.class);
+      Method method = logClass.getMethod("parseFrom", ByteString.class);
+      if (logClassName.contains("SlowLogResponses")) {
+        SlowLogResponses slowLogResponses = (SlowLogResponses) method
+          .invoke(null, logEntry.getLogMessage());
+        List<LogEntry> logEntries = new ArrayList<>();
+        for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogResponses.getSlowLogPayloadsList()) {
+          logEntries.add(ProtobufUtil.getSlowLogRecord(slowLogPayload));
+        }
+        return logEntries;
+      }
+    } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
+      | InvocationTargetException e) {
+      throw new RuntimeException("Error while retrieving response from server");
+    }
+    throw new RuntimeException("Invalid response from server");
+  }
+
+  /**
+   * Convert {@link ClearSlowLogResponses} to boolean
+   *
+   * @param clearSlowLogResponses Clear slowlog response protobuf instance
+   * @return boolean representing clear slowlog response
+   */
+  public static boolean toClearSlowLogPayload(final ClearSlowLogResponses clearSlowLogResponses) {
+    return clearSlowLogResponses.getIsCleaned();
+  }
+
+  public static List<LogEntry> toBalancerDecisionResponse(
+      HBaseProtos.LogEntry logEntry) {
+    try {
+      final String logClassName = logEntry.getLogClassName();
+      Class<?> logClass = Class.forName(logClassName).asSubclass(Message.class);
+      Method method = logClass.getMethod("parseFrom", ByteString.class);
+      if (logClassName.contains("BalancerDecisionsResponse")) {
+        MasterProtos.BalancerDecisionsResponse response =
+          (MasterProtos.BalancerDecisionsResponse) method
+            .invoke(null, logEntry.getLogMessage());
+        return getBalancerDecisionEntries(response);
+      }
+    } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
+      | InvocationTargetException e) {
+      throw new RuntimeException("Error while retrieving response from server");
+    }
+    throw new RuntimeException("Invalid response from server");
+  }
+
+  public static List<LogEntry> getBalancerDecisionEntries(
+      MasterProtos.BalancerDecisionsResponse response) {
+    List<RecentLogs.BalancerDecision> balancerDecisions = response.getBalancerDecisionList();
+    if (CollectionUtils.isEmpty(balancerDecisions)) {
+      return Collections.emptyList();
+    }
+    List<LogEntry> logEntries = new ArrayList<>();
+    for (RecentLogs.BalancerDecision balancerDecision : balancerDecisions) {
+      BalancerDecision bd =
+        new BalancerDecision.Builder().setInitTotalCost(balancerDecision.getInitTotalCost())
+          .setInitialFunctionCosts(balancerDecision.getInitialFunctionCosts())
+          .setComputedTotalCost(balancerDecision.getComputedTotalCost())
+          .setFinalFunctionCosts(balancerDecision.getFinalFunctionCosts())
+          .setComputedSteps(balancerDecision.getComputedSteps())
+          .setRegionPlans(balancerDecision.getRegionPlansList()).build();
+      logEntries.add(bd);
+    }
+    return logEntries;
+  }
+
+  public static HBaseProtos.LogRequest toBalancerDecisionRequest(int limit) {
+    MasterProtos.BalancerDecisionsRequest balancerDecisionsRequest =
+      MasterProtos.BalancerDecisionsRequest.newBuilder().setLimit(limit).build();
+    return HBaseProtos.LogRequest.newBuilder()
+      .setLogClassName(balancerDecisionsRequest.getClass().getName())
+      .setLogMessage(balancerDecisionsRequest.toByteString())
+      .build();
+  }
+
+  /**
    * Get the Meta region state from the passed data bytes. Can handle both old and new style
    * server names.
    * @param data protobuf serialized data with meta server name.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 31e69cc..d794bc0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -19,8 +19,11 @@ package org.apache.hadoop.hbase.protobuf;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 import java.util.regex.Pattern;
 
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.ByteStringer;
 
@@ -48,6 +51,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
@@ -59,6 +63,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsReques
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
@@ -77,6 +82,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Col
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
@@ -1895,4 +1901,77 @@ public final class RequestConverter {
     return IsSnapshotCleanupEnabledRequest.newBuilder().build();
   }
 
+  /**
+   * Build RPC request payload for getLogEntries
+   *
+   * @param filterParams map of filter params
+   * @param limit limit for no of records that server returns
+   * @param logType type of the log records
+   * @return request payload HBaseProtos.LogRequest
+   */
+  public static HBaseProtos.LogRequest buildSlowLogResponseRequest(
+      final Map<String, Object> filterParams, final int limit, final String logType) {
+    SlowLogResponseRequest.Builder builder = SlowLogResponseRequest.newBuilder();
+    builder.setLimit(limit);
+    if (logType.equals("SLOW_LOG")) {
+      builder.setLogType(SlowLogResponseRequest.LogType.SLOW_LOG);
+    } else if (logType.equals("LARGE_LOG")) {
+      builder.setLogType(SlowLogResponseRequest.LogType.LARGE_LOG);
+    }
+    boolean filterByAnd = false;
+    if (MapUtils.isNotEmpty(filterParams)) {
+      if (filterParams.containsKey("clientAddress")) {
+        final String clientAddress = (String) filterParams.get("clientAddress");
+        if (StringUtils.isNotEmpty(clientAddress)) {
+          builder.setClientAddress(clientAddress);
+        }
+      }
+      if (filterParams.containsKey("regionName")) {
+        final String regionName = (String) filterParams.get("regionName");
+        if (StringUtils.isNotEmpty(regionName)) {
+          builder.setRegionName(regionName);
+        }
+      }
+      if (filterParams.containsKey("tableName")) {
+        final String tableName = (String) filterParams.get("tableName");
+        if (StringUtils.isNotEmpty(tableName)) {
+          builder.setTableName(tableName);
+        }
+      }
+      if (filterParams.containsKey("userName")) {
+        final String userName = (String) filterParams.get("userName");
+        if (StringUtils.isNotEmpty(userName)) {
+          builder.setUserName(userName);
+        }
+      }
+      if (filterParams.containsKey("filterByOperator")) {
+        final String filterByOperator = (String) filterParams.get("filterByOperator");
+        if (StringUtils.isNotEmpty(filterByOperator)) {
+          if (filterByOperator.toUpperCase().equals("AND")) {
+            filterByAnd = true;
+          }
+        }
+      }
+    }
+    if (filterByAnd) {
+      builder.setFilterByOperator(SlowLogResponseRequest.FilterByOperator.AND);
+    } else {
+      builder.setFilterByOperator(SlowLogResponseRequest.FilterByOperator.OR);
+    }
+    SlowLogResponseRequest slowLogResponseRequest = builder.build();
+    return HBaseProtos.LogRequest.newBuilder()
+      .setLogClassName(slowLogResponseRequest.getClass().getName())
+      .setLogMessage(slowLogResponseRequest.toByteString())
+      .build();
+  }
+
+  /**
+   * Create a protocol buffer {@link ClearSlowLogResponseRequest}
+   *
+   * @return a protocol buffer ClearSlowLogResponseRequest
+   */
+  public static ClearSlowLogResponseRequest buildClearSlowLogResponseRequest() {
+    return ClearSlowLogResponseRequest.newBuilder().build();
+  }
+
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java
new file mode 100644
index 0000000..1d32355
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java
@@ -0,0 +1,154 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.slowlog;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Slowlog Accessor to record slow/large RPC log identified at each RegionServer RpcServer level.
+ * This can be done only optionally to record the entire history of slow/large rpc calls
+ * since RingBuffer can handle only limited latest records.
+ */
+@InterfaceAudience.Private
+public final class SlowLogTableAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SlowLogTableAccessor.class);
+
+  private static final Random RANDOM = new Random();
+
+  private static Connection connection;
+
+  private SlowLogTableAccessor() {
+  }
+
+  /**
+   * hbase:slowlog table name - can be enabled
+   * with config - hbase.regionserver.slowlog.systable.enabled
+   */
+  public static final TableName SLOW_LOG_TABLE_NAME =
+    TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "slowlog");
+
+  private static void doPut(final Connection connection, final List<Put> puts)
+    throws IOException {
+    try (Table table = connection.getTable(SLOW_LOG_TABLE_NAME)) {
+      table.put(puts);
+    }
+  }
+
+  /**
+   * Add slow/large log records to hbase:slowlog table
+   * @param slowLogPayloads List of SlowLogPayload to process
+   * @param configuration Configuration to use for connection
+   */
+  public static void addSlowLogRecords(final List<TooSlowLog.SlowLogPayload> slowLogPayloads,
+    final Configuration configuration) {
+    List<Put> puts = new ArrayList<>(slowLogPayloads.size());
+    for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
+      final byte[] rowKey = getRowKey(slowLogPayload);
+      final Put put = new Put(rowKey);
+      put.setDurability(Durability.SKIP_WAL);
+      put.setPriority(HConstants.NORMAL_QOS);
+      put.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("call_details"),
+        Bytes.toBytes(slowLogPayload.getCallDetails()))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("client_address"),
+          Bytes.toBytes(slowLogPayload.getClientAddress()))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("method_name"),
+          Bytes.toBytes(slowLogPayload.getMethodName()))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("param"),
+          Bytes.toBytes(slowLogPayload.getParam()))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("processing_time"),
+          Bytes.toBytes(Integer.toString(slowLogPayload.getProcessingTime())))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
+          Bytes.toBytes(Integer.toString(slowLogPayload.getQueueTime())))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("region_name"),
+          Bytes.toBytes(slowLogPayload.getRegionName()))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("response_size"),
+          Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize())))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("server_class"),
+          Bytes.toBytes(slowLogPayload.getServerClass()))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),
+          Bytes.toBytes(Long.toString(slowLogPayload.getStartTime())))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("type"),
+          Bytes.toBytes(slowLogPayload.getType().name()))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("username"),
+          Bytes.toBytes(slowLogPayload.getUserName()));
+      puts.add(put);
+    }
+    try {
+      if (connection == null) {
+        createConnection(configuration);
+      }
+      doPut(connection, puts);
+    } catch (Exception e) {
+      LOG.warn("Failed to add slow/large log records to hbase:slowlog table.", e);
+    }
+  }
+
+  private static synchronized void createConnection(Configuration configuration)
+    throws IOException {
+    Configuration conf = new Configuration(configuration);
+    // rpc timeout: 20s
+    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 20000);
+    // retry count: 5
+    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
+    conf.setInt("hbase.client.serverside.retries.multiplier", 1);
+    connection = ConnectionFactory.createConnection(conf);
+  }
+
+  /**
+   * Create rowKey: currentTimeMillis APPEND slowLogPayload.hashcode
+   * Scan on slowlog table should keep records with sorted order of time, however records
+   * added at the very same time (currentTimeMillis) could be in random order.
+   *
+   * @param slowLogPayload SlowLogPayload to process
+   * @return rowKey byte[]
+   */
+  private static byte[] getRowKey(final TooSlowLog.SlowLogPayload slowLogPayload) {
+    String hashcode = String.valueOf(slowLogPayload.hashCode());
+    String lastFiveDig =
+      hashcode.substring((hashcode.length() > 5) ? (hashcode.length() - 5) : 0);
+    if (lastFiveDig.startsWith("-")) {
+      lastFiveDig = String.valueOf(RANDOM.nextInt(99999));
+    }
+    final long currentTimeMillis = EnvironmentEdgeManager.currentTime();
+    final String timeAndHashcode = currentTimeMillis + lastFiveDig;
+    final long rowKeyLong = Long.parseLong(timeAndHashcode);
+    return Bytes.toBytes(rowKeyLong);
+  }
+
+}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 41e7525..e660fe3 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1401,6 +1401,22 @@ public final class HConstants {
    */
   public static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
 
+  public static final int DEFAULT_SLOW_LOG_RING_BUFFER_SIZE = 256;
+
+  public static final String SLOW_LOG_BUFFER_ENABLED_KEY =
+    "hbase.regionserver.slowlog.buffer.enabled";
+  public static final boolean DEFAULT_ONLINE_LOG_PROVIDER_ENABLED = false;
+
+  /** The slowlog info family as a string*/
+  private static final String SLOWLOG_INFO_FAMILY_STR = "info";
+
+  /** The slowlog info family */
+  public static final byte [] SLOWLOG_INFO_FAMILY = Bytes.toBytes(SLOWLOG_INFO_FAMILY_STR);
+
+  public static final String SLOW_LOG_SYS_TABLE_ENABLED_KEY =
+    "hbase.regionserver.slowlog.systable.enabled";
+  public static final boolean DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY = false;
+
   private HConstants() {
     // Can't be instantiated with this ctor.
   }
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 7420308..009b841 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1690,4 +1690,60 @@ possible configurations would overwhelm and obscure the important.
       too large batch request.
     </description>
   </property>
+  <property>
+    <name>hbase.regionserver.slowlog.ringbuffer.size</name>
+    <value>256</value>
+    <description>
+      Default size of ringbuffer to be maintained by each RegionServer in order
+      to store online slowlog responses. This is an in-memory ring buffer of
+      requests that were judged to be too slow in addition to the responseTooSlow
+      logging. The in-memory representation would be complete.
+      For more details, please look into Doc Section:
+      Get Slow Response Log from shell
+    </description>
+  </property>
+  <property>
+    <name>hbase.regionserver.slowlog.buffer.enabled</name>
+    <value>false</value>
+    <description>
+      Indicates whether RegionServers have ring buffer running for storing
+      Online Slow logs in FIFO manner with limited entries. The size of
+      the ring buffer is indicated by config: hbase.regionserver.slowlog.ringbuffer.size
+      The default value is false, turn this on and get latest slowlog
+      responses with complete data.
+    </description>
+  </property>
+  <property>
+    <name>hbase.regionserver.slowlog.systable.enabled</name>
+    <value>false</value>
+    <description>
+      Should be enabled only if hbase.regionserver.slowlog.buffer.enabled is enabled. If enabled
+      (true), all slow/large RPC logs would be persisted to system table hbase:slowlog (in addition
+      to in-memory ring buffer at each RegionServer). The records are stored in increasing
+      order of time. Operators can scan the table with various combination of ColumnValueFilter.
+      More details are provided in the doc section:
+      "Get Slow/Large Response Logs from System table hbase:slowlog"
+    </description>
+  </property>
+  <property>
+    <name>hbase.namedqueue.provider.classes</name>
+    <value>org.apache.hadoop.hbase.namequeues.impl.SlowLogQueueService,org.apache.hadoop.hbase.namequeues.impl.BalancerDecisionQueueService</value>
+    <description>
+      Default values for NamedQueueService implementors. This comma separated full class names
+      represent all implementors of NamedQueueService that we would like to be invoked by
+      LogEvent handler service. One example of NamedQueue service is SlowLogQueueService which
+      is used to store slow/large RPC logs in ringbuffer at each RegionServer.
+      All implementors of NamedQueueService should be found under package:
+      "org.apache.hadoop.hbase.namequeues.impl"
+    </description>
+  </property>
+  <property>
+    <name>hbase.master.balancer.decision.buffer.enabled</name>
+    <value>false</value>
+    <description>
+      Indicates whether active HMaster has ring buffer running for storing
+      balancer decisions in FIFO manner with limited entries. The size of
+      the ring buffer is indicated by config: hbase.master.balancer.decision.queue.size
+    </description>
+  </property>
 </configuration>
diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml
index 9509a5f..0adac0c 100644
--- a/hbase-protocol/pom.xml
+++ b/hbase-protocol/pom.xml
@@ -191,6 +191,7 @@
                           <include>Procedure.proto</include>
                           <include>Quota.proto</include>
                           <include>RPC.proto</include>
+                          <include>RecentLogs.proto</include>
                           <include>RegionNormalizer.proto</include>
                           <include>RegionServerStatus.proto</include>
                           <include>RowProcessor.proto</include>
@@ -201,6 +202,7 @@
                           <include>Snapshot.proto</include>
                           <include>SnapshotCleanup.proto</include>
                           <include>Table.proto</include>
+                          <include>TooSlowLog.proto</include>
                           <include>Tracing.proto</include>
                           <include>VisibilityLabels.proto</include>
                           <include>WAL.proto</include>
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java
index b69b157..25f1a58 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java
@@ -23304,6 +23304,2999 @@ public final class AdminProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.UpdateConfigurationResponse)
   }
 
+  public interface SlowLogResponseRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional string region_name = 1;
+    /**
+     * <code>optional string region_name = 1;</code>
+     */
+    boolean hasRegionName();
+    /**
+     * <code>optional string region_name = 1;</code>
+     */
+    java.lang.String getRegionName();
+    /**
+     * <code>optional string region_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getRegionNameBytes();
+
+    // optional string table_name = 2;
+    /**
+     * <code>optional string table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>optional string table_name = 2;</code>
+     */
+    java.lang.String getTableName();
+    /**
+     * <code>optional string table_name = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getTableNameBytes();
+
+    // optional string client_address = 3;
+    /**
+     * <code>optional string client_address = 3;</code>
+     */
+    boolean hasClientAddress();
+    /**
+     * <code>optional string client_address = 3;</code>
+     */
+    java.lang.String getClientAddress();
+    /**
+     * <code>optional string client_address = 3;</code>
+     */
+    com.google.protobuf.ByteString
+        getClientAddressBytes();
+
+    // optional string user_name = 4;
+    /**
+     * <code>optional string user_name = 4;</code>
+     */
+    boolean hasUserName();
+    /**
+     * <code>optional string user_name = 4;</code>
+     */
+    java.lang.String getUserName();
+    /**
+     * <code>optional string user_name = 4;</code>
+     */
+    com.google.protobuf.ByteString
+        getUserNameBytes();
+
+    // optional uint32 limit = 5 [default = 10];
+    /**
+     * <code>optional uint32 limit = 5 [default = 10];</code>
+     */
+    boolean hasLimit();
+    /**
+     * <code>optional uint32 limit = 5 [default = 10];</code>
+     */
+    int getLimit();
+
+    // optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];
+    /**
+     * <code>optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];</code>
+     */
+    boolean hasFilterByOperator();
+    /**
+     * <code>optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator getFilterByOperator();
+
+    // optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;
+    /**
+     * <code>optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;</code>
+     */
+    boolean hasLogType();
+    /**
+     * <code>optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType getLogType();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.SlowLogResponseRequest}
+   *
+   * <pre>
+   **
+   * Slow/Large log (LogRequest) use-case specific RPC request. This request payload will be
+   * converted in bytes and sent to generic RPC API: GetLogEntries
+   * LogRequest message has two params:
+   * 1. log_class_name: SlowLogResponseRequest (for Slow/Large log use-case)
+   * 2. log_message: SlowLogResponseRequest converted in bytes (for Slow/Large log use-case)
+   * </pre>
+   */
+  public static final class SlowLogResponseRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements SlowLogResponseRequestOrBuilder {
+    // Use SlowLogResponseRequest.newBuilder() to construct.
+    private SlowLogResponseRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SlowLogResponseRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SlowLogResponseRequest defaultInstance;
+    public static SlowLogResponseRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SlowLogResponseRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SlowLogResponseRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              regionName_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              tableName_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              clientAddress_ = input.readBytes();
+              break;
+            }
+            case 34: {
+              bitField0_ |= 0x00000008;
+              userName_ = input.readBytes();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              limit_ = input.readUInt32();
+              break;
+            }
+            case 48: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator value = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(6, rawValue);
+              } else {
+                bitField0_ |= 0x00000020;
+                filterByOperator_ = value;
+              }
+              break;
+            }
+            case 56: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType value = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(7, rawValue);
+              } else {
+                bitField0_ |= 0x00000040;
+                logType_ = value;
+              }
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponseRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponseRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<SlowLogResponseRequest> PARSER =
+        new com.google.protobuf.AbstractParser<SlowLogResponseRequest>() {
+      public SlowLogResponseRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new SlowLogResponseRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<SlowLogResponseRequest> getParserForType() {
+      return PARSER;
+    }
+
+    /**
+     * Protobuf enum {@code hbase.pb.SlowLogResponseRequest.FilterByOperator}
+     */
+    public enum FilterByOperator
+        implements com.google.protobuf.ProtocolMessageEnum {
+      /**
+       * <code>AND = 0;</code>
+       */
+      AND(0, 0),
+      /**
+       * <code>OR = 1;</code>
+       */
+      OR(1, 1),
+      ;
+
+      /**
+       * <code>AND = 0;</code>
+       */
+      public static final int AND_VALUE = 0;
+      /**
+       * <code>OR = 1;</code>
+       */
+      public static final int OR_VALUE = 1;
+
+
+      public final int getNumber() { return value; }
+
+      public static FilterByOperator valueOf(int value) {
+        switch (value) {
+          case 0: return AND;
+          case 1: return OR;
+          default: return null;
+        }
+      }
+
+      public static com.google.protobuf.Internal.EnumLiteMap<FilterByOperator>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<FilterByOperator>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<FilterByOperator>() {
+              public FilterByOperator findValueByNumber(int number) {
+                return FilterByOperator.valueOf(number);
+              }
+            };
+
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.getDescriptor().getEnumTypes().get(0);
+      }
+
+      private static final FilterByOperator[] VALUES = values();
+
+      public static FilterByOperator valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+
+      private final int index;
+      private final int value;
+
+      private FilterByOperator(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+
+      // @@protoc_insertion_point(enum_scope:hbase.pb.SlowLogResponseRequest.FilterByOperator)
+    }
+
+    /**
+     * Protobuf enum {@code hbase.pb.SlowLogResponseRequest.LogType}
+     */
+    public enum LogType
+        implements com.google.protobuf.ProtocolMessageEnum {
+      /**
+       * <code>SLOW_LOG = 0;</code>
+       */
+      SLOW_LOG(0, 0),
+      /**
+       * <code>LARGE_LOG = 1;</code>
+       */
+      LARGE_LOG(1, 1),
+      ;
+
+      /**
+       * <code>SLOW_LOG = 0;</code>
+       */
+      public static final int SLOW_LOG_VALUE = 0;
+      /**
+       * <code>LARGE_LOG = 1;</code>
+       */
+      public static final int LARGE_LOG_VALUE = 1;
+
+
+      public final int getNumber() { return value; }
+
+      public static LogType valueOf(int value) {
+        switch (value) {
+          case 0: return SLOW_LOG;
+          case 1: return LARGE_LOG;
+          default: return null;
+        }
+      }
+
+      public static com.google.protobuf.Internal.EnumLiteMap<LogType>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<LogType>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<LogType>() {
+              public LogType findValueByNumber(int number) {
+                return LogType.valueOf(number);
+              }
+            };
+
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.getDescriptor().getEnumTypes().get(1);
+      }
+
+      private static final LogType[] VALUES = values();
+
+      public static LogType valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+
+      private final int index;
+      private final int value;
+
+      private LogType(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+
+      // @@protoc_insertion_point(enum_scope:hbase.pb.SlowLogResponseRequest.LogType)
+    }
+
+    private int bitField0_;
+    // optional string region_name = 1;
+    public static final int REGION_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object regionName_;
+    /**
+     * <code>optional string region_name = 1;</code>
+     */
+    public boolean hasRegionName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional string region_name = 1;</code>
+     */
+    public java.lang.String getRegionName() {
+      java.lang.Object ref = regionName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          regionName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string region_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getRegionNameBytes() {
+      java.lang.Object ref = regionName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        regionName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string table_name = 2;
+    public static final int TABLE_NAME_FIELD_NUMBER = 2;
+    private java.lang.Object tableName_;
+    /**
+     * <code>optional string table_name = 2;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional string table_name = 2;</code>
+     */
+    public java.lang.String getTableName() {
+      java.lang.Object ref = tableName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          tableName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string table_name = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getTableNameBytes() {
+      java.lang.Object ref = tableName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        tableName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string client_address = 3;
+    public static final int CLIENT_ADDRESS_FIELD_NUMBER = 3;
+    private java.lang.Object clientAddress_;
+    /**
+     * <code>optional string client_address = 3;</code>
+     */
+    public boolean hasClientAddress() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional string client_address = 3;</code>
+     */
+    public java.lang.String getClientAddress() {
+      java.lang.Object ref = clientAddress_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          clientAddress_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string client_address = 3;</code>
+     */
+    public com.google.protobuf.ByteString
+        getClientAddressBytes() {
+      java.lang.Object ref = clientAddress_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        clientAddress_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string user_name = 4;
+    public static final int USER_NAME_FIELD_NUMBER = 4;
+    private java.lang.Object userName_;
+    /**
+     * <code>optional string user_name = 4;</code>
+     */
+    public boolean hasUserName() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional string user_name = 4;</code>
+     */
+    public java.lang.String getUserName() {
+      java.lang.Object ref = userName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          userName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string user_name = 4;</code>
+     */
+    public com.google.protobuf.ByteString
+        getUserNameBytes() {
+      java.lang.Object ref = userName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        userName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional uint32 limit = 5 [default = 10];
+    public static final int LIMIT_FIELD_NUMBER = 5;
+    private int limit_;
+    /**
+     * <code>optional uint32 limit = 5 [default = 10];</code>
+     */
+    public boolean hasLimit() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional uint32 limit = 5 [default = 10];</code>
+     */
+    public int getLimit() {
+      return limit_;
+    }
+
+    // optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];
+    public static final int FILTER_BY_OPERATOR_FIELD_NUMBER = 6;
+    private org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator filterByOperator_;
+    /**
+     * <code>optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];</code>
+     */
+    public boolean hasFilterByOperator() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator getFilterByOperator() {
+      return filterByOperator_;
+    }
+
+    // optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;
+    public static final int LOG_TYPE_FIELD_NUMBER = 7;
+    private org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType logType_;
+    /**
+     * <code>optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;</code>
+     */
+    public boolean hasLogType() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType getLogType() {
+      return logType_;
+    }
+
+    private void initFields() {
+      regionName_ = "";
+      tableName_ = "";
+      clientAddress_ = "";
+      userName_ = "";
+      limit_ = 10;
+      filterByOperator_ = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator.OR;
+      logType_ = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType.SLOW_LOG;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getRegionNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getTableNameBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, getClientAddressBytes());
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBytes(4, getUserNameBytes());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt32(5, limit_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeEnum(6, filterByOperator_.getNumber());
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeEnum(7, logType_.getNumber());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getRegionNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getTableNameBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getClientAddressBytes());
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(4, getUserNameBytes());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(5, limit_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(6, filterByOperator_.getNumber());
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(7, logType_.getNumber());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest other = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest) obj;
+
+      boolean result = true;
+      result = result && (hasRegionName() == other.hasRegionName());
+      if (hasRegionName()) {
+        result = result && getRegionName()
+            .equals(other.getRegionName());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasClientAddress() == other.hasClientAddress());
+      if (hasClientAddress()) {
+        result = result && getClientAddress()
+            .equals(other.getClientAddress());
+      }
+      result = result && (hasUserName() == other.hasUserName());
+      if (hasUserName()) {
+        result = result && getUserName()
+            .equals(other.getUserName());
+      }
+      result = result && (hasLimit() == other.hasLimit());
+      if (hasLimit()) {
+        result = result && (getLimit()
+            == other.getLimit());
+      }
+      result = result && (hasFilterByOperator() == other.hasFilterByOperator());
+      if (hasFilterByOperator()) {
+        result = result &&
+            (getFilterByOperator() == other.getFilterByOperator());
+      }
+      result = result && (hasLogType() == other.hasLogType());
+      if (hasLogType()) {
+        result = result &&
+            (getLogType() == other.getLogType());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRegionName()) {
+        hash = (37 * hash) + REGION_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionName().hashCode();
+      }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasClientAddress()) {
+        hash = (37 * hash) + CLIENT_ADDRESS_FIELD_NUMBER;
+        hash = (53 * hash) + getClientAddress().hashCode();
+      }
+      if (hasUserName()) {
+        hash = (37 * hash) + USER_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getUserName().hashCode();
+      }
+      if (hasLimit()) {
+        hash = (37 * hash) + LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + getLimit();
+      }
+      if (hasFilterByOperator()) {
+        hash = (37 * hash) + FILTER_BY_OPERATOR_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getFilterByOperator());
+      }
+      if (hasLogType()) {
+        hash = (37 * hash) + LOG_TYPE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getLogType());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.SlowLogResponseRequest}
+     *
+     * <pre>
+     **
+     * Slow/Large log (LogRequest) use-case specific RPC request. This request payload will be
+     * converted in bytes and sent to generic RPC API: GetLogEntries
+     * LogRequest message has two params:
+     * 1. log_class_name: SlowLogResponseRequest (for Slow/Large log use-case)
+     * 2. log_message: SlowLogResponseRequest converted in bytes (for Slow/Large log use-case)
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponseRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponseRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        regionName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        tableName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        clientAddress_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        userName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000008);
+        limit_ = 10;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        filterByOperator_ = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator.OR;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        logType_ = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType.SLOW_LOG;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponseRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest result = new org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.regionName_ = regionName_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.tableName_ = tableName_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.clientAddress_ = clientAddress_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.userName_ = userName_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.limit_ = limit_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.filterByOperator_ = filterByOperator_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.logType_ = logType_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.getDefaultInstance()) return this;
+        if (other.hasRegionName()) {
+          bitField0_ |= 0x00000001;
+          regionName_ = other.regionName_;
+          onChanged();
+        }
+        if (other.hasTableName()) {
+          bitField0_ |= 0x00000002;
+          tableName_ = other.tableName_;
+          onChanged();
+        }
+        if (other.hasClientAddress()) {
+          bitField0_ |= 0x00000004;
+          clientAddress_ = other.clientAddress_;
+          onChanged();
+        }
+        if (other.hasUserName()) {
+          bitField0_ |= 0x00000008;
+          userName_ = other.userName_;
+          onChanged();
+        }
+        if (other.hasLimit()) {
+          setLimit(other.getLimit());
+        }
+        if (other.hasFilterByOperator()) {
+          setFilterByOperator(other.getFilterByOperator());
+        }
+        if (other.hasLogType()) {
+          setLogType(other.getLogType());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional string region_name = 1;
+      private java.lang.Object regionName_ = "";
+      /**
+       * <code>optional string region_name = 1;</code>
+       */
+      public boolean hasRegionName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional string region_name = 1;</code>
+       */
+      public java.lang.String getRegionName() {
+        java.lang.Object ref = regionName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          regionName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string region_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getRegionNameBytes() {
+        java.lang.Object ref = regionName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          regionName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string region_name = 1;</code>
+       */
+      public Builder setRegionName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        regionName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string region_name = 1;</code>
+       */
+      public Builder clearRegionName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        regionName_ = getDefaultInstance().getRegionName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string region_name = 1;</code>
+       */
+      public Builder setRegionNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        regionName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string table_name = 2;
+      private java.lang.Object tableName_ = "";
+      /**
+       * <code>optional string table_name = 2;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional string table_name = 2;</code>
+       */
+      public java.lang.String getTableName() {
+        java.lang.Object ref = tableName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          tableName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string table_name = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getTableNameBytes() {
+        java.lang.Object ref = tableName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          tableName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string table_name = 2;</code>
+       */
+      public Builder setTableName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        tableName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string table_name = 2;</code>
+       */
+      public Builder clearTableName() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        tableName_ = getDefaultInstance().getTableName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string table_name = 2;</code>
+       */
+      public Builder setTableNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        tableName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string client_address = 3;
+      private java.lang.Object clientAddress_ = "";
+      /**
+       * <code>optional string client_address = 3;</code>
+       */
+      public boolean hasClientAddress() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional string client_address = 3;</code>
+       */
+      public java.lang.String getClientAddress() {
+        java.lang.Object ref = clientAddress_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          clientAddress_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string client_address = 3;</code>
+       */
+      public com.google.protobuf.ByteString
+          getClientAddressBytes() {
+        java.lang.Object ref = clientAddress_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          clientAddress_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string client_address = 3;</code>
+       */
+      public Builder setClientAddress(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        clientAddress_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string client_address = 3;</code>
+       */
+      public Builder clearClientAddress() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        clientAddress_ = getDefaultInstance().getClientAddress();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string client_address = 3;</code>
+       */
+      public Builder setClientAddressBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        clientAddress_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string user_name = 4;
+      private java.lang.Object userName_ = "";
+      /**
+       * <code>optional string user_name = 4;</code>
+       */
+      public boolean hasUserName() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional string user_name = 4;</code>
+       */
+      public java.lang.String getUserName() {
+        java.lang.Object ref = userName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          userName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string user_name = 4;</code>
+       */
+      public com.google.protobuf.ByteString
+          getUserNameBytes() {
+        java.lang.Object ref = userName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          userName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string user_name = 4;</code>
+       */
+      public Builder setUserName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        userName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string user_name = 4;</code>
+       */
+      public Builder clearUserName() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        userName_ = getDefaultInstance().getUserName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string user_name = 4;</code>
+       */
+      public Builder setUserNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        userName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional uint32 limit = 5 [default = 10];
+      private int limit_ = 10;
+      /**
+       * <code>optional uint32 limit = 5 [default = 10];</code>
+       */
+      public boolean hasLimit() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional uint32 limit = 5 [default = 10];</code>
+       */
+      public int getLimit() {
+        return limit_;
+      }
+      /**
+       * <code>optional uint32 limit = 5 [default = 10];</code>
+       */
+      public Builder setLimit(int value) {
+        bitField0_ |= 0x00000010;
+        limit_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 limit = 5 [default = 10];</code>
+       */
+      public Builder clearLimit() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        limit_ = 10;
+        onChanged();
+        return this;
+      }
+
+      // optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];
+      private org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator filterByOperator_ = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator.OR;
+      /**
+       * <code>optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];</code>
+       */
+      public boolean hasFilterByOperator() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator getFilterByOperator() {
+        return filterByOperator_;
+      }
+      /**
+       * <code>optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];</code>
+       */
+      public Builder setFilterByOperator(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000020;
+        filterByOperator_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SlowLogResponseRequest.FilterByOperator filter_by_operator = 6 [default = OR];</code>
+       */
+      public Builder clearFilterByOperator() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        filterByOperator_ = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.FilterByOperator.OR;
+        onChanged();
+        return this;
+      }
+
+      // optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;
+      private org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType logType_ = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType.SLOW_LOG;
+      /**
+       * <code>optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;</code>
+       */
+      public boolean hasLogType() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType getLogType() {
+        return logType_;
+      }
+      /**
+       * <code>optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;</code>
+       */
+      public Builder setLogType(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000040;
+        logType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.SlowLogResponseRequest.LogType log_type = 7;</code>
+       */
+      public Builder clearLogType() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        logType_ = org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest.LogType.SLOW_LOG;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SlowLogResponseRequest)
+    }
+
+    static {
+      defaultInstance = new SlowLogResponseRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SlowLogResponseRequest)
+  }
+
+  public interface SlowLogResponsesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload> 
+        getSlowLogPayloadsList();
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload getSlowLogPayloads(int index);
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    int getSlowLogPayloadsCount();
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder> 
+        getSlowLogPayloadsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder getSlowLogPayloadsOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.SlowLogResponses}
+   *
+   * <pre>
+   **
+   * Slow/Large log (LogEntry) use-case specific RPC response. This response payload will be
+   * converted in bytes by servers and sent as response to generic RPC API: GetLogEntries
+   * LogEntry message has two params:
+   * 1. log_class_name: SlowLogResponses (for Slow/Large log use-case)
+   * 2. log_message: SlowLogResponses converted in bytes (for Slow/Large log use-case)
+   * </pre>
+   */
+  public static final class SlowLogResponses extends
+      com.google.protobuf.GeneratedMessage
+      implements SlowLogResponsesOrBuilder {
+    // Use SlowLogResponses.newBuilder() to construct.
+    private SlowLogResponses(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SlowLogResponses(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SlowLogResponses defaultInstance;
+    public static SlowLogResponses getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SlowLogResponses getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SlowLogResponses(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                slowLogPayloads_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              slowLogPayloads_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          slowLogPayloads_ = java.util.Collections.unmodifiableList(slowLogPayloads_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponses_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponses_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<SlowLogResponses> PARSER =
+        new com.google.protobuf.AbstractParser<SlowLogResponses>() {
+      public SlowLogResponses parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new SlowLogResponses(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<SlowLogResponses> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;
+    public static final int SLOW_LOG_PAYLOADS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload> slowLogPayloads_;
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload> getSlowLogPayloadsList() {
+      return slowLogPayloads_;
+    }
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder> 
+        getSlowLogPayloadsOrBuilderList() {
+      return slowLogPayloads_;
+    }
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    public int getSlowLogPayloadsCount() {
+      return slowLogPayloads_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload getSlowLogPayloads(int index) {
+      return slowLogPayloads_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder getSlowLogPayloadsOrBuilder(
+        int index) {
+      return slowLogPayloads_.get(index);
+    }
+
+    private void initFields() {
+      slowLogPayloads_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      for (int i = 0; i < getSlowLogPayloadsCount(); i++) {
+        if (!getSlowLogPayloads(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < slowLogPayloads_.size(); i++) {
+        output.writeMessage(1, slowLogPayloads_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < slowLogPayloads_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, slowLogPayloads_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses other = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses) obj;
+
+      boolean result = true;
+      result = result && getSlowLogPayloadsList()
+          .equals(other.getSlowLogPayloadsList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getSlowLogPayloadsCount() > 0) {
+        hash = (37 * hash) + SLOW_LOG_PAYLOADS_FIELD_NUMBER;
+        hash = (53 * hash) + getSlowLogPayloadsList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.SlowLogResponses}
+     *
+     * <pre>
+     **
+     * Slow/Large log (LogEntry) use-case specific RPC response. This response payload will be
+     * converted in bytes by servers and sent as response to generic RPC API: GetLogEntries
+     * LogEntry message has two params:
+     * 1. log_class_name: SlowLogResponses (for Slow/Large log use-case)
+     * 2. log_message: SlowLogResponses converted in bytes (for Slow/Large log use-case)
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponsesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponses_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponses_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getSlowLogPayloadsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (slowLogPayloadsBuilder_ == null) {
+          slowLogPayloads_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          slowLogPayloadsBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_SlowLogResponses_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses build() {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses result = new org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses(this);
+        int from_bitField0_ = bitField0_;
+        if (slowLogPayloadsBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            slowLogPayloads_ = java.util.Collections.unmodifiableList(slowLogPayloads_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.slowLogPayloads_ = slowLogPayloads_;
+        } else {
+          result.slowLogPayloads_ = slowLogPayloadsBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses.getDefaultInstance()) return this;
+        if (slowLogPayloadsBuilder_ == null) {
+          if (!other.slowLogPayloads_.isEmpty()) {
+            if (slowLogPayloads_.isEmpty()) {
+              slowLogPayloads_ = other.slowLogPayloads_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureSlowLogPayloadsIsMutable();
+              slowLogPayloads_.addAll(other.slowLogPayloads_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.slowLogPayloads_.isEmpty()) {
+            if (slowLogPayloadsBuilder_.isEmpty()) {
+              slowLogPayloadsBuilder_.dispose();
+              slowLogPayloadsBuilder_ = null;
+              slowLogPayloads_ = other.slowLogPayloads_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              slowLogPayloadsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getSlowLogPayloadsFieldBuilder() : null;
+            } else {
+              slowLogPayloadsBuilder_.addAllMessages(other.slowLogPayloads_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getSlowLogPayloadsCount(); i++) {
+          if (!getSlowLogPayloads(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload> slowLogPayloads_ =
+        java.util.Collections.emptyList();
+      private void ensureSlowLogPayloadsIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          slowLogPayloads_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload>(slowLogPayloads_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder> slowLogPayloadsBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload> getSlowLogPayloadsList() {
+        if (slowLogPayloadsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(slowLogPayloads_);
+        } else {
+          return slowLogPayloadsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public int getSlowLogPayloadsCount() {
+        if (slowLogPayloadsBuilder_ == null) {
+          return slowLogPayloads_.size();
+        } else {
+          return slowLogPayloadsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload getSlowLogPayloads(int index) {
+        if (slowLogPayloadsBuilder_ == null) {
+          return slowLogPayloads_.get(index);
+        } else {
+          return slowLogPayloadsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder setSlowLogPayloads(
+          int index, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload value) {
+        if (slowLogPayloadsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureSlowLogPayloadsIsMutable();
+          slowLogPayloads_.set(index, value);
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder setSlowLogPayloads(
+          int index, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder builderForValue) {
+        if (slowLogPayloadsBuilder_ == null) {
+          ensureSlowLogPayloadsIsMutable();
+          slowLogPayloads_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder addSlowLogPayloads(org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload value) {
+        if (slowLogPayloadsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureSlowLogPayloadsIsMutable();
+          slowLogPayloads_.add(value);
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder addSlowLogPayloads(
+          int index, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload value) {
+        if (slowLogPayloadsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureSlowLogPayloadsIsMutable();
+          slowLogPayloads_.add(index, value);
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder addSlowLogPayloads(
+          org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder builderForValue) {
+        if (slowLogPayloadsBuilder_ == null) {
+          ensureSlowLogPayloadsIsMutable();
+          slowLogPayloads_.add(builderForValue.build());
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder addSlowLogPayloads(
+          int index, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder builderForValue) {
+        if (slowLogPayloadsBuilder_ == null) {
+          ensureSlowLogPayloadsIsMutable();
+          slowLogPayloads_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder addAllSlowLogPayloads(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload> values) {
+        if (slowLogPayloadsBuilder_ == null) {
+          ensureSlowLogPayloadsIsMutable();
+          super.addAll(values, slowLogPayloads_);
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder clearSlowLogPayloads() {
+        if (slowLogPayloadsBuilder_ == null) {
+          slowLogPayloads_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public Builder removeSlowLogPayloads(int index) {
+        if (slowLogPayloadsBuilder_ == null) {
+          ensureSlowLogPayloadsIsMutable();
+          slowLogPayloads_.remove(index);
+          onChanged();
+        } else {
+          slowLogPayloadsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder getSlowLogPayloadsBuilder(
+          int index) {
+        return getSlowLogPayloadsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder getSlowLogPayloadsOrBuilder(
+          int index) {
+        if (slowLogPayloadsBuilder_ == null) {
+          return slowLogPayloads_.get(index);  } else {
+          return slowLogPayloadsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder> 
+           getSlowLogPayloadsOrBuilderList() {
+        if (slowLogPayloadsBuilder_ != null) {
+          return slowLogPayloadsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(slowLogPayloads_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder addSlowLogPayloadsBuilder() {
+        return getSlowLogPayloadsFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder addSlowLogPayloadsBuilder(
+          int index) {
+        return getSlowLogPayloadsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.SlowLogPayload slow_log_payloads = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder> 
+           getSlowLogPayloadsBuilderList() {
+        return getSlowLogPayloadsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder> 
+          getSlowLogPayloadsFieldBuilder() {
+        if (slowLogPayloadsBuilder_ == null) {
+          slowLogPayloadsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload.Builder, org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayloadOrBuilder>(
+                  slowLogPayloads_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          slowLogPayloads_ = null;
+        }
+        return slowLogPayloadsBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SlowLogResponses)
+    }
+
+    static {
+      defaultInstance = new SlowLogResponses(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SlowLogResponses)
+  }
+
+  public interface ClearSlowLogResponseRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  /**
+   * Protobuf type {@code hbase.pb.ClearSlowLogResponseRequest}
+   */
+  public static final class ClearSlowLogResponseRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements ClearSlowLogResponseRequestOrBuilder {
+    // Use ClearSlowLogResponseRequest.newBuilder() to construct.
+    private ClearSlowLogResponseRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ClearSlowLogResponseRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ClearSlowLogResponseRequest defaultInstance;
+    public static ClearSlowLogResponseRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ClearSlowLogResponseRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ClearSlowLogResponseRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponseRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponseRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ClearSlowLogResponseRequest> PARSER =
+        new com.google.protobuf.AbstractParser<ClearSlowLogResponseRequest>() {
+      public ClearSlowLogResponseRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ClearSlowLogResponseRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ClearSlowLogResponseRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest other = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest) obj;
+
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.ClearSlowLogResponseRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponseRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponseRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponseRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest result = new org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest(this);
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.ClearSlowLogResponseRequest)
+    }
+
+    static {
+      defaultInstance = new ClearSlowLogResponseRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ClearSlowLogResponseRequest)
+  }
+
+  public interface ClearSlowLogResponsesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required bool is_cleaned = 1;
+    /**
+     * <code>required bool is_cleaned = 1;</code>
+     */
+    boolean hasIsCleaned();
+    /**
+     * <code>required bool is_cleaned = 1;</code>
+     */
+    boolean getIsCleaned();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.ClearSlowLogResponses}
+   */
+  public static final class ClearSlowLogResponses extends
+      com.google.protobuf.GeneratedMessage
+      implements ClearSlowLogResponsesOrBuilder {
+    // Use ClearSlowLogResponses.newBuilder() to construct.
+    private ClearSlowLogResponses(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ClearSlowLogResponses(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ClearSlowLogResponses defaultInstance;
+    public static ClearSlowLogResponses getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ClearSlowLogResponses getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ClearSlowLogResponses(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              isCleaned_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponses_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponses_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ClearSlowLogResponses> PARSER =
+        new com.google.protobuf.AbstractParser<ClearSlowLogResponses>() {
+      public ClearSlowLogResponses parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ClearSlowLogResponses(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ClearSlowLogResponses> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required bool is_cleaned = 1;
+    public static final int IS_CLEANED_FIELD_NUMBER = 1;
+    private boolean isCleaned_;
+    /**
+     * <code>required bool is_cleaned = 1;</code>
+     */
+    public boolean hasIsCleaned() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required bool is_cleaned = 1;</code>
+     */
+    public boolean getIsCleaned() {
+      return isCleaned_;
+    }
+
+    private void initFields() {
+      isCleaned_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasIsCleaned()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBool(1, isCleaned_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(1, isCleaned_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses other = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses) obj;
+
+      boolean result = true;
+      result = result && (hasIsCleaned() == other.hasIsCleaned());
+      if (hasIsCleaned()) {
+        result = result && (getIsCleaned()
+            == other.getIsCleaned());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasIsCleaned()) {
+        hash = (37 * hash) + IS_CLEANED_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getIsCleaned());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.ClearSlowLogResponses}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponsesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponses_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponses_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        isCleaned_ = false;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_hbase_pb_ClearSlowLogResponses_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses build() {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses result = new org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.isCleaned_ = isCleaned_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.getDefaultInstance()) return this;
+        if (other.hasIsCleaned()) {
+          setIsCleaned(other.getIsCleaned());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasIsCleaned()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required bool is_cleaned = 1;
+      private boolean isCleaned_ ;
+      /**
+       * <code>required bool is_cleaned = 1;</code>
+       */
+      public boolean hasIsCleaned() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required bool is_cleaned = 1;</code>
+       */
+      public boolean getIsCleaned() {
+        return isCleaned_;
+      }
+      /**
+       * <code>required bool is_cleaned = 1;</code>
+       */
+      public Builder setIsCleaned(boolean value) {
+        bitField0_ |= 0x00000001;
+        isCleaned_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool is_cleaned = 1;</code>
+       */
+      public Builder clearIsCleaned() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        isCleaned_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.ClearSlowLogResponses)
+    }
+
+    static {
+      defaultInstance = new ClearSlowLogResponses(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ClearSlowLogResponses)
+  }
+
   /**
    * Protobuf service {@code hbase.pb.AdminService}
    */
@@ -23456,6 +26449,22 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest request,
           com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse> done);
 
+      /**
+       * <code>rpc ClearSlowLogsResponses(.hbase.pb.ClearSlowLogResponseRequest) returns (.hbase.pb.ClearSlowLogResponses);</code>
+       */
+      public abstract void clearSlowLogsResponses(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses> done);
+
+      /**
+       * <code>rpc GetLogEntries(.hbase.pb.LogRequest) returns (.hbase.pb.LogEntry);</code>
+       */
+      public abstract void getLogEntries(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry> done);
+
     }
 
     public static com.google.protobuf.Service newReflectiveService(
@@ -23605,6 +26614,22 @@ public final class AdminProtos {
           impl.updateConfiguration(controller, request, done);
         }
 
+        @java.lang.Override
+        public  void clearSlowLogsResponses(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses> done) {
+          impl.clearSlowLogsResponses(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLogEntries(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry> done) {
+          impl.getLogEntries(controller, request, done);
+        }
+
       };
     }
 
@@ -23663,6 +26688,10 @@ public final class AdminProtos {
               return impl.updateFavoredNodes(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest)request);
             case 17:
               return impl.updateConfiguration(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest)request);
+            case 18:
+              return impl.clearSlowLogsResponses(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest)request);
+            case 19:
+              return impl.getLogEntries(controller, (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -23713,6 +26742,10 @@ public final class AdminProtos {
               return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.getDefaultInstance();
             case 17:
               return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest.getDefaultInstance();
+            case 18:
+              return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.getDefaultInstance();
+            case 19:
+              return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -23763,6 +26796,10 @@ public final class AdminProtos {
               return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance();
             case 17:
               return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance();
+            case 18:
+              return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.getDefaultInstance();
+            case 19:
+              return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -23915,6 +26952,22 @@ public final class AdminProtos {
         org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest request,
         com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse> done);
 
+    /**
+     * <code>rpc ClearSlowLogsResponses(.hbase.pb.ClearSlowLogResponseRequest) returns (.hbase.pb.ClearSlowLogResponses);</code>
+     */
+    public abstract void clearSlowLogsResponses(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses> done);
+
+    /**
+     * <code>rpc GetLogEntries(.hbase.pb.LogRequest) returns (.hbase.pb.LogEntry);</code>
+     */
+    public abstract void getLogEntries(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry> done);
+
     public static final
         com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
@@ -24027,6 +27080,16 @@ public final class AdminProtos {
             com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse>specializeCallback(
               done));
           return;
+        case 18:
+          this.clearSlowLogsResponses(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses>specializeCallback(
+              done));
+          return;
+        case 19:
+          this.getLogEntries(controller, (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry>specializeCallback(
+              done));
+          return;
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -24077,6 +27140,10 @@ public final class AdminProtos {
           return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.getDefaultInstance();
         case 17:
           return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest.getDefaultInstance();
+        case 18:
+          return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest.getDefaultInstance();
+        case 19:
+          return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -24127,6 +27194,10 @@ public final class AdminProtos {
           return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance();
         case 17:
           return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance();
+        case 18:
+          return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.getDefaultInstance();
+        case 19:
+          return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -24417,6 +27488,36 @@ public final class AdminProtos {
             org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.class,
             org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance()));
       }
+
+      public  void clearSlowLogsResponses(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(18),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.class,
+            org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.getDefaultInstance()));
+      }
+
+      public  void getLogEntries(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(19),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.class,
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance()));
+      }
     }
 
     public static BlockingInterface newBlockingStub(
@@ -24514,6 +27615,16 @@ public final class AdminProtos {
           com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest request)
           throws com.google.protobuf.ServiceException;
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses clearSlowLogsResponses(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request)
+          throws com.google.protobuf.ServiceException;
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry getLogEntries(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request)
+          throws com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
@@ -24738,6 +27849,30 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance());
       }
 
+
+      public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses clearSlowLogsResponses(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(18),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses.getDefaultInstance());
+      }
+
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry getLogEntries(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(19),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance());
+      }
+
     }
 
     // @@protoc_insertion_point(class_scope:hbase.pb.AdminService)
@@ -24933,6 +28068,26 @@ public final class AdminProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_UpdateConfigurationResponse_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SlowLogResponseRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_SlowLogResponseRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SlowLogResponses_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_SlowLogResponses_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_ClearSlowLogResponseRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_ClearSlowLogResponseRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_ClearSlowLogResponses_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_ClearSlowLogResponses_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -24943,122 +28098,139 @@ public final class AdminProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\013Admin.proto\022\010hbase.pb\032\014Client.proto\032\013H" +
-      "Base.proto\032\tWAL.proto\"[\n\024GetRegionInfoRe" +
-      "quest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionS" +
-      "pecifier\022\030\n\020compaction_state\030\002 \001(\010\"\353\001\n\025G" +
-      "etRegionInfoResponse\022)\n\013region_info\030\001 \002(" +
-      "\0132\024.hbase.pb.RegionInfo\022I\n\020compaction_st" +
-      "ate\030\002 \001(\0162/.hbase.pb.GetRegionInfoRespon" +
-      "se.CompactionState\022\024\n\014isRecovering\030\003 \001(\010" +
-      "\"F\n\017CompactionState\022\010\n\004NONE\020\000\022\t\n\005MINOR\020\001" +
-      "\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_AND_MINOR\020\003\"P\n\023GetS",
-      "toreFileRequest\022)\n\006region\030\001 \002(\0132\031.hbase." +
-      "pb.RegionSpecifier\022\016\n\006family\030\002 \003(\014\"*\n\024Ge" +
-      "tStoreFileResponse\022\022\n\nstore_file\030\001 \003(\t\"\030" +
-      "\n\026GetOnlineRegionRequest\"D\n\027GetOnlineReg" +
-      "ionResponse\022)\n\013region_info\030\001 \003(\0132\024.hbase" +
-      ".pb.RegionInfo\"\263\002\n\021OpenRegionRequest\022=\n\t" +
-      "open_info\030\001 \003(\0132*.hbase.pb.OpenRegionReq" +
-      "uest.RegionOpenInfo\022\027\n\017serverStartCode\030\002" +
-      " \001(\004\022\032\n\022master_system_time\030\005 \001(\004\032\251\001\n\016Reg" +
-      "ionOpenInfo\022$\n\006region\030\001 \002(\0132\024.hbase.pb.R",
-      "egionInfo\022\037\n\027version_of_offline_node\030\002 \001" +
-      "(\r\022+\n\rfavored_nodes\030\003 \003(\0132\024.hbase.pb.Ser" +
-      "verName\022#\n\033openForDistributedLogReplay\030\004" +
-      " \001(\010\"\246\001\n\022OpenRegionResponse\022F\n\ropening_s" +
-      "tate\030\001 \003(\0162/.hbase.pb.OpenRegionResponse" +
-      ".RegionOpeningState\"H\n\022RegionOpeningStat" +
-      "e\022\n\n\006OPENED\020\000\022\022\n\016ALREADY_OPENED\020\001\022\022\n\016FAI" +
-      "LED_OPENING\020\002\"?\n\023WarmupRegionRequest\022(\n\n" +
-      "regionInfo\030\001 \002(\0132\024.hbase.pb.RegionInfo\"\026" +
-      "\n\024WarmupRegionResponse\"\313\001\n\022CloseRegionRe",
-      "quest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionS" +
-      "pecifier\022\037\n\027version_of_closing_node\030\002 \001(" +
-      "\r\022\036\n\020transition_in_ZK\030\003 \001(\010:\004true\0220\n\022des" +
-      "tination_server\030\004 \001(\0132\024.hbase.pb.ServerN" +
-      "ame\022\027\n\017serverStartCode\030\005 \001(\004\"%\n\023CloseReg" +
-      "ionResponse\022\016\n\006closed\030\001 \002(\010\"y\n\022FlushRegi" +
-      "onRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Reg" +
-      "ionSpecifier\022\030\n\020if_older_than_ts\030\002 \001(\004\022\036" +
-      "\n\026write_flush_wal_marker\030\003 \001(\010\"_\n\023FlushR" +
-      "egionResponse\022\027\n\017last_flush_time\030\001 \002(\004\022\017",
-      "\n\007flushed\030\002 \001(\010\022\036\n\026wrote_flush_wal_marke" +
-      "r\030\003 \001(\010\"T\n\022SplitRegionRequest\022)\n\006region\030" +
-      "\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\023\n\013spli" +
-      "t_point\030\002 \001(\014\"\025\n\023SplitRegionResponse\"`\n\024" +
-      "CompactRegionRequest\022)\n\006region\030\001 \002(\0132\031.h" +
-      "base.pb.RegionSpecifier\022\r\n\005major\030\002 \001(\010\022\016" +
-      "\n\006family\030\003 \001(\014\"\027\n\025CompactRegionResponse\"" +
-      "*\n\027CompactionSwitchRequest\022\017\n\007enabled\030\001 " +
-      "\002(\010\".\n\030CompactionSwitchResponse\022\022\n\nprev_" +
-      "state\030\001 \002(\010\"\315\001\n\031UpdateFavoredNodesReques",
-      "t\022I\n\013update_info\030\001 \003(\01324.hbase.pb.Update" +
-      "FavoredNodesRequest.RegionUpdateInfo\032e\n\020" +
-      "RegionUpdateInfo\022$\n\006region\030\001 \002(\0132\024.hbase" +
-      ".pb.RegionInfo\022+\n\rfavored_nodes\030\002 \003(\0132\024." +
-      "hbase.pb.ServerName\".\n\032UpdateFavoredNode" +
-      "sResponse\022\020\n\010response\030\001 \001(\r\"\244\001\n\023MergeReg" +
-      "ionsRequest\022+\n\010region_a\030\001 \002(\0132\031.hbase.pb" +
-      ".RegionSpecifier\022+\n\010region_b\030\002 \002(\0132\031.hba" +
-      "se.pb.RegionSpecifier\022\027\n\010forcible\030\003 \001(\010:" +
-      "\005false\022\032\n\022master_system_time\030\004 \001(\004\"\026\n\024Me",
-      "rgeRegionsResponse\"a\n\010WALEntry\022\035\n\003key\030\001 " +
-      "\002(\0132\020.hbase.pb.WALKey\022\027\n\017key_value_bytes" +
-      "\030\002 \003(\014\022\035\n\025associated_cell_count\030\003 \001(\005\"\242\001" +
-      "\n\030ReplicateWALEntryRequest\022!\n\005entry\030\001 \003(" +
-      "\0132\022.hbase.pb.WALEntry\022\034\n\024replicationClus" +
-      "terId\030\002 \001(\t\022\"\n\032sourceBaseNamespaceDirPat" +
-      "h\030\003 \001(\t\022!\n\031sourceHFileArchiveDirPath\030\004 \001" +
-      "(\t\"\033\n\031ReplicateWALEntryResponse\"\026\n\024RollW" +
-      "ALWriterRequest\"0\n\025RollWALWriterResponse" +
-      "\022\027\n\017region_to_flush\030\001 \003(\014\"#\n\021StopServerR",
-      "equest\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServerResp" +
-      "onse\"\026\n\024GetServerInfoRequest\"K\n\nServerIn" +
-      "fo\022)\n\013server_name\030\001 \002(\0132\024.hbase.pb.Serve" +
-      "rName\022\022\n\nwebui_port\030\002 \001(\r\"B\n\025GetServerIn" +
-      "foResponse\022)\n\013server_info\030\001 \002(\0132\024.hbase." +
-      "pb.ServerInfo\"\034\n\032UpdateConfigurationRequ" +
-      "est\"\035\n\033UpdateConfigurationResponse2\342\013\n\014A" +
-      "dminService\022P\n\rGetRegionInfo\022\036.hbase.pb." +
-      "GetRegionInfoRequest\032\037.hbase.pb.GetRegio" +
-      "nInfoResponse\022M\n\014GetStoreFile\022\035.hbase.pb",
-      ".GetStoreFileRequest\032\036.hbase.pb.GetStore" +
-      "FileResponse\022V\n\017GetOnlineRegion\022 .hbase." +
-      "pb.GetOnlineRegionRequest\032!.hbase.pb.Get" +
-      "OnlineRegionResponse\022G\n\nOpenRegion\022\033.hba" +
-      "se.pb.OpenRegionRequest\032\034.hbase.pb.OpenR" +
-      "egionResponse\022M\n\014WarmupRegion\022\035.hbase.pb" +
-      ".WarmupRegionRequest\032\036.hbase.pb.WarmupRe" +
-      "gionResponse\022J\n\013CloseRegion\022\034.hbase.pb.C" +
-      "loseRegionRequest\032\035.hbase.pb.CloseRegion" +
-      "Response\022J\n\013FlushRegion\022\034.hbase.pb.Flush",
-      "RegionRequest\032\035.hbase.pb.FlushRegionResp" +
-      "onse\022J\n\013SplitRegion\022\034.hbase.pb.SplitRegi" +
-      "onRequest\032\035.hbase.pb.SplitRegionResponse" +
-      "\022Y\n\020CompactionSwitch\022!.hbase.pb.Compacti" +
-      "onSwitchRequest\032\".hbase.pb.CompactionSwi" +
-      "tchResponse\022P\n\rCompactRegion\022\036.hbase.pb." +
-      "CompactRegionRequest\032\037.hbase.pb.CompactR" +
-      "egionResponse\022M\n\014MergeRegions\022\035.hbase.pb" +
-      ".MergeRegionsRequest\032\036.hbase.pb.MergeReg" +
-      "ionsResponse\022\\\n\021ReplicateWALEntry\022\".hbas",
+      "Base.proto\032\tWAL.proto\032\020TooSlowLog.proto\"" +
+      "[\n\024GetRegionInfoRequest\022)\n\006region\030\001 \002(\0132" +
+      "\031.hbase.pb.RegionSpecifier\022\030\n\020compaction" +
+      "_state\030\002 \001(\010\"\353\001\n\025GetRegionInfoResponse\022)" +
+      "\n\013region_info\030\001 \002(\0132\024.hbase.pb.RegionInf" +
+      "o\022I\n\020compaction_state\030\002 \001(\0162/.hbase.pb.G" +
+      "etRegionInfoResponse.CompactionState\022\024\n\014" +
+      "isRecovering\030\003 \001(\010\"F\n\017CompactionState\022\010\n" +
+      "\004NONE\020\000\022\t\n\005MINOR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_A",
+      "ND_MINOR\020\003\"P\n\023GetStoreFileRequest\022)\n\006reg" +
+      "ion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\016\n\006" +
+      "family\030\002 \003(\014\"*\n\024GetStoreFileResponse\022\022\n\n" +
+      "store_file\030\001 \003(\t\"\030\n\026GetOnlineRegionReque" +
+      "st\"D\n\027GetOnlineRegionResponse\022)\n\013region_" +
+      "info\030\001 \003(\0132\024.hbase.pb.RegionInfo\"\263\002\n\021Ope" +
+      "nRegionRequest\022=\n\topen_info\030\001 \003(\0132*.hbas" +
+      "e.pb.OpenRegionRequest.RegionOpenInfo\022\027\n" +
+      "\017serverStartCode\030\002 \001(\004\022\032\n\022master_system_" +
+      "time\030\005 \001(\004\032\251\001\n\016RegionOpenInfo\022$\n\006region\030",
+      "\001 \002(\0132\024.hbase.pb.RegionInfo\022\037\n\027version_o" +
+      "f_offline_node\030\002 \001(\r\022+\n\rfavored_nodes\030\003 " +
+      "\003(\0132\024.hbase.pb.ServerName\022#\n\033openForDist" +
+      "ributedLogReplay\030\004 \001(\010\"\246\001\n\022OpenRegionRes" +
+      "ponse\022F\n\ropening_state\030\001 \003(\0162/.hbase.pb." +
+      "OpenRegionResponse.RegionOpeningState\"H\n" +
+      "\022RegionOpeningState\022\n\n\006OPENED\020\000\022\022\n\016ALREA" +
+      "DY_OPENED\020\001\022\022\n\016FAILED_OPENING\020\002\"?\n\023Warmu" +
+      "pRegionRequest\022(\n\nregionInfo\030\001 \002(\0132\024.hba" +
+      "se.pb.RegionInfo\"\026\n\024WarmupRegionResponse",
+      "\"\313\001\n\022CloseRegionRequest\022)\n\006region\030\001 \002(\0132" +
+      "\031.hbase.pb.RegionSpecifier\022\037\n\027version_of" +
+      "_closing_node\030\002 \001(\r\022\036\n\020transition_in_ZK\030" +
+      "\003 \001(\010:\004true\0220\n\022destination_server\030\004 \001(\0132" +
+      "\024.hbase.pb.ServerName\022\027\n\017serverStartCode" +
+      "\030\005 \001(\004\"%\n\023CloseRegionResponse\022\016\n\006closed\030" +
+      "\001 \002(\010\"y\n\022FlushRegionRequest\022)\n\006region\030\001 " +
+      "\002(\0132\031.hbase.pb.RegionSpecifier\022\030\n\020if_old" +
+      "er_than_ts\030\002 \001(\004\022\036\n\026write_flush_wal_mark" +
+      "er\030\003 \001(\010\"_\n\023FlushRegionResponse\022\027\n\017last_",
+      "flush_time\030\001 \002(\004\022\017\n\007flushed\030\002 \001(\010\022\036\n\026wro" +
+      "te_flush_wal_marker\030\003 \001(\010\"T\n\022SplitRegion" +
+      "Request\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Regio" +
+      "nSpecifier\022\023\n\013split_point\030\002 \001(\014\"\025\n\023Split" +
+      "RegionResponse\"`\n\024CompactRegionRequest\022)" +
+      "\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifie" +
+      "r\022\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003 \001(\014\"\027\n\025Comp" +
+      "actRegionResponse\"*\n\027CompactionSwitchReq" +
+      "uest\022\017\n\007enabled\030\001 \002(\010\".\n\030CompactionSwitc" +
+      "hResponse\022\022\n\nprev_state\030\001 \002(\010\"\315\001\n\031Update",
+      "FavoredNodesRequest\022I\n\013update_info\030\001 \003(\013" +
+      "24.hbase.pb.UpdateFavoredNodesRequest.Re" +
+      "gionUpdateInfo\032e\n\020RegionUpdateInfo\022$\n\006re" +
+      "gion\030\001 \002(\0132\024.hbase.pb.RegionInfo\022+\n\rfavo" +
+      "red_nodes\030\002 \003(\0132\024.hbase.pb.ServerName\".\n" +
+      "\032UpdateFavoredNodesResponse\022\020\n\010response\030" +
+      "\001 \001(\r\"\244\001\n\023MergeRegionsRequest\022+\n\010region_" +
+      "a\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022+\n\010re" +
+      "gion_b\030\002 \002(\0132\031.hbase.pb.RegionSpecifier\022" +
+      "\027\n\010forcible\030\003 \001(\010:\005false\022\032\n\022master_syste",
+      "m_time\030\004 \001(\004\"\026\n\024MergeRegionsResponse\"a\n\010" +
+      "WALEntry\022\035\n\003key\030\001 \002(\0132\020.hbase.pb.WALKey\022" +
+      "\027\n\017key_value_bytes\030\002 \003(\014\022\035\n\025associated_c" +
+      "ell_count\030\003 \001(\005\"\242\001\n\030ReplicateWALEntryReq" +
+      "uest\022!\n\005entry\030\001 \003(\0132\022.hbase.pb.WALEntry\022" +
+      "\034\n\024replicationClusterId\030\002 \001(\t\022\"\n\032sourceB" +
+      "aseNamespaceDirPath\030\003 \001(\t\022!\n\031sourceHFile" +
+      "ArchiveDirPath\030\004 \001(\t\"\033\n\031ReplicateWALEntr" +
+      "yResponse\"\026\n\024RollWALWriterRequest\"0\n\025Rol" +
+      "lWALWriterResponse\022\027\n\017region_to_flush\030\001 ",
+      "\003(\014\"#\n\021StopServerRequest\022\016\n\006reason\030\001 \002(\t" +
+      "\"\024\n\022StopServerResponse\"\026\n\024GetServerInfoR" +
+      "equest\"K\n\nServerInfo\022)\n\013server_name\030\001 \002(" +
+      "\0132\024.hbase.pb.ServerName\022\022\n\nwebui_port\030\002 " +
+      "\001(\r\"B\n\025GetServerInfoResponse\022)\n\013server_i" +
+      "nfo\030\001 \002(\0132\024.hbase.pb.ServerInfo\"\034\n\032Updat" +
+      "eConfigurationRequest\"\035\n\033UpdateConfigura" +
+      "tionResponse\"\333\002\n\026SlowLogResponseRequest\022" +
+      "\023\n\013region_name\030\001 \001(\t\022\022\n\ntable_name\030\002 \001(\t" +
+      "\022\026\n\016client_address\030\003 \001(\t\022\021\n\tuser_name\030\004 ",
+      "\001(\t\022\021\n\005limit\030\005 \001(\r:\00210\022Q\n\022filter_by_oper" +
+      "ator\030\006 \001(\01621.hbase.pb.SlowLogResponseReq" +
+      "uest.FilterByOperator:\002OR\022:\n\010log_type\030\007 " +
+      "\001(\0162(.hbase.pb.SlowLogResponseRequest.Lo" +
+      "gType\"#\n\020FilterByOperator\022\007\n\003AND\020\000\022\006\n\002OR" +
+      "\020\001\"&\n\007LogType\022\014\n\010SLOW_LOG\020\000\022\r\n\tLARGE_LOG" +
+      "\020\001\"G\n\020SlowLogResponses\0223\n\021slow_log_paylo" +
+      "ads\030\001 \003(\0132\030.hbase.pb.SlowLogPayload\"\035\n\033C" +
+      "learSlowLogResponseRequest\"+\n\025ClearSlowL" +
+      "ogResponses\022\022\n\nis_cleaned\030\001 \002(\0102\377\014\n\014Admi",
+      "nService\022P\n\rGetRegionInfo\022\036.hbase.pb.Get" +
+      "RegionInfoRequest\032\037.hbase.pb.GetRegionIn" +
+      "foResponse\022M\n\014GetStoreFile\022\035.hbase.pb.Ge" +
+      "tStoreFileRequest\032\036.hbase.pb.GetStoreFil" +
+      "eResponse\022V\n\017GetOnlineRegion\022 .hbase.pb." +
+      "GetOnlineRegionRequest\032!.hbase.pb.GetOnl" +
+      "ineRegionResponse\022G\n\nOpenRegion\022\033.hbase." +
+      "pb.OpenRegionRequest\032\034.hbase.pb.OpenRegi" +
+      "onResponse\022M\n\014WarmupRegion\022\035.hbase.pb.Wa" +
+      "rmupRegionRequest\032\036.hbase.pb.WarmupRegio",
+      "nResponse\022J\n\013CloseRegion\022\034.hbase.pb.Clos" +
+      "eRegionRequest\032\035.hbase.pb.CloseRegionRes" +
+      "ponse\022J\n\013FlushRegion\022\034.hbase.pb.FlushReg" +
+      "ionRequest\032\035.hbase.pb.FlushRegionRespons" +
+      "e\022J\n\013SplitRegion\022\034.hbase.pb.SplitRegionR" +
+      "equest\032\035.hbase.pb.SplitRegionResponse\022Y\n" +
+      "\020CompactionSwitch\022!.hbase.pb.CompactionS" +
+      "witchRequest\032\".hbase.pb.CompactionSwitch" +
+      "Response\022P\n\rCompactRegion\022\036.hbase.pb.Com" +
+      "pactRegionRequest\032\037.hbase.pb.CompactRegi",
+      "onResponse\022M\n\014MergeRegions\022\035.hbase.pb.Me" +
+      "rgeRegionsRequest\032\036.hbase.pb.MergeRegion" +
+      "sResponse\022\\\n\021ReplicateWALEntry\022\".hbase.p" +
+      "b.ReplicateWALEntryRequest\032#.hbase.pb.Re" +
+      "plicateWALEntryResponse\022Q\n\006Replay\022\".hbas" +
       "e.pb.ReplicateWALEntryRequest\032#.hbase.pb" +
-      ".ReplicateWALEntryResponse\022Q\n\006Replay\022\".h" +
-      "base.pb.ReplicateWALEntryRequest\032#.hbase" +
-      ".pb.ReplicateWALEntryResponse\022P\n\rRollWAL" +
-      "Writer\022\036.hbase.pb.RollWALWriterRequest\032\037" +
-      ".hbase.pb.RollWALWriterResponse\022P\n\rGetSe" +
-      "rverInfo\022\036.hbase.pb.GetServerInfoRequest" +
-      "\032\037.hbase.pb.GetServerInfoResponse\022G\n\nSto" +
-      "pServer\022\033.hbase.pb.StopServerRequest\032\034.h" +
-      "base.pb.StopServerResponse\022_\n\022UpdateFavo",
-      "redNodes\022#.hbase.pb.UpdateFavoredNodesRe" +
-      "quest\032$.hbase.pb.UpdateFavoredNodesRespo" +
-      "nse\022b\n\023UpdateConfiguration\022$.hbase.pb.Up" +
-      "dateConfigurationRequest\032%.hbase.pb.Upda" +
-      "teConfigurationResponseBA\n*org.apache.ha" +
-      "doop.hbase.protobuf.generatedB\013AdminProt" +
-      "osH\001\210\001\001\240\001\001"
+      ".ReplicateWALEntryResponse\022P\n\rRollWALWri" +
+      "ter\022\036.hbase.pb.RollWALWriterRequest\032\037.hb" +
+      "ase.pb.RollWALWriterResponse\022P\n\rGetServe" +
+      "rInfo\022\036.hbase.pb.GetServerInfoRequest\032\037.",
+      "hbase.pb.GetServerInfoResponse\022G\n\nStopSe" +
+      "rver\022\033.hbase.pb.StopServerRequest\032\034.hbas" +
+      "e.pb.StopServerResponse\022_\n\022UpdateFavored" +
+      "Nodes\022#.hbase.pb.UpdateFavoredNodesReque" +
+      "st\032$.hbase.pb.UpdateFavoredNodesResponse" +
+      "\022b\n\023UpdateConfiguration\022$.hbase.pb.Updat" +
+      "eConfigurationRequest\032%.hbase.pb.UpdateC" +
+      "onfigurationResponse\022`\n\026ClearSlowLogsRes" +
+      "ponses\022%.hbase.pb.ClearSlowLogResponseRe" +
+      "quest\032\037.hbase.pb.ClearSlowLogResponses\0229",
+      "\n\rGetLogEntries\022\024.hbase.pb.LogRequest\032\022." +
+      "hbase.pb.LogEntryBA\n*org.apache.hadoop.h" +
+      "base.protobuf.generatedB\013AdminProtosH\001\210\001" +
+      "\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -25293,6 +28465,30 @@ public final class AdminProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_UpdateConfigurationResponse_descriptor,
               new java.lang.String[] { });
+          internal_static_hbase_pb_SlowLogResponseRequest_descriptor =
+            getDescriptor().getMessageTypes().get(36);
+          internal_static_hbase_pb_SlowLogResponseRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_SlowLogResponseRequest_descriptor,
+              new java.lang.String[] { "RegionName", "TableName", "ClientAddress", "UserName", "Limit", "FilterByOperator", "LogType", });
+          internal_static_hbase_pb_SlowLogResponses_descriptor =
+            getDescriptor().getMessageTypes().get(37);
+          internal_static_hbase_pb_SlowLogResponses_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_SlowLogResponses_descriptor,
+              new java.lang.String[] { "SlowLogPayloads", });
+          internal_static_hbase_pb_ClearSlowLogResponseRequest_descriptor =
+            getDescriptor().getMessageTypes().get(38);
+          internal_static_hbase_pb_ClearSlowLogResponseRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_ClearSlowLogResponseRequest_descriptor,
+              new java.lang.String[] { });
+          internal_static_hbase_pb_ClearSlowLogResponses_descriptor =
+            getDescriptor().getMessageTypes().get(39);
+          internal_static_hbase_pb_ClearSlowLogResponses_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_ClearSlowLogResponses_descriptor,
+              new java.lang.String[] { "IsCleaned", });
           return null;
         }
       };
@@ -25302,6 +28498,7 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.protobuf.generated.ClientProtos.getDescriptor(),
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.protobuf.generated.WALProtos.getDescriptor(),
+          org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.getDescriptor(),
         }, assigner);
   }
 
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
index f86370d..9f0a6eb 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
@@ -20250,6 +20250,1242 @@ public final class HBaseProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotDescription)
   }
 
+  public interface LogRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string log_class_name = 1;
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    boolean hasLogClassName();
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    java.lang.String getLogClassName();
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getLogClassNameBytes();
+
+    // required bytes log_message = 2;
+    /**
+     * <code>required bytes log_message = 2;</code>
+     */
+    boolean hasLogMessage();
+    /**
+     * <code>required bytes log_message = 2;</code>
+     */
+    com.google.protobuf.ByteString getLogMessage();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.LogRequest}
+   */
+  public static final class LogRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements LogRequestOrBuilder {
+    // Use LogRequest.newBuilder() to construct.
+    private LogRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private LogRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final LogRequest defaultInstance;
+    public static LogRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public LogRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private LogRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              logClassName_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              logMessage_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<LogRequest> PARSER =
+        new com.google.protobuf.AbstractParser<LogRequest>() {
+      public LogRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new LogRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<LogRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string log_class_name = 1;
+    public static final int LOG_CLASS_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object logClassName_;
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    public boolean hasLogClassName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    public java.lang.String getLogClassName() {
+      java.lang.Object ref = logClassName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          logClassName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getLogClassNameBytes() {
+      java.lang.Object ref = logClassName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        logClassName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required bytes log_message = 2;
+    public static final int LOG_MESSAGE_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString logMessage_;
+    /**
+     * <code>required bytes log_message = 2;</code>
+     */
+    public boolean hasLogMessage() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required bytes log_message = 2;</code>
+     */
+    public com.google.protobuf.ByteString getLogMessage() {
+      return logMessage_;
+    }
+
+    private void initFields() {
+      logClassName_ = "";
+      logMessage_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasLogClassName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLogMessage()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getLogClassNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, logMessage_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getLogClassNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, logMessage_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest) obj;
+
+      boolean result = true;
+      result = result && (hasLogClassName() == other.hasLogClassName());
+      if (hasLogClassName()) {
+        result = result && getLogClassName()
+            .equals(other.getLogClassName());
+      }
+      result = result && (hasLogMessage() == other.hasLogMessage());
+      if (hasLogMessage()) {
+        result = result && getLogMessage()
+            .equals(other.getLogMessage());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasLogClassName()) {
+        hash = (37 * hash) + LOG_CLASS_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getLogClassName().hashCode();
+      }
+      if (hasLogMessage()) {
+        hash = (37 * hash) + LOG_MESSAGE_FIELD_NUMBER;
+        hash = (53 * hash) + getLogMessage().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.LogRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        logClassName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        logMessage_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.logClassName_ = logClassName_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.logMessage_ = logMessage_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.getDefaultInstance()) return this;
+        if (other.hasLogClassName()) {
+          bitField0_ |= 0x00000001;
+          logClassName_ = other.logClassName_;
+          onChanged();
+        }
+        if (other.hasLogMessage()) {
+          setLogMessage(other.getLogMessage());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasLogClassName()) {
+          
+          return false;
+        }
+        if (!hasLogMessage()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string log_class_name = 1;
+      private java.lang.Object logClassName_ = "";
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public boolean hasLogClassName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public java.lang.String getLogClassName() {
+        java.lang.Object ref = logClassName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          logClassName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getLogClassNameBytes() {
+        java.lang.Object ref = logClassName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          logClassName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public Builder setLogClassName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        logClassName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public Builder clearLogClassName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        logClassName_ = getDefaultInstance().getLogClassName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public Builder setLogClassNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        logClassName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required bytes log_message = 2;
+      private com.google.protobuf.ByteString logMessage_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes log_message = 2;</code>
+       */
+      public boolean hasLogMessage() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required bytes log_message = 2;</code>
+       */
+      public com.google.protobuf.ByteString getLogMessage() {
+        return logMessage_;
+      }
+      /**
+       * <code>required bytes log_message = 2;</code>
+       */
+      public Builder setLogMessage(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        logMessage_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bytes log_message = 2;</code>
+       */
+      public Builder clearLogMessage() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        logMessage_ = getDefaultInstance().getLogMessage();
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.LogRequest)
+    }
+
+    static {
+      defaultInstance = new LogRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.LogRequest)
+  }
+
+  public interface LogEntryOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string log_class_name = 1;
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    boolean hasLogClassName();
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    java.lang.String getLogClassName();
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getLogClassNameBytes();
+
+    // required bytes log_message = 2;
+    /**
+     * <code>required bytes log_message = 2;</code>
+     */
+    boolean hasLogMessage();
+    /**
+     * <code>required bytes log_message = 2;</code>
+     */
+    com.google.protobuf.ByteString getLogMessage();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.LogEntry}
+   */
+  public static final class LogEntry extends
+      com.google.protobuf.GeneratedMessage
+      implements LogEntryOrBuilder {
+    // Use LogEntry.newBuilder() to construct.
+    private LogEntry(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private LogEntry(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final LogEntry defaultInstance;
+    public static LogEntry getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public LogEntry getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private LogEntry(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              logClassName_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              logMessage_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogEntry_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogEntry_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<LogEntry> PARSER =
+        new com.google.protobuf.AbstractParser<LogEntry>() {
+      public LogEntry parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new LogEntry(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<LogEntry> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string log_class_name = 1;
+    public static final int LOG_CLASS_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object logClassName_;
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    public boolean hasLogClassName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    public java.lang.String getLogClassName() {
+      java.lang.Object ref = logClassName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          logClassName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string log_class_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getLogClassNameBytes() {
+      java.lang.Object ref = logClassName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        logClassName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required bytes log_message = 2;
+    public static final int LOG_MESSAGE_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString logMessage_;
+    /**
+     * <code>required bytes log_message = 2;</code>
+     */
+    public boolean hasLogMessage() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required bytes log_message = 2;</code>
+     */
+    public com.google.protobuf.ByteString getLogMessage() {
+      return logMessage_;
+    }
+
+    private void initFields() {
+      logClassName_ = "";
+      logMessage_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasLogClassName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLogMessage()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getLogClassNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, logMessage_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getLogClassNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, logMessage_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry) obj;
+
+      boolean result = true;
+      result = result && (hasLogClassName() == other.hasLogClassName());
+      if (hasLogClassName()) {
+        result = result && getLogClassName()
+            .equals(other.getLogClassName());
+      }
+      result = result && (hasLogMessage() == other.hasLogMessage());
+      if (hasLogMessage()) {
+        result = result && getLogMessage()
+            .equals(other.getLogMessage());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasLogClassName()) {
+        hash = (37 * hash) + LOG_CLASS_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getLogClassName().hashCode();
+      }
+      if (hasLogMessage()) {
+        hash = (37 * hash) + LOG_MESSAGE_FIELD_NUMBER;
+        hash = (53 * hash) + getLogMessage().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.LogEntry}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntryOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogEntry_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogEntry_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        logClassName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        logMessage_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LogEntry_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry build() {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.logClassName_ = logClassName_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.logMessage_ = logMessage_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance()) return this;
+        if (other.hasLogClassName()) {
+          bitField0_ |= 0x00000001;
+          logClassName_ = other.logClassName_;
+          onChanged();
+        }
+        if (other.hasLogMessage()) {
+          setLogMessage(other.getLogMessage());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasLogClassName()) {
+          
+          return false;
+        }
+        if (!hasLogMessage()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string log_class_name = 1;
+      private java.lang.Object logClassName_ = "";
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public boolean hasLogClassName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public java.lang.String getLogClassName() {
+        java.lang.Object ref = logClassName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          logClassName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getLogClassNameBytes() {
+        java.lang.Object ref = logClassName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          logClassName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public Builder setLogClassName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        logClassName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public Builder clearLogClassName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        logClassName_ = getDefaultInstance().getLogClassName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string log_class_name = 1;</code>
+       */
+      public Builder setLogClassNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        logClassName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required bytes log_message = 2;
+      private com.google.protobuf.ByteString logMessage_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes log_message = 2;</code>
+       */
+      public boolean hasLogMessage() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required bytes log_message = 2;</code>
+       */
+      public com.google.protobuf.ByteString getLogMessage() {
+        return logMessage_;
+      }
+      /**
+       * <code>required bytes log_message = 2;</code>
+       */
+      public Builder setLogMessage(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        logMessage_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bytes log_message = 2;</code>
+       */
+      public Builder clearLogMessage() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        logMessage_ = getDefaultInstance().getLogMessage();
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.LogEntry)
+    }
+
+    static {
+      defaultInstance = new LogEntry(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.LogEntry)
+  }
+
   public interface RegionLocationOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -21248,6 +22484,16 @@ public final class HBaseProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_LogRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_LogRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_LogEntry_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_LogEntry_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionLocation_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -21321,17 +22567,20 @@ public final class HBaseProtos {
       " \001(\005\022\r\n\005owner\030\006 \001(\t\022<\n\025users_and_permiss" +
       "ions\030\007 \001(\0132\035.hbase.pb.UsersAndPermission" +
       "s\022\016\n\003ttl\030\010 \001(\003:\0010\".\n\004Type\022\014\n\010DISABLED\020\000\022",
-      "\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"w\n\016RegionLocat" +
-      "ion\022)\n\013region_info\030\001 \002(\0132\024.hbase.pb.Regi" +
-      "onInfo\022)\n\013server_name\030\002 \001(\0132\024.hbase.pb.S" +
-      "erverName\022\017\n\007seq_num\030\003 \002(\003*r\n\013CompareTyp" +
-      "e\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020" +
-      "\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013" +
-      "\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NA" +
-      "NOSECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISE" +
-      "CONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOU" +
-      "RS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache.hadoop.hbas",
-      "e.protobuf.generatedB\013HBaseProtosH\001\240\001\001"
+      "\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"9\n\nLogRequest\022" +
+      "\026\n\016log_class_name\030\001 \002(\t\022\023\n\013log_message\030\002" +
+      " \002(\014\"7\n\010LogEntry\022\026\n\016log_class_name\030\001 \002(\t" +
+      "\022\023\n\013log_message\030\002 \002(\014\"w\n\016RegionLocation\022" +
+      ")\n\013region_info\030\001 \002(\0132\024.hbase.pb.RegionIn" +
+      "fo\022)\n\013server_name\030\002 \001(\0132\024.hbase.pb.Serve" +
+      "rName\022\017\n\007seq_num\030\003 \002(\003*r\n\013CompareType\022\010\n" +
+      "\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n" +
+      "\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GR" +
+      "EATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NANOSE",
+      "CONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISECOND" +
+      "S\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS\020\006" +
+      "\022\010\n\004DAYS\020\007B>\n*org.apache.hadoop.hbase.pr" +
+      "otobuf.generatedB\013HBaseProtosH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -21488,8 +22737,20 @@ public final class HBaseProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_SnapshotDescription_descriptor,
               new java.lang.String[] { "Name", "Table", "CreationTime", "Type", "Version", "Owner", "UsersAndPermissions", "Ttl", });
-          internal_static_hbase_pb_RegionLocation_descriptor =
+          internal_static_hbase_pb_LogRequest_descriptor =
             getDescriptor().getMessageTypes().get(25);
+          internal_static_hbase_pb_LogRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_LogRequest_descriptor,
+              new java.lang.String[] { "LogClassName", "LogMessage", });
+          internal_static_hbase_pb_LogEntry_descriptor =
+            getDescriptor().getMessageTypes().get(26);
+          internal_static_hbase_pb_LogEntry_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_LogEntry_descriptor,
+              new java.lang.String[] { "LogClassName", "LogMessage", });
+          internal_static_hbase_pb_RegionLocation_descriptor =
+            getDescriptor().getMessageTypes().get(27);
           internal_static_hbase_pb_RegionLocation_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_RegionLocation_descriptor,
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index 2eaed11..9fa92e1 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -65819,6 +65819,1197 @@ public final class MasterProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.IsSnapshotCleanupEnabledResponse)
   }
 
+  public interface BalancerDecisionsRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional uint32 limit = 1;
+    /**
+     * <code>optional uint32 limit = 1;</code>
+     */
+    boolean hasLimit();
+    /**
+     * <code>optional uint32 limit = 1;</code>
+     */
+    int getLimit();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.BalancerDecisionsRequest}
+   *
+   * <pre>
+   **
+   * BalancerDecision (LogRequest) use-case specific RPC request. This request payload will be
+   * converted in bytes and sent to generic RPC API: GetLogEntries
+   * LogRequest message has two params:
+   * 1. log_class_name: BalancerDecisionsRequest (for BalancerDecision use-case)
+   * 2. log_message: BalancerDecisionsRequest converted in bytes (for BalancerDecision use-case)
+   * </pre>
+   */
+  public static final class BalancerDecisionsRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements BalancerDecisionsRequestOrBuilder {
+    // Use BalancerDecisionsRequest.newBuilder() to construct.
+    private BalancerDecisionsRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private BalancerDecisionsRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final BalancerDecisionsRequest defaultInstance;
+    public static BalancerDecisionsRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public BalancerDecisionsRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private BalancerDecisionsRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              limit_ = input.readUInt32();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<BalancerDecisionsRequest> PARSER =
+        new com.google.protobuf.AbstractParser<BalancerDecisionsRequest>() {
+      public BalancerDecisionsRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new BalancerDecisionsRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<BalancerDecisionsRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional uint32 limit = 1;
+    public static final int LIMIT_FIELD_NUMBER = 1;
+    private int limit_;
+    /**
+     * <code>optional uint32 limit = 1;</code>
+     */
+    public boolean hasLimit() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional uint32 limit = 1;</code>
+     */
+    public int getLimit() {
+      return limit_;
+    }
+
+    private void initFields() {
+      limit_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt32(1, limit_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(1, limit_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest) obj;
+
+      boolean result = true;
+      result = result && (hasLimit() == other.hasLimit());
+      if (hasLimit()) {
+        result = result && (getLimit()
+            == other.getLimit());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasLimit()) {
+        hash = (37 * hash) + LIMIT_FIELD_NUMBER;
+        hash = (53 * hash) + getLimit();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.BalancerDecisionsRequest}
+     *
+     * <pre>
+     **
+     * BalancerDecision (LogRequest) use-case specific RPC request. This request payload will be
+     * converted in bytes and sent to generic RPC API: GetLogEntries
+     * LogRequest message has two params:
+     * 1. log_class_name: BalancerDecisionsRequest (for BalancerDecision use-case)
+     * 2. log_message: BalancerDecisionsRequest converted in bytes (for BalancerDecision use-case)
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        limit_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.limit_ = limit_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest.getDefaultInstance()) return this;
+        if (other.hasLimit()) {
+          setLimit(other.getLimit());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional uint32 limit = 1;
+      private int limit_ ;
+      /**
+       * <code>optional uint32 limit = 1;</code>
+       */
+      public boolean hasLimit() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional uint32 limit = 1;</code>
+       */
+      public int getLimit() {
+        return limit_;
+      }
+      /**
+       * <code>optional uint32 limit = 1;</code>
+       */
+      public Builder setLimit(int value) {
+        bitField0_ |= 0x00000001;
+        limit_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 limit = 1;</code>
+       */
+      public Builder clearLimit() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        limit_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.BalancerDecisionsRequest)
+    }
+
+    static {
+      defaultInstance = new BalancerDecisionsRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.BalancerDecisionsRequest)
+  }
+
+  public interface BalancerDecisionsResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .hbase.pb.BalancerDecision balancer_decision = 1;
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision> 
+        getBalancerDecisionList();
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision getBalancerDecision(int index);
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    int getBalancerDecisionCount();
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder> 
+        getBalancerDecisionOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder getBalancerDecisionOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.BalancerDecisionsResponse}
+   *
+   * <pre>
+   **
+   * BalancerDecision (LogEntry) use-case specific RPC response. This response payload will be
+   * converted in bytes by servers and sent as response to generic RPC API: GetLogEntries
+   * LogEntry message has two params:
+   * 1. log_class_name: BalancerDecisionsResponse (for BalancerDecision use-case)
+   * 2. log_message: BalancerDecisionsResponse converted in bytes (for BalancerDecision use-case)
+   * </pre>
+   */
+  public static final class BalancerDecisionsResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements BalancerDecisionsResponseOrBuilder {
+    // Use BalancerDecisionsResponse.newBuilder() to construct.
+    private BalancerDecisionsResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private BalancerDecisionsResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final BalancerDecisionsResponse defaultInstance;
+    public static BalancerDecisionsResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public BalancerDecisionsResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private BalancerDecisionsResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                balancerDecision_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              balancerDecision_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          balancerDecision_ = java.util.Collections.unmodifiableList(balancerDecision_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<BalancerDecisionsResponse> PARSER =
+        new com.google.protobuf.AbstractParser<BalancerDecisionsResponse>() {
+      public BalancerDecisionsResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new BalancerDecisionsResponse(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<BalancerDecisionsResponse> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated .hbase.pb.BalancerDecision balancer_decision = 1;
+    public static final int BALANCER_DECISION_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision> balancerDecision_;
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision> getBalancerDecisionList() {
+      return balancerDecision_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder> 
+        getBalancerDecisionOrBuilderList() {
+      return balancerDecision_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    public int getBalancerDecisionCount() {
+      return balancerDecision_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision getBalancerDecision(int index) {
+      return balancerDecision_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder getBalancerDecisionOrBuilder(
+        int index) {
+      return balancerDecision_.get(index);
+    }
+
+    private void initFields() {
+      balancerDecision_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      for (int i = 0; i < getBalancerDecisionCount(); i++) {
+        if (!getBalancerDecision(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < balancerDecision_.size(); i++) {
+        output.writeMessage(1, balancerDecision_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < balancerDecision_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, balancerDecision_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse) obj;
+
+      boolean result = true;
+      result = result && getBalancerDecisionList()
+          .equals(other.getBalancerDecisionList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getBalancerDecisionCount() > 0) {
+        hash = (37 * hash) + BALANCER_DECISION_FIELD_NUMBER;
+        hash = (53 * hash) + getBalancerDecisionList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.BalancerDecisionsResponse}
+     *
+     * <pre>
+     **
+     * BalancerDecision (LogEntry) use-case specific RPC response. This response payload will be
+     * converted in bytes by servers and sent as response to generic RPC API: GetLogEntries
+     * LogEntry message has two params:
+     * 1. log_class_name: BalancerDecisionsResponse (for BalancerDecision use-case)
+     * 2. log_message: BalancerDecisionsResponse converted in bytes (for BalancerDecision use-case)
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBalancerDecisionFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (balancerDecisionBuilder_ == null) {
+          balancerDecision_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          balancerDecisionBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalancerDecisionsResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse(this);
+        int from_bitField0_ = bitField0_;
+        if (balancerDecisionBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            balancerDecision_ = java.util.Collections.unmodifiableList(balancerDecision_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.balancerDecision_ = balancerDecision_;
+        } else {
+          result.balancerDecision_ = balancerDecisionBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse.getDefaultInstance()) return this;
+        if (balancerDecisionBuilder_ == null) {
+          if (!other.balancerDecision_.isEmpty()) {
+            if (balancerDecision_.isEmpty()) {
+              balancerDecision_ = other.balancerDecision_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureBalancerDecisionIsMutable();
+              balancerDecision_.addAll(other.balancerDecision_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.balancerDecision_.isEmpty()) {
+            if (balancerDecisionBuilder_.isEmpty()) {
+              balancerDecisionBuilder_.dispose();
+              balancerDecisionBuilder_ = null;
+              balancerDecision_ = other.balancerDecision_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              balancerDecisionBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getBalancerDecisionFieldBuilder() : null;
+            } else {
+              balancerDecisionBuilder_.addAllMessages(other.balancerDecision_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getBalancerDecisionCount(); i++) {
+          if (!getBalancerDecision(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalancerDecisionsResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated .hbase.pb.BalancerDecision balancer_decision = 1;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision> balancerDecision_ =
+        java.util.Collections.emptyList();
+      private void ensureBalancerDecisionIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          balancerDecision_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision>(balancerDecision_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder> balancerDecisionBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision> getBalancerDecisionList() {
+        if (balancerDecisionBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(balancerDecision_);
+        } else {
+          return balancerDecisionBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public int getBalancerDecisionCount() {
+        if (balancerDecisionBuilder_ == null) {
+          return balancerDecision_.size();
+        } else {
+          return balancerDecisionBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision getBalancerDecision(int index) {
+        if (balancerDecisionBuilder_ == null) {
+          return balancerDecision_.get(index);
+        } else {
+          return balancerDecisionBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder setBalancerDecision(
+          int index, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision value) {
+        if (balancerDecisionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBalancerDecisionIsMutable();
+          balancerDecision_.set(index, value);
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder setBalancerDecision(
+          int index, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder builderForValue) {
+        if (balancerDecisionBuilder_ == null) {
+          ensureBalancerDecisionIsMutable();
+          balancerDecision_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder addBalancerDecision(org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision value) {
+        if (balancerDecisionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBalancerDecisionIsMutable();
+          balancerDecision_.add(value);
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder addBalancerDecision(
+          int index, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision value) {
+        if (balancerDecisionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBalancerDecisionIsMutable();
+          balancerDecision_.add(index, value);
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder addBalancerDecision(
+          org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder builderForValue) {
+        if (balancerDecisionBuilder_ == null) {
+          ensureBalancerDecisionIsMutable();
+          balancerDecision_.add(builderForValue.build());
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder addBalancerDecision(
+          int index, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder builderForValue) {
+        if (balancerDecisionBuilder_ == null) {
+          ensureBalancerDecisionIsMutable();
+          balancerDecision_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder addAllBalancerDecision(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision> values) {
+        if (balancerDecisionBuilder_ == null) {
+          ensureBalancerDecisionIsMutable();
+          super.addAll(values, balancerDecision_);
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder clearBalancerDecision() {
+        if (balancerDecisionBuilder_ == null) {
+          balancerDecision_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public Builder removeBalancerDecision(int index) {
+        if (balancerDecisionBuilder_ == null) {
+          ensureBalancerDecisionIsMutable();
+          balancerDecision_.remove(index);
+          onChanged();
+        } else {
+          balancerDecisionBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder getBalancerDecisionBuilder(
+          int index) {
+        return getBalancerDecisionFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder getBalancerDecisionOrBuilder(
+          int index) {
+        if (balancerDecisionBuilder_ == null) {
+          return balancerDecision_.get(index);  } else {
+          return balancerDecisionBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder> 
+           getBalancerDecisionOrBuilderList() {
+        if (balancerDecisionBuilder_ != null) {
+          return balancerDecisionBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(balancerDecision_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder addBalancerDecisionBuilder() {
+        return getBalancerDecisionFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder addBalancerDecisionBuilder(
+          int index) {
+        return getBalancerDecisionFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.BalancerDecision balancer_decision = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder> 
+           getBalancerDecisionBuilderList() {
+        return getBalancerDecisionFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder> 
+          getBalancerDecisionFieldBuilder() {
+        if (balancerDecisionBuilder_ == null) {
+          balancerDecisionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder>(
+                  balancerDecision_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          balancerDecision_ = null;
+        }
+        return balancerDecisionBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.BalancerDecisionsResponse)
+    }
+
+    static {
+      defaultInstance = new BalancerDecisionsResponse(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.BalancerDecisionsResponse)
+  }
+
   public interface GetClusterIdRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
   }
@@ -71226,6 +72417,14 @@ public final class MasterProtos {
           com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledResponse> done);
 
       /**
+       * <code>rpc GetLogEntries(.hbase.pb.LogRequest) returns (.hbase.pb.LogEntry);</code>
+       */
+      public abstract void getLogEntries(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry> done);
+
+      /**
        * <code>rpc GetTableState(.hbase.pb.GetTableStateRequest) returns (.hbase.pb.GetTableStateResponse);</code>
        *
        * <pre>
@@ -71755,6 +72954,14 @@ public final class MasterProtos {
         }
 
         @java.lang.Override
+        public  void getLogEntries(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry> done) {
+          impl.getLogEntries(controller, request, done);
+        }
+
+        @java.lang.Override
         public  void getTableState(
             com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request,
@@ -71913,6 +73120,8 @@ public final class MasterProtos {
             case 63:
               return impl.isSnapshotCleanupEnabled(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledRequest)request);
             case 64:
+              return impl.getLogEntries(controller, (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest)request);
+            case 65:
               return impl.getTableState(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -72057,6 +73266,8 @@ public final class MasterProtos {
             case 63:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledRequest.getDefaultInstance();
             case 64:
+              return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.getDefaultInstance();
+            case 65:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -72201,6 +73412,8 @@ public final class MasterProtos {
             case 63:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledResponse.getDefaultInstance();
             case 64:
+              return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance();
+            case 65:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -73013,6 +74226,14 @@ public final class MasterProtos {
         com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledResponse> done);
 
     /**
+     * <code>rpc GetLogEntries(.hbase.pb.LogRequest) returns (.hbase.pb.LogEntry);</code>
+     */
+    public abstract void getLogEntries(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry> done);
+
+    /**
      * <code>rpc GetTableState(.hbase.pb.GetTableStateRequest) returns (.hbase.pb.GetTableStateResponse);</code>
      *
      * <pre>
@@ -73367,6 +74588,11 @@ public final class MasterProtos {
               done));
           return;
         case 64:
+          this.getLogEntries(controller, (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry>specializeCallback(
+              done));
+          return;
+        case 65:
           this.getTableState(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest)request,
             com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse>specializeCallback(
               done));
@@ -73514,6 +74740,8 @@ public final class MasterProtos {
         case 63:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledRequest.getDefaultInstance();
         case 64:
+          return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest.getDefaultInstance();
+        case 65:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
@@ -73658,6 +74886,8 @@ public final class MasterProtos {
         case 63:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledResponse.getDefaultInstance();
         case 64:
+          return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance();
+        case 65:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
@@ -74640,12 +75870,27 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledResponse.getDefaultInstance()));
       }
 
+      public  void getLogEntries(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(64),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.class,
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance()));
+      }
+
       public  void getTableState(
           com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request,
           com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(64),
+          getDescriptor().getMethods().get(65),
           controller,
           request,
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(),
@@ -74982,6 +76227,11 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledRequest request)
           throws com.google.protobuf.ServiceException;
 
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry getLogEntries(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request)
+          throws com.google.protobuf.ServiceException;
+
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse getTableState(
           com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request)
@@ -75763,12 +77013,24 @@ public final class MasterProtos {
       }
 
 
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry getLogEntries(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(64),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LogEntry.getDefaultInstance());
+      }
+
+
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse getTableState(
           com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request)
           throws com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(64),
+          getDescriptor().getMethods().get(65),
           controller,
           request,
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance());
@@ -76897,6 +78159,16 @@ public final class MasterProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_IsSnapshotCleanupEnabledResponse_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_BalancerDecisionsRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_BalancerDecisionsRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_BalancerDecisionsResponse_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_BalancerDecisionsResponse_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetClusterIdRequest_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -76953,385 +78225,391 @@ public final class MasterProtos {
       "\n\014Master.proto\022\010hbase.pb\032\013Table.proto\032\013H" +
       "Base.proto\032\014Client.proto\032\023ClusterStatus." +
       "proto\032\023ErrorHandling.proto\032\017Procedure.pr" +
-      "oto\032\013Quota.proto\032\016Snapshot.proto\"\234\001\n\020Add" +
-      "ColumnRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbas" +
-      "e.pb.TableName\0225\n\017column_families\030\002 \002(\0132" +
-      "\034.hbase.pb.ColumnFamilySchema\022\026\n\013nonce_g" +
-      "roup\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\023\n\021AddCo" +
-      "lumnResponse\"}\n\023DeleteColumnRequest\022\'\n\nt" +
-      "able_name\030\001 \002(\0132\023.hbase.pb.TableName\022\023\n\013",
-      "column_name\030\002 \002(\014\022\026\n\013nonce_group\030\003 \001(\004:\001" +
-      "0\022\020\n\005nonce\030\004 \001(\004:\0010\"\026\n\024DeleteColumnRespo" +
-      "nse\"\237\001\n\023ModifyColumnRequest\022\'\n\ntable_nam" +
-      "e\030\001 \002(\0132\023.hbase.pb.TableName\0225\n\017column_f" +
-      "amilies\030\002 \002(\0132\034.hbase.pb.ColumnFamilySch" +
-      "ema\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001" +
-      "(\004:\0010\"\026\n\024ModifyColumnResponse\"n\n\021MoveReg" +
-      "ionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re" +
-      "gionSpecifier\022.\n\020dest_server_name\030\002 \001(\0132" +
-      "\024.hbase.pb.ServerName\"\024\n\022MoveRegionRespo",
-      "nse\"\222\001\n\035DispatchMergingRegionsRequest\022+\n" +
-      "\010region_a\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
-      "er\022+\n\010region_b\030\002 \002(\0132\031.hbase.pb.RegionSp" +
-      "ecifier\022\027\n\010forcible\030\003 \001(\010:\005false\" \n\036Disp" +
-      "atchMergingRegionsResponse\"@\n\023AssignRegi" +
-      "onRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Reg" +
-      "ionSpecifier\"\026\n\024AssignRegionResponse\"X\n\025" +
-      "UnassignRegionRequest\022)\n\006region\030\001 \002(\0132\031." +
-      "hbase.pb.RegionSpecifier\022\024\n\005force\030\002 \001(\010:" +
-      "\005false\"\030\n\026UnassignRegionResponse\"A\n\024Offl",
-      "ineRegionRequest\022)\n\006region\030\001 \002(\0132\031.hbase" +
-      ".pb.RegionSpecifier\"\027\n\025OfflineRegionResp" +
-      "onse\"\177\n\022CreateTableRequest\022+\n\014table_sche" +
-      "ma\030\001 \002(\0132\025.hbase.pb.TableSchema\022\022\n\nsplit" +
-      "_keys\030\002 \003(\014\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005n" +
-      "once\030\004 \001(\004:\0010\"&\n\023CreateTableResponse\022\017\n\007" +
-      "proc_id\030\001 \001(\004\"g\n\022DeleteTableRequest\022\'\n\nt" +
-      "able_name\030\001 \002(\0132\023.hbase.pb.TableName\022\026\n\013" +
-      "nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"&" +
-      "\n\023DeleteTableResponse\022\017\n\007proc_id\030\001 \001(\004\"\207",
-      "\001\n\024TruncateTableRequest\022&\n\ttableName\030\001 \002" +
-      "(\0132\023.hbase.pb.TableName\022\035\n\016preserveSplit" +
-      "s\030\002 \001(\010:\005false\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020" +
-      "\n\005nonce\030\004 \001(\004:\0010\"\027\n\025TruncateTableRespons" +
-      "e\"g\n\022EnableTableRequest\022\'\n\ntable_name\030\001 " +
-      "\002(\0132\023.hbase.pb.TableName\022\026\n\013nonce_group\030" +
-      "\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"&\n\023EnableTabl" +
-      "eResponse\022\017\n\007proc_id\030\001 \001(\004\"h\n\023DisableTab" +
-      "leRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb" +
-      ".TableName\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005no",
-      "nce\030\003 \001(\004:\0010\"\'\n\024DisableTableResponse\022\017\n\007" +
-      "proc_id\030\001 \001(\004\"\224\001\n\022ModifyTableRequest\022\'\n\n" +
-      "table_name\030\001 \002(\0132\023.hbase.pb.TableName\022+\n" +
-      "\014table_schema\030\002 \002(\0132\025.hbase.pb.TableSche" +
-      "ma\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(" +
-      "\004:\0010\"\025\n\023ModifyTableResponse\"~\n\026CreateNam" +
-      "espaceRequest\022:\n\023namespaceDescriptor\030\001 \002" +
-      "(\0132\035.hbase.pb.NamespaceDescriptor\022\026\n\013non" +
-      "ce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"\031\n\027C" +
-      "reateNamespaceResponse\"Y\n\026DeleteNamespac",
-      "eRequest\022\025\n\rnamespaceName\030\001 \002(\t\022\026\n\013nonce" +
-      "_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"\031\n\027Del" +
-      "eteNamespaceResponse\"~\n\026ModifyNamespaceR" +
-      "equest\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.hb" +
-      "ase.pb.NamespaceDescriptor\022\026\n\013nonce_grou" +
-      "p\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"\031\n\027ModifyNa" +
-      "mespaceResponse\"6\n\035GetNamespaceDescripto" +
-      "rRequest\022\025\n\rnamespaceName\030\001 \002(\t\"\\\n\036GetNa" +
-      "mespaceDescriptorResponse\022:\n\023namespaceDe" +
-      "scriptor\030\001 \002(\0132\035.hbase.pb.NamespaceDescr",
-      "iptor\"\027\n\025ListNamespacesRequest\"/\n\026ListNa" +
-      "mespacesResponse\022\025\n\rnamespaceName\030\001 \003(\t\"" +
-      "!\n\037ListNamespaceDescriptorsRequest\"^\n Li" +
-      "stNamespaceDescriptorsResponse\022:\n\023namesp" +
-      "aceDescriptor\030\001 \003(\0132\035.hbase.pb.Namespace" +
-      "Descriptor\"?\n&ListTableDescriptorsByName" +
-      "spaceRequest\022\025\n\rnamespaceName\030\001 \002(\t\"U\n\'L" +
-      "istTableDescriptorsByNamespaceResponse\022*" +
-      "\n\013tableSchema\030\001 \003(\0132\025.hbase.pb.TableSche" +
-      "ma\"9\n ListTableNamesByNamespaceRequest\022\025",
-      "\n\rnamespaceName\030\001 \002(\t\"K\n!ListTableNamesB" +
-      "yNamespaceResponse\022&\n\ttableName\030\001 \003(\0132\023." +
-      "hbase.pb.TableName\"\021\n\017ShutdownRequest\"\022\n" +
-      "\020ShutdownResponse\"\023\n\021StopMasterRequest\"\024" +
-      "\n\022StopMasterResponse\"\034\n\032IsInMaintenanceM" +
-      "odeRequest\"8\n\033IsInMaintenanceModeRespons" +
-      "e\022\031\n\021inMaintenanceMode\030\001 \002(\010\"\037\n\016BalanceR" +
-      "equest\022\r\n\005force\030\001 \001(\010\"\'\n\017BalanceResponse" +
-      "\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031SetBalancerRun" +
-      "ningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013synchronous\030\002",
-      " \001(\010\"8\n\032SetBalancerRunningResponse\022\032\n\022pr" +
-      "ev_balance_value\030\001 \001(\010\"\032\n\030IsBalancerEnab" +
-      "ledRequest\",\n\031IsBalancerEnabledResponse\022" +
-      "\017\n\007enabled\030\001 \002(\010\"w\n\035SetSplitOrMergeEnabl" +
-      "edRequest\022\017\n\007enabled\030\001 \002(\010\022\023\n\013synchronou" +
-      "s\030\002 \001(\010\0220\n\014switch_types\030\003 \003(\0162\032.hbase.pb" +
-      ".MasterSwitchType\"4\n\036SetSplitOrMergeEnab" +
-      "ledResponse\022\022\n\nprev_value\030\001 \003(\010\"O\n\034IsSpl" +
-      "itOrMergeEnabledRequest\022/\n\013switch_type\030\001" +
-      " \002(\0162\032.hbase.pb.MasterSwitchType\"0\n\035IsSp",
-      "litOrMergeEnabledResponse\022\017\n\007enabled\030\001 \002" +
-      "(\010\"\022\n\020NormalizeRequest\"+\n\021NormalizeRespo" +
-      "nse\022\026\n\016normalizer_ran\030\001 \002(\010\")\n\033SetNormal" +
-      "izerRunningRequest\022\n\n\002on\030\001 \002(\010\"=\n\034SetNor" +
-      "malizerRunningResponse\022\035\n\025prev_normalize" +
-      "r_value\030\001 \001(\010\"\034\n\032IsNormalizerEnabledRequ" +
-      "est\".\n\033IsNormalizerEnabledResponse\022\017\n\007en" +
-      "abled\030\001 \002(\010\"\027\n\025RunCatalogScanRequest\"-\n\026" +
-      "RunCatalogScanResponse\022\023\n\013scan_result\030\001 " +
-      "\001(\005\"-\n\033EnableCatalogJanitorRequest\022\016\n\006en",
-      "able\030\001 \002(\010\"2\n\034EnableCatalogJanitorRespon" +
-      "se\022\022\n\nprev_value\030\001 \001(\010\" \n\036IsCatalogJanit" +
-      "orEnabledRequest\"0\n\037IsCatalogJanitorEnab" +
-      "ledResponse\022\r\n\005value\030\001 \002(\010\"\030\n\026RunCleaner" +
-      "ChoreRequest\"4\n\027RunCleanerChoreResponse\022" +
-      "\031\n\021cleaner_chore_ran\030\001 \002(\010\"+\n\035SetCleaner" +
-      "ChoreRunningRequest\022\n\n\002on\030\001 \002(\010\"4\n\036SetCl" +
-      "eanerChoreRunningResponse\022\022\n\nprev_value\030" +
-      "\001 \001(\010\"\036\n\034IsCleanerChoreEnabledRequest\".\n" +
-      "\035IsCleanerChoreEnabledResponse\022\r\n\005value\030",
-      "\001 \002(\010\"B\n\017SnapshotRequest\022/\n\010snapshot\030\001 \002" +
-      "(\0132\035.hbase.pb.SnapshotDescription\",\n\020Sna" +
-      "pshotResponse\022\030\n\020expected_timeout\030\001 \002(\003\"" +
-      "\036\n\034GetCompletedSnapshotsRequest\"Q\n\035GetCo" +
-      "mpletedSnapshotsResponse\0220\n\tsnapshots\030\001 " +
-      "\003(\0132\035.hbase.pb.SnapshotDescription\"H\n\025De" +
-      "leteSnapshotRequest\022/\n\010snapshot\030\001 \002(\0132\035." +
-      "hbase.pb.SnapshotDescription\"\030\n\026DeleteSn" +
-      "apshotResponse\"d\n\026RestoreSnapshotRequest" +
-      "\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.SnapshotDe",
-      "scription\022\031\n\nrestoreACL\030\002 \001(\010:\005false\"\031\n\027" +
-      "RestoreSnapshotResponse\"H\n\025IsSnapshotDon" +
-      "eRequest\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb.Sn" +
-      "apshotDescription\"^\n\026IsSnapshotDoneRespo" +
-      "nse\022\023\n\004done\030\001 \001(\010:\005false\022/\n\010snapshot\030\002 \001" +
-      "(\0132\035.hbase.pb.SnapshotDescription\"O\n\034IsR" +
-      "estoreSnapshotDoneRequest\022/\n\010snapshot\030\001 " +
-      "\001(\0132\035.hbase.pb.SnapshotDescription\"4\n\035Is" +
-      "RestoreSnapshotDoneResponse\022\023\n\004done\030\001 \001(" +
-      "\010:\005false\"F\n\033GetSchemaAlterStatusRequest\022",
-      "\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.TableName" +
-      "\"T\n\034GetSchemaAlterStatusResponse\022\035\n\025yet_" +
-      "to_update_regions\030\001 \001(\r\022\025\n\rtotal_regions" +
-      "\030\002 \001(\r\"\213\001\n\032GetTableDescriptorsRequest\022(\n" +
-      "\013table_names\030\001 \003(\0132\023.hbase.pb.TableName\022" +
-      "\r\n\005regex\030\002 \001(\t\022!\n\022include_sys_tables\030\003 \001" +
-      "(\010:\005false\022\021\n\tnamespace\030\004 \001(\t\"J\n\033GetTable" +
-      "DescriptorsResponse\022+\n\014table_schema\030\001 \003(" +
-      "\0132\025.hbase.pb.TableSchema\"[\n\024GetTableName" +
-      "sRequest\022\r\n\005regex\030\001 \001(\t\022!\n\022include_sys_t",
-      "ables\030\002 \001(\010:\005false\022\021\n\tnamespace\030\003 \001(\t\"A\n" +
-      "\025GetTableNamesResponse\022(\n\013table_names\030\001 " +
-      "\003(\0132\023.hbase.pb.TableName\"?\n\024GetTableStat" +
-      "eRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb." +
-      "TableName\"B\n\025GetTableStateResponse\022)\n\013ta" +
-      "ble_state\030\001 \002(\0132\024.hbase.pb.TableState\"\031\n" +
-      "\027GetClusterStatusRequest\"K\n\030GetClusterSt" +
-      "atusResponse\022/\n\016cluster_status\030\001 \002(\0132\027.h" +
-      "base.pb.ClusterStatus\"\030\n\026IsMasterRunning" +
-      "Request\"4\n\027IsMasterRunningResponse\022\031\n\021is",
-      "_master_running\030\001 \002(\010\"I\n\024ExecProcedureRe" +
-      "quest\0221\n\tprocedure\030\001 \002(\0132\036.hbase.pb.Proc" +
-      "edureDescription\"F\n\025ExecProcedureRespons" +
-      "e\022\030\n\020expected_timeout\030\001 \001(\003\022\023\n\013return_da" +
-      "ta\030\002 \001(\014\"K\n\026IsProcedureDoneRequest\0221\n\tpr" +
-      "ocedure\030\001 \001(\0132\036.hbase.pb.ProcedureDescri" +
-      "ption\"`\n\027IsProcedureDoneResponse\022\023\n\004done" +
-      "\030\001 \001(\010:\005false\0220\n\010snapshot\030\002 \001(\0132\036.hbase." +
-      "pb.ProcedureDescription\",\n\031GetProcedureR" +
-      "esultRequest\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032GetPro",
-      "cedureResultResponse\0229\n\005state\030\001 \002(\0162*.hb" +
-      "ase.pb.GetProcedureResultResponse.State\022" +
-      "\022\n\nstart_time\030\002 \001(\004\022\023\n\013last_update\030\003 \001(\004" +
-      "\022\016\n\006result\030\004 \001(\014\0224\n\texception\030\005 \001(\0132!.hb" +
-      "ase.pb.ForeignExceptionMessage\"1\n\005State\022" +
-      "\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020" +
-      "\002\"M\n\025AbortProcedureRequest\022\017\n\007proc_id\030\001 " +
-      "\002(\004\022#\n\025mayInterruptIfRunning\030\002 \001(\010:\004true" +
-      "\"6\n\026AbortProcedureResponse\022\034\n\024is_procedu" +
-      "re_aborted\030\001 \002(\010\"\027\n\025ListProceduresReques",
-      "t\"@\n\026ListProceduresResponse\022&\n\tprocedure" +
-      "\030\001 \003(\0132\023.hbase.pb.Procedure\"\315\001\n\017SetQuota" +
-      "Request\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser_group" +
-      "\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\ntable_name\030" +
-      "\004 \001(\0132\023.hbase.pb.TableName\022\022\n\nremove_all" +
-      "\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010\022+\n\010thrott" +
-      "le\030\007 \001(\0132\031.hbase.pb.ThrottleRequest\"\022\n\020S" +
-      "etQuotaResponse\"J\n\037MajorCompactionTimest" +
-      "ampRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.p" +
-      "b.TableName\"U\n(MajorCompactionTimestampF",
-      "orRegionRequest\022)\n\006region\030\001 \002(\0132\031.hbase." +
-      "pb.RegionSpecifier\"@\n MajorCompactionTim" +
-      "estampResponse\022\034\n\024compaction_timestamp\030\001" +
-      " \002(\003\"\035\n\033SecurityCapabilitiesRequest\"\354\001\n\034" +
-      "SecurityCapabilitiesResponse\022G\n\014capabili" +
-      "ties\030\001 \003(\01621.hbase.pb.SecurityCapabiliti" +
-      "esResponse.Capability\"\202\001\n\nCapability\022\031\n\025" +
-      "SIMPLE_AUTHENTICATION\020\000\022\031\n\025SECURE_AUTHEN" +
-      "TICATION\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022CELL_AU" +
-      "THORIZATION\020\003\022\023\n\017CELL_VISIBILITY\020\004\"D\n\027Cl",
-      "earDeadServersRequest\022)\n\013server_name\030\001 \003" +
-      "(\0132\024.hbase.pb.ServerName\"E\n\030ClearDeadSer" +
-      "versResponse\022)\n\013server_name\030\001 \003(\0132\024.hbas" +
-      "e.pb.ServerName\"A\n\031SetSnapshotCleanupReq" +
-      "uest\022\017\n\007enabled\030\001 \002(\010\022\023\n\013synchronous\030\002 \001" +
-      "(\010\";\n\032SetSnapshotCleanupResponse\022\035\n\025prev" +
-      "_snapshot_cleanup\030\001 \002(\010\"!\n\037IsSnapshotCle" +
-      "anupEnabledRequest\"3\n IsSnapshotCleanupE" +
-      "nabledResponse\022\017\n\007enabled\030\001 \002(\010\"\025\n\023GetCl" +
-      "usterIdRequest\"*\n\024GetClusterIdResponse\022\022",
-      "\n\ncluster_id\030\001 \001(\t\"\023\n\021GetMastersRequest\"" +
-      "W\n\027GetMastersResponseEntry\022)\n\013server_nam" +
-      "e\030\001 \002(\0132\024.hbase.pb.ServerName\022\021\n\tis_acti" +
-      "ve\030\002 \002(\010\"O\n\022GetMastersResponse\0229\n\016master" +
-      "_servers\030\001 \003(\0132!.hbase.pb.GetMastersResp" +
-      "onseEntry\"\037\n\035GetMetaRegionLocationsReque" +
-      "st\"R\n\036GetMetaRegionLocationsResponse\0220\n\016" +
-      "meta_locations\030\001 \003(\0132\030.hbase.pb.RegionLo" +
-      "cation\"\025\n\023GetNumLiveRSRequest\"2\n\024GetNumL" +
-      "iveRSResponse\022\032\n\022num_region_servers\030\001 \002(",
-      "\005*(\n\020MasterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERG" +
-      "E\020\0012\363.\n\rMasterService\022e\n\024GetSchemaAlterS" +
-      "tatus\022%.hbase.pb.GetSchemaAlterStatusReq" +
-      "uest\032&.hbase.pb.GetSchemaAlterStatusResp" +
-      "onse\022b\n\023GetTableDescriptors\022$.hbase.pb.G" +
-      "etTableDescriptorsRequest\032%.hbase.pb.Get" +
-      "TableDescriptorsResponse\022P\n\rGetTableName" +
-      "s\022\036.hbase.pb.GetTableNamesRequest\032\037.hbas" +
-      "e.pb.GetTableNamesResponse\022Y\n\020GetCluster" +
-      "Status\022!.hbase.pb.GetClusterStatusReques",
-      "t\032\".hbase.pb.GetClusterStatusResponse\022V\n" +
-      "\017IsMasterRunning\022 .hbase.pb.IsMasterRunn" +
-      "ingRequest\032!.hbase.pb.IsMasterRunningRes" +
-      "ponse\022D\n\tAddColumn\022\032.hbase.pb.AddColumnR" +
-      "equest\032\033.hbase.pb.AddColumnResponse\022M\n\014D" +
-      "eleteColumn\022\035.hbase.pb.DeleteColumnReque" +
-      "st\032\036.hbase.pb.DeleteColumnResponse\022M\n\014Mo" +
-      "difyColumn\022\035.hbase.pb.ModifyColumnReques" +
-      "t\032\036.hbase.pb.ModifyColumnResponse\022G\n\nMov" +
-      "eRegion\022\033.hbase.pb.MoveRegionRequest\032\034.h",
-      "base.pb.MoveRegionResponse\022k\n\026DispatchMe" +
-      "rgingRegions\022\'.hbase.pb.DispatchMergingR" +
-      "egionsRequest\032(.hbase.pb.DispatchMerging" +
-      "RegionsResponse\022M\n\014AssignRegion\022\035.hbase." +
-      "pb.AssignRegionRequest\032\036.hbase.pb.Assign" +
-      "RegionResponse\022S\n\016UnassignRegion\022\037.hbase" +
-      ".pb.UnassignRegionRequest\032 .hbase.pb.Una" +
-      "ssignRegionResponse\022P\n\rOfflineRegion\022\036.h" +
-      "base.pb.OfflineRegionRequest\032\037.hbase.pb." +
-      "OfflineRegionResponse\022J\n\013DeleteTable\022\034.h",
-      "base.pb.DeleteTableRequest\032\035.hbase.pb.De" +
-      "leteTableResponse\022P\n\rtruncateTable\022\036.hba" +
-      "se.pb.TruncateTableRequest\032\037.hbase.pb.Tr" +
-      "uncateTableResponse\022J\n\013EnableTable\022\034.hba" +
-      "se.pb.EnableTableRequest\032\035.hbase.pb.Enab" +
-      "leTableResponse\022M\n\014DisableTable\022\035.hbase." +
-      "pb.DisableTableRequest\032\036.hbase.pb.Disabl" +
-      "eTableResponse\022J\n\013ModifyTable\022\034.hbase.pb" +
-      ".ModifyTableRequest\032\035.hbase.pb.ModifyTab" +
-      "leResponse\022J\n\013CreateTable\022\034.hbase.pb.Cre",
-      "ateTableRequest\032\035.hbase.pb.CreateTableRe" +
-      "sponse\022A\n\010Shutdown\022\031.hbase.pb.ShutdownRe" +
-      "quest\032\032.hbase.pb.ShutdownResponse\022G\n\nSto" +
-      "pMaster\022\033.hbase.pb.StopMasterRequest\032\034.h" +
-      "base.pb.StopMasterResponse\022h\n\031IsMasterIn" +
-      "MaintenanceMode\022$.hbase.pb.IsInMaintenan" +
-      "ceModeRequest\032%.hbase.pb.IsInMaintenance" +
-      "ModeResponse\022>\n\007Balance\022\030.hbase.pb.Balan" +
-      "ceRequest\032\031.hbase.pb.BalanceResponse\022_\n\022" +
-      "SetBalancerRunning\022#.hbase.pb.SetBalance",
-      "rRunningRequest\032$.hbase.pb.SetBalancerRu" +
-      "nningResponse\022\\\n\021IsBalancerEnabled\022\".hba" +
-      "se.pb.IsBalancerEnabledRequest\032#.hbase.p" +
-      "b.IsBalancerEnabledResponse\022k\n\026SetSplitO" +
-      "rMergeEnabled\022\'.hbase.pb.SetSplitOrMerge" +
-      "EnabledRequest\032(.hbase.pb.SetSplitOrMerg" +
-      "eEnabledResponse\022h\n\025IsSplitOrMergeEnable" +
-      "d\022&.hbase.pb.IsSplitOrMergeEnabledReques" +
-      "t\032\'.hbase.pb.IsSplitOrMergeEnabledRespon" +
-      "se\022D\n\tNormalize\022\032.hbase.pb.NormalizeRequ",
-      "est\032\033.hbase.pb.NormalizeResponse\022e\n\024SetN" +
-      "ormalizerRunning\022%.hbase.pb.SetNormalize" +
-      "rRunningRequest\032&.hbase.pb.SetNormalizer" +
-      "RunningResponse\022b\n\023IsNormalizerEnabled\022$" +
-      ".hbase.pb.IsNormalizerEnabledRequest\032%.h" +
-      "base.pb.IsNormalizerEnabledResponse\022S\n\016R" +
-      "unCatalogScan\022\037.hbase.pb.RunCatalogScanR" +
-      "equest\032 .hbase.pb.RunCatalogScanResponse" +
-      "\022e\n\024EnableCatalogJanitor\022%.hbase.pb.Enab" +
-      "leCatalogJanitorRequest\032&.hbase.pb.Enabl",
-      "eCatalogJanitorResponse\022n\n\027IsCatalogJani" +
-      "torEnabled\022(.hbase.pb.IsCatalogJanitorEn" +
-      "abledRequest\032).hbase.pb.IsCatalogJanitor" +
-      "EnabledResponse\022V\n\017RunCleanerChore\022 .hba" +
-      "se.pb.RunCleanerChoreRequest\032!.hbase.pb." +
-      "RunCleanerChoreResponse\022k\n\026SetCleanerCho" +
-      "reRunning\022\'.hbase.pb.SetCleanerChoreRunn" +
-      "ingRequest\032(.hbase.pb.SetCleanerChoreRun" +
-      "ningResponse\022h\n\025IsCleanerChoreEnabled\022&." +
-      "hbase.pb.IsCleanerChoreEnabledRequest\032\'.",
-      "hbase.pb.IsCleanerChoreEnabledResponse\022^" +
-      "\n\021ExecMasterService\022#.hbase.pb.Coprocess" +
-      "orServiceRequest\032$.hbase.pb.CoprocessorS" +
-      "erviceResponse\022A\n\010Snapshot\022\031.hbase.pb.Sn" +
-      "apshotRequest\032\032.hbase.pb.SnapshotRespons" +
-      "e\022h\n\025GetCompletedSnapshots\022&.hbase.pb.Ge" +
-      "tCompletedSnapshotsRequest\032\'.hbase.pb.Ge" +
-      "tCompletedSnapshotsResponse\022S\n\016DeleteSna" +
-      "pshot\022\037.hbase.pb.DeleteSnapshotRequest\032 " +
-      ".hbase.pb.DeleteSnapshotResponse\022S\n\016IsSn",
-      "apshotDone\022\037.hbase.pb.IsSnapshotDoneRequ" +
-      "est\032 .hbase.pb.IsSnapshotDoneResponse\022V\n" +
-      "\017RestoreSnapshot\022 .hbase.pb.RestoreSnaps" +
-      "hotRequest\032!.hbase.pb.RestoreSnapshotRes" +
-      "ponse\022h\n\025IsRestoreSnapshotDone\022&.hbase.p" +
-      "b.IsRestoreSnapshotDoneRequest\032\'.hbase.p" +
-      "b.IsRestoreSnapshotDoneResponse\022P\n\rExecP" +
-      "rocedure\022\036.hbase.pb.ExecProcedureRequest" +
-      "\032\037.hbase.pb.ExecProcedureResponse\022W\n\024Exe" +
-      "cProcedureWithRet\022\036.hbase.pb.ExecProcedu",
-      "reRequest\032\037.hbase.pb.ExecProcedureRespon" +
-      "se\022V\n\017IsProcedureDone\022 .hbase.pb.IsProce" +
-      "dureDoneRequest\032!.hbase.pb.IsProcedureDo" +
-      "neResponse\022V\n\017ModifyNamespace\022 .hbase.pb" +
-      ".ModifyNamespaceRequest\032!.hbase.pb.Modif" +
-      "yNamespaceResponse\022V\n\017CreateNamespace\022 ." +
-      "hbase.pb.CreateNamespaceRequest\032!.hbase." +
-      "pb.CreateNamespaceResponse\022V\n\017DeleteName" +
-      "space\022 .hbase.pb.DeleteNamespaceRequest\032" +
-      "!.hbase.pb.DeleteNamespaceResponse\022k\n\026Ge",
-      "tNamespaceDescriptor\022\'.hbase.pb.GetNames" +
-      "paceDescriptorRequest\032(.hbase.pb.GetName" +
-      "spaceDescriptorResponse\022q\n\030ListNamespace" +
-      "Descriptors\022).hbase.pb.ListNamespaceDesc" +
-      "riptorsRequest\032*.hbase.pb.ListNamespaceD" +
-      "escriptorsResponse\022\206\001\n\037ListTableDescript" +
-      "orsByNamespace\0220.hbase.pb.ListTableDescr" +
-      "iptorsByNamespaceRequest\0321.hbase.pb.List" +
-      "TableDescriptorsByNamespaceResponse\022t\n\031L" +
-      "istTableNamesByNamespace\022*.hbase.pb.List",
-      "TableNamesByNamespaceRequest\032+.hbase.pb." +
-      "ListTableNamesByNamespaceResponse\022A\n\010Set" +
-      "Quota\022\031.hbase.pb.SetQuotaRequest\032\032.hbase" +
-      ".pb.SetQuotaResponse\022x\n\037getLastMajorComp" +
-      "actionTimestamp\022).hbase.pb.MajorCompacti" +
-      "onTimestampRequest\032*.hbase.pb.MajorCompa" +
-      "ctionTimestampResponse\022\212\001\n(getLastMajorC" +
-      "ompactionTimestampForRegion\0222.hbase.pb.M" +
-      "ajorCompactionTimestampForRegionRequest\032" +
-      "*.hbase.pb.MajorCompactionTimestampRespo",
-      "nse\022_\n\022getProcedureResult\022#.hbase.pb.Get" +
-      "ProcedureResultRequest\032$.hbase.pb.GetPro" +
-      "cedureResultResponse\022h\n\027getSecurityCapab" +
-      "ilities\022%.hbase.pb.SecurityCapabilitiesR" +
-      "equest\032&.hbase.pb.SecurityCapabilitiesRe" +
-      "sponse\022S\n\016AbortProcedure\022\037.hbase.pb.Abor" +
-      "tProcedureRequest\032 .hbase.pb.AbortProced" +
-      "ureResponse\022S\n\016ListProcedures\022\037.hbase.pb" +
-      ".ListProceduresRequest\032 .hbase.pb.ListPr" +
-      "oceduresResponse\022Y\n\020ClearDeadServers\022!.h",
-      "base.pb.ClearDeadServersRequest\032\".hbase." +
-      "pb.ClearDeadServersResponse\022S\n\016ListNames" +
-      "paces\022\037.hbase.pb.ListNamespacesRequest\032 " +
-      ".hbase.pb.ListNamespacesResponse\022b\n\025Swit" +
-      "chSnapshotCleanup\022#.hbase.pb.SetSnapshot" +
-      "CleanupRequest\032$.hbase.pb.SetSnapshotCle" +
-      "anupResponse\022q\n\030IsSnapshotCleanupEnabled" +
-      "\022).hbase.pb.IsSnapshotCleanupEnabledRequ" +
-      "est\032*.hbase.pb.IsSnapshotCleanupEnabledR" +
-      "esponse\022P\n\rGetTableState\022\036.hbase.pb.GetT",
-      "ableStateRequest\032\037.hbase.pb.GetTableStat" +
-      "eResponse2\347\002\n\021ClientMetaService\022M\n\014GetCl" +
-      "usterId\022\035.hbase.pb.GetClusterIdRequest\032\036" +
-      ".hbase.pb.GetClusterIdResponse\022G\n\nGetMas" +
-      "ters\022\033.hbase.pb.GetMastersRequest\032\034.hbas" +
-      "e.pb.GetMastersResponse\022k\n\026GetMetaRegion" +
-      "Locations\022\'.hbase.pb.GetMetaRegionLocati" +
-      "onsRequest\032(.hbase.pb.GetMetaRegionLocat" +
-      "ionsResponse\022M\n\014GetNumLiveRS\022\035.hbase.pb." +
-      "GetNumLiveRSRequest\032\036.hbase.pb.GetNumLiv",
-      "eRSResponseBB\n*org.apache.hadoop.hbase.p" +
-      "rotobuf.generatedB\014MasterProtosH\001\210\001\001\240\001\001"
+      "oto\032\013Quota.proto\032\016Snapshot.proto\032\020Recent" +
+      "Logs.proto\"\234\001\n\020AddColumnRequest\022\'\n\ntable" +
+      "_name\030\001 \002(\0132\023.hbase.pb.TableName\0225\n\017colu" +
+      "mn_families\030\002 \002(\0132\034.hbase.pb.ColumnFamil" +
+      "ySchema\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce" +
+      "\030\004 \001(\004:\0010\"\023\n\021AddColumnResponse\"}\n\023Delete" +
+      "ColumnRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbas",
+      "e.pb.TableName\022\023\n\013column_name\030\002 \002(\014\022\026\n\013n" +
+      "once_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\026\n" +
+      "\024DeleteColumnResponse\"\237\001\n\023ModifyColumnRe" +
+      "quest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.Tab" +
+      "leName\0225\n\017column_families\030\002 \002(\0132\034.hbase." +
+      "pb.ColumnFamilySchema\022\026\n\013nonce_group\030\003 \001" +
+      "(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\026\n\024ModifyColumnR" +
+      "esponse\"n\n\021MoveRegionRequest\022)\n\006region\030\001" +
+      " \002(\0132\031.hbase.pb.RegionSpecifier\022.\n\020dest_" +
+      "server_name\030\002 \001(\0132\024.hbase.pb.ServerName\"",
+      "\024\n\022MoveRegionResponse\"\222\001\n\035DispatchMergin" +
+      "gRegionsRequest\022+\n\010region_a\030\001 \002(\0132\031.hbas" +
+      "e.pb.RegionSpecifier\022+\n\010region_b\030\002 \002(\0132\031" +
+      ".hbase.pb.RegionSpecifier\022\027\n\010forcible\030\003 " +
+      "\001(\010:\005false\" \n\036DispatchMergingRegionsResp" +
+      "onse\"@\n\023AssignRegionRequest\022)\n\006region\030\001 " +
+      "\002(\0132\031.hbase.pb.RegionSpecifier\"\026\n\024Assign" +
+      "RegionResponse\"X\n\025UnassignRegionRequest\022" +
+      ")\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
+      "er\022\024\n\005force\030\002 \001(\010:\005false\"\030\n\026UnassignRegi",
+      "onResponse\"A\n\024OfflineRegionRequest\022)\n\006re" +
+      "gion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\"\027\n" +
+      "\025OfflineRegionResponse\"\177\n\022CreateTableReq" +
+      "uest\022+\n\014table_schema\030\001 \002(\0132\025.hbase.pb.Ta" +
+      "bleSchema\022\022\n\nsplit_keys\030\002 \003(\014\022\026\n\013nonce_g" +
+      "roup\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"&\n\023Creat" +
+      "eTableResponse\022\017\n\007proc_id\030\001 \001(\004\"g\n\022Delet" +
+      "eTableRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbas" +
+      "e.pb.TableName\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020" +
+      "\n\005nonce\030\003 \001(\004:\0010\"&\n\023DeleteTableResponse\022",
+      "\017\n\007proc_id\030\001 \001(\004\"\207\001\n\024TruncateTableReques" +
+      "t\022&\n\ttableName\030\001 \002(\0132\023.hbase.pb.TableNam" +
+      "e\022\035\n\016preserveSplits\030\002 \001(\010:\005false\022\026\n\013nonc" +
+      "e_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\027\n\025Tr" +
+      "uncateTableResponse\"g\n\022EnableTableReques" +
+      "t\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.TableNa" +
+      "me\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(" +
+      "\004:\0010\"&\n\023EnableTableResponse\022\017\n\007proc_id\030\001" +
+      " \001(\004\"h\n\023DisableTableRequest\022\'\n\ntable_nam" +
+      "e\030\001 \002(\0132\023.hbase.pb.TableName\022\026\n\013nonce_gr",
+      "oup\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"\'\n\024Disabl" +
+      "eTableResponse\022\017\n\007proc_id\030\001 \001(\004\"\224\001\n\022Modi" +
+      "fyTableRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hba" +
+      "se.pb.TableName\022+\n\014table_schema\030\002 \002(\0132\025." +
+      "hbase.pb.TableSchema\022\026\n\013nonce_group\030\003 \001(" +
+      "\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\025\n\023ModifyTableRes" +
+      "ponse\"~\n\026CreateNamespaceRequest\022:\n\023names" +
+      "paceDescriptor\030\001 \002(\0132\035.hbase.pb.Namespac" +
+      "eDescriptor\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005n" +
+      "once\030\003 \001(\004:\0010\"\031\n\027CreateNamespaceResponse",
+      "\"Y\n\026DeleteNamespaceRequest\022\025\n\rnamespaceN" +
+      "ame\030\001 \002(\t\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005non" +
+      "ce\030\003 \001(\004:\0010\"\031\n\027DeleteNamespaceResponse\"~" +
+      "\n\026ModifyNamespaceRequest\022:\n\023namespaceDes" +
+      "criptor\030\001 \002(\0132\035.hbase.pb.NamespaceDescri" +
+      "ptor\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 " +
+      "\001(\004:\0010\"\031\n\027ModifyNamespaceResponse\"6\n\035Get" +
+      "NamespaceDescriptorRequest\022\025\n\rnamespaceN" +
+      "ame\030\001 \002(\t\"\\\n\036GetNamespaceDescriptorRespo" +
+      "nse\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.hbase",
+      ".pb.NamespaceDescriptor\"\027\n\025ListNamespace" +
+      "sRequest\"/\n\026ListNamespacesResponse\022\025\n\rna" +
+      "mespaceName\030\001 \003(\t\"!\n\037ListNamespaceDescri" +
+      "ptorsRequest\"^\n ListNamespaceDescriptors" +
+      "Response\022:\n\023namespaceDescriptor\030\001 \003(\0132\035." +
+      "hbase.pb.NamespaceDescriptor\"?\n&ListTabl" +
+      "eDescriptorsByNamespaceRequest\022\025\n\rnamesp" +
+      "aceName\030\001 \002(\t\"U\n\'ListTableDescriptorsByN" +
+      "amespaceResponse\022*\n\013tableSchema\030\001 \003(\0132\025." +
+      "hbase.pb.TableSchema\"9\n ListTableNamesBy",
+      "NamespaceRequest\022\025\n\rnamespaceName\030\001 \002(\t\"" +
+      "K\n!ListTableNamesByNamespaceResponse\022&\n\t" +
+      "tableName\030\001 \003(\0132\023.hbase.pb.TableName\"\021\n\017" +
+      "ShutdownRequest\"\022\n\020ShutdownResponse\"\023\n\021S" +
+      "topMasterRequest\"\024\n\022StopMasterResponse\"\034" +
+      "\n\032IsInMaintenanceModeRequest\"8\n\033IsInMain" +
+      "tenanceModeResponse\022\031\n\021inMaintenanceMode" +
+      "\030\001 \002(\010\"\037\n\016BalanceRequest\022\r\n\005force\030\001 \001(\010\"" +
+      "\'\n\017BalanceResponse\022\024\n\014balancer_ran\030\001 \002(\010" +
+      "\"<\n\031SetBalancerRunningRequest\022\n\n\002on\030\001 \002(",
+      "\010\022\023\n\013synchronous\030\002 \001(\010\"8\n\032SetBalancerRun" +
+      "ningResponse\022\032\n\022prev_balance_value\030\001 \001(\010" +
+      "\"\032\n\030IsBalancerEnabledRequest\",\n\031IsBalanc" +
+      "erEnabledResponse\022\017\n\007enabled\030\001 \002(\010\"w\n\035Se" +
+      "tSplitOrMergeEnabledRequest\022\017\n\007enabled\030\001" +
+      " \002(\010\022\023\n\013synchronous\030\002 \001(\010\0220\n\014switch_type" +
+      "s\030\003 \003(\0162\032.hbase.pb.MasterSwitchType\"4\n\036S" +
+      "etSplitOrMergeEnabledResponse\022\022\n\nprev_va" +
+      "lue\030\001 \003(\010\"O\n\034IsSplitOrMergeEnabledReques" +
+      "t\022/\n\013switch_type\030\001 \002(\0162\032.hbase.pb.Master",
+      "SwitchType\"0\n\035IsSplitOrMergeEnabledRespo" +
+      "nse\022\017\n\007enabled\030\001 \002(\010\"\022\n\020NormalizeRequest" +
+      "\"+\n\021NormalizeResponse\022\026\n\016normalizer_ran\030" +
+      "\001 \002(\010\")\n\033SetNormalizerRunningRequest\022\n\n\002" +
+      "on\030\001 \002(\010\"=\n\034SetNormalizerRunningResponse" +
+      "\022\035\n\025prev_normalizer_value\030\001 \001(\010\"\034\n\032IsNor" +
+      "malizerEnabledRequest\".\n\033IsNormalizerEna" +
+      "bledResponse\022\017\n\007enabled\030\001 \002(\010\"\027\n\025RunCata" +
+      "logScanRequest\"-\n\026RunCatalogScanResponse" +
+      "\022\023\n\013scan_result\030\001 \001(\005\"-\n\033EnableCatalogJa",
+      "nitorRequest\022\016\n\006enable\030\001 \002(\010\"2\n\034EnableCa" +
+      "talogJanitorResponse\022\022\n\nprev_value\030\001 \001(\010" +
+      "\" \n\036IsCatalogJanitorEnabledRequest\"0\n\037Is" +
+      "CatalogJanitorEnabledResponse\022\r\n\005value\030\001" +
+      " \002(\010\"\030\n\026RunCleanerChoreRequest\"4\n\027RunCle" +
+      "anerChoreResponse\022\031\n\021cleaner_chore_ran\030\001" +
+      " \002(\010\"+\n\035SetCleanerChoreRunningRequest\022\n\n" +
+      "\002on\030\001 \002(\010\"4\n\036SetCleanerChoreRunningRespo" +
+      "nse\022\022\n\nprev_value\030\001 \001(\010\"\036\n\034IsCleanerChor" +
+      "eEnabledRequest\".\n\035IsCleanerChoreEnabled",
+      "Response\022\r\n\005value\030\001 \002(\010\"B\n\017SnapshotReque" +
+      "st\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.Snapshot" +
+      "Description\",\n\020SnapshotResponse\022\030\n\020expec" +
+      "ted_timeout\030\001 \002(\003\"\036\n\034GetCompletedSnapsho" +
+      "tsRequest\"Q\n\035GetCompletedSnapshotsRespon" +
+      "se\0220\n\tsnapshots\030\001 \003(\0132\035.hbase.pb.Snapsho" +
+      "tDescription\"H\n\025DeleteSnapshotRequest\022/\n" +
+      "\010snapshot\030\001 \002(\0132\035.hbase.pb.SnapshotDescr" +
+      "iption\"\030\n\026DeleteSnapshotResponse\"d\n\026Rest" +
+      "oreSnapshotRequest\022/\n\010snapshot\030\001 \002(\0132\035.h",
+      "base.pb.SnapshotDescription\022\031\n\nrestoreAC" +
+      "L\030\002 \001(\010:\005false\"\031\n\027RestoreSnapshotRespons" +
+      "e\"H\n\025IsSnapshotDoneRequest\022/\n\010snapshot\030\001" +
+      " \001(\0132\035.hbase.pb.SnapshotDescription\"^\n\026I" +
+      "sSnapshotDoneResponse\022\023\n\004done\030\001 \001(\010:\005fal" +
+      "se\022/\n\010snapshot\030\002 \001(\0132\035.hbase.pb.Snapshot" +
+      "Description\"O\n\034IsRestoreSnapshotDoneRequ" +
+      "est\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb.Snapsho" +
+      "tDescription\"4\n\035IsRestoreSnapshotDoneRes" +
+      "ponse\022\023\n\004done\030\001 \001(\010:\005false\"F\n\033GetSchemaA",
+      "lterStatusRequest\022\'\n\ntable_name\030\001 \002(\0132\023." +
+      "hbase.pb.TableName\"T\n\034GetSchemaAlterStat" +
+      "usResponse\022\035\n\025yet_to_update_regions\030\001 \001(" +
+      "\r\022\025\n\rtotal_regions\030\002 \001(\r\"\213\001\n\032GetTableDes" +
+      "criptorsRequest\022(\n\013table_names\030\001 \003(\0132\023.h" +
+      "base.pb.TableName\022\r\n\005regex\030\002 \001(\t\022!\n\022incl" +
+      "ude_sys_tables\030\003 \001(\010:\005false\022\021\n\tnamespace" +
+      "\030\004 \001(\t\"J\n\033GetTableDescriptorsResponse\022+\n" +
+      "\014table_schema\030\001 \003(\0132\025.hbase.pb.TableSche" +
+      "ma\"[\n\024GetTableNamesRequest\022\r\n\005regex\030\001 \001(",
+      "\t\022!\n\022include_sys_tables\030\002 \001(\010:\005false\022\021\n\t" +
+      "namespace\030\003 \001(\t\"A\n\025GetTableNamesResponse" +
+      "\022(\n\013table_names\030\001 \003(\0132\023.hbase.pb.TableNa" +
+      "me\"?\n\024GetTableStateRequest\022\'\n\ntable_name" +
+      "\030\001 \002(\0132\023.hbase.pb.TableName\"B\n\025GetTableS" +
+      "tateResponse\022)\n\013table_state\030\001 \002(\0132\024.hbas" +
+      "e.pb.TableState\"\031\n\027GetClusterStatusReque" +
+      "st\"K\n\030GetClusterStatusResponse\022/\n\016cluste" +
+      "r_status\030\001 \002(\0132\027.hbase.pb.ClusterStatus\"" +
+      "\030\n\026IsMasterRunningRequest\"4\n\027IsMasterRun",
+      "ningResponse\022\031\n\021is_master_running\030\001 \002(\010\"" +
+      "I\n\024ExecProcedureRequest\0221\n\tprocedure\030\001 \002" +
+      "(\0132\036.hbase.pb.ProcedureDescription\"F\n\025Ex" +
+      "ecProcedureResponse\022\030\n\020expected_timeout\030" +
+      "\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"K\n\026IsProcedur" +
+      "eDoneRequest\0221\n\tprocedure\030\001 \001(\0132\036.hbase." +
+      "pb.ProcedureDescription\"`\n\027IsProcedureDo" +
+      "neResponse\022\023\n\004done\030\001 \001(\010:\005false\0220\n\010snaps" +
+      "hot\030\002 \001(\0132\036.hbase.pb.ProcedureDescriptio" +
+      "n\",\n\031GetProcedureResultRequest\022\017\n\007proc_i",
+      "d\030\001 \002(\004\"\371\001\n\032GetProcedureResultResponse\0229" +
+      "\n\005state\030\001 \002(\0162*.hbase.pb.GetProcedureRes" +
+      "ultResponse.State\022\022\n\nstart_time\030\002 \001(\004\022\023\n" +
+      "\013last_update\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\0224\n\tex" +
+      "ception\030\005 \001(\0132!.hbase.pb.ForeignExceptio" +
+      "nMessage\"1\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNN" +
+      "ING\020\001\022\014\n\010FINISHED\020\002\"M\n\025AbortProcedureReq" +
+      "uest\022\017\n\007proc_id\030\001 \002(\004\022#\n\025mayInterruptIfR" +
+      "unning\030\002 \001(\010:\004true\"6\n\026AbortProcedureResp" +
+      "onse\022\034\n\024is_procedure_aborted\030\001 \002(\010\"\027\n\025Li",
+      "stProceduresRequest\"@\n\026ListProceduresRes" +
+      "ponse\022&\n\tprocedure\030\001 \003(\0132\023.hbase.pb.Proc" +
+      "edure\"\315\001\n\017SetQuotaRequest\022\021\n\tuser_name\030\001" +
+      " \001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\tnamespace\030\003 " +
+      "\001(\t\022\'\n\ntable_name\030\004 \001(\0132\023.hbase.pb.Table" +
+      "Name\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypass_globa" +
+      "ls\030\006 \001(\010\022+\n\010throttle\030\007 \001(\0132\031.hbase.pb.Th" +
+      "rottleRequest\"\022\n\020SetQuotaResponse\"J\n\037Maj" +
+      "orCompactionTimestampRequest\022\'\n\ntable_na" +
+      "me\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(MajorCo",
+      "mpactionTimestampForRegionRequest\022)\n\006reg" +
+      "ion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\"@\n " +
+      "MajorCompactionTimestampResponse\022\034\n\024comp" +
+      "action_timestamp\030\001 \002(\003\"\035\n\033SecurityCapabi" +
+      "litiesRequest\"\354\001\n\034SecurityCapabilitiesRe" +
+      "sponse\022G\n\014capabilities\030\001 \003(\01621.hbase.pb." +
+      "SecurityCapabilitiesResponse.Capability\"" +
+      "\202\001\n\nCapability\022\031\n\025SIMPLE_AUTHENTICATION\020" +
+      "\000\022\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rAUTHORIZ" +
+      "ATION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017CELL_",
+      "VISIBILITY\020\004\"D\n\027ClearDeadServersRequest\022" +
+      ")\n\013server_name\030\001 \003(\0132\024.hbase.pb.ServerNa" +
+      "me\"E\n\030ClearDeadServersResponse\022)\n\013server" +
+      "_name\030\001 \003(\0132\024.hbase.pb.ServerName\"A\n\031Set" +
+      "SnapshotCleanupRequest\022\017\n\007enabled\030\001 \002(\010\022" +
+      "\023\n\013synchronous\030\002 \001(\010\";\n\032SetSnapshotClean" +
+      "upResponse\022\035\n\025prev_snapshot_cleanup\030\001 \002(" +
+      "\010\"!\n\037IsSnapshotCleanupEnabledRequest\"3\n " +
+      "IsSnapshotCleanupEnabledResponse\022\017\n\007enab" +
+      "led\030\001 \002(\010\")\n\030BalancerDecisionsRequest\022\r\n",
+      "\005limit\030\001 \001(\r\"R\n\031BalancerDecisionsRespons" +
+      "e\0225\n\021balancer_decision\030\001 \003(\0132\032.hbase.pb." +
+      "BalancerDecision\"\025\n\023GetClusterIdRequest\"" +
+      "*\n\024GetClusterIdResponse\022\022\n\ncluster_id\030\001 " +
+      "\001(\t\"\023\n\021GetMastersRequest\"W\n\027GetMastersRe" +
+      "sponseEntry\022)\n\013server_name\030\001 \002(\0132\024.hbase" +
+      ".pb.ServerName\022\021\n\tis_active\030\002 \002(\010\"O\n\022Get" +
+      "MastersResponse\0229\n\016master_servers\030\001 \003(\0132" +
+      "!.hbase.pb.GetMastersResponseEntry\"\037\n\035Ge" +
+      "tMetaRegionLocationsRequest\"R\n\036GetMetaRe",
+      "gionLocationsResponse\0220\n\016meta_locations\030" +
+      "\001 \003(\0132\030.hbase.pb.RegionLocation\"\025\n\023GetNu" +
+      "mLiveRSRequest\"2\n\024GetNumLiveRSResponse\022\032" +
+      "\n\022num_region_servers\030\001 \002(\005*(\n\020MasterSwit" +
+      "chType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\256/\n\rMasterS" +
+      "ervice\022e\n\024GetSchemaAlterStatus\022%.hbase.p" +
+      "b.GetSchemaAlterStatusRequest\032&.hbase.pb" +
+      ".GetSchemaAlterStatusResponse\022b\n\023GetTabl" +
+      "eDescriptors\022$.hbase.pb.GetTableDescript" +
+      "orsRequest\032%.hbase.pb.GetTableDescriptor",
+      "sResponse\022P\n\rGetTableNames\022\036.hbase.pb.Ge" +
+      "tTableNamesRequest\032\037.hbase.pb.GetTableNa" +
+      "mesResponse\022Y\n\020GetClusterStatus\022!.hbase." +
+      "pb.GetClusterStatusRequest\032\".hbase.pb.Ge" +
+      "tClusterStatusResponse\022V\n\017IsMasterRunnin" +
+      "g\022 .hbase.pb.IsMasterRunningRequest\032!.hb" +
+      "ase.pb.IsMasterRunningResponse\022D\n\tAddCol" +
+      "umn\022\032.hbase.pb.AddColumnRequest\032\033.hbase." +
+      "pb.AddColumnResponse\022M\n\014DeleteColumn\022\035.h" +
+      "base.pb.DeleteColumnRequest\032\036.hbase.pb.D",
+      "eleteColumnResponse\022M\n\014ModifyColumn\022\035.hb" +
+      "ase.pb.ModifyColumnRequest\032\036.hbase.pb.Mo" +
+      "difyColumnResponse\022G\n\nMoveRegion\022\033.hbase" +
+      ".pb.MoveRegionRequest\032\034.hbase.pb.MoveReg" +
+      "ionResponse\022k\n\026DispatchMergingRegions\022\'." +
+      "hbase.pb.DispatchMergingRegionsRequest\032(" +
+      ".hbase.pb.DispatchMergingRegionsResponse" +
+      "\022M\n\014AssignRegion\022\035.hbase.pb.AssignRegion" +
+      "Request\032\036.hbase.pb.AssignRegionResponse\022" +
+      "S\n\016UnassignRegion\022\037.hbase.pb.UnassignReg",
+      "ionRequest\032 .hbase.pb.UnassignRegionResp" +
+      "onse\022P\n\rOfflineRegion\022\036.hbase.pb.Offline" +
+      "RegionRequest\032\037.hbase.pb.OfflineRegionRe" +
+      "sponse\022J\n\013DeleteTable\022\034.hbase.pb.DeleteT" +
+      "ableRequest\032\035.hbase.pb.DeleteTableRespon" +
+      "se\022P\n\rtruncateTable\022\036.hbase.pb.TruncateT" +
+      "ableRequest\032\037.hbase.pb.TruncateTableResp" +
+      "onse\022J\n\013EnableTable\022\034.hbase.pb.EnableTab" +
+      "leRequest\032\035.hbase.pb.EnableTableResponse" +
+      "\022M\n\014DisableTable\022\035.hbase.pb.DisableTable",
+      "Request\032\036.hbase.pb.DisableTableResponse\022" +
+      "J\n\013ModifyTable\022\034.hbase.pb.ModifyTableReq" +
+      "uest\032\035.hbase.pb.ModifyTableResponse\022J\n\013C" +
+      "reateTable\022\034.hbase.pb.CreateTableRequest" +
+      "\032\035.hbase.pb.CreateTableResponse\022A\n\010Shutd" +
+      "own\022\031.hbase.pb.ShutdownRequest\032\032.hbase.p" +
+      "b.ShutdownResponse\022G\n\nStopMaster\022\033.hbase" +
+      ".pb.StopMasterRequest\032\034.hbase.pb.StopMas" +
+      "terResponse\022h\n\031IsMasterInMaintenanceMode" +
+      "\022$.hbase.pb.IsInMaintenanceModeRequest\032%",
+      ".hbase.pb.IsInMaintenanceModeResponse\022>\n" +
+      "\007Balance\022\030.hbase.pb.BalanceRequest\032\031.hba" +
+      "se.pb.BalanceResponse\022_\n\022SetBalancerRunn" +
+      "ing\022#.hbase.pb.SetBalancerRunningRequest" +
+      "\032$.hbase.pb.SetBalancerRunningResponse\022\\" +
+      "\n\021IsBalancerEnabled\022\".hbase.pb.IsBalance" +
+      "rEnabledRequest\032#.hbase.pb.IsBalancerEna" +
+      "bledResponse\022k\n\026SetSplitOrMergeEnabled\022\'" +
+      ".hbase.pb.SetSplitOrMergeEnabledRequest\032" +
+      "(.hbase.pb.SetSplitOrMergeEnabledRespons",
+      "e\022h\n\025IsSplitOrMergeEnabled\022&.hbase.pb.Is" +
+      "SplitOrMergeEnabledRequest\032\'.hbase.pb.Is" +
+      "SplitOrMergeEnabledResponse\022D\n\tNormalize" +
+      "\022\032.hbase.pb.NormalizeRequest\032\033.hbase.pb." +
+      "NormalizeResponse\022e\n\024SetNormalizerRunnin" +
+      "g\022%.hbase.pb.SetNormalizerRunningRequest" +
+      "\032&.hbase.pb.SetNormalizerRunningResponse" +
+      "\022b\n\023IsNormalizerEnabled\022$.hbase.pb.IsNor" +
+      "malizerEnabledRequest\032%.hbase.pb.IsNorma" +
+      "lizerEnabledResponse\022S\n\016RunCatalogScan\022\037",
+      ".hbase.pb.RunCatalogScanRequest\032 .hbase." +
+      "pb.RunCatalogScanResponse\022e\n\024EnableCatal" +
+      "ogJanitor\022%.hbase.pb.EnableCatalogJanito" +
+      "rRequest\032&.hbase.pb.EnableCatalogJanitor" +
+      "Response\022n\n\027IsCatalogJanitorEnabled\022(.hb" +
+      "ase.pb.IsCatalogJanitorEnabledRequest\032)." +
+      "hbase.pb.IsCatalogJanitorEnabledResponse" +
+      "\022V\n\017RunCleanerChore\022 .hbase.pb.RunCleane" +
+      "rChoreRequest\032!.hbase.pb.RunCleanerChore" +
+      "Response\022k\n\026SetCleanerChoreRunning\022\'.hba",
+      "se.pb.SetCleanerChoreRunningRequest\032(.hb" +
+      "ase.pb.SetCleanerChoreRunningResponse\022h\n" +
+      "\025IsCleanerChoreEnabled\022&.hbase.pb.IsClea" +
+      "nerChoreEnabledRequest\032\'.hbase.pb.IsClea" +
+      "nerChoreEnabledResponse\022^\n\021ExecMasterSer" +
+      "vice\022#.hbase.pb.CoprocessorServiceReques" +
+      "t\032$.hbase.pb.CoprocessorServiceResponse\022" +
+      "A\n\010Snapshot\022\031.hbase.pb.SnapshotRequest\032\032" +
+      ".hbase.pb.SnapshotResponse\022h\n\025GetComplet" +
+      "edSnapshots\022&.hbase.pb.GetCompletedSnaps",
+      "hotsRequest\032\'.hbase.pb.GetCompletedSnaps" +
+      "hotsResponse\022S\n\016DeleteSnapshot\022\037.hbase.p" +
+      "b.DeleteSnapshotRequest\032 .hbase.pb.Delet" +
+      "eSnapshotResponse\022S\n\016IsSnapshotDone\022\037.hb" +
+      "ase.pb.IsSnapshotDoneRequest\032 .hbase.pb." +
+      "IsSnapshotDoneResponse\022V\n\017RestoreSnapsho" +
+      "t\022 .hbase.pb.RestoreSnapshotRequest\032!.hb" +
+      "ase.pb.RestoreSnapshotResponse\022h\n\025IsRest" +
+      "oreSnapshotDone\022&.hbase.pb.IsRestoreSnap" +
+      "shotDoneRequest\032\'.hbase.pb.IsRestoreSnap",
+      "shotDoneResponse\022P\n\rExecProcedure\022\036.hbas" +
+      "e.pb.ExecProcedureRequest\032\037.hbase.pb.Exe" +
+      "cProcedureResponse\022W\n\024ExecProcedureWithR" +
+      "et\022\036.hbase.pb.ExecProcedureRequest\032\037.hba" +
+      "se.pb.ExecProcedureResponse\022V\n\017IsProcedu" +
+      "reDone\022 .hbase.pb.IsProcedureDoneRequest" +
+      "\032!.hbase.pb.IsProcedureDoneResponse\022V\n\017M" +
+      "odifyNamespace\022 .hbase.pb.ModifyNamespac" +
+      "eRequest\032!.hbase.pb.ModifyNamespaceRespo" +
+      "nse\022V\n\017CreateNamespace\022 .hbase.pb.Create",
+      "NamespaceRequest\032!.hbase.pb.CreateNamesp" +
+      "aceResponse\022V\n\017DeleteNamespace\022 .hbase.p" +
+      "b.DeleteNamespaceRequest\032!.hbase.pb.Dele" +
+      "teNamespaceResponse\022k\n\026GetNamespaceDescr" +
+      "iptor\022\'.hbase.pb.GetNamespaceDescriptorR" +
+      "equest\032(.hbase.pb.GetNamespaceDescriptor" +
+      "Response\022q\n\030ListNamespaceDescriptors\022).h" +
+      "base.pb.ListNamespaceDescriptorsRequest\032" +
+      "*.hbase.pb.ListNamespaceDescriptorsRespo" +
+      "nse\022\206\001\n\037ListTableDescriptorsByNamespace\022",
+      "0.hbase.pb.ListTableDescriptorsByNamespa" +
+      "ceRequest\0321.hbase.pb.ListTableDescriptor" +
+      "sByNamespaceResponse\022t\n\031ListTableNamesBy" +
+      "Namespace\022*.hbase.pb.ListTableNamesByNam" +
+      "espaceRequest\032+.hbase.pb.ListTableNamesB" +
+      "yNamespaceResponse\022A\n\010SetQuota\022\031.hbase.p" +
+      "b.SetQuotaRequest\032\032.hbase.pb.SetQuotaRes" +
+      "ponse\022x\n\037getLastMajorCompactionTimestamp" +
+      "\022).hbase.pb.MajorCompactionTimestampRequ" +
+      "est\032*.hbase.pb.MajorCompactionTimestampR",
+      "esponse\022\212\001\n(getLastMajorCompactionTimest" +
+      "ampForRegion\0222.hbase.pb.MajorCompactionT" +
+      "imestampForRegionRequest\032*.hbase.pb.Majo" +
+      "rCompactionTimestampResponse\022_\n\022getProce" +
+      "dureResult\022#.hbase.pb.GetProcedureResult" +
+      "Request\032$.hbase.pb.GetProcedureResultRes" +
+      "ponse\022h\n\027getSecurityCapabilities\022%.hbase" +
+      ".pb.SecurityCapabilitiesRequest\032&.hbase." +
+      "pb.SecurityCapabilitiesResponse\022S\n\016Abort" +
+      "Procedure\022\037.hbase.pb.AbortProcedureReque",
+      "st\032 .hbase.pb.AbortProcedureResponse\022S\n\016" +
+      "ListProcedures\022\037.hbase.pb.ListProcedures" +
+      "Request\032 .hbase.pb.ListProceduresRespons" +
+      "e\022Y\n\020ClearDeadServers\022!.hbase.pb.ClearDe" +
+      "adServersRequest\032\".hbase.pb.ClearDeadSer" +
+      "versResponse\022S\n\016ListNamespaces\022\037.hbase.p" +
+      "b.ListNamespacesRequest\032 .hbase.pb.ListN" +
+      "amespacesResponse\022b\n\025SwitchSnapshotClean" +
+      "up\022#.hbase.pb.SetSnapshotCleanupRequest\032" +
+      "$.hbase.pb.SetSnapshotCleanupResponse\022q\n",
+      "\030IsSnapshotCleanupEnabled\022).hbase.pb.IsS" +
+      "napshotCleanupEnabledRequest\032*.hbase.pb." +
+      "IsSnapshotCleanupEnabledResponse\0229\n\rGetL" +
+      "ogEntries\022\024.hbase.pb.LogRequest\032\022.hbase." +
+      "pb.LogEntry\022P\n\rGetTableState\022\036.hbase.pb." +
+      "GetTableStateRequest\032\037.hbase.pb.GetTable" +
+      "StateResponse2\347\002\n\021ClientMetaService\022M\n\014G" +
+      "etClusterId\022\035.hbase.pb.GetClusterIdReque" +
+      "st\032\036.hbase.pb.GetClusterIdResponse\022G\n\nGe" +
+      "tMasters\022\033.hbase.pb.GetMastersRequest\032\034.",
+      "hbase.pb.GetMastersResponse\022k\n\026GetMetaRe" +
+      "gionLocations\022\'.hbase.pb.GetMetaRegionLo" +
+      "cationsRequest\032(.hbase.pb.GetMetaRegionL" +
+      "ocationsResponse\022M\n\014GetNumLiveRS\022\035.hbase" +
+      ".pb.GetNumLiveRSRequest\032\036.hbase.pb.GetNu" +
+      "mLiveRSResponseBB\n*org.apache.hadoop.hba" +
+      "se.protobuf.generatedB\014MasterProtosH\001\210\001\001" +
+      "\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -78088,56 +79366,68 @@ public final class MasterProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_IsSnapshotCleanupEnabledResponse_descriptor,
               new java.lang.String[] { "Enabled", });
-          internal_static_hbase_pb_GetClusterIdRequest_descriptor =
+          internal_static_hbase_pb_BalancerDecisionsRequest_descriptor =
             getDescriptor().getMessageTypes().get(125);
+          internal_static_hbase_pb_BalancerDecisionsRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_BalancerDecisionsRequest_descriptor,
+              new java.lang.String[] { "Limit", });
+          internal_static_hbase_pb_BalancerDecisionsResponse_descriptor =
+            getDescriptor().getMessageTypes().get(126);
+          internal_static_hbase_pb_BalancerDecisionsResponse_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_BalancerDecisionsResponse_descriptor,
+              new java.lang.String[] { "BalancerDecision", });
+          internal_static_hbase_pb_GetClusterIdRequest_descriptor =
+            getDescriptor().getMessageTypes().get(127);
           internal_static_hbase_pb_GetClusterIdRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetClusterIdRequest_descriptor,
               new java.lang.String[] { });
           internal_static_hbase_pb_GetClusterIdResponse_descriptor =
-            getDescriptor().getMessageTypes().get(126);
+            getDescriptor().getMessageTypes().get(128);
           internal_static_hbase_pb_GetClusterIdResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetClusterIdResponse_descriptor,
               new java.lang.String[] { "ClusterId", });
           internal_static_hbase_pb_GetMastersRequest_descriptor =
-            getDescriptor().getMessageTypes().get(127);
+            getDescriptor().getMessageTypes().get(129);
           internal_static_hbase_pb_GetMastersRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetMastersRequest_descriptor,
               new java.lang.String[] { });
           internal_static_hbase_pb_GetMastersResponseEntry_descriptor =
-            getDescriptor().getMessageTypes().get(128);
+            getDescriptor().getMessageTypes().get(130);
           internal_static_hbase_pb_GetMastersResponseEntry_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetMastersResponseEntry_descriptor,
               new java.lang.String[] { "ServerName", "IsActive", });
           internal_static_hbase_pb_GetMastersResponse_descriptor =
-            getDescriptor().getMessageTypes().get(129);
+            getDescriptor().getMessageTypes().get(131);
           internal_static_hbase_pb_GetMastersResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetMastersResponse_descriptor,
               new java.lang.String[] { "MasterServers", });
           internal_static_hbase_pb_GetMetaRegionLocationsRequest_descriptor =
-            getDescriptor().getMessageTypes().get(130);
+            getDescriptor().getMessageTypes().get(132);
           internal_static_hbase_pb_GetMetaRegionLocationsRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetMetaRegionLocationsRequest_descriptor,
               new java.lang.String[] { });
           internal_static_hbase_pb_GetMetaRegionLocationsResponse_descriptor =
-            getDescriptor().getMessageTypes().get(131);
+            getDescriptor().getMessageTypes().get(133);
           internal_static_hbase_pb_GetMetaRegionLocationsResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetMetaRegionLocationsResponse_descriptor,
               new java.lang.String[] { "MetaLocations", });
           internal_static_hbase_pb_GetNumLiveRSRequest_descriptor =
-            getDescriptor().getMessageTypes().get(132);
+            getDescriptor().getMessageTypes().get(134);
           internal_static_hbase_pb_GetNumLiveRSRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetNumLiveRSRequest_descriptor,
               new java.lang.String[] { });
           internal_static_hbase_pb_GetNumLiveRSResponse_descriptor =
-            getDescriptor().getMessageTypes().get(133);
+            getDescriptor().getMessageTypes().get(135);
           internal_static_hbase_pb_GetNumLiveRSResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetNumLiveRSResponse_descriptor,
@@ -78156,6 +79446,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.getDescriptor(),
           org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.getDescriptor(),
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.getDescriptor(),
+          org.apache.hadoop.hbase.protobuf.generated.RecentLogs.getDescriptor(),
         }, assigner);
   }
 
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RecentLogs.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RecentLogs.java
new file mode 100644
index 0000000..3b6d873
--- /dev/null
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RecentLogs.java
@@ -0,0 +1,1229 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: RecentLogs.proto
+
+package org.apache.hadoop.hbase.protobuf.generated;
+
+public final class RecentLogs {
+  private RecentLogs() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface BalancerDecisionOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string initial_function_costs = 1;
+    /**
+     * <code>required string initial_function_costs = 1;</code>
+     */
+    boolean hasInitialFunctionCosts();
+    /**
+     * <code>required string initial_function_costs = 1;</code>
+     */
+    java.lang.String getInitialFunctionCosts();
+    /**
+     * <code>required string initial_function_costs = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getInitialFunctionCostsBytes();
+
+    // required string final_function_costs = 2;
+    /**
+     * <code>required string final_function_costs = 2;</code>
+     */
+    boolean hasFinalFunctionCosts();
+    /**
+     * <code>required string final_function_costs = 2;</code>
+     */
+    java.lang.String getFinalFunctionCosts();
+    /**
+     * <code>required string final_function_costs = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getFinalFunctionCostsBytes();
+
+    // required double init_total_cost = 3;
+    /**
+     * <code>required double init_total_cost = 3;</code>
+     */
+    boolean hasInitTotalCost();
+    /**
+     * <code>required double init_total_cost = 3;</code>
+     */
+    double getInitTotalCost();
+
+    // required double computed_total_cost = 4;
+    /**
+     * <code>required double computed_total_cost = 4;</code>
+     */
+    boolean hasComputedTotalCost();
+    /**
+     * <code>required double computed_total_cost = 4;</code>
+     */
+    double getComputedTotalCost();
+
+    // required uint64 computed_steps = 5;
+    /**
+     * <code>required uint64 computed_steps = 5;</code>
+     */
+    boolean hasComputedSteps();
+    /**
+     * <code>required uint64 computed_steps = 5;</code>
+     */
+    long getComputedSteps();
+
+    // repeated string region_plans = 6;
+    /**
+     * <code>repeated string region_plans = 6;</code>
+     */
+    java.util.List<java.lang.String>
+    getRegionPlansList();
+    /**
+     * <code>repeated string region_plans = 6;</code>
+     */
+    int getRegionPlansCount();
+    /**
+     * <code>repeated string region_plans = 6;</code>
+     */
+    java.lang.String getRegionPlans(int index);
+    /**
+     * <code>repeated string region_plans = 6;</code>
+     */
+    com.google.protobuf.ByteString
+        getRegionPlansBytes(int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.BalancerDecision}
+   */
+  public static final class BalancerDecision extends
+      com.google.protobuf.GeneratedMessage
+      implements BalancerDecisionOrBuilder {
+    // Use BalancerDecision.newBuilder() to construct.
+    private BalancerDecision(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private BalancerDecision(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final BalancerDecision defaultInstance;
+    public static BalancerDecision getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public BalancerDecision getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private BalancerDecision(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              initialFunctionCosts_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              finalFunctionCosts_ = input.readBytes();
+              break;
+            }
+            case 25: {
+              bitField0_ |= 0x00000004;
+              initTotalCost_ = input.readDouble();
+              break;
+            }
+            case 33: {
+              bitField0_ |= 0x00000008;
+              computedTotalCost_ = input.readDouble();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              computedSteps_ = input.readUInt64();
+              break;
+            }
+            case 50: {
+              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+                regionPlans_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000020;
+              }
+              regionPlans_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+          regionPlans_ = new com.google.protobuf.UnmodifiableLazyStringList(regionPlans_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.RecentLogs.internal_static_hbase_pb_BalancerDecision_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.RecentLogs.internal_static_hbase_pb_BalancerDecision_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.class, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<BalancerDecision> PARSER =
+        new com.google.protobuf.AbstractParser<BalancerDecision>() {
+      public BalancerDecision parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new BalancerDecision(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<BalancerDecision> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string initial_function_costs = 1;
+    public static final int INITIAL_FUNCTION_COSTS_FIELD_NUMBER = 1;
+    private java.lang.Object initialFunctionCosts_;
+    /**
+     * <code>required string initial_function_costs = 1;</code>
+     */
+    public boolean hasInitialFunctionCosts() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string initial_function_costs = 1;</code>
+     */
+    public java.lang.String getInitialFunctionCosts() {
+      java.lang.Object ref = initialFunctionCosts_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          initialFunctionCosts_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string initial_function_costs = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getInitialFunctionCostsBytes() {
+      java.lang.Object ref = initialFunctionCosts_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        initialFunctionCosts_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required string final_function_costs = 2;
+    public static final int FINAL_FUNCTION_COSTS_FIELD_NUMBER = 2;
+    private java.lang.Object finalFunctionCosts_;
+    /**
+     * <code>required string final_function_costs = 2;</code>
+     */
+    public boolean hasFinalFunctionCosts() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required string final_function_costs = 2;</code>
+     */
+    public java.lang.String getFinalFunctionCosts() {
+      java.lang.Object ref = finalFunctionCosts_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          finalFunctionCosts_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string final_function_costs = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getFinalFunctionCostsBytes() {
+      java.lang.Object ref = finalFunctionCosts_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        finalFunctionCosts_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required double init_total_cost = 3;
+    public static final int INIT_TOTAL_COST_FIELD_NUMBER = 3;
+    private double initTotalCost_;
+    /**
+     * <code>required double init_total_cost = 3;</code>
+     */
+    public boolean hasInitTotalCost() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required double init_total_cost = 3;</code>
+     */
+    public double getInitTotalCost() {
+      return initTotalCost_;
+    }
+
+    // required double computed_total_cost = 4;
+    public static final int COMPUTED_TOTAL_COST_FIELD_NUMBER = 4;
+    private double computedTotalCost_;
+    /**
+     * <code>required double computed_total_cost = 4;</code>
+     */
+    public boolean hasComputedTotalCost() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required double computed_total_cost = 4;</code>
+     */
+    public double getComputedTotalCost() {
+      return computedTotalCost_;
+    }
+
+    // required uint64 computed_steps = 5;
+    public static final int COMPUTED_STEPS_FIELD_NUMBER = 5;
+    private long computedSteps_;
+    /**
+     * <code>required uint64 computed_steps = 5;</code>
+     */
+    public boolean hasComputedSteps() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>required uint64 computed_steps = 5;</code>
+     */
+    public long getComputedSteps() {
+      return computedSteps_;
+    }
+
+    // repeated string region_plans = 6;
+    public static final int REGION_PLANS_FIELD_NUMBER = 6;
+    private com.google.protobuf.LazyStringList regionPlans_;
+    /**
+     * <code>repeated string region_plans = 6;</code>
+     */
+    public java.util.List<java.lang.String>
+        getRegionPlansList() {
+      return regionPlans_;
+    }
+    /**
+     * <code>repeated string region_plans = 6;</code>
+     */
+    public int getRegionPlansCount() {
+      return regionPlans_.size();
+    }
+    /**
+     * <code>repeated string region_plans = 6;</code>
+     */
+    public java.lang.String getRegionPlans(int index) {
+      return regionPlans_.get(index);
+    }
+    /**
+     * <code>repeated string region_plans = 6;</code>
+     */
+    public com.google.protobuf.ByteString
+        getRegionPlansBytes(int index) {
+      return regionPlans_.getByteString(index);
+    }
+
+    private void initFields() {
+      initialFunctionCosts_ = "";
+      finalFunctionCosts_ = "";
+      initTotalCost_ = 0D;
+      computedTotalCost_ = 0D;
+      computedSteps_ = 0L;
+      regionPlans_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasInitialFunctionCosts()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasFinalFunctionCosts()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasInitTotalCost()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasComputedTotalCost()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasComputedSteps()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getInitialFunctionCostsBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getFinalFunctionCostsBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeDouble(3, initTotalCost_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeDouble(4, computedTotalCost_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt64(5, computedSteps_);
+      }
+      for (int i = 0; i < regionPlans_.size(); i++) {
+        output.writeBytes(6, regionPlans_.getByteString(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getInitialFunctionCostsBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getFinalFunctionCostsBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeDoubleSize(3, initTotalCost_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeDoubleSize(4, computedTotalCost_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(5, computedSteps_);
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < regionPlans_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(regionPlans_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getRegionPlansList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision other = (org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision) obj;
+
+      boolean result = true;
+      result = result && (hasInitialFunctionCosts() == other.hasInitialFunctionCosts());
+      if (hasInitialFunctionCosts()) {
+        result = result && getInitialFunctionCosts()
+            .equals(other.getInitialFunctionCosts());
+      }
+      result = result && (hasFinalFunctionCosts() == other.hasFinalFunctionCosts());
+      if (hasFinalFunctionCosts()) {
+        result = result && getFinalFunctionCosts()
+            .equals(other.getFinalFunctionCosts());
+      }
+      result = result && (hasInitTotalCost() == other.hasInitTotalCost());
+      if (hasInitTotalCost()) {
+        result = result && (Double.doubleToLongBits(getInitTotalCost())    == Double.doubleToLongBits(other.getInitTotalCost()));
+      }
+      result = result && (hasComputedTotalCost() == other.hasComputedTotalCost());
+      if (hasComputedTotalCost()) {
+        result = result && (Double.doubleToLongBits(getComputedTotalCost())    == Double.doubleToLongBits(other.getComputedTotalCost()));
+      }
+      result = result && (hasComputedSteps() == other.hasComputedSteps());
+      if (hasComputedSteps()) {
+        result = result && (getComputedSteps()
+            == other.getComputedSteps());
+      }
+      result = result && getRegionPlansList()
+          .equals(other.getRegionPlansList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasInitialFunctionCosts()) {
+        hash = (37 * hash) + INITIAL_FUNCTION_COSTS_FIELD_NUMBER;
+        hash = (53 * hash) + getInitialFunctionCosts().hashCode();
+      }
+      if (hasFinalFunctionCosts()) {
+        hash = (37 * hash) + FINAL_FUNCTION_COSTS_FIELD_NUMBER;
+        hash = (53 * hash) + getFinalFunctionCosts().hashCode();
+      }
+      if (hasInitTotalCost()) {
+        hash = (37 * hash) + INIT_TOTAL_COST_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(
+            Double.doubleToLongBits(getInitTotalCost()));
+      }
+      if (hasComputedTotalCost()) {
+        hash = (37 * hash) + COMPUTED_TOTAL_COST_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(
+            Double.doubleToLongBits(getComputedTotalCost()));
+      }
+      if (hasComputedSteps()) {
+        hash = (37 * hash) + COMPUTED_STEPS_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getComputedSteps());
+      }
+      if (getRegionPlansCount() > 0) {
+        hash = (37 * hash) + REGION_PLANS_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionPlansList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.BalancerDecision}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecisionOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.RecentLogs.internal_static_hbase_pb_BalancerDecision_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.RecentLogs.internal_static_hbase_pb_BalancerDecision_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.class, org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        initialFunctionCosts_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        finalFunctionCosts_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        initTotalCost_ = 0D;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        computedTotalCost_ = 0D;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        computedSteps_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        regionPlans_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.RecentLogs.internal_static_hbase_pb_BalancerDecision_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision build() {
+        org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision result = new org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.initialFunctionCosts_ = initialFunctionCosts_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.finalFunctionCosts_ = finalFunctionCosts_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.initTotalCost_ = initTotalCost_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.computedTotalCost_ = computedTotalCost_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.computedSteps_ = computedSteps_;
+        if (((bitField0_ & 0x00000020) == 0x00000020)) {
+          regionPlans_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              regionPlans_);
+          bitField0_ = (bitField0_ & ~0x00000020);
+        }
+        result.regionPlans_ = regionPlans_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision.getDefaultInstance()) return this;
+        if (other.hasInitialFunctionCosts()) {
+          bitField0_ |= 0x00000001;
+          initialFunctionCosts_ = other.initialFunctionCosts_;
+          onChanged();
+        }
+        if (other.hasFinalFunctionCosts()) {
+          bitField0_ |= 0x00000002;
+          finalFunctionCosts_ = other.finalFunctionCosts_;
+          onChanged();
+        }
+        if (other.hasInitTotalCost()) {
+          setInitTotalCost(other.getInitTotalCost());
+        }
+        if (other.hasComputedTotalCost()) {
+          setComputedTotalCost(other.getComputedTotalCost());
+        }
+        if (other.hasComputedSteps()) {
+          setComputedSteps(other.getComputedSteps());
+        }
+        if (!other.regionPlans_.isEmpty()) {
+          if (regionPlans_.isEmpty()) {
+            regionPlans_ = other.regionPlans_;
+            bitField0_ = (bitField0_ & ~0x00000020);
+          } else {
+            ensureRegionPlansIsMutable();
+            regionPlans_.addAll(other.regionPlans_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasInitialFunctionCosts()) {
+          
+          return false;
+        }
+        if (!hasFinalFunctionCosts()) {
+          
+          return false;
+        }
+        if (!hasInitTotalCost()) {
+          
+          return false;
+        }
+        if (!hasComputedTotalCost()) {
+          
+          return false;
+        }
+        if (!hasComputedSteps()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.RecentLogs.BalancerDecision) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string initial_function_costs = 1;
+      private java.lang.Object initialFunctionCosts_ = "";
+      /**
+       * <code>required string initial_function_costs = 1;</code>
+       */
+      public boolean hasInitialFunctionCosts() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string initial_function_costs = 1;</code>
+       */
+      public java.lang.String getInitialFunctionCosts() {
+        java.lang.Object ref = initialFunctionCosts_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          initialFunctionCosts_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string initial_function_costs = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getInitialFunctionCostsBytes() {
+        java.lang.Object ref = initialFunctionCosts_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          initialFunctionCosts_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string initial_function_costs = 1;</code>
+       */
+      public Builder setInitialFunctionCosts(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        initialFunctionCosts_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string initial_function_costs = 1;</code>
+       */
+      public Builder clearInitialFunctionCosts() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        initialFunctionCosts_ = getDefaultInstance().getInitialFunctionCosts();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string initial_function_costs = 1;</code>
+       */
+      public Builder setInitialFunctionCostsBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        initialFunctionCosts_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required string final_function_costs = 2;
+      private java.lang.Object finalFunctionCosts_ = "";
+      /**
+       * <code>required string final_function_costs = 2;</code>
+       */
+      public boolean hasFinalFunctionCosts() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required string final_function_costs = 2;</code>
+       */
+      public java.lang.String getFinalFunctionCosts() {
+        java.lang.Object ref = finalFunctionCosts_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          finalFunctionCosts_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string final_function_costs = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getFinalFunctionCostsBytes() {
+        java.lang.Object ref = finalFunctionCosts_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          finalFunctionCosts_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string final_function_costs = 2;</code>
+       */
+      public Builder setFinalFunctionCosts(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        finalFunctionCosts_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string final_function_costs = 2;</code>
+       */
+      public Builder clearFinalFunctionCosts() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        finalFunctionCosts_ = getDefaultInstance().getFinalFunctionCosts();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string final_function_costs = 2;</code>
+       */
+      public Builder setFinalFunctionCostsBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        finalFunctionCosts_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required double init_total_cost = 3;
+      private double initTotalCost_ ;
+      /**
+       * <code>required double init_total_cost = 3;</code>
+       */
+      public boolean hasInitTotalCost() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required double init_total_cost = 3;</code>
+       */
+      public double getInitTotalCost() {
+        return initTotalCost_;
+      }
+      /**
+       * <code>required double init_total_cost = 3;</code>
+       */
+      public Builder setInitTotalCost(double value) {
+        bitField0_ |= 0x00000004;
+        initTotalCost_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required double init_total_cost = 3;</code>
+       */
+      public Builder clearInitTotalCost() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        initTotalCost_ = 0D;
+        onChanged();
+        return this;
+      }
+
+      // required double computed_total_cost = 4;
+      private double computedTotalCost_ ;
+      /**
+       * <code>required double computed_total_cost = 4;</code>
+       */
+      public boolean hasComputedTotalCost() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required double computed_total_cost = 4;</code>
+       */
+      public double getComputedTotalCost() {
+        return computedTotalCost_;
+      }
+      /**
+       * <code>required double computed_total_cost = 4;</code>
+       */
+      public Builder setComputedTotalCost(double value) {
+        bitField0_ |= 0x00000008;
+        computedTotalCost_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required double computed_total_cost = 4;</code>
+       */
+      public Builder clearComputedTotalCost() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        computedTotalCost_ = 0D;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 computed_steps = 5;
+      private long computedSteps_ ;
+      /**
+       * <code>required uint64 computed_steps = 5;</code>
+       */
+      public boolean hasComputedSteps() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>required uint64 computed_steps = 5;</code>
+       */
+      public long getComputedSteps() {
+        return computedSteps_;
+      }
+      /**
+       * <code>required uint64 computed_steps = 5;</code>
+       */
+      public Builder setComputedSteps(long value) {
+        bitField0_ |= 0x00000010;
+        computedSteps_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 computed_steps = 5;</code>
+       */
+      public Builder clearComputedSteps() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        computedSteps_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // repeated string region_plans = 6;
+      private com.google.protobuf.LazyStringList regionPlans_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureRegionPlansIsMutable() {
+        if (!((bitField0_ & 0x00000020) == 0x00000020)) {
+          regionPlans_ = new com.google.protobuf.LazyStringArrayList(regionPlans_);
+          bitField0_ |= 0x00000020;
+         }
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public java.util.List<java.lang.String>
+          getRegionPlansList() {
+        return java.util.Collections.unmodifiableList(regionPlans_);
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public int getRegionPlansCount() {
+        return regionPlans_.size();
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public java.lang.String getRegionPlans(int index) {
+        return regionPlans_.get(index);
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public com.google.protobuf.ByteString
+          getRegionPlansBytes(int index) {
+        return regionPlans_.getByteString(index);
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public Builder setRegionPlans(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureRegionPlansIsMutable();
+        regionPlans_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public Builder addRegionPlans(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureRegionPlansIsMutable();
+        regionPlans_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public Builder addAllRegionPlans(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureRegionPlansIsMutable();
+        super.addAll(values, regionPlans_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public Builder clearRegionPlans() {
+        regionPlans_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string region_plans = 6;</code>
+       */
+      public Builder addRegionPlansBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureRegionPlansIsMutable();
+        regionPlans_.add(value);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.BalancerDecision)
+    }
+
+    static {
+      defaultInstance = new BalancerDecision(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.BalancerDecision)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_BalancerDecision_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_BalancerDecision_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\020RecentLogs.proto\022\010hbase.pb\"\264\001\n\020Balance" +
+      "rDecision\022\036\n\026initial_function_costs\030\001 \002(" +
+      "\t\022\034\n\024final_function_costs\030\002 \002(\t\022\027\n\017init_" +
+      "total_cost\030\003 \002(\001\022\033\n\023computed_total_cost\030" +
+      "\004 \002(\001\022\026\n\016computed_steps\030\005 \002(\004\022\024\n\014region_" +
+      "plans\030\006 \003(\tB=\n*org.apache.hadoop.hbase.p" +
+      "rotobuf.generatedB\nRecentLogsH\001\240\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_hbase_pb_BalancerDecision_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_hbase_pb_BalancerDecision_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_BalancerDecision_descriptor,
+              new java.lang.String[] { "InitialFunctionCosts", "FinalFunctionCosts", "InitTotalCost", "ComputedTotalCost", "ComputedSteps", "RegionPlans", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/TooSlowLog.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/TooSlowLog.java
new file mode 100644
index 0000000..c29ff07
--- /dev/null
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/TooSlowLog.java
@@ -0,0 +1,2465 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: TooSlowLog.proto
+
+package org.apache.hadoop.hbase.protobuf.generated;
+
+public final class TooSlowLog {
+  private TooSlowLog() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface SlowLogPayloadOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required int64 start_time = 1;
+    /**
+     * <code>required int64 start_time = 1;</code>
+     */
+    boolean hasStartTime();
+    /**
+     * <code>required int64 start_time = 1;</code>
+     */
+    long getStartTime();
+
+    // required int32 processing_time = 2;
+    /**
+     * <code>required int32 processing_time = 2;</code>
+     */
+    boolean hasProcessingTime();
+    /**
+     * <code>required int32 processing_time = 2;</code>
+     */
+    int getProcessingTime();
+
+    // required int32 queue_time = 3;
+    /**
+     * <code>required int32 queue_time = 3;</code>
+     */
+    boolean hasQueueTime();
+    /**
+     * <code>required int32 queue_time = 3;</code>
+     */
+    int getQueueTime();
+
+    // required int64 response_size = 4;
... 7108 lines suppressed ...