Skip to content

Commit 00fec24

Browse files
authored
HBASE-25790 NamedQueue 'BalancerRejection' for recent history of balancer skipping (#3182)
Signed-off-by: Viraj Jasani <vjasani@apache.org>
1 parent accfceb commit 00fec24

File tree

20 files changed

+726
-18
lines changed

20 files changed

+726
-18
lines changed

hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
6363
"hbase.master.balancer.decision.buffer.enabled";
6464
public static final boolean DEFAULT_BALANCER_DECISION_BUFFER_ENABLED = false;
6565

66+
public static final String BALANCER_REJECTION_BUFFER_ENABLED =
67+
"hbase.master.balancer.rejection.buffer.enabled";
68+
public static final boolean DEFAULT_BALANCER_REJECTION_BUFFER_ENABLED = false;
69+
6670
protected static final int MIN_SERVER_BALANCE = 2;
6771
private volatile boolean stopped = false;
6872

Lines changed: 116 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,116 @@
1+
/*
2+
*
3+
* Licensed to the Apache Software Foundation (ASF) under one
4+
* or more contributor license agreements. See the NOTICE file
5+
* distributed with this work for additional information
6+
* regarding copyright ownership. The ASF licenses this file
7+
* to you under the Apache License, Version 2.0 (the
8+
* "License"); you may not use this file except in compliance
9+
* with the License. You may obtain a copy of the License at
10+
*
11+
* http://www.apache.org/licenses/LICENSE-2.0
12+
*
13+
* Unless required by applicable law or agreed to in writing, software
14+
* distributed under the License is distributed on an "AS IS" BASIS,
15+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16+
* See the License for the specific language governing permissions and
17+
* limitations under the License.
18+
*/
19+
20+
package org.apache.hadoop.hbase.client;
21+
22+
import java.util.ArrayList;
23+
import java.util.Collections;
24+
import java.util.List;
25+
26+
import org.apache.commons.lang3.builder.ToStringBuilder;
27+
import org.apache.hadoop.hbase.util.GsonUtil;
28+
import org.apache.yetus.audience.InterfaceAudience;
29+
import org.apache.yetus.audience.InterfaceStability;
30+
31+
import org.apache.hbase.thirdparty.com.google.gson.Gson;
32+
import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
33+
34+
/**
35+
* History of detail information that balancer movements was rejected
36+
*/
37+
@InterfaceAudience.Public
38+
@InterfaceStability.Evolving
39+
final public class BalancerRejection extends LogEntry {
40+
//The reason why balancer was rejected
41+
private final String reason;
42+
private final List<String> costFuncInfoList;
43+
44+
// used to convert object to pretty printed format
45+
// used by toJsonPrettyPrint()
46+
private static final Gson GSON = GsonUtil.createGson()
47+
.setPrettyPrinting()
48+
.disableHtmlEscaping()
49+
.registerTypeAdapter(BalancerRejection.class, (JsonSerializer<BalancerRejection>)
50+
(balancerRejection, type, jsonSerializationContext) -> {
51+
Gson gson = new Gson();
52+
return gson.toJsonTree(balancerRejection);
53+
}).create();
54+
55+
private BalancerRejection(String reason, List<String> costFuncInfoList) {
56+
this.reason = reason;
57+
if(costFuncInfoList == null){
58+
this.costFuncInfoList = Collections.emptyList();
59+
}
60+
else {
61+
this.costFuncInfoList = costFuncInfoList;
62+
}
63+
}
64+
65+
public String getReason() {
66+
return reason;
67+
}
68+
69+
public List<String> getCostFuncInfoList() {
70+
return costFuncInfoList;
71+
}
72+
73+
@Override
74+
public String toString() {
75+
return new ToStringBuilder(this)
76+
.append("reason", reason)
77+
.append("costFuncInfoList", costFuncInfoList.toString())
78+
.toString();
79+
}
80+
81+
@Override
82+
public String toJsonPrettyPrint() {
83+
return GSON.toJson(this);
84+
}
85+
86+
public static class Builder {
87+
private String reason;
88+
private List<String> costFuncInfoList;
89+
90+
public Builder setReason(String reason) {
91+
this.reason = reason;
92+
return this;
93+
}
94+
95+
public void addCostFuncInfo(String funcName, double cost, float multiplier){
96+
if(costFuncInfoList == null){
97+
costFuncInfoList = new ArrayList<>();
98+
}
99+
costFuncInfoList.add(
100+
new StringBuilder()
101+
.append(funcName)
102+
.append(" cost:").append(cost)
103+
.append(" multiplier:").append(multiplier)
104+
.toString());
105+
}
106+
107+
public Builder setCostFuncInfoList(List<String> costFuncInfoList){
108+
this.costFuncInfoList = costFuncInfoList;
109+
return this;
110+
}
111+
112+
public BalancerRejection build() {
113+
return new BalancerRejection(reason, costFuncInfoList);
114+
}
115+
}
116+
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java

Lines changed: 30 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -4202,26 +4202,44 @@ private CompletableFuture<List<LogEntry>> getBalancerDecisions(final int limit)
42024202
.call();
42034203
}
42044204

4205+
private CompletableFuture<List<LogEntry>> getBalancerRejections(final int limit) {
4206+
return this.<List<LogEntry>>newMasterCaller()
4207+
.action((controller, stub) ->
4208+
this.call(controller, stub,
4209+
ProtobufUtil.toBalancerRejectionRequest(limit),
4210+
MasterService.Interface::getLogEntries, ProtobufUtil::toBalancerRejectionResponse))
4211+
.call();
4212+
}
4213+
42054214
@Override
42064215
public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames,
42074216
String logType, ServerType serverType, int limit,
42084217
Map<String, Object> filterParams) {
42094218
if (logType == null || serverType == null) {
42104219
throw new IllegalArgumentException("logType and/or serverType cannot be empty");
42114220
}
4212-
if (logType.equals("SLOW_LOG") || logType.equals("LARGE_LOG")) {
4213-
if (ServerType.MASTER.equals(serverType)) {
4214-
throw new IllegalArgumentException("Slow/Large logs are not maintained by HMaster");
4215-
}
4216-
return getSlowLogResponses(filterParams, serverNames, limit, logType);
4217-
} else if (logType.equals("BALANCER_DECISION")) {
4218-
if (ServerType.REGION_SERVER.equals(serverType)) {
4219-
throw new IllegalArgumentException(
4220-
"Balancer Decision logs are not maintained by HRegionServer");
4221-
}
4222-
return getBalancerDecisions(limit);
4221+
switch (logType){
4222+
case "SLOW_LOG":
4223+
case "LARGE_LOG":
4224+
if (ServerType.MASTER.equals(serverType)) {
4225+
throw new IllegalArgumentException("Slow/Large logs are not maintained by HMaster");
4226+
}
4227+
return getSlowLogResponses(filterParams, serverNames, limit, logType);
4228+
case "BALANCER_DECISION":
4229+
if (ServerType.REGION_SERVER.equals(serverType)) {
4230+
throw new IllegalArgumentException(
4231+
"Balancer Decision logs are not maintained by HRegionServer");
4232+
}
4233+
return getBalancerDecisions(limit);
4234+
case "BALANCER_REJECTION":
4235+
if (ServerType.REGION_SERVER.equals(serverType)) {
4236+
throw new IllegalArgumentException(
4237+
"Balancer Rejection logs are not maintained by HRegionServer");
4238+
}
4239+
return getBalancerRejections(limit);
4240+
default:
4241+
return CompletableFuture.completedFuture(Collections.emptyList());
42234242
}
4224-
return CompletableFuture.completedFuture(Collections.emptyList());
42254243
}
42264244

42274245
}

hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -68,6 +68,7 @@
6868
import org.apache.hadoop.hbase.ServerName;
6969
import org.apache.hadoop.hbase.TableName;
7070
import org.apache.hadoop.hbase.client.Append;
71+
import org.apache.hadoop.hbase.client.BalancerRejection;
7172
import org.apache.hadoop.hbase.client.BalancerDecision;
7273
import org.apache.hadoop.hbase.client.CheckAndMutate;
7374
import org.apache.hadoop.hbase.client.ClientUtil;
@@ -3776,6 +3777,25 @@ public static List<LogEntry> toBalancerDecisionResponse(
37763777
throw new RuntimeException("Invalid response from server");
37773778
}
37783779

3780+
public static List<LogEntry> toBalancerRejectionResponse(
3781+
HBaseProtos.LogEntry logEntry) {
3782+
try {
3783+
final String logClassName = logEntry.getLogClassName();
3784+
Class<?> logClass = Class.forName(logClassName).asSubclass(Message.class);
3785+
Method method = logClass.getMethod("parseFrom", ByteString.class);
3786+
if (logClassName.contains("BalancerRejectionsResponse")) {
3787+
MasterProtos.BalancerRejectionsResponse response =
3788+
(MasterProtos.BalancerRejectionsResponse) method
3789+
.invoke(null, logEntry.getLogMessage());
3790+
return getBalancerRejectionEntries(response);
3791+
}
3792+
} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
3793+
| InvocationTargetException e) {
3794+
throw new RuntimeException("Error while retrieving response from server");
3795+
}
3796+
throw new RuntimeException("Invalid response from server");
3797+
}
3798+
37793799
public static List<LogEntry> getBalancerDecisionEntries(
37803800
MasterProtos.BalancerDecisionsResponse response) {
37813801
List<RecentLogs.BalancerDecision> balancerDecisions = response.getBalancerDecisionList();
@@ -3792,6 +3812,19 @@ public static List<LogEntry> getBalancerDecisionEntries(
37923812
.collect(Collectors.toList());
37933813
}
37943814

3815+
public static List<LogEntry> getBalancerRejectionEntries(
3816+
MasterProtos.BalancerRejectionsResponse response) {
3817+
List<RecentLogs.BalancerRejection> balancerRejections = response.getBalancerRejectionList();
3818+
if (CollectionUtils.isEmpty(balancerRejections)) {
3819+
return Collections.emptyList();
3820+
}
3821+
return balancerRejections.stream().map(balancerRejection -> new BalancerRejection.Builder()
3822+
.setReason(balancerRejection.getReason())
3823+
.setCostFuncInfoList(balancerRejection.getCostFuncInfoList())
3824+
.build())
3825+
.collect(Collectors.toList());
3826+
}
3827+
37953828
public static HBaseProtos.LogRequest toBalancerDecisionRequest(int limit) {
37963829
MasterProtos.BalancerDecisionsRequest balancerDecisionsRequest =
37973830
MasterProtos.BalancerDecisionsRequest.newBuilder().setLimit(limit).build();
@@ -3801,4 +3834,13 @@ public static HBaseProtos.LogRequest toBalancerDecisionRequest(int limit) {
38013834
.build();
38023835
}
38033836

3837+
public static HBaseProtos.LogRequest toBalancerRejectionRequest(int limit) {
3838+
MasterProtos.BalancerRejectionsRequest balancerRejectionsRequest =
3839+
MasterProtos.BalancerRejectionsRequest.newBuilder().setLimit(limit).build();
3840+
return HBaseProtos.LogRequest.newBuilder()
3841+
.setLogClassName(balancerRejectionsRequest.getClass().getName())
3842+
.setLogMessage(balancerRejectionsRequest.toByteString())
3843+
.build();
3844+
}
3845+
38043846
}

hbase-common/src/main/resources/hbase-default.xml

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2007,7 +2007,7 @@ possible configurations would overwhelm and obscure the important.
20072007
</property>
20082008
<property>
20092009
<name>hbase.namedqueue.provider.classes</name>
2010-
<value>org.apache.hadoop.hbase.namequeues.impl.SlowLogQueueService,org.apache.hadoop.hbase.namequeues.impl.BalancerDecisionQueueService</value>
2010+
<value>org.apache.hadoop.hbase.namequeues.impl.SlowLogQueueService,org.apache.hadoop.hbase.namequeues.impl.BalancerDecisionQueueService,org.apache.hadoop.hbase.namequeues.impl.BalancerRejectionQueueService</value>
20112011
<description>
20122012
Default values for NamedQueueService implementors. This comma separated full class names
20132013
represent all implementors of NamedQueueService that we would like to be invoked by
@@ -2026,4 +2026,13 @@ possible configurations would overwhelm and obscure the important.
20262026
the ring buffer is indicated by config: hbase.master.balancer.decision.queue.size
20272027
</description>
20282028
</property>
2029+
<property>
2030+
<name>hbase.master.balancer.rejection.buffer.enabled</name>
2031+
<value>false</value>
2032+
<description>
2033+
Indicates whether active HMaster has ring buffer running for storing
2034+
balancer rejection in FIFO manner with limited entries. The size of
2035+
the ring buffer is indicated by config: hbase.master.balancer.rejection.queue.size
2036+
</description>
2037+
</property>
20292038
</configuration>

hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -706,6 +706,13 @@ message BalancerDecisionsRequest {
706706
optional uint32 limit = 1;
707707
}
708708

709+
/**
710+
* Same as BalancerDecision but used for BalancerRejection
711+
*/
712+
message BalancerRejectionsRequest {
713+
optional uint32 limit = 1;
714+
}
715+
709716
/**
710717
* BalancerDecision (LogEntry) use-case specific RPC response. This response payload will be
711718
* converted in bytes by servers and sent as response to generic RPC API: GetLogEntries
@@ -717,6 +724,10 @@ message BalancerDecisionsResponse {
717724
repeated BalancerDecision balancer_decision = 1;
718725
}
719726

727+
message BalancerRejectionsResponse {
728+
repeated BalancerRejection balancer_rejection = 1;
729+
}
730+
720731
service MasterService {
721732
/** Used by the client to get the number of regions that have received the updated schema */
722733
rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)

hbase-protocol-shaded/src/main/protobuf/server/master/RecentLogs.proto

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,3 +37,8 @@ message BalancerDecision {
3737
repeated string region_plans = 6;
3838

3939
}
40+
41+
message BalancerRejection {
42+
required string reason = 1;
43+
repeated string cost_func_info = 2;
44+
}

hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java

Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -84,6 +84,7 @@
8484
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
8585
import org.apache.hadoop.hbase.mob.MobUtils;
8686
import org.apache.hadoop.hbase.namequeues.BalancerDecisionDetails;
87+
import org.apache.hadoop.hbase.namequeues.BalancerRejectionDetails;
8788
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
8889
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
8990
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
@@ -3396,6 +3397,16 @@ public HBaseProtos.LogEntry getLogEntries(RpcController controller,
33963397
.setLogClassName(balancerDecisionsResponse.getClass().getName())
33973398
.setLogMessage(balancerDecisionsResponse.toByteString())
33983399
.build();
3400+
}else if (logClassName.contains("BalancerRejectionsRequest")){
3401+
MasterProtos.BalancerRejectionsRequest balancerRejectionsRequest =
3402+
(MasterProtos.BalancerRejectionsRequest) method
3403+
.invoke(null, request.getLogMessage());
3404+
MasterProtos.BalancerRejectionsResponse balancerRejectionsResponse =
3405+
getBalancerRejections(balancerRejectionsRequest);
3406+
return HBaseProtos.LogEntry.newBuilder()
3407+
.setLogClassName(balancerRejectionsResponse.getClass().getName())
3408+
.setLogMessage(balancerRejectionsResponse.toByteString())
3409+
.build();
33993410
}
34003411
} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
34013412
| InvocationTargetException e) {
@@ -3423,4 +3434,22 @@ private MasterProtos.BalancerDecisionsResponse getBalancerDecisions(
34233434
.addAllBalancerDecision(balancerDecisions).build();
34243435
}
34253436

3437+
private MasterProtos.BalancerRejectionsResponse getBalancerRejections(
3438+
MasterProtos.BalancerRejectionsRequest request) {
3439+
final NamedQueueRecorder namedQueueRecorder = this.regionServer.getNamedQueueRecorder();
3440+
if (namedQueueRecorder == null) {
3441+
return MasterProtos.BalancerRejectionsResponse.newBuilder()
3442+
.addAllBalancerRejection(Collections.emptyList()).build();
3443+
}
3444+
final NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
3445+
namedQueueGetRequest.setNamedQueueEvent(BalancerRejectionDetails.BALANCER_REJECTION_EVENT);
3446+
namedQueueGetRequest.setBalancerRejectionsRequest(request);
3447+
NamedQueueGetResponse namedQueueGetResponse =
3448+
namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
3449+
List<RecentLogs.BalancerRejection> balancerRejections =
3450+
namedQueueGetResponse.getBalancerRejections();
3451+
return MasterProtos.BalancerRejectionsResponse.newBuilder()
3452+
.addAllBalancerRejection(balancerRejections).build();
3453+
}
3454+
34263455
}

0 commit comments

Comments
 (0)