Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-24528 : BalancerDecision queue implementation in HMaster with Admin API #2261

Closed
wants to merge 22 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
22 commits
Select commit Hold shift + click to select a range
b5a42c8
HBASE-24528 : BalancerDecision implementation with Admin API to retri…
virajjasani Aug 15, 2020
4b4f8f4
minor change
virajjasani Aug 16, 2020
f6ce577
addressing review comments
virajjasani Aug 18, 2020
cc3b83f
generic Admin API for ring buffer use-cases
virajjasani Aug 18, 2020
6556972
Merge branch 'master' of github.com:apache/hbase into HBASE-24528-trunk
virajjasani Aug 18, 2020
637f1dc
latest review
virajjasani Aug 20, 2020
eb47d80
s\isBalancerDecisionEnabled\isBalancerDecisionRecording
virajjasani Aug 20, 2020
215757b
review : including limit as separate argument rather than as request …
virajjasani Aug 22, 2020
b69bf09
update javadoc of LogQueryFilter
virajjasani Aug 25, 2020
208a51e
updating javadoc of getSlowLogResponses
virajjasani Aug 26, 2020
b665ebc
common RPC call
virajjasani Aug 27, 2020
cd516f2
Revert "updating javadoc of getSlowLogResponses"
virajjasani Aug 27, 2020
b50e19d
Revert "update javadoc of LogQueryFilter"
virajjasani Aug 27, 2020
4d276af
bring limit to request payload
virajjasani Aug 27, 2020
554c66b
Merge branch 'master' of github.com:apache/hbase into HBASE-24528-trunk
virajjasani Aug 31, 2020
8ae912a
fixing checkstyle
virajjasani Aug 31, 2020
7b6bbfb
simplify Admin API with mandatory arguments and additional ones as map
virajjasani Sep 4, 2020
5665727
keep LogType weakly typed to handle multiple client-server upgrade sc…
virajjasani Sep 9, 2020
9562d31
Merge branch 'master' of github.com:apache/hbase into HBASE-24528-trunk
virajjasani Sep 9, 2020
7d11205
addendum for recent review
virajjasani Sep 12, 2020
5c58136
Merge branch 'master' of github.com:apache/hbase into HBASE-24528-trunk
virajjasani Sep 13, 2020
57cd46f
Merge branch 'master' of github.com:apache/hbase into HBASE-24528-trunk
virajjasani Sep 15, 2020
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,14 @@
import java.io.IOException;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CacheEvictionStats;
Expand Down Expand Up @@ -2339,9 +2341,30 @@ 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.
apurtell marked this conversation as resolved.
Show resolved Hide resolved
* Use {@link #getLogEntries(Set, String, ServerType, int, Map)} instead.
*/
List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
final LogQueryFilter logQueryFilter) throws IOException;
@Deprecated
default List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
apurtell marked this conversation as resolved.
Show resolved Hide resolved
final LogQueryFilter logQueryFilter) throws IOException {
String logType;
if (LogQueryFilter.Type.LARGE_LOG.equals(logQueryFilter.getType())) {
logType = "LARGE_LOG";
} else {
logType = "SLOW_LOG";
}
Map<String, Object> filterParams = new HashMap<>();
filterParams.put("regionName", logQueryFilter.getRegionName());
filterParams.put("clientAddress", logQueryFilter.getClientAddress());
filterParams.put("tableName", logQueryFilter.getTableName());
filterParams.put("userName", logQueryFilter.getUserName());
filterParams.put("filterByOperator", logQueryFilter.getFilterByOperator().toString());
List<LogEntry> logEntries =
getLogEntries(serverNames, logType, ServerType.REGION_SERVER, logQueryFilter.getLimit(),
filterParams);
return logEntries.stream().map(logEntry -> (OnlineLogRecord) logEntry)
.collect(Collectors.toList());
}

/**
* Clears online slow/large RPC logs from the provided list of
Expand Down Expand Up @@ -2472,4 +2495,20 @@ Pair<List<String>, List<TableName>> getConfiguredNamespacesAndTablesInRSGroup(St
*/
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 serverNames servers to retrieve records from, useful in case of records maintained
* by RegionServer as we can select specific server. In case of servertype=MASTER, logs will
* only come from the currently active master.
* @param logType string representing type of log records
* @param serverType enum for server type: HMaster or RegionServer
* @param limit put a limit to list of records that server should send in response
* @param filterParams additional filter params
* @return Log entries representing online records from servers
* @throws IOException if a remote or network exception occurs
*/
List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -974,12 +974,6 @@ public boolean isSnapshotCleanupEnabled() throws IOException {
return get(admin.isSnapshotCleanupEnabled());
}

@Override
public List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
final LogQueryFilter logQueryFilter) throws IOException {
return get(admin.getSlowLogResponses(serverNames, logQueryFilter));
}

@Override
public List<Boolean> clearSlowLogResponses(final Set<ServerName> serverNames)
throws IOException {
Expand Down Expand Up @@ -1057,4 +1051,11 @@ public void updateRSGroupConfig(String groupName, Map<String, String> configurat
throws IOException {
get(admin.updateRSGroupConfig(groupName, configuration));
}

@Override
public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
ServerType serverType, int limit, Map<String, Object> filterParams)
throws IOException {
return get(admin.getLogEntries(serverNames, logType, serverType, limit, filterParams));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,15 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.CacheEvictionStats;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
Expand Down Expand Up @@ -1543,9 +1545,31 @@ default CompletableFuture<List<Boolean>> hasUserPermissions(List<Permission> per
* @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}
* @deprecated since 2.4.0 and will be removed in 4.0.0.
* Use {@link #getLogEntries(Set, String, ServerType, int, Map)} instead.
*/
CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(final Set<ServerName> serverNames,
final LogQueryFilter logQueryFilter);
@Deprecated
default CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(
apurtell marked this conversation as resolved.
Show resolved Hide resolved
final Set<ServerName> serverNames, final LogQueryFilter logQueryFilter) {
String logType;
if (LogQueryFilter.Type.LARGE_LOG.equals(logQueryFilter.getType())) {
logType = "LARGE_LOG";
} else {
logType = "SLOW_LOG";
}
Map<String, Object> filterParams = new HashMap<>();
filterParams.put("regionName", logQueryFilter.getRegionName());
filterParams.put("clientAddress", logQueryFilter.getClientAddress());
filterParams.put("tableName", logQueryFilter.getTableName());
filterParams.put("userName", logQueryFilter.getUserName());
filterParams.put("filterByOperator", logQueryFilter.getFilterByOperator().toString());
CompletableFuture<List<LogEntry>> logEntries =
getLogEntries(serverNames, logType, ServerType.REGION_SERVER, logQueryFilter.getLimit(),
filterParams);
return logEntries.thenApply(
logEntryList -> logEntryList.stream().map(logEntry -> (OnlineLogRecord) logEntry)
.collect(Collectors.toList()));
}

/**
* Clears online slow RPC logs from the provided list of
Expand Down Expand Up @@ -1673,4 +1697,20 @@ CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(final Set<ServerNam
* @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 RegionServer as we can select specific server. In case of servertype=MASTER, logs will
* only come from the currently active master.
* @param logType string representing type of log records
* @param serverType enum for server type: HMaster or RegionServer
* @param limit put a limit to list of records that server should send in response
* @param filterParams additional filter params
* @return Log entries representing online records from servers
*/
CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType,
ServerType serverType, int limit, Map<String, Object> filterParams);
}
Original file line number Diff line number Diff line change
Expand Up @@ -852,12 +852,6 @@ public CompletableFuture<Boolean> isSnapshotCleanupEnabled() {
return wrap(rawAdmin.isSnapshotCleanupEnabled());
}

@Override
public CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(
final Set<ServerName> serverNames, final LogQueryFilter logQueryFilter) {
return wrap(rawAdmin.getSlowLogResponses(serverNames, logQueryFilter));
}

@Override
public CompletableFuture<List<Boolean>> clearSlowLogResponses(Set<ServerName> serverNames) {
return wrap(rawAdmin.clearSlowLogResponses(serverNames));
Expand Down Expand Up @@ -934,4 +928,11 @@ public CompletableFuture<Void> renameRSGroup(String oldName, String newName) {
updateRSGroupConfig(String groupName, Map<String, String> configuration) {
return wrap(rawAdmin.updateRSGroupConfig(groupName, configuration));
}

@Override
public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames,
String logType, ServerType serverType, int limit,
Map<String, Object> filterParams) {
return wrap(rawAdmin.getLogEntries(serverNames, logType, serverType, limit, filterParams));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hadoop.hbase.client;

import java.util.List;

import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.hadoop.hbase.util.GsonUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;

import org.apache.hbase.thirdparty.com.google.gson.Gson;
import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;

/**
* History of balancer decisions taken for region movements.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
final public class BalancerDecision extends LogEntry {

private final String initialFunctionCosts;
private final String finalFunctionCosts;
private final double initTotalCost;
private final double computedTotalCost;
private final long computedSteps;
private final List<String> regionPlans;

// used to convert object to pretty printed format
// used by toJsonPrettyPrint()
private static final Gson GSON = GsonUtil.createGson()
.setPrettyPrinting()
.registerTypeAdapter(BalancerDecision.class, (JsonSerializer<BalancerDecision>)
(balancerDecision, type, jsonSerializationContext) -> {
Gson gson = new Gson();
return gson.toJsonTree(balancerDecision);
}).create();

private BalancerDecision(String initialFunctionCosts, String finalFunctionCosts,
double initTotalCost, double computedTotalCost, List<String> regionPlans,
long computedSteps) {
this.initialFunctionCosts = initialFunctionCosts;
this.finalFunctionCosts = finalFunctionCosts;
this.initTotalCost = initTotalCost;
this.computedTotalCost = computedTotalCost;
this.regionPlans = regionPlans;
this.computedSteps = computedSteps;
}

public String getInitialFunctionCosts() {
return initialFunctionCosts;
}

public String getFinalFunctionCosts() {
return finalFunctionCosts;
}

public double getInitTotalCost() {
return initTotalCost;
}

public double getComputedTotalCost() {
return computedTotalCost;
}

public List<String> getRegionPlans() {
return regionPlans;
}

public long getComputedSteps() {
return computedSteps;
}

@Override
public String toString() {
return new ToStringBuilder(this)
.append("initialFunctionCosts", initialFunctionCosts)
.append("finalFunctionCosts", finalFunctionCosts)
.append("initTotalCost", initTotalCost)
.append("computedTotalCost", computedTotalCost)
.append("computedSteps", computedSteps)
.append("regionPlans", regionPlans)
.toString();
}

@Override
public String toJsonPrettyPrint() {
return GSON.toJson(this);
}

public static class Builder {
private String initialFunctionCosts;
private String finalFunctionCosts;
private double initTotalCost;
private double computedTotalCost;
private long computedSteps;
private List<String> regionPlans;

public Builder setInitialFunctionCosts(String initialFunctionCosts) {
this.initialFunctionCosts = initialFunctionCosts;
return this;
}

public Builder setFinalFunctionCosts(String finalFunctionCosts) {
this.finalFunctionCosts = finalFunctionCosts;
return this;
}

public Builder setInitTotalCost(double initTotalCost) {
this.initTotalCost = initTotalCost;
return this;
}

public Builder setComputedTotalCost(double computedTotalCost) {
this.computedTotalCost = computedTotalCost;
return this;
}

public Builder setRegionPlans(List<String> regionPlans) {
this.regionPlans = regionPlans;
return this;
}

public Builder setComputedSteps(long computedSteps) {
this.computedSteps = computedSteps;
return this;
}

public BalancerDecision build() {
return new BalancerDecision(initialFunctionCosts, finalFunctionCosts,
initTotalCost, computedTotalCost, regionPlans, computedSteps);
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hadoop.hbase.client;

import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;

/**
* Abstract response class representing online logs response from ring-buffer use-cases
* e.g slow/large RPC logs, balancer decision logs
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public abstract class LogEntry {

/**
* Based on response sent by server, provide pretty printed Json representation in string
* @return Pretty printed Json representation
*/
public abstract String toJsonPrettyPrint();

}
Loading