You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/08/15 16:25:19 UTC

[GitHub] [hbase] virajjasani opened a new pull request #2261: HBASE-24528 : BalancerDecision implementation with Admin API to retrieve data from HMaster queue

virajjasani opened a new pull request #2261:
URL: https://github.com/apache/hbase/pull/2261


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r478596596



##########
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:
       Done, now both use same RPC API: `getLogEntries`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r473676689



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ *
+ * 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.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision request payload with filter attributes
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionRequest extends LogRequest {
+
+  private int limit = 250;
+
+  public int getLimit() {

Review comment:
       This is the request param coming to Admin API. Since this class is extending `LogRequest`.
   
   Detailed comment on Admin API: `limit` is included in request payload by individual use-cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-691682560


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 27s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 18s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 30s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 32s |  master passed  |
   | -0 :warning: |  patch  |  12m 13s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 26s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 19s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 30s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-client generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | -0 :warning: |  javadoc  |   2m  1s |  root generated 1 new + 46 unchanged - 0 fixed = 47 total (was 46)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 325m 17s |  root in the patch failed.  |
   |  |   | 363m 38s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0504a5f9f0d2 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe776306d0 |
   | Default Java | 1.8.0_232 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/testReport/ |
   | Max. process+thread count | 5230 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-693096819


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 25s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 22s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 47s |  master passed  |
   | -0 :warning: |  patch  |  13m  1s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 37s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-client generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | -0 :warning: |  javadoc  |   2m 18s |  root generated 1 new + 46 unchanged - 0 fixed = 47 total (was 46)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 262m 34s |  root in the patch failed.  |
   |  |   | 302m 35s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 811e3a00c47b 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 325317ff9e |
   | Default Java | 1.8.0_232 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/testReport/ |
   | Max. process+thread count | 3761 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487301224



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471696078



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,7 +30,8 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG,

Review comment:
       I wish I had caught this earlier. 
   If we have to use an Enum here, add a constructor that defines an ordinal for each type so we can maintain compatibility by instantiating by our ordinal, e.g.
   
   NamedQueueEvent(int ordinal) { ... }
   
   SLOW_LOG(1),
   BALANCE_DECISION(2),.
   ...
   
   NamedQueueEvent getByOrdinal(int ordinal)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471696078



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,7 +30,8 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG,

Review comment:
       I wish I had caught this earlier. 
   If we have to use an Enum here, add a constructor that defines an ordinal for each type so we can maintain compatibility by instantiating by our ordinal, e.g.
   
   NamedQueueEvent(int ordinal) { ... }
   
   SLOW_LOG(1),
   BALANCE_DECISION(2),.
   ...




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-689549362






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-674497260


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 27s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  1s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 57s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 22s |  master passed  |
   | +0 :ok: |  refguide  |   5m 55s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  20m 45s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 29s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 57s |  root: The patch generated 17 new + 387 unchanged - 0 fixed = 404 total (was 387)  |
   | -0 :warning: |  rubocop  |   0m 13s |  The patch generated 4 new + 364 unchanged - 0 fixed = 368 total (was 364)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   6m 20s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 33s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   8m 23s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  24m  1s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 34s |  The patch does not generate ASF License warnings.  |
   |  |   | 106m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux a0e5ba5dcb3c 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 122 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r482600399



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1682,13 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   */
+  CompletableFuture<List<LogEntry>> getLogEntries(LogRequest logRequest);

Review comment:
       Please define a limit parameter for the Admin API. I'm open to other suggestions, but by "individual use cases"  I believe you mean the RPC message classes, and those are not user facing public API. The user facing API here is the Admin API. This is the place to do this.
   
   As an alternative you can do the same thing as you did for the SlowLog API where a filter can be supplied, and one thing the filter interface lets you do is specify a limit. 
   
   I won't approve this without a limit option of some kind in the public user facing admin API. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-686629676


   Who is the consumer of the Admin API? This is where all of this is ultimately exposed. 
   
   The consumer of the Admin API is an admin level user. They import org.apache.hadoop.hbase.client.*. They use the interfaces in Admin.java. Their only interaction with a connection is Connection, ConnectionFactory, Admin, Table, etc. This is where the limit must be expressed. Users don't hand craft RPC messages. RPC internals, like particular *Request and *Response classes, are internal implementation detail, are not directly exposed to users, and are not expected to be used by users. Users should not be made to deal with them just to limit a potential firehose of ring entries.
   
   Hope that helps!


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-689549565


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  4s |  https://github.com/apache/hbase/pull/2261 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/14/console |
   | versions | git=2.17.1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471676264



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       If we have this type of API, it should be possible to provide a limit, e.g.
   
   public List getLogEntries(String name, int limit);
   
   so that a client that is memory constrained (or wants to be frugal) doesn't have to worry about invoking this and maybe getting back a list of 5000 entries or whatever, by passing in a limit of 100, or 10, or ...
   
   It would be fine to also provide a method that doesn't accept a limit, for convenience. Prerequisite: The methods are generic enough so we don't add a pair (or more!) for every type. E.g.
   
   public List getLogEntries(String name) { this(name, Integer.MAX_VALUE); }
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675593553


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 17s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  4s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 24s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 15s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 57s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 13s |  root in master failed.  |
   | -0 :warning: |  patch  |  10m 43s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 39s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  7s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  7s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 24s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 15s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 58s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 13s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 243m 40s |  root in the patch passed.  |
   |  |   | 285m  9s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 753447dee88d 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea26463a33 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/testReport/ |
   | Max. process+thread count | 4430 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani edited a comment on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani edited a comment on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675434004


   I believe we should have generic Admin API for similar use-cases. For existing slowLog use-case, I can make it default implementation pointing to generic API and provide deprecation notice (with removal in 4.0.0 release).
   
   1. Addressed concerns other than providing generic Admin API for ring buffer use-cases.
   2. Will add generic API changes as separate commit so that it becomes easier to review commit by commit.
   3. I can take up thrift changes as a follow up? (to avoid too many changes going in same PR)


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r475896191



##########
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, maybe a filter would want to filter more? Not a big deal, just wondering.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487408079



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani edited a comment on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani edited a comment on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675434004


   I believe we should have generic Admin API for similar use-cases. For existing slowLog use-case, I can make it default implementation pointing to generic API and provide deprecation notice (with removal in 4.0.0 release).
   
   1. Addressed concerns other than providing generic Admin API for ring buffer use-cases.
   2. Will add generic API changes as separate commit so that it becomes easier to review commit by commit on PR.
   3. I can take up thrift changes as a follow up? (to avoid too many changes going in same PR)


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471676264



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       If we have this type of API, it should be possible to provide a limit, e.g.
   
   public List\<LogEntry\> getLogEntries(String name, int limit);
   
   so that a client that is memory constrained (or wants to be frugal) doesn't have to worry about invoking this and maybe getting back a list of 5000 entries or whatever, by passing in a limit of 100, or 10, or ...
   
   It would be fine to also provide a method that doesn't accept a limit, for convenience. Prerequisite: The methods are generic enough so we don't add a pair (or more!) for every type. E.g.
   
   public List\<LogEntry\> getLogEntries(String name) { this(name, Integer.MAX_VALUE); }




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-678628648


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m  6s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 35s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 12s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 23s |  master passed  |
   | +0 :ok: |  refguide  |   5m 34s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  18m 42s |  master passed  |
   | -0 :warning: |  patch  |  10m 33s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 25s |  root: The patch generated 22 new + 506 unchanged - 0 fixed = 528 total (was 506)  |
   | -0 :warning: |  rubocop  |   0m 23s |  The patch generated 16 new + 591 unchanged - 5 fixed = 607 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   5m 35s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 16s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 24s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  20m  0s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 26s |  The patch does not generate ASF License warnings.  |
   |  |   |  96m 40s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 740d728e6d05 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4021f4577c |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 122 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-677783532


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 24s |  master passed  |
   | +1 :green_heart: |  compile  |   4m 16s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 52s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 29s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 47s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   1m  5s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  root in master failed.  |
   | -0 :warning: |  patch  |  12m 58s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 37s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 37s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  0s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 30s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 51s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   1m 10s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 167m 30s |  root in the patch passed.  |
   |  |   | 214m 14s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d7e7fb224cfe 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 1164531d5a |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/testReport/ |
   | Max. process+thread count | 6540 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r476354758



##########
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:
       Same as above:
   Using common API for proto seems almost impossible. One use-case of slowLog is being handled by RSRpc services whereas the other for balancerDecision is being managed by MasterRpc services. Both have different sets of Rpc implementations written and handled at different places.
   Moreover, at RPC level, we can't achieve abstraction since protobuf doesn't support `is-a` relationship among proto `messages`.

##########
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:
       This is deprecated API and by IA definitions, we can't change the signature of this without breaking minor release compatibility? That's why we made this default implementation to use new API internally.

##########
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:
       Using common API for proto seems almost impossible. One use-case of slowLog is being handled by RSRpc services whereas the other for balancerDecision is being managed by MasterRpc services. Both have different sets of Rpc implementations written and handled at different places.
   Moreover, at RPC level, we can't achieve abstraction since protobuf doesn't support `is-a` relationship among proto `messages`.

##########
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:
       This is private utility method being used by generic API on L4230:
   ```
   public CompletableFuture<List<LogEntry>> getLogEntries(LogRequest logRequest, int limit) {
   ```
   Anything specific I can mention in method name to improve readability and the fact that this is utility method?

##########
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:
       This is deprecated API and by IA definitions, we can't change the signature of this without breaking minor release compatibility? That's why we made this default implementation to use new API internally.

##########
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:
       At RPC level, we can't achieve abstraction since protobuf doesn't support `is-a` relationship among proto `messages`.

##########
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 filter payload might be used to filter more as you said, you are right. But let me update javadoc to remove `limit` from description. Also, we don't need to remove mention of slow log because this filter is slowLog specific. Parent request payload, which is generic is `LogRequest` (which is empty so far but I have kept it IS.Evolving just in case we can identify some common filter param applicable to all use-cases).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r476351625



##########
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 filter payload might be used to filter more as you said, you are right. But let me update javadoc to remove `limit` from description. Also, we don't need to remove mention of slow log because this filter is slowLog specific. Parent request payload, which is generic is `LogRequest` (which is empty so far but I have kept it IS.Evolving just in case we can identify some common filter param applicable to all use-cases).
   
   Edit: Updated the javadoc description.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-682246210


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 44s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 22s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 33s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 41s |  master passed  |
   | -0 :warning: |  patch  |  12m 20s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 22s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 31s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 39s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 263m 42s |  root in the patch failed.  |
   |  |   | 302m 35s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3acceed62648 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 047e0618d2 |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/testReport/ |
   | Max. process+thread count | 4245 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675841830


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 22s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 57s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 59s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in master failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 50s |  hbase-thrift in master failed.  |
   | -0 :warning: |  patch  |  10m 16s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 49s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 49s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 53s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 53s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 165m 20s |  root in the patch passed.  |
   |  |   | 203m 57s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 578ffe23e4c4 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 98e35842eb |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/testReport/ |
   | Max. process+thread count | 6395 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-692988092


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 11s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 55s |  master passed  |
   | +0 :ok: |  refguide  |   8m 25s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  26m 21s |  master passed  |
   | -0 :warning: |  patch  |  14m 56s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 48s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 54s |  root: The patch generated 9 new + 510 unchanged - 0 fixed = 519 total (was 510)  |
   | -0 :warning: |  rubocop  |   0m 34s |  The patch generated 15 new + 587 unchanged - 9 fixed = 602 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   7m 22s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 58s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |  10m 10s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  28m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 48s |  The patch does not generate ASF License warnings.  |
   |  |   | 128m 18s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux ee1b039253cf 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 325317ff9e |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 123 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-678638980


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 46s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 46s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 52s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in master failed.  |
   | -0 :warning: |  patch  |  10m 16s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 47s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 52s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 50s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 163m 32s |  root in the patch passed.  |
   |  |   | 200m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 45f2cc3a4bf3 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4021f4577c |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/testReport/ |
   | Max. process+thread count | 6631 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-683283809


   We have two separate RPC APIs generic for Master and RS Rpcs:
   
   **RSRpcServices:**
   ```
     public AdminLogEntry getLogEntries(RpcController controller,
         AdminLogRequest request) throws ServiceException
   ```
   
   **MasterRpcServices:**
   ```
     public MasterProtos.MasterLogEntry getLogEntries(RpcController controller,
         MasterProtos.MasterLogRequest request) throws ServiceException
   ```
   
   And, we will be using ByteString from request/response objects to RPC APIs.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani edited a comment on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani edited a comment on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-683283809


   We have two separate RPC APIs generic for Master and RS Rpcs:
   
   **RSRpcServices:**
   ```
     public AdminLogEntry getLogEntries(RpcController controller,
         AdminLogRequest request) throws ServiceException
   ```
   
   **MasterRpcServices:**
   ```
     public MasterProtos.MasterLogEntry getLogEntries(RpcController controller,
         MasterProtos.MasterLogRequest request) throws ServiceException
   ```
   
   And, we will be using ByteString from request/response objects to RPC APIs.
   
   **Admin** API for both use-cases (different ring buffers at Master as well as RS level) is generic:
   ```
     List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487301224



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471672318



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       This would mean you'd have to go back and modify any admin API related to the slow log, which is fine, and desirable (if you accept the premise of this feedback)

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -2458,4 +2458,11 @@ boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous)
    */
   void updateRSGroupConfig(String groupName, Map<String, String> configuration) throws IOException;
 
+  /**
+   * Retrieve recent balancer decision factors with region plans from HMaster in-memory ringbuffer
+   *
+   * @return list of balancer decision records
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<BalancerDecisionRecords> getBalancerDecisions() throws IOException;

Review comment:
       Just call this BalancerDecision ? Of course it's a record, or an item, or an element (choose your term for individual item in a collection) so that just adds letters for no clearer meaning.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRecords.java
##########
@@ -0,0 +1,147 @@
+/*
+ *
+ * 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.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+import org.apache.yetus.audience.InterfaceAudience;
+import java.util.List;
+
+/**
+ * History of balancer decisions taken for region movements.
+ */
+@InterfaceAudience.Private
+final public class BalancerDecisionRecords {
+
+  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(OnlineLogRecord.class, (JsonSerializer<OnlineLogRecord>)
+      (slowLogPayload, type, jsonSerializationContext) -> {

Review comment:
       Since this is being used for more than just the slowlog, this parameter _slowLogPayload_ should be renamed. (First question that comes to mind is what does slow log have to do with the balancer). Call it _logPayload_?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       Something to consider: Rather than adding new API for every ringbuffer backed type, since the ringbuffers are named, can we just have one API that retrieves records from a buffer specified by name? 
   
   E.g.
   
   public List<LogEntry> getLogEntries(String name)
   
   Then, LogEntry is a generic type capable of accepting any protobuf encoding. Then, we derive new types from LogEntry such as BalancerDecision. Have a static method in RingEntry for instantiating the subclasses based on what type is communicated by the protobuf. 
   
   If _LogEntry_ is too generic a name, consider _RingEntry_. (I'm not the best at naming, maybe someone else has a better idea...)
   
   It is a lot easier to add or remove specialized classes as these things evolve than add or remove methods from public/stable admin APIs. 

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRecords.java
##########
@@ -0,0 +1,147 @@
+/*
+ *
+ * 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.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+import org.apache.yetus.audience.InterfaceAudience;
+import java.util.List;
+
+/**
+ * History of balancer decisions taken for region movements.
+ */
+@InterfaceAudience.Private
+final public class BalancerDecisionRecords {

Review comment:
       Just BalancerDecision? 'Records' doesn't add any meaning. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r482600399



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1682,13 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   */
+  CompletableFuture<List<LogEntry>> getLogEntries(LogRequest logRequest);

Review comment:
       Please define a limit parameter for the Admin API. I'm open to other suggestions, but by "individual use cases"  I believe you mean the RPC message classes, and those are not public API.
   
   As an alternative you can do the same thing as you did for the SlowLog API where a filter can be supplied, and one thing the filter interface lets you do is specify a limit. 
   
   I won't approve this without a limit option of some kind in the public user facing admin API. 

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ *
+ * 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.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision request payload with filter attributes
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionRequest extends LogRequest {
+
+  private int limit = 250;
+
+  public int getLimit() {

Review comment:
       -1 on limits in non public RPC classes. The Admin API is the public interface here. These RPC classes are implementation detail and not the place to be doing this. 

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1682,13 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   */
+  CompletableFuture<List<LogEntry>> getLogEntries(LogRequest logRequest);

Review comment:
       In the future please do not mark conversations that are not resolved as resolved. 

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -2472,4 +2482,13 @@ boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous)
    */
   void updateRSGroupConfig(String groupName, Map<String, String> configuration) throws IOException;
 
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;

Review comment:
       
   Please define a limit parameter for the Admin API. I'm open to other suggestions, but by "individual use cases"  I believe you mean the RPC message classes, and those are not public API.
   
   As an alternative you can do the same thing as you did for the SlowLog API where a filter can be supplied, and one thing the filter interface lets you do is specify a limit.
   
   I won't approve this without a limit option of some kind in the public user facing admin API.

##########
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:
       -1 to limits on non public RPC request classes.

##########
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:
       In the future please do not mark unresolved conversations as resolved. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-680186315


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 20s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 45s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 54s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 59s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  root in master failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 48s |  hbase-thrift in master failed.  |
   | -0 :warning: |  patch  |  11m 16s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  7s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 58s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 58s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 51s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 20s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 190m 46s |  root in the patch passed.  |
   |  |   | 232m 13s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d1414d55dc18 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ebe321a99b |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/testReport/ |
   | Max. process+thread count | 6773 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471690690



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       Definitely, adding new APIs not desirable, however few reasons why I had to choose this way: 
   
   1. slowLog released in 2.3 already.
   2. slowLog is in each RS and the API has flexibility to choose serverName to get records from (filter by serverName) whereas balancer is HMaster queue meaning activeMasterStub is to be identified by client, hence no need of serverName filter.
   3. client has to identify whom to make a call - HMaster or RegionServer (or limited no of RS) based on use-case, not a big deal but maybe too much logic for client interface?
   4. genericAPI might need to include abstract request and response classes, and respective use-cases to ensure if filter is required at server side (not a big deal)
   
   For 1), I believe trunk and 2.4.0 both can have generic API end-point.
   2) and 3) seem bit of headache for single API.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r477627879



##########
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:
       Not impossible at all.
   
   getRingEntries RPC in RS RPC servcies
   
   getRingEntries RPC in master RPC services
   
   > we can't achieve abstraction since protobuf doesn't support is-a relationship among proto messages.
   
   You have an outer message type that is a class name and a byte string, and the protobuf for the inner type is serialized into the byte string while the class name field of the outer type is set to the type to use for parsing the inner type. 
   
   Otherwise you have made moot any work toward making things generic, there will be new message types and RPC methods for every ring. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r482698935



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ *
+ * 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.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision request payload with filter attributes
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionRequest extends LogRequest {
+
+  private int limit = 250;
+
+  public int getLimit() {

Review comment:
       This is public non-RPC class (Admin API request payload). It extends `LogRequest` and public Admin API endpoint is:
   ```
     List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;
   ```
   And hence, this class goes as request param to this API by extending `LogRequest`, so we anyways have this `limit` on public class for imposing all limits and filters (BalancerDecision so far doesn't have filters though, so just limit).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471701524



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       I agree, that's the trade off here. Some pain now for flexibility later, or not much pain and no flexibility now or later. New APIs for each new type. 
   
   Either decision is not wrong or right. 
   
   A benefit to API per ring type is pressure from API compat concerns and method proliferation will put a lot of pressure on new potential use cases for this mechanism, will set a really high bar. Maybe too high, hence my concerns.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471677530



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRecords.java
##########
@@ -0,0 +1,147 @@
+/*
+ *
+ * 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.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+import org.apache.yetus.audience.InterfaceAudience;
+import java.util.List;
+
+/**
+ * History of balancer decisions taken for region movements.
+ */
+@InterfaceAudience.Private
+final public class BalancerDecisionRecords {
+
+  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(OnlineLogRecord.class, (JsonSerializer<OnlineLogRecord>)
+      (slowLogPayload, type, jsonSerializationContext) -> {

Review comment:
       Can't believe this miss! This Gson serialization function was a copy paste :|

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRecords.java
##########
@@ -0,0 +1,147 @@
+/*
+ *
+ * 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.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+import org.apache.yetus.audience.InterfaceAudience;
+import java.util.List;
+
+/**
+ * History of balancer decisions taken for region movements.
+ */
+@InterfaceAudience.Private
+final public class BalancerDecisionRecords {

Review comment:
       Sure

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       Definitely, adding new APIs not desirable, however the reason why I had to choose this way: 
   
   1. slowLog released in 2.3 already.
   2. slowLog is in each RS and the API has flexibility to choose serverName to get records from (filter by serverName) whereas balancer is HMaster queue meaning activeMasterStub is to be identified by client, hence no need of serverName filter.
   3. client has to identify whom to make a call - HMaster or RegionServer (or limited no of RS) based on use-case, not a big deal but maybe too much logic for client interface?
   4. genericAPI might need to include abstract request and response classes, and respective use-cases to ensure if filter is required at server side (not a big deal)
   
   For 1), I believe trunk and 2.4.0 both can have generic API end-point.
   2) and 3) seem bit of headache for single API.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-674513635


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 41s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  1s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 29s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 13s |  root in master failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 57s |  hbase-thrift in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  4s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  4s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 20s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 15s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 43s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 58s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 14s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 240m 14s |  root in the patch passed.  |
   |  |   | 279m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9a5b74e5ccd7 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/testReport/ |
   | Max. process+thread count | 4446 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r475073514



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -4006,9 +4007,9 @@ private void getProcedureResult(long procId, CompletableFuture<Void> future, int
     }
   }
 
-  private CompletableFuture<List<OnlineLogRecord>> getSlowLogResponseFromServer(
+  private CompletableFuture<List<LogEntry>> getSlowLogResponseFromServer(

Review comment:
       This has been taken care of.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani edited a comment on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani edited a comment on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-683283809


   We have two separate RPC APIs generic for Master and RS Rpcs:
   
   **RSRpcServices:**
   ```
     public AdminLogEntry getLogEntries(RpcController controller,
         AdminLogRequest request) throws ServiceException
   ```
   
   **MasterRpcServices:**
   ```
     public MasterProtos.MasterLogEntry getLogEntries(RpcController controller,
         MasterProtos.MasterLogRequest request) throws ServiceException
   ```
   
   And, we will be using ByteString from request/response objects to RPC APIs.
   
   Admin API for both use-cases (different ring buffers at Master as well as RS level) is generic:
   ```
     List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675817866


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   3m 26s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 11s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 24s |  master passed  |
   | +0 :ok: |  refguide  |   5m 28s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  20m 58s |  master passed  |
   | -0 :warning: |  patch  |   2m 27s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 19s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 49s |  root: The patch generated 26 new + 437 unchanged - 0 fixed = 463 total (was 437)  |
   | -0 :warning: |  rubocop  |   0m 26s |  The patch generated 5 new + 594 unchanged - 2 fixed = 599 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   6m 17s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 31s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   8m 17s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  23m 57s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 34s |  The patch does not generate ASF License warnings.  |
   |  |   | 108m 37s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux fa91c148e4a2 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 98e35842eb |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 122 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r482700776



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -2472,4 +2482,13 @@ boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous)
    */
   void updateRSGroupConfig(String groupName, Map<String, String> configuration) throws IOException;
 
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;

Review comment:
       This is our Admin endpoint:
   ```
   List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;
   ```
   Now when client uses this to get slowLog response, they can provide `limit` in specific payload which extends `LogRequest` and send it over, similarly they can provide `limit` in BalancerDecisionRequest which extends `LogRequest` and get the response accordingly, both request payloads that extend `LogRequest` are anyways public interfaces for clients. Now, client would rather provide `limit` in request payload than as separate argument on this Admin endpoint (method signature). Does that sound good?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ *
+ * 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.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision request payload with filter attributes
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionRequest extends LogRequest {
+
+  private int limit = 250;
+
+  public int getLimit() {

Review comment:
       This is public non-RPC class. It extends `LogRequest` and public Admin API endpoint is:
   ```
     List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;
   ```
   And hence, this class goes as request param to this API by extending `LogRequest`, so we anyways have this `limit` on public class for imposing all limits and filters (BalancerDecision so far doesn't have filters though, so just limit).

##########
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:
       > Limits by way of filter will be fine if you want to do it this way.
   
   This is exactly what I am trying to do: 
   1. For SlowLog use-case, this class exposes `limit`:
       `public class LogQueryFilter extends LogRequest`
   2. For BalancerDecision use-case, this class exposes `limit`:
       `public class BalancerDecisionRequest extends LogRequest`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675631929


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 23s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m  5s |  master passed  |
   | -0 :warning: |  patch  |  12m 15s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 55s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 58s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 49s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m 10s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 320m  9s |  root in the patch failed.  |
   |  |   | 361m 48s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 35e267241f42 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea26463a33 |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/testReport/ |
   | Max. process+thread count | 2859 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471690236



##########
File path: hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
##########
@@ -1277,4 +1278,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     throw new NotImplementedException("updateRSGroupConfig not supported in ThriftAdmin");
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       If we had one admin API for master and regionservers that returned protobuf encoded entries, then the thrift API could support it. First cut could just pass through the encoded PB. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r478595968



##########
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:
       Done

##########
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:
       Brought back `limit` to request payload.

##########
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:
       Done




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-680097112


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 24s |  master passed  |
   | +0 :ok: |  refguide  |   5m 16s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  18m 36s |  master passed  |
   | -0 :warning: |  patch  |   2m 21s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 27s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m  8s |  root: The patch generated 22 new + 507 unchanged - 0 fixed = 529 total (was 507)  |
   | -0 :warning: |  rubocop  |   0m 24s |  The patch generated 16 new + 591 unchanged - 5 fixed = 607 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   5m 19s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 53s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 38s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  20m 26s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 36s |  The patch does not generate ASF License warnings.  |
   |  |   |  93m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 696c3dc9fda1 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ebe321a99b |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 137 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani edited a comment on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani edited a comment on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-683283809


   We have two separate RPC APIs generic for Master and RS Rpcs:
   
   **RSRpcServices:**
   ```
     public AdminProtos.AdminLogEntry getLogEntries(RpcController controller,
         AdminProtos.AdminLogRequest request) throws ServiceException
   ```
   
   **MasterRpcServices:**
   ```
     public MasterProtos.MasterLogEntry getLogEntries(RpcController controller,
         MasterProtos.MasterLogRequest request) throws ServiceException
   ```
   
   And, we will be using ByteString from request/response objects to RPC APIs.
   
   **Admin** API for both type of ring-buffer use-cases (Master and RS) is generic:
   ```
     List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r477627879



##########
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:
       Not impossible at all.
   
   getRingEntries RPC in RS RPC servcies
   
   getRingEntries RPC in master RPC services
   
   > we can't achieve abstraction since protobuf doesn't support is-a relationship among proto messages.
   
   You have an outer message type that is a class name and a byte string, and the protobuf for the inner type is serialized into the byte string while the class name is set to the type you need to use to deserialize. 
   
   Otherwise you have made moot any work toward making things generic, there will be new message types and RPC methods for every ring. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-689730734


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 11s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 56s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 25s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 49s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 28s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   1m  4s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in master failed.  |
   | -0 :warning: |  patch  |  12m 36s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 52s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 23s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 49s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 28s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   1m  2s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 259m 49s |  root in the patch passed.  |
   |  |   | 304m 19s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c0b90445b623 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2e96a5b2d3 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/testReport/ |
   | Max. process+thread count | 4484 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-683767501


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 23s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 40s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  4s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 19s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 14s |  root in master failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 16s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 57s |  hbase-thrift in master failed.  |
   | -0 :warning: |  patch  |  11m 31s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  4s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  4s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 18s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 16s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 59s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 13s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 235m 50s |  root in the patch passed.  |
   |  |   | 275m 56s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 1acd909374c3 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 25fcc40f6a |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/testReport/ |
   | Max. process+thread count | 4389 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471676264



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       If we have this type of API, it should be possible to provide a limit, e.g.
   
   public List getLogEntries(String name, int limit);
   
   so that a client that is memory constrained (or wants to be frugal) doesn't have to worry about invoking this and maybe getting back a list of 5000 entries or whatever, by passing in a limit of 100, or 10, or ...




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487408079



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-691654076


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 56s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 29s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 25s |  master passed  |
   | +0 :ok: |  refguide  |   5m 40s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  18m 50s |  master passed  |
   | -0 :warning: |  patch  |  10m 35s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 23s |  root: The patch generated 8 new + 511 unchanged - 0 fixed = 519 total (was 511)  |
   | -0 :warning: |  rubocop  |   0m 24s |  The patch generated 15 new + 587 unchanged - 9 fixed = 602 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   5m 37s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 22s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 35s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  20m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 27s |  The patch does not generate ASF License warnings.  |
   |  |   |  97m 46s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 2578ca0f47cf 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe776306d0 |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 122 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-691674833


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 47s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 20s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 29s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 46s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   1m  1s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 14s |  root in master failed.  |
   | -0 :warning: |  patch  |  12m 29s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 20s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 20s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 45s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 28s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   1m  2s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 255m 26s |  root in the patch passed.  |
   |  |   | 298m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d1256e8b6594 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe776306d0 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/testReport/ |
   | Max. process+thread count | 4440 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/18/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471706020



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       Ok, but if you dont' accept the majority of this feedback, which would be fine, we at least need the limit parameter. And also add one for the slow log if there isn't one there.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-674454611


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   7m 58s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 17s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 40s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  4s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m  1s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  7s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 48s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 48s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 20s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 52s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 322m 30s |  root in the patch failed.  |
   |  |   | 370m 14s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux af906ce8d83f 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/testReport/ |
   | Max. process+thread count | 3021 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471701524



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       I agree, that's the trade off here. Some pain now for flexibility later, or not much pain and no flexibility now or later. 
   
   New APIs for each new type is not a wrong choice per se. A benefit to API per ring type is pressure from API compat concerns and method proliferation will put a lot of pressure on new potential use cases for this mechanism, will set a really high bar. Maybe too high, hence my concerns.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675491925


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 24s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 28s |  master passed  |
   | +0 :ok: |  refguide  |   5m 55s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  21m  6s |  master passed  |
   | -0 :warning: |  patch  |  11m 36s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 16s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 51s |  root: The patch generated 19 new + 436 unchanged - 1 fixed = 455 total (was 437)  |
   | -0 :warning: |  rubocop  |   0m 24s |  The patch generated 5 new + 566 unchanged - 2 fixed = 571 total (was 568)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   6m 15s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 26s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   8m 22s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  23m 59s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 31s |  The patch does not generate ASF License warnings.  |
   |  |   | 106m 57s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 144e1f1e63c8 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea26463a33 |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 123 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r482601358



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -2472,4 +2482,13 @@ boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous)
    */
   void updateRSGroupConfig(String groupName, Map<String, String> configuration) throws IOException;
 
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;

Review comment:
       Please define a limit parameter for the Admin API. I'm open to other suggestions, but by "individual use cases"  I believe you mean the RPC message classes, and those are not the user facing API, they are an implementation detail. 
   
   As an alternative you can do the same thing as you did for the SlowLog API where a filter can be supplied, and one thing the filter interface lets you do is specify a limit.
   
   I won't approve this without a limit option of some kind in the public user facing admin API.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-686667507


   I am considering to use the same filter based approach I did for the slow log method in Admin.java, where a `LogEntryFilter` can express a limit.
   `LogEntryFilter` expresses "limit" for slow/large log and similarly `BalancerDecisionRequest` expresses "limit" for balancerDecision.
   Both have built-in "limit" values and user can change it, but without constructing LogEntryFilter/BalancerDecisionRequest objects, user can not even invoke Admin.getLogEntries API. So either default or user provided "limit" will definitely reach to Admin.getLogEntries().
   
   ```
   1.
   public class BalancerDecisionRequest extends LogRequest {
   ...
     private int limit = 250;
   ...
   ...
   
   2.
   public class LogQueryFilter extends LogRequest {
   ...
   ...
     private int limit = 10;
   ...
   ...
   ```
   Admin.getLogEntries() takes one argument, which is object of either of above classes, hence we are already passing `limit` with this object.
   
   Here is how shell invokes call to Admin.getLogEntries:
   ```
       def get_balancer_decisions(args)
         balancer_decisions_req = org.apache.hadoop.hbase.client.BalancerDecisionRequest.new
         if args.key? 'LIMIT'   
           limit = args['LIMIT']
           balancer_decisions_req.setLimit(limit)   # if user provided limit in shell argument, use it, else default will be used.
         end
         balancer_decisions_responses = @admin.getLogEntries(balancer_decisions_req)
   ...
   ...
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471710161



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       I agree method proliferation is a concern, something to consider and once we come to a decision, there is no way back it seems.
   For limit param, we have limit in slowLog use-case because client is calling every single RS. Here, since it's just HMaster, the real limit should be kept in HMaster's ring buffer size. However, memory constrains for client makes limiting useful even for single server.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r473435176



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -4006,9 +4007,9 @@ private void getProcedureResult(long procId, CompletableFuture<Void> future, int
     }
   }
 
-  private CompletableFuture<List<OnlineLogRecord>> getSlowLogResponseFromServer(
+  private CompletableFuture<List<LogEntry>> getSlowLogResponseFromServer(

Review comment:
       Comment on admin interface has implications here. (_This needs an optional parameter to allow constrained clients to limit the size of the returned list of LogEntry._)

##########
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 BalancerDecisionRequest {
+  optional uint32 limit = 5 [default = 250];

Review comment:
       This default seems low, and should be an interface method param option and a class file constant, not a proto constant, IMHO.
   
   The limit should be generic to the getList() interface and not specific to the log entry type. 
   
   The default should align with the server side default ring buffer size.  The idea is the client will get all of the entries by default, but can ask for fewer depending on use case or constraints. 

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1051,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException {

Review comment:
       Comment on admin interface has implications here. (_This needs an optional parameter to allow constrained clients to limit the size of the returned list of LogEntry._)

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ *
+ * 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.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision request payload with filter attributes
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionRequest extends LogRequest {
+
+  private int limit = 250;
+
+  public int getLimit() {

Review comment:
       Huh, limit supported here but not in admin API. What am I missing?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
##########
@@ -222,6 +226,14 @@ public synchronized void setConf(Configuration conf) {
 
     curFunctionCosts= new Double[costFunctions.size()];
     tempFunctionCosts= new Double[costFunctions.size()];
+
+    boolean isBalancerDecisionEnabled = getConf()

Review comment:
       For this and other variables controlling whether or not we record decisions or events, some variant of "record" could be in the variable name to make this plain. This is "record" as verb, not as noun. Consider isBalancerDecisionRecording or isBalancerDecisionRecordingEnabled.  We don't need to call the elements FooBarRecords, but when deciding to put them in a ring buffer or not we could be said to be "recording" FooBar entries or not. Helps with clarity I think but just mentioned for your consideration. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java
##########
@@ -40,7 +40,7 @@
 
   public RpcLogDetails(RpcCall rpcCall, Message param, String clientAddress, long responseSize,
       String className, boolean isSlowLog, boolean isLargeLog) {
-    super(NamedQueueEvent.SLOW_LOG);
+    super(0);

Review comment:
       The idea of using ordinals is we can encode them into message types and be more gentle at runtime if missing one (as opposed to what Java would do by default, a linkage error I think). However for code I think enums provide documentation of intent and magic constants should be avoided. Or, at least, use a static integer constant named "SLOW_LOG"...

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
##########
@@ -211,6 +210,11 @@ public String explainFailure() throws Exception {
       LOG.info("====== Move meta done ======");
       Thread.sleep(5000);
     }
+    BalancerDecisionRequest balancerDecisionRequest = new BalancerDecisionRequest();
+    balancerDecisionRequest.setLimit(2);

Review comment:
       Per other comments limit should be lifted into admin API not set per request type. 

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -2472,4 +2482,13 @@ boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous)
    */
   void updateRSGroupConfig(String groupName, Map<String, String> configuration) throws IOException;
 
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;

Review comment:
       This needs an optional parameter to allow constrained clients to limit the size of the returned list of LogEntry. 

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -2339,9 +2342,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.

Review comment:
       As I mentioned in my earlier comments, I recognize it's unfortunate to have an almost immediate deprecation of this but it's better to move to a generic API now then bake in more special case APIs per ring buffer element type. 
   
   So +1 to this.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -64,6 +66,7 @@
 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;

Review comment:
       Is there a nonshaded collections util available? Might be an issue when backporting. Just a nit.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1682,13 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   */
+  CompletableFuture<List<LogEntry>> getLogEntries(LogRequest logRequest);

Review comment:
       Comment on admin interface has implications here. (_This needs an optional parameter to allow constrained clients to limit the size of the returned list of LogEntry._)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,38 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),
+    BALANCE_DECISION(1);
+
+    private final int value;
+
+    NamedQueueEvent(int i) {
+      this.value = i;
+    }
+
+    public static NamedQueueEvent getEventByOrdinal(int value){
+      switch (value) {
+        case 0: {
+          return SLOW_LOG;
+        }
+        case 1: {
+          return BALANCE_DECISION;
+        }
+        default: {
+          throw new IllegalArgumentException("Failed to retrieve NamedQueue Event");

Review comment:
       "NamedQueue event with ordinal %d not defined"

##########
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:
       Just a thought:
   Can we leave this blank by default? And then do a runtime scan of classes implementing the log queue interface if this param is blank, to include them all by default. Over the long run this would be less prone to error.

##########
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 BalancerDecisionRequest {

Review comment:
       Probably should be BalancerDecisionsRequest, but just a nit (usually there will be more than one decision, right?)

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
##########
@@ -1605,6 +1605,10 @@
    */
   public static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
 
+  public static final String BALANCER_DECISION_BUFFER_ENABLED =

Review comment:
       Should probably be defined in a base load balancer class or interface instead. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/BalancerDecisionDetails.java
##########
@@ -0,0 +1,49 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.namequeues;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.client.BalancerDecision;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision details that would be passed on to ring buffer for history
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionDetails extends NamedQueuePayload {
+
+  private final BalancerDecision balancerDecision;
+
+  public BalancerDecisionDetails(BalancerDecision balancerDecision) {
+    super(1);
+    this.balancerDecision = balancerDecision;
+  }
+
+  public BalancerDecision getBalancerDecisionRecords() {

Review comment:
       Could just be getBalancerDecisions()

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/BalancerDecisionDetails.java
##########
@@ -0,0 +1,49 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.namequeues;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.client.BalancerDecision;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision details that would be passed on to ring buffer for history
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionDetails extends NamedQueuePayload {
+
+  private final BalancerDecision balancerDecision;
+
+  public BalancerDecisionDetails(BalancerDecision balancerDecision) {
+    super(1);
+    this.balancerDecision = balancerDecision;
+  }
+
+  public BalancerDecision getBalancerDecisionRecords() {
+    return balancerDecision;
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this)
+      .append("balancerDecisionRecords", balancerDecision)

Review comment:
       Could be just "balancerDecisions"

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
##########
@@ -234,26 +246,21 @@ private void loadCustomCostFunctions(Configuration conf) {
       return;
     }
 
-    costFunctions.addAll(Arrays.stream(functionsNames)
-            .map(c -> {
-              Class<? extends CostFunction> klass = null;
-              try {
-                klass = (Class<? extends CostFunction>) Class.forName(c);
-              } catch (ClassNotFoundException e) {
-                LOG.warn("Cannot load class " + c + "': " + e.getMessage());
-              }
-              if (null == klass) {
-                return null;
-              }
-
-              CostFunction reflected = ReflectionUtils.newInstance(klass, conf);
-              LOG.info("Successfully loaded custom CostFunction '" +
-                      reflected.getClass().getSimpleName() + "'");
-
-              return reflected;
-            })
-            .filter(Objects::nonNull)
-            .collect(Collectors.toList()));
+    costFunctions.addAll(Arrays.stream(functionsNames).map(c -> {

Review comment:
       I hope this won't be too difficult to port back to Java 7 for branch-1. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
##########
@@ -3966,7 +3966,7 @@ public SlowLogResponses getSlowLogResponses(final RpcController controller,
     }
     List<SlowLogPayload> slowLogPayloads;
     NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
-    namedQueueGetRequest.setNamedQueueEvent(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
+    namedQueueGetRequest.setNamedQueueEvent(0);

Review comment:
       See related comment.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java
##########
@@ -201,7 +201,7 @@ public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request)
       slowLogPayloads = getSlowLogPayloads(slowLogResponseRequest);
     }
     NamedQueueGetResponse response = new NamedQueueGetResponse();
-    response.setNamedQueueEvent(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
+    response.setNamedQueueEvent(0);

Review comment:
       See related comment. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/request/NamedQueueGetRequest.java
##########
@@ -46,20 +48,29 @@ public void setSlowLogResponseRequest(
     this.slowLogResponseRequest = slowLogResponseRequest;
   }
 
+  public MasterProtos.BalancerDecisionRequest getBalancerDecisionRequest() {
+    return balancerDecisionRequest;
+  }
+
+  public void setBalancerDecisionRequest(
+      MasterProtos.BalancerDecisionRequest balancerDecisionRequest) {
+    this.balancerDecisionRequest = balancerDecisionRequest;
+  }
+
   public NamedQueuePayload.NamedQueueEvent getNamedQueueEvent() {
     return namedQueueEvent;
   }
 
-  public void setNamedQueueEvent(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
-    this.namedQueueEvent = namedQueueEvent;
+  public void setNamedQueueEvent(int eventOrdinal) {
+    this.namedQueueEvent = NamedQueuePayload.NamedQueueEvent.getEventByOrdinal(eventOrdinal);
   }
 
   @Override
   public String toString() {
     return new ToStringBuilder(this)
       .append("slowLogResponseRequest", slowLogResponseRequest)
       .append("namedQueueEvent", namedQueueEvent)
+      .append("balancerDecisionRequest", balancerDecisionRequest)

Review comment:
       "balancerDecisionsRequest"




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r478595968



##########
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:
       Done




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675434004


   I believe we should have generic Admin API for similar use-cases. For existing slowLog use-case, I can make it default implementation pointing to generic API and provide deprecation notice (with removal in 4.0.0 release).
   
   1. Addressed concerns other than providing generic Admin API for ring buffer use-cases.
   2. Let me take that up now and add as separate commit so that it becomes easier to review commit by commit.
   3. I can take up thrift changes as a follow up? (to avoid too many changes going in same commit)


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-674438974


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m  1s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 47s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in master failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 51s |  hbase-thrift in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 41s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 50s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 137m 23s |  root in the patch failed.  |
   |  |   | 176m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux f14e36d274da 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/testReport/ |
   | Max. process+thread count | 4628 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-693090757


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 39s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  4s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 20s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 15s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 58s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 14s |  root in master failed.  |
   | -0 :warning: |  patch  |  11m 38s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  5s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 21s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 15s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 57s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 14s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 238m 42s |  root in the patch passed.  |
   |  |   | 280m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2c4daf389985 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 325317ff9e |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/testReport/ |
   | Max. process+thread count | 4421 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/19/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani closed pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani closed pull request #2261:
URL: https://github.com/apache/hbase/pull/2261


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API to retrieve data

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-674432727


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   7m 58s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 28s |  master passed  |
   | +0 :ok: |  refguide  |   5m 33s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  20m 18s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 17s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 44s |  root: The patch generated 17 new + 387 unchanged - 0 fixed = 404 total (was 387)  |
   | -0 :warning: |  rubocop  |   0m 14s |  The patch generated 4 new + 364 unchanged - 0 fixed = 368 total (was 364)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   6m  9s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 22s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 52s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  23m 22s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 24s |  The patch does not generate ASF License warnings.  |
   |  |   | 110m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux a6136c5885a1 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 122 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-678602243


   Ok, let me take care of this today.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r478097593



##########
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:
       > If the LogQueryFilter can impose a limit, then that would work too.
   
   You mean to say the previous approach where individual request payloads had limit themselves (LogQueryFilter and BalancerDecisionRequest both have limit as a param) was good enough? Or you are suggesting to just let LogQueryFilter (which represents slowLog use-case) have limit param?
   I somehow still feel, instead of exposing `limit` separately on Admin API, we should let individual request payloads have `limit` (like previous commit)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-678651779


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 32s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 16s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 36s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 29s |  master passed  |
   | -0 :warning: |  patch  |  11m 11s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 17s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 20s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 20s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 33s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 34s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 284m 27s |  root in the patch failed.  |
   |  |   | 319m 57s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 312229e7f31d 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4021f4577c |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/testReport/ |
   | Max. process+thread count | 4847 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487408079



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same as above, last line of method after try/catch:
   ```
       throw new RuntimeException("Invalid response from server");
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       I understand your concern but this is anyways server side code and at server side, we need to add new ring buffer implementor that implements `NamedQueueService`.
   And one of the methods need to be implemented is:
   ```
     /**
      * Retrieve event type for NamedQueueService implementation.
      *
      * @return {@link NamedQueuePayload.NamedQueueEvent}
      */
     NamedQueuePayload.NamedQueueEvent getEvent();
   ```
   Which is used by main LogEventHandler to maintain map of Event -> Implementor logic. I believe creating new enum entry with ordinal should be good enough for server side implementation.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is over:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Although specialized RPC does not exist anymore, we would want client to encode use-case specific request (BalancerDecisionsRequest) in bytes to generic RPC and also retrieve response and then decode bytes into use-case specific response (BalancerDecisionsResponse, which wraps `BalancerDecision` proto).
   
   Again, this decision comes for relatively easy-to-understand code. What do we need to do for a new ring buffer use-case?
   1. Define request and response message. Which will be encoded in bytes and sent to generic RPC API `getLogEntries`  (BalancerDecisionsRequest and BalancerDecisionsResponse in this case)
   2. Define message for use-case specific payload that we want to return to end user. (message BalancerDecision in this case)
   3. Add parsing logic in ProtobufUtil.
   For our use-cases:
   ```
   if (logClassName.contains("SlowLogResponses")) {
   ```
   and 
   ```
   if (logClassName.contains("BalancerDecisionsResponse")) {
   ```
   
   We don't need a new RPC or Admin API, but good to have new request/response message which can be encoded within generic `LogRequest` and `LogResponse` and the relevant parsing logic becomes easy to grasp.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Dang, it doesn't look good now. Context was different earlier. Let me keep it `log_message`, it's payload now.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       Yeah trivial improvement. Let's keep it if you are fine?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-691510632






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-683820139


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 28s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 18s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 31s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 37s |  master passed  |
   | -0 :warning: |  patch  |  12m 19s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 25s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 19s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 35s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 32s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 325m 15s |  root in the patch failed.  |
   |  |   | 363m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 104ebe445641 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 25fcc40f6a |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/testReport/ |
   | Max. process+thread count | 5758 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-691510632






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-691510632






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487301224



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471671371



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       Something to consider: Rather than adding new API for every ringbuffer backed type, since the ringbuffers are named, can we just have one API that retrieves records from a buffer specified by name? 
   
   E.g.
   
   public List<LogEntry> getLogEntries(String name)
   
   Then, LogEntry is a generic type capable of accepting any protobuf encoding. Then, we derive new types from LogEntry such as BalancerDecision. Have a static method in LogEntry for instantiating the subclasses by reflection based on what type is communicated by the protobuf. 
   
   If _LogEntry_ is too generic a name, consider _RingEntry_. (I'm not the best at naming, maybe someone else has a better idea...)
   
   It is a lot easier to add or remove specialized classes as these things evolve than add or remove methods from public/stable admin APIs. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-682173905


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 44s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m  1s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 50s |  master passed  |
   | +0 :ok: |  refguide  |   6m 58s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  25m 44s |  master passed  |
   | -0 :warning: |  patch  |   2m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 27s |  root: The patch generated 25 new + 514 unchanged - 0 fixed = 539 total (was 514)  |
   | -0 :warning: |  rubocop  |   0m 30s |  The patch generated 5 new + 594 unchanged - 2 fixed = 599 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   6m 11s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 55s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 49s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  21m 47s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 49s |  The patch does not generate ASF License warnings.  |
   |  |   | 108m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux f46e25181ff3 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 047e0618d2 |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 138 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r482602133



##########
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:
       Limits by way of filter will be fine if you want to do it this way. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-687268031


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 29s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 11s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 25s |  master passed  |
   | +0 :ok: |  refguide  |   5m 32s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  20m 58s |  master passed  |
   | -0 :warning: |  patch  |   2m 22s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 38s |  root: The patch generated 4 new + 512 unchanged - 0 fixed = 516 total (was 512)  |
   | -0 :warning: |  rubocop  |   0m 27s |  The patch generated 16 new + 587 unchanged - 9 fixed = 603 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   6m 32s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 42s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   8m 20s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  25m  7s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 30s |  The patch does not generate ASF License warnings.  |
   |  |   | 108m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 2cdacd09d851 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 1e8db480b3 |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 122 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r473904758



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1051,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException {

Review comment:
       Same as above, `limit` is included in request payload by individual use-cases.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ *
+ * 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.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision request payload with filter attributes
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionRequest extends LogRequest {
+
+  private int limit = 250;
+
+  public int getLimit() {

Review comment:
       This is the request payload coming to Admin API. Since this class is extending `LogRequest`.
   
   Detailed comment on Admin API: `limit` is included in request payload by individual use-cases.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
##########
@@ -222,6 +226,14 @@ public synchronized void setConf(Configuration conf) {
 
     curFunctionCosts= new Double[costFunctions.size()];
     tempFunctionCosts= new Double[costFunctions.size()];
+
+    boolean isBalancerDecisionEnabled = getConf()

Review comment:
       Definitely, with verb it looks 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 BalancerDecisionRequest {

Review comment:
       Oh I see, this is just request payload we are sending over to RPC endpoint. But yeah, it could be `Decisions`

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -2472,4 +2482,13 @@ boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous)
    */
   void updateRSGroupConfig(String groupName, Map<String, String> configuration) throws IOException;
 
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;

Review comment:
       We can consider `limit` as also one of the request attribute right? Hence, we are leaving upto individual use-cases to provide limit and both slowLog and balancerDecision do have `limit` provided as part of their request payload and both use-cases consider limit:
   (All references are for 1. BalancerDecision followed by 2. SlowLog)
   ```
   1.
   public class BalancerDecisionRequest extends LogRequest {
   
     private int limit = 250;
   
     public int getLimit() {
   ...
   ...
   
   2.
   public class LogQueryFilter extends LogRequest {
   ...
   ...
     private int limit = 10;
   ...
   ...
   
   ```
   
   And, here on the server side, we consider limiting records for both use-cases:
   ```
   1.
   public class BalancerDecisionQueueService implements NamedQueueService {
   ...
   ...
     @Override
     public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
   ...
   ...
       int limit = Math.min(request.getBalancerDecisionRequest().getLimit(), balancerDecisions.size());
       // filter limit if provided
       balancerDecisions = balancerDecisions.subList(0, limit);
   ...
   ...
   
   2.
   public class LogHandlerUtils {
   ...
   ...
     public static List<TooSlowLog.SlowLogPayload> getFilteredLogs(
         AdminProtos.SlowLogResponseRequest request, List<TooSlowLog.SlowLogPayload> logPayloadList) {
   ...
   ...
       int limit = Math.min(request.getLimit(), logPayloadList.size());
       return logPayloadList.subList(0, limit);
     }
   
   ```
   Also, this is where we convert Admin request payload to Protobuf request payload before passing on to servers over network call:
   ```
   1.
     private CompletableFuture<List<LogEntry>> getBalancerDecisions(
         BalancerDecisionRequest balancerDecisionRequest) {
       return this.<List<LogEntry>>newMasterCaller()
         .action((controller, stub) ->
           this.call(controller, stub,
             MasterProtos.BalancerDecisionRequest.newBuilder()
   /** => **/            .setLimit(balancerDecisionRequest.getLimit()).build(),
             MasterService.Interface::getBalancerDecisions, ProtobufUtil::toBalancerDecisionResponse))
         .call();
     }
   
   2.
   public final class RequestConverter {
   ...
   ...
     public static SlowLogResponseRequest buildSlowLogResponseRequest(
         final LogQueryFilter logQueryFilter) {
   ...
   ...
       return builder.setLimit(logQueryFilter.getLimit()).build();
     }
   
   ```

##########
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:
       Hmm, well keeping it blank by default is not a bad choice but Java's way of identifying what all classes are implementing an interface seems bit dirty to me :(  (unless you are aware of better reflection libraries which can nail this in better way)
   In previous PR https://github.com/apache/hbase/pull/2052 , I discussed this with Wellington also. Initially, I thought yeah, let's find all implementors run time, should not be a big deal, but I could not find one way simple enough to not ruin our code, and hence ended up doing this. Everytime we add new namedQueue implmentation, we can update `hbase-default` and it's functionalities will be taken care of.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
##########
@@ -234,26 +246,21 @@ private void loadCustomCostFunctions(Configuration conf) {
       return;
     }
 
-    costFunctions.addAll(Arrays.stream(functionsNames)
-            .map(c -> {
-              Class<? extends CostFunction> klass = null;
-              try {
-                klass = (Class<? extends CostFunction>) Class.forName(c);
-              } catch (ClassNotFoundException e) {
-                LOG.warn("Cannot load class " + c + "': " + e.getMessage());
-              }
-              if (null == klass) {
-                return null;
-              }
-
-              CostFunction reflected = ReflectionUtils.newInstance(klass, conf);
-              LOG.info("Successfully loaded custom CostFunction '" +
-                      reflected.getClass().getSimpleName() + "'");
-
-              return reflected;
-            })
-            .filter(Objects::nonNull)
-            .collect(Collectors.toList()));
+    costFunctions.addAll(Arrays.stream(functionsNames).map(c -> {

Review comment:
       That's true, nothing to worry!

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1682,13 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * Retrieve recent online records from HMaster / RegionServers.
+   * Examples include slow/large RPC logs, balancer decisions by master.
+   *
+   * @param logRequest request payload with possible filters
+   * @return Log entries representing online records from servers
+   */
+  CompletableFuture<List<LogEntry>> getLogEntries(LogRequest logRequest);

Review comment:
       Same as above, `limit` is included in request payload by individual use-cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r477627879



##########
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:
       Not impossible at all.
   
   getRingEntries RPC in RS RPC servcies
   
   getRingEntries RPC in master RPC services
   
   > we can't achieve abstraction since protobuf doesn't support is-a relationship among proto messages.
   
   You have an outer message type that is a class name and a byte string, and the inner protobuf is stuffed into the byte string while the class name is set to the type you need to use to deserialize. 
   
   Otherwise you have made moot any work toward making things generic, there will be new message types and RPC methods for every ring. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487310800



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       Just to be clear above is a suggestion, feel free to ignore it. ^^




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-686694208


   I was so focused on limit that I didn't see the other issues here too, although this idea was reflected in other feedback. 
   
   The Admin API should be fully generic:
   
       List<LogEntry> getLogEntries(ServerName server, LogName log, int (or long) limit) throws IOException;
   
   One method to ask any server, either master or regionserver, for any log of any name (with a limit 😃 ) Does that help clarify things? This is why I made the other comment about the RPC level changes. No special case RPC to the master for BalancerLog specifically. One generic facility to ask any server for the entries of any named ring.
   
   So going forward we can add a ring anywhere for any reason and not need to change Public/Stable API, nor define new RPC methods for same. 
   
   Now, we may want to add *additional* convenience classes in org.apache.hadoop.hbase.client.* , but the client-server API would be unaffected by changes to this sugar.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-677712834


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 25s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m  4s |  master passed  |
   | +0 :ok: |  refguide  |   4m 59s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  17m 10s |  master passed  |
   | -0 :warning: |  patch  |   9m 30s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m  5s |  root: The patch generated 22 new + 437 unchanged - 0 fixed = 459 total (was 437)  |
   | -0 :warning: |  rubocop  |   0m 21s |  The patch generated 5 new + 594 unchanged - 2 fixed = 599 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   5m  4s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  11m  8s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 14s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  18m 27s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 42s |  The patch does not generate ASF License warnings.  |
   |  |   |  86m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 65b880e7f412 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 1164531d5a |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 137 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-675607377


   @apurtell 
   Update so far:
   Addressed all concerns including generic Admin API for future use-cases.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471710161



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       I agree method proliferation is a concern, something to consider and once we come to a decision, there is no way back it seems.
   For limit param, we have limit in slowLog use-case because client is calling every single RS. Here, since it's just HMaster, the real limit should be kept in HMaster's ring buffer size. However, memory constraint makes limiting useful even for single server.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471704886



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       So just to be clear, please take some time to think about this, but it is not a request for change per se.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-683695690


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 37s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 57s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 28s |  master passed  |
   | +0 :ok: |  refguide  |   5m 49s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  19m 57s |  master passed  |
   | -0 :warning: |  patch  |  11m 16s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 51s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   2m 22s |  the patch passed  |
   | -0 :warning: |  rubocop  |   0m 24s |  The patch generated 5 new + 594 unchanged - 2 fixed = 599 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   5m 38s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  15m  8s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   9m  7s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  24m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 41s |  The patch does not generate ASF License warnings.  |
   |  |   | 109m  4s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 0e5def4b3d65 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 25fcc40f6a |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-general-check/output/branch-site/book.html |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 123 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/12/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani edited a comment on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani edited a comment on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-683283809


   We have two separate RPC APIs generic for Master and RS Rpcs:
   
   **RSRpcServices:**
   ```
     public AdminProtos.AdminLogEntry getLogEntries(RpcController controller,
         AdminProtos.AdminLogRequest request) throws ServiceException
   ```
   
   **MasterRpcServices:**
   ```
     public MasterProtos.MasterLogEntry getLogEntries(RpcController controller,
         MasterProtos.MasterLogRequest request) throws ServiceException
   ```
   
   And, we will be using ByteString from request/response objects to RPC APIs.
   
   **Admin** API for both use-cases (different ring buffers at Master as well as RS level) is generic:
   ```
     List<LogEntry> getLogEntries(LogRequest logRequest) throws IOException;
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487407947



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize it's bit difficult to catch while reading since it's last line of the method.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471692736



##########
File path: hbase-shell/src/main/ruby/hbase/admin.rb
##########
@@ -1639,6 +1639,18 @@ def recommission_regionserver(server_name_string, encoded_region_names)
       @admin.recommissionRegionServer(server_name, region_names_in_bytes)
     end
 
+    #----------------------------------------------------------------------------------------------
+    # Retrieve latest balancer decisions made by LoadBalancers
+    def get_balancer_decisions
+      balancer_decisions_responses = @admin.getBalancerDecisions
+      balancer_decisions_resp_arr = []
+      balancer_decisions_responses.each { |balancer_dec_resp|
+        balancer_decisions_resp_arr << balancer_dec_resp.toJsonPrettyPrint
+      }
+      puts 'Retrieved BalancerDecision Responses from HMaster'
+      puts balancer_decisions_resp_arr

Review comment:
       Only return the array here, this may be used for programmatic things. Put the part that prints the result into the command impl which invokes this admin.rb function.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-678450344


   By API, I mean Admin.java . The RPC stuff (Request/Response classes) is not user facing. 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471671371



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       Something to consider: Rather than adding new API for every ringbuffer backed type, since the ringbuffers are named, can we just have one API that retrieves records from a buffer specified by name? 
   
   E.g.
   
   public List\<LogEntry\> getLogEntries(String name)
   
   Then, LogEntry is a generic type capable of accepting any protobuf encoding. Then, we derive new types from LogEntry such as BalancerDecision. Have a static method in LogEntry for instantiating the subclasses by reflection based on what type is communicated by the protobuf. 
   
   If _LogEntry_ is too generic a name, consider _RingEntry_. (I'm not the best at naming, maybe someone else has a better idea...)
   
   It is a lot easier to add or remove specialized classes as these things evolve than add or remove methods from public/stable admin APIs. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r489023341



##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Ok.
   Please add a comment that explains this in the proto file where these definitions are made. Can be done at commit time, np. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r482602787



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ *
+ * 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.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision request payload with filter attributes
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionRequest extends LogRequest {
+
+  private int limit = 250;
+
+  public int getLimit() {

Review comment:
       -1 on limits in non public RPC classes. The Admin API is the public interface here. These RPC FooRequest and FooResponse classes are implementation detail and not the place to be doing this. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r473676689



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/BalancerDecisionRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ *
+ * 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.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Balancer decision request payload with filter attributes
+ */
+@InterfaceAudience.Private
+public class BalancerDecisionRequest extends LogRequest {
+
+  private int limit = 250;
+
+  public int getLimit() {

Review comment:
       This is the request param coming to Admin API.
   
   Detailed comment on Admin API: `limit` is included in request payload by individual use-cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r477623990



##########
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

##########
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:
       If we can limit the slow log result set size with this filter that is slow log specific, then we don't need a limit param on the existing admin API for slow log. So keep this?
   
   Apologies for the back and forth on this. 

##########
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:
       Yes, this is what I meant by "add a method and leave this one for backwards compat". Additional methods on an interface are binary compat per Java spec and our compat guidelines as long as existing method signatures are maintained. 
   
   If the LogQueryFilter can impose a limit, then that would work too. 

##########
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:
       MasterProtos.BalancerDecisionsRequest is not generic. This is the issue. We should be using one common generic RPC API to get slow log and balancer ring entries, and any other named ring entry type. This uses a new special master RPC for balancer decision.

##########
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:
       Not impossible at all.
   
   getRingEntries RPC in RS RPC servcies
   
   getRingEntries RPC in master RPC services
   
   > we can't achieve abstraction since protobuf doesn't support is-a relationship among proto messages.
   
   You have an outer message type that is a class name and a byte string, and the inner protobuf is stuffed into the byte string while the class name is set to the type you need to use to deserialize. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] apurtell commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487301224



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
##########
@@ -1673,4 +1697,20 @@
    * @throws IOException if a remote or network exception occurs
    */
   CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
+
+  /**
+   * 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

Review comment:
       What happens when we have multiple masters? I think just a doc update is needed here to indicate the log for servertype=MASTER will only come from the currently active master. Can be done at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerType.java
##########
@@ -0,0 +1,33 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * 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
+public enum ServerType {
+  HMASTER,

Review comment:
       We try to not use 'H' prefixes in new code. Please just call these MASTER and REGION_SERVER. Can be fixed at commit time.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3626,4 +3643,49 @@ public static CheckAndMutate toCheckAndMutate(ClientProtos.Condition condition,
       throw new DoNotRetryIOException(e.getMessage());
     }
   }
+
+  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")) {

Review comment:
       Same comment as above.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       This is fine but leaves it up to the client to wonder if the empty list is because there were no SlowLogResponses or if what was returned was valid protobuf but we found something else besides the expected entry type. Maybe throw an exception if the expected type is not encoded? Would help catch future changes that break compat.

##########
File path: hbase-protocol-shaded/src/main/protobuf/HBase.proto
##########
@@ -273,4 +273,14 @@ message RegionLocation {
   required RegionInfo region_info = 1;
   optional ServerName server_name = 2;
   required int64 seq_num = 3;
-}
\ No newline at end of file
+}
+
+message LogRequest {
+  required string log_class_name = 1;
+  required bytes log_initializer_message = 2;

Review comment:
       Why "initializer"? What does that mean?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       With generic GetLogEntries RPC now, we should be maximally flexible and specify these things by string, not by enum (even with ordinal). If you still want the enum, define string constants, switch with the string value, and call the static lookup function getEventByName. 

##########
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 {
+  optional uint32 limit = 1;
+}
+
+message BalancerDecisionsResponse {

Review comment:
       Do we still need these?
   
   We have generic LogRequest now, and the log request gives the name of the log we want to return (encapsulated) results from. 
   
   Protobuf for BalancerDecisions is fine!
   
   But the BalancerDecisionsRequest and BalancerDecisionsResponse proto types are for an RPC interface from an earlier revision of this patch, and that (specialized) RPC no longer exists. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/NamedQueuePayload.java
##########
@@ -30,16 +30,39 @@
 public class NamedQueuePayload {
 
   public enum NamedQueueEvent {
-    SLOW_LOG
+    SLOW_LOG(0),

Review comment:
       In the request RPC, client should give the name of the queue they want entries from, as a string.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
##########
@@ -37,7 +37,7 @@
   private int metricsSize = 1000;
   private int mruCap = calcMruCap(metricsSize);
 
-  private Map<String, Map<String, Double>> stochasticCosts =
+  private final Map<String, Map<String, Double>> stochasticCosts =

Review comment:
       This seems like an extraneous change.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-682220175


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 58s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 38s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 38s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 25s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 28s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 46s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 56s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 16s |  root in master failed.  |
   | -0 :warning: |  patch  |  13m 11s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 36s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 43s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   1m  3s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 15s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 172m 33s |  root in the patch passed.  |
   |  |   | 217m 29s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3806ac4649f4 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 047e0618d2 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/testReport/ |
   | Max. process+thread count | 7421 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-689768988


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 35s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 27s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 23s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 30s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 37s |  master passed  |
   | -0 :warning: |  patch  |  12m 19s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 18s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 18s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 36s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-client generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | -0 :warning: |  javadoc  |   2m  8s |  root generated 1 new + 45 unchanged - 0 fixed = 46 total (was 45)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 340m 11s |  root in the patch failed.  |
   |  |   | 379m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9fb4abfc81cc 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2e96a5b2d3 |
   | Default Java | 1.8.0_232 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/testReport/ |
   | Max. process+thread count | 6333 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-689616185


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 16s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 21s |  master passed  |
   | +0 :ok: |  refguide  |   5m 47s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  21m 55s |  master passed  |
   | -0 :warning: |  patch  |  12m  3s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 29s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 35s |  root: The patch generated 8 new + 511 unchanged - 0 fixed = 519 total (was 511)  |
   | -0 :warning: |  rubocop  |   0m 25s |  The patch generated 15 new + 587 unchanged - 9 fixed = 602 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   5m 48s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  13m  2s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 50s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  24m 27s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 32s |  The patch does not generate ASF License warnings.  |
   |  |   | 106m  3s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 64afbef1e046 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2e96a5b2d3 |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 123 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/15/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r471710161



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
##########
@@ -1057,4 +1057,9 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
       throws IOException {
     get(admin.updateRSGroupConfig(groupName, configuration));
   }
+
+  @Override
+  public List<BalancerDecisionRecords> getBalancerDecisions() throws IOException {

Review comment:
       I agree method proliferation is a concern, something to consider and once we come to a decision, there is no way back it seems.
   For limit param, we have limit in slowLog use-case because client is calling every single RS. Here, since it's just HMaster, the real limit should be kept in HMaster's ring buffer size. However, memory constraints for client makes limiting useful even for single server.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-677838306


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 43s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 43s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 38s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 23s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m 37s |  master passed  |
   | -0 :warning: |  patch  |  14m 13s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 55s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 55s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 284m 33s |  root in the patch failed.  |
   |  |   | 329m 41s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d48e04bbd23f 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 1164531d5a |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/testReport/ |
   | Max. process+thread count | 4807 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-691510632


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 33s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 47s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 15s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 33s |  master passed  |
   | +0 :ok: |  refguide  |   5m 39s |  branch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  spotbugs  |  20m 54s |  master passed  |
   | -0 :warning: |  patch  |  11m 30s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 11s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 46s |  root: The patch generated 8 new + 511 unchanged - 0 fixed = 519 total (was 511)  |
   | -0 :warning: |  rubocop  |   0m 27s |  The patch generated 15 new + 587 unchanged - 9 fixed = 602 total (was 596)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +0 :ok: |  refguide  |   6m 24s |  patch has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 36s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   8m 17s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  24m 46s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 39s |  The patch does not generate ASF License warnings.  |
   |  |   | 108m 14s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/16/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle refguide xml cc hbaseprotoc prototool rubocop |
   | uname | Linux 9ee01311a561 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e5ca9adc54 |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/16/artifact/yetus-general-check/output/branch-site/book.html |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/16/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/16/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | refguide | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/16/artifact/yetus-general-check/output/patch-site/book.html |
   | Max. process+thread count | 122 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/16/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#issuecomment-687320712


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   6m  2s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 10s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 44s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m  1s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 29s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 52s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   1m  5s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  root in master failed.  |
   | -0 :warning: |  patch  |  14m 10s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 51s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 51s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 29s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 50s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   1m 12s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  root in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 165m 59s |  root in the patch passed.  |
   |  |   | 221m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2261 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a050ca43fea6 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 1e8db480b3 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-root.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/testReport/ |
   | Max. process+thread count | 6513 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-hadoop-compat hbase-client hbase-server hbase-thrift hbase-shell . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2261/13/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r487407947



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
##########
@@ -3503,14 +3508,26 @@ private static OnlineLogRecord getSlowLogRecord(
   /**
    * Convert  AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
    *
-   * @param slowLogResponses slowlog response protobuf instance
+   * @param logEntry slowlog response protobuf instance
    * @return list of SlowLog payloads for client usecase
    */
-  public static List<OnlineLogRecord> toSlowLogPayloads(
-      final AdminProtos.SlowLogResponses slowLogResponses) {
-    List<OnlineLogRecord> onlineLogRecords = slowLogResponses.getSlowLogPayloadsList()
-      .stream().map(ProtobufUtil::getSlowLogRecord).collect(Collectors.toList());
-    return onlineLogRecords;
+  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")) {

Review comment:
       Oh, we don't leave it up to client, we do throw Exception. The last line of this method after try/catch is:
   ```
       throw new RuntimeException("Invalid response from server");
   ```
   I realize since it's last line of the method, while reading, it's bit difficult to catch.
   Any specific Exception message recommendation? The reason why I kept "Invalid response from server" is because  ultimately we are parsing response from RPC call. 
   Open to update error message for better recommendation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2261: HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2261:
URL: https://github.com/apache/hbase/pull/2261#discussion_r478097593



##########
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:
       > If the LogQueryFilter can impose a limit, then that would work too.
   
   You mean to say the previous approach where individual request payloads had limit themselves (LogQueryFilter and BalancerDecisionRequest both have limit as a param) was good enough? Or you are suggesting to just let LogQueryFilter (which represents slowLog use-case) have limit param?
   I somehow still feel, instead of exposing `limit` separately on Admin API, we should let individual request payloads have `limit` (like previous commit)

##########
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:
       Yeah exactly, I am also more inclined to keep `limit` in request payloads. Instead, let me keep `limit` in abstract payload `LogRequest` which is empty right now(IS.evolving) and also going to be extended by every new use-case's own request payloads.

##########
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:
       > You have an outer message type that is a class name and a byte string, and the protobuf for the inner type is serialized into the byte string while the class name field of the outer type is set to the type to use for parsing the inner type.
   
   Thanks a lot, I was not aware of this usage. Yes, passing an object as ByteString should be good enough.




----------------------------------------------------------------
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:
users@infra.apache.org