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