apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r475894386



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -2339,9 +2340,16 @@ boolean snapshotCleanupSwitch(final boolean on, final 
boolean synchronous)
    * @param logQueryFilter filter to be used if provided (determines slow / 
large RPC logs)
    * @return online slowlog response list
    * @throws IOException if a remote or network exception occurs
+   * @deprecated since 2.4.0 and will be removed in 4.0.0.
+   *   Use {@link #getLogEntries(LogRequest, int)} instead.
    */
-  List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
-      final LogQueryFilter logQueryFilter) throws IOException;
+  default List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> 
serverNames,

Review comment:
       No 'limit' parameter here. Improvement should track the changes we are 
making to the generic API? Add a method and leave this one for backwards 
compat? 

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogQueryFilter.java
##########
@@ -22,22 +22,27 @@
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import java.util.Collections;
+import java.util.Set;
 
 /**
  * Slow/Large Log Query Filter with all filter and limit parameters
  * Used by Admin API: getSlowLogResponses
  */
-@InterfaceAudience.Private
-public class LogQueryFilter {
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class LogQueryFilter extends LogRequest {
 
   private String regionName;
   private String clientAddress;
   private String tableName;
   private String userName;
-  private int limit = 10;

Review comment:
       This change doesn't track with the javadoc for this class, which says 
"Slow/Large Log Query Filter with all filter and limit parameters". Also I 
think the comment should be updated to not mention the slow log too. 
   
   Anyway, does it make sense to have the limit in the admin API but also here? 
Maybe a filter would want to filter more? Not a big deal, just wondering.

##########
File path: hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
##########
@@ -1123,6 +1132,9 @@ service MasterService {
 
   rpc UpdateRSGroupConfig(UpdateRSGroupConfigRequest)
   returns (UpdateRSGroupConfigResponse);
+
+  rpc GetBalancerDecisions(BalancerDecisionsRequest)

Review comment:
       Remove. Use getLogEntries() for this instead. 

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1543,9 +1544,17 @@
    * @param serverNames Server names to get slowlog responses from
    * @param logQueryFilter filter to be used if provided
    * @return Online slowlog response list. The return value wrapped by a 
{@link CompletableFuture}
-   */
-  CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(final 
Set<ServerName> serverNames,
-      final LogQueryFilter logQueryFilter);
+   * @deprecated since 2.4.0 and will be removed in 4.0.0.
+   *   Use {@link #getLogEntries(LogRequest, int)} instead.
+   */
+  default CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(

Review comment:
       See above comment

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -4212,4 +4213,29 @@ private void getProcedureResult(long procId, 
CompletableFuture<Void> future, int
               (s, c, req, done) -> s.updateRSGroupConfig(c, req, done), resp 
-> null))
         ).call();
   }
+
+  private CompletableFuture<List<LogEntry>> getBalancerDecisions(

Review comment:
       Use the generic API getLogEntries() for this instead of adding one for 
balancer decisions.

##########
File path: hbase-common/src/main/resources/hbase-default.xml
##########
@@ -1994,7 +1994,7 @@ possible configurations would overwhelm and obscure the 
important.
   </property>
   <property>
     <name>hbase.namedqueue.provider.classes</name>
-    <value>org.apache.hadoop.hbase.namequeues.impl.SlowLogQueueService</value>
+    
<value>org.apache.hadoop.hbase.namequeues.impl.SlowLogQueueService,org.apache.hadoop.hbase.namequeues.impl.BalancerDecisionQueueService</value>

Review comment:
       Ok, sounds good.

##########
File path: hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
##########
@@ -693,6 +694,14 @@ message SwitchExceedThrottleQuotaResponse {
   required bool previous_exceed_throttle_quota_enabled = 1;
 }
 
+message BalancerDecisionsRequest {

Review comment:
       Remove. Use getLogEntries() instead.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -3289,4 +3295,24 @@ public UpdateRSGroupConfigResponse 
updateRSGroupConfig(RpcController controller,
     }
     return builder.build();
   }
+
+  @Override
+  public MasterProtos.BalancerDecisionsResponse 
getBalancerDecisions(RpcController controller,

Review comment:
       This should implement getLogEntries() instead, or be removed if 
getLogEntries is already implemented elsewhere. 
   
   Use getLogEntries() to retrieve balancer decisions




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to