HBASE-23935 : Backport HBASE-22978, HBASE-24528, HBASE-24718 to branch-1 (#2312)
* HBASE-22978: Online slow response log (with sub-tasks: Large logs, Replicate logs to HDFS, Filter support in query) * HBASE-24718: Generic NamedQueue framework for recent in-memory history * HBASE-24528: Improve balancer decision observability Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Reid Chan <reidchan@apache.org>
This commit is contained in:
parent
b7d63f96a9
commit
fb25a7d530
|
@ -22,6 +22,7 @@ import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
@ -1681,5 +1682,33 @@ public interface Admin extends Abortable, Closeable {
|
||||||
*/
|
*/
|
||||||
boolean isSnapshotCleanupEnabled() throws IOException;
|
boolean isSnapshotCleanupEnabled() throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clears online slow/large RPC logs from the provided list of
|
||||||
|
* RegionServers
|
||||||
|
*
|
||||||
|
* @param serverNames Set of Server names to clean slowlog responses from
|
||||||
|
* @return List of booleans representing if online slowlog response buffer is cleaned
|
||||||
|
* from each RegionServer
|
||||||
|
* @throws IOException if a remote or network exception occurs
|
||||||
|
*/
|
||||||
|
List<Boolean> clearSlowLogResponses(final Set<ServerName> serverNames)
|
||||||
|
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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,160 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.lang.reflect.Type;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.commons.lang.builder.ToStringBuilder;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.hbase.util.GsonUtil;
|
||||||
|
|
||||||
|
import org.apache.hbase.thirdparty.com.google.gson.Gson;
|
||||||
|
import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
|
||||||
|
import org.apache.hbase.thirdparty.com.google.gson.JsonSerializationContext;
|
||||||
|
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, new JsonSerializer<BalancerDecision>() {
|
||||||
|
@Override
|
||||||
|
public JsonElement serialize(BalancerDecision balancerDecision, Type type,
|
||||||
|
JsonSerializationContext 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);
|
||||||
|
}
|
||||||
|
|
||||||
|
@InterfaceAudience.Public
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
|
||||||
|
@ -2145,6 +2146,12 @@ class ConnectionManager {
|
||||||
return stub.isSnapshotCleanupEnabled(controller, request);
|
return stub.isSnapshotCleanupEnabled(controller, request);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HBaseProtos.LogEntry getLogEntries(RpcController controller,
|
||||||
|
HBaseProtos.LogRequest request) throws ServiceException {
|
||||||
|
return stub.getLogEntries(controller, request);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller,
|
public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller,
|
||||||
ListNamespaceDescriptorsRequest request) throws ServiceException {
|
ListNamespaceDescriptorsRequest request) throws ServiceException {
|
||||||
|
|
|
@ -26,11 +26,13 @@ import java.io.IOException;
|
||||||
import java.io.InterruptedIOException;
|
import java.io.InterruptedIOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -79,6 +81,7 @@ import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
|
||||||
|
@ -175,6 +178,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
|
||||||
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
|
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
|
||||||
import org.apache.hadoop.hbase.util.Addressing;
|
import org.apache.hadoop.hbase.util.Addressing;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.CollectionUtils;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
@ -5089,4 +5093,92 @@ public class HBaseAdmin implements Admin {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private List<LogEntry> getSlowLogResponses(
|
||||||
|
final Map<String, Object> filterParams, final Set<ServerName> serverNames, final int limit,
|
||||||
|
final String logType) {
|
||||||
|
if (CollectionUtils.isEmpty(serverNames)) {
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
List<LogEntry> logRecords = new ArrayList<>();
|
||||||
|
for (ServerName serverName : serverNames) {
|
||||||
|
try {
|
||||||
|
logRecords.addAll(getSlowLogs(serverName, filterParams, limit, logType));
|
||||||
|
} catch (ServiceException | IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return logRecords;
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<LogEntry> getSlowLogs(ServerName serverName, Map<String, Object> filterParams,
|
||||||
|
int limit, String logType) throws IOException, ServiceException {
|
||||||
|
AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
|
||||||
|
HBaseRpcController controller = rpcControllerFactory.newController();
|
||||||
|
HBaseProtos.LogRequest logRequest =
|
||||||
|
RequestConverter.buildSlowLogResponseRequest(filterParams, limit, logType);
|
||||||
|
HBaseProtos.LogEntry logEntry = admin.getLogEntries(controller, logRequest);
|
||||||
|
return ProtobufUtil.toSlowLogPayloads(logEntry);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Boolean> clearSlowLogResponses(Set<ServerName> serverNames)
|
||||||
|
throws IOException {
|
||||||
|
if (CollectionUtils.isEmpty(serverNames)) {
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
List<Boolean> logsCleared = new ArrayList<>();
|
||||||
|
for (ServerName serverName : serverNames) {
|
||||||
|
try {
|
||||||
|
logsCleared.add(clearSlowLogsResponses(serverName));
|
||||||
|
} catch (ServiceException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return logsCleared;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Boolean clearSlowLogsResponses(final ServerName serverName)
|
||||||
|
throws IOException, ServiceException {
|
||||||
|
AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
|
||||||
|
HBaseRpcController controller = rpcControllerFactory.newController();
|
||||||
|
AdminProtos.ClearSlowLogResponses clearSlowLogResponses =
|
||||||
|
admin.clearSlowLogsResponses(controller,
|
||||||
|
RequestConverter.buildClearSlowLogResponseRequest());
|
||||||
|
return ProtobufUtil.toClearSlowLogPayload(clearSlowLogResponses);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
|
||||||
|
ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException {
|
||||||
|
if (logType == null || serverType == null) {
|
||||||
|
throw new IllegalArgumentException("logType and/or serverType cannot be empty");
|
||||||
|
}
|
||||||
|
if (logType.equals("SLOW_LOG") || logType.equals("LARGE_LOG")) {
|
||||||
|
if (ServerType.MASTER.equals(serverType)) {
|
||||||
|
throw new IllegalArgumentException("Slow/Large logs are not maintained by HMaster");
|
||||||
|
}
|
||||||
|
return getSlowLogResponses(filterParams, serverNames, limit, logType);
|
||||||
|
} else if (logType.equals("BALANCER_DECISION")) {
|
||||||
|
if (ServerType.REGION_SERVER.equals(serverType)) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Balancer Decision logs are not maintained by HRegionServer");
|
||||||
|
}
|
||||||
|
return getBalancerDecisions(limit);
|
||||||
|
}
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<LogEntry> getBalancerDecisions(final int limit) throws IOException {
|
||||||
|
return executeCallable(new MasterCallable<List<LogEntry>>(getConnection()) {
|
||||||
|
@Override
|
||||||
|
public List<LogEntry> call(int callTimeout) throws Exception {
|
||||||
|
HBaseRpcController controller = rpcControllerFactory.newController();
|
||||||
|
controller.setCallTimeout(callTimeout);
|
||||||
|
HBaseProtos.LogEntry logEntry =
|
||||||
|
master.getLogEntries(controller, ProtobufUtil.toBalancerDecisionRequest(limit));
|
||||||
|
return ProtobufUtil.toBalancerDecisionResponse(logEntry);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.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();
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,329 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.lang.reflect.Type;
|
||||||
|
|
||||||
|
import org.apache.commons.lang.builder.EqualsBuilder;
|
||||||
|
import org.apache.commons.lang.builder.HashCodeBuilder;
|
||||||
|
import org.apache.commons.lang.builder.ToStringBuilder;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.hbase.util.GsonUtil;
|
||||||
|
|
||||||
|
import org.apache.hbase.thirdparty.com.google.gson.Gson;
|
||||||
|
import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
|
||||||
|
import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
|
||||||
|
import org.apache.hbase.thirdparty.com.google.gson.JsonSerializationContext;
|
||||||
|
import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Slow/Large Log payload for hbase-client, to be used by Admin API get_slow_responses and
|
||||||
|
* get_large_responses
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Public
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
final public class OnlineLogRecord extends LogEntry {
|
||||||
|
// used to convert object to pretty printed format
|
||||||
|
// used by toJsonPrettyPrint()
|
||||||
|
private static final Gson GSON = GsonUtil.createGson().setPrettyPrinting()
|
||||||
|
.registerTypeAdapter(OnlineLogRecord.class, new JsonSerializer<OnlineLogRecord>() {
|
||||||
|
@Override
|
||||||
|
public JsonElement serialize(OnlineLogRecord slowLogPayload, Type type,
|
||||||
|
JsonSerializationContext jsonSerializationContext) {
|
||||||
|
Gson gson = new Gson();
|
||||||
|
JsonObject jsonObj = (JsonObject) gson.toJsonTree(slowLogPayload);
|
||||||
|
if (slowLogPayload.getMultiGetsCount() == 0) {
|
||||||
|
jsonObj.remove("multiGetsCount");
|
||||||
|
}
|
||||||
|
if (slowLogPayload.getMultiMutationsCount() == 0) {
|
||||||
|
jsonObj.remove("multiMutationsCount");
|
||||||
|
}
|
||||||
|
if (slowLogPayload.getMultiServiceCalls() == 0) {
|
||||||
|
jsonObj.remove("multiServiceCalls");
|
||||||
|
}
|
||||||
|
return jsonObj;
|
||||||
|
}
|
||||||
|
}).create();
|
||||||
|
|
||||||
|
private final long startTime;
|
||||||
|
private final int processingTime;
|
||||||
|
private final int queueTime;
|
||||||
|
private final long responseSize;
|
||||||
|
private final String clientAddress;
|
||||||
|
private final String serverClass;
|
||||||
|
private final String methodName;
|
||||||
|
private final String callDetails;
|
||||||
|
private final String param;
|
||||||
|
// we don't want to serialize region name, it is just for the filter purpose
|
||||||
|
// hence avoiding deserialization
|
||||||
|
private final transient String regionName;
|
||||||
|
private final String userName;
|
||||||
|
private final int multiGetsCount;
|
||||||
|
private final int multiMutationsCount;
|
||||||
|
private final int multiServiceCalls;
|
||||||
|
|
||||||
|
public long getStartTime() {
|
||||||
|
return startTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getProcessingTime() {
|
||||||
|
return processingTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getQueueTime() {
|
||||||
|
return queueTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getResponseSize() {
|
||||||
|
return responseSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getClientAddress() {
|
||||||
|
return clientAddress;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getServerClass() {
|
||||||
|
return serverClass;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getMethodName() {
|
||||||
|
return methodName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getCallDetails() {
|
||||||
|
return callDetails;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getParam() {
|
||||||
|
return param;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getRegionName() {
|
||||||
|
return regionName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getUserName() {
|
||||||
|
return userName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getMultiGetsCount() {
|
||||||
|
return multiGetsCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getMultiMutationsCount() {
|
||||||
|
return multiMutationsCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getMultiServiceCalls() {
|
||||||
|
return multiServiceCalls;
|
||||||
|
}
|
||||||
|
|
||||||
|
private OnlineLogRecord(final long startTime, final int processingTime, final int queueTime,
|
||||||
|
final long responseSize, final String clientAddress, final String serverClass,
|
||||||
|
final String methodName, final String callDetails, final String param,
|
||||||
|
final String regionName, final String userName, final int multiGetsCount,
|
||||||
|
final int multiMutationsCount, final int multiServiceCalls) {
|
||||||
|
this.startTime = startTime;
|
||||||
|
this.processingTime = processingTime;
|
||||||
|
this.queueTime = queueTime;
|
||||||
|
this.responseSize = responseSize;
|
||||||
|
this.clientAddress = clientAddress;
|
||||||
|
this.serverClass = serverClass;
|
||||||
|
this.methodName = methodName;
|
||||||
|
this.callDetails = callDetails;
|
||||||
|
this.param = param;
|
||||||
|
this.regionName = regionName;
|
||||||
|
this.userName = userName;
|
||||||
|
this.multiGetsCount = multiGetsCount;
|
||||||
|
this.multiMutationsCount = multiMutationsCount;
|
||||||
|
this.multiServiceCalls = multiServiceCalls;
|
||||||
|
}
|
||||||
|
|
||||||
|
@InterfaceAudience.Public
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
public static class OnlineLogRecordBuilder {
|
||||||
|
private long startTime;
|
||||||
|
private int processingTime;
|
||||||
|
private int queueTime;
|
||||||
|
private long responseSize;
|
||||||
|
private String clientAddress;
|
||||||
|
private String serverClass;
|
||||||
|
private String methodName;
|
||||||
|
private String callDetails;
|
||||||
|
private String param;
|
||||||
|
private String regionName;
|
||||||
|
private String userName;
|
||||||
|
private int multiGetsCount;
|
||||||
|
private int multiMutationsCount;
|
||||||
|
private int multiServiceCalls;
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setStartTime(long startTime) {
|
||||||
|
this.startTime = startTime;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setProcessingTime(int processingTime) {
|
||||||
|
this.processingTime = processingTime;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setQueueTime(int queueTime) {
|
||||||
|
this.queueTime = queueTime;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setResponseSize(long responseSize) {
|
||||||
|
this.responseSize = responseSize;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setClientAddress(String clientAddress) {
|
||||||
|
this.clientAddress = clientAddress;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setServerClass(String serverClass) {
|
||||||
|
this.serverClass = serverClass;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setMethodName(String methodName) {
|
||||||
|
this.methodName = methodName;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setCallDetails(String callDetails) {
|
||||||
|
this.callDetails = callDetails;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setParam(String param) {
|
||||||
|
this.param = param;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setRegionName(String regionName) {
|
||||||
|
this.regionName = regionName;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setUserName(String userName) {
|
||||||
|
this.userName = userName;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setMultiGetsCount(int multiGetsCount) {
|
||||||
|
this.multiGetsCount = multiGetsCount;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setMultiMutationsCount(int multiMutationsCount) {
|
||||||
|
this.multiMutationsCount = multiMutationsCount;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecordBuilder setMultiServiceCalls(int multiServiceCalls) {
|
||||||
|
this.multiServiceCalls = multiServiceCalls;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OnlineLogRecord build() {
|
||||||
|
return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize,
|
||||||
|
clientAddress, serverClass, methodName, callDetails, param, regionName,
|
||||||
|
userName, multiGetsCount, multiMutationsCount, multiServiceCalls);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
OnlineLogRecord that = (OnlineLogRecord) o;
|
||||||
|
|
||||||
|
return new EqualsBuilder()
|
||||||
|
.append(startTime, that.startTime)
|
||||||
|
.append(processingTime, that.processingTime)
|
||||||
|
.append(queueTime, that.queueTime)
|
||||||
|
.append(responseSize, that.responseSize)
|
||||||
|
.append(multiGetsCount, that.multiGetsCount)
|
||||||
|
.append(multiMutationsCount, that.multiMutationsCount)
|
||||||
|
.append(multiServiceCalls, that.multiServiceCalls)
|
||||||
|
.append(clientAddress, that.clientAddress)
|
||||||
|
.append(serverClass, that.serverClass)
|
||||||
|
.append(methodName, that.methodName)
|
||||||
|
.append(callDetails, that.callDetails)
|
||||||
|
.append(param, that.param)
|
||||||
|
.append(regionName, that.regionName)
|
||||||
|
.append(userName, that.userName)
|
||||||
|
.isEquals();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return new HashCodeBuilder(17, 37)
|
||||||
|
.append(startTime)
|
||||||
|
.append(processingTime)
|
||||||
|
.append(queueTime)
|
||||||
|
.append(responseSize)
|
||||||
|
.append(clientAddress)
|
||||||
|
.append(serverClass)
|
||||||
|
.append(methodName)
|
||||||
|
.append(callDetails)
|
||||||
|
.append(param)
|
||||||
|
.append(regionName)
|
||||||
|
.append(userName)
|
||||||
|
.append(multiGetsCount)
|
||||||
|
.append(multiMutationsCount)
|
||||||
|
.append(multiServiceCalls)
|
||||||
|
.toHashCode();
|
||||||
|
}
|
||||||
|
|
||||||
|
public String toJsonPrettyPrint() {
|
||||||
|
return GSON.toJson(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new ToStringBuilder(this)
|
||||||
|
.append("startTime", startTime)
|
||||||
|
.append("processingTime", processingTime)
|
||||||
|
.append("queueTime", queueTime)
|
||||||
|
.append("responseSize", responseSize)
|
||||||
|
.append("clientAddress", clientAddress)
|
||||||
|
.append("serverClass", serverClass)
|
||||||
|
.append("methodName", methodName)
|
||||||
|
.append("callDetails", callDetails)
|
||||||
|
.append("param", param)
|
||||||
|
.append("regionName", regionName)
|
||||||
|
.append("userName", userName)
|
||||||
|
.append("multiGetsCount", multiGetsCount)
|
||||||
|
.append("multiMutationsCount", multiMutationsCount)
|
||||||
|
.append("multiServiceCalls", multiServiceCalls)
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,35 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
@InterfaceStability.Stable
|
||||||
|
public enum ServerType {
|
||||||
|
MASTER,
|
||||||
|
REGION_SERVER
|
||||||
|
}
|
|
@ -0,0 +1,89 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.lang.StringUtils;
|
||||||
|
import org.apache.commons.lang.builder.EqualsBuilder;
|
||||||
|
import org.apache.commons.lang.builder.HashCodeBuilder;
|
||||||
|
import org.apache.commons.lang.builder.ToStringBuilder;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* SlowLog params object that contains detailed info as params and region name : to be used
|
||||||
|
* for filter purpose
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class SlowLogParams {
|
||||||
|
|
||||||
|
private final String regionName;
|
||||||
|
private final String params;
|
||||||
|
|
||||||
|
public SlowLogParams(String regionName, String params) {
|
||||||
|
this.regionName = regionName;
|
||||||
|
this.params = params;
|
||||||
|
}
|
||||||
|
|
||||||
|
public SlowLogParams(String params) {
|
||||||
|
this.regionName = StringUtils.EMPTY;
|
||||||
|
this.params = params;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getRegionName() {
|
||||||
|
return regionName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getParams() {
|
||||||
|
return params;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new ToStringBuilder(this)
|
||||||
|
.append("regionName", regionName)
|
||||||
|
.append("params", params)
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
SlowLogParams that = (SlowLogParams) o;
|
||||||
|
|
||||||
|
return new EqualsBuilder()
|
||||||
|
.append(regionName, that.regionName)
|
||||||
|
.append(params, that.params)
|
||||||
|
.isEquals();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return new HashCodeBuilder(17, 37)
|
||||||
|
.append(regionName)
|
||||||
|
.append(params)
|
||||||
|
.toHashCode();
|
||||||
|
}
|
||||||
|
}
|
|
@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.Tag;
|
import org.apache.hadoop.hbase.Tag;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.Append;
|
import org.apache.hadoop.hbase.client.Append;
|
||||||
|
import org.apache.hadoop.hbase.client.BalancerDecision;
|
||||||
import org.apache.hadoop.hbase.client.ClientUtil;
|
import org.apache.hadoop.hbase.client.ClientUtil;
|
||||||
import org.apache.hadoop.hbase.client.Consistency;
|
import org.apache.hadoop.hbase.client.Consistency;
|
||||||
import org.apache.hadoop.hbase.client.Cursor;
|
import org.apache.hadoop.hbase.client.Cursor;
|
||||||
|
@ -84,12 +85,15 @@ import org.apache.hadoop.hbase.client.Delete;
|
||||||
import org.apache.hadoop.hbase.client.Durability;
|
import org.apache.hadoop.hbase.client.Durability;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
import org.apache.hadoop.hbase.client.Increment;
|
import org.apache.hadoop.hbase.client.Increment;
|
||||||
|
import org.apache.hadoop.hbase.client.LogEntry;
|
||||||
import org.apache.hadoop.hbase.client.Mutation;
|
import org.apache.hadoop.hbase.client.Mutation;
|
||||||
|
import org.apache.hadoop.hbase.client.OnlineLogRecord;
|
||||||
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
|
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
|
import org.apache.hadoop.hbase.client.SlowLogParams;
|
||||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||||
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
||||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
|
@ -101,6 +105,7 @@ import org.apache.hadoop.hbase.master.RegionState;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
|
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
|
||||||
|
@ -114,6 +119,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRespon
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||||
|
@ -150,9 +156,11 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableReques
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.RecentLogs;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.TableProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.TableProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||||
|
@ -177,6 +185,7 @@ import org.apache.hadoop.hbase.security.visibility.CellVisibility;
|
||||||
import org.apache.hadoop.hbase.util.Addressing;
|
import org.apache.hadoop.hbase.util.Addressing;
|
||||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.CollectionUtils;
|
||||||
import org.apache.hadoop.hbase.util.DynamicClassLoader;
|
import org.apache.hadoop.hbase.util.DynamicClassLoader;
|
||||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||||
import org.apache.hadoop.hbase.util.Methods;
|
import org.apache.hadoop.hbase.util.Methods;
|
||||||
|
@ -3034,6 +3043,62 @@ public final class ProtobufUtil {
|
||||||
return Bytes.toStringBinary(bs.toByteArray());
|
return Bytes.toStringBinary(bs.toByteArray());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return SlowLogParams to maintain recent online slowlog responses
|
||||||
|
*
|
||||||
|
* @param message Message object {@link Message}
|
||||||
|
* @return SlowLogParams with regionName(for filter queries) and params
|
||||||
|
*/
|
||||||
|
public static SlowLogParams getSlowLogParams(Message message) {
|
||||||
|
if (message == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
if (message instanceof ScanRequest) {
|
||||||
|
ScanRequest scanRequest = (ScanRequest) message;
|
||||||
|
String regionName = getStringForByteString(scanRequest.getRegion().getValue());
|
||||||
|
String params = TextFormat.shortDebugString(message);
|
||||||
|
return new SlowLogParams(regionName, params);
|
||||||
|
} else if (message instanceof MutationProto) {
|
||||||
|
MutationProto mutationProto = (MutationProto) message;
|
||||||
|
String params = "type= " + mutationProto.getMutateType().toString();
|
||||||
|
return new SlowLogParams(params);
|
||||||
|
} else if (message instanceof GetRequest) {
|
||||||
|
GetRequest getRequest = (GetRequest) message;
|
||||||
|
String regionName = getStringForByteString(getRequest.getRegion().getValue());
|
||||||
|
String params =
|
||||||
|
"region= " + regionName + ", row= " + getStringForByteString(getRequest.getGet().getRow());
|
||||||
|
return new SlowLogParams(regionName, params);
|
||||||
|
} else if (message instanceof ClientProtos.MultiRequest) {
|
||||||
|
ClientProtos.MultiRequest multiRequest = (ClientProtos.MultiRequest) message;
|
||||||
|
int actionsCount = 0;
|
||||||
|
for (ClientProtos.RegionAction regionAction : multiRequest.getRegionActionList()) {
|
||||||
|
actionsCount += regionAction.getActionCount();
|
||||||
|
}
|
||||||
|
ClientProtos.RegionAction actions = multiRequest.getRegionActionList().get(0);
|
||||||
|
String row = actions.getActionCount() <= 0 ? "" :
|
||||||
|
getStringForByteString(actions.getAction(0).hasGet() ?
|
||||||
|
actions.getAction(0).getGet().getRow() :
|
||||||
|
actions.getAction(0).getMutation().getRow());
|
||||||
|
String regionName = getStringForByteString(actions.getRegion().getValue());
|
||||||
|
String params =
|
||||||
|
"region= " + regionName + ", for " + actionsCount + " action(s) and 1st row key=" + row;
|
||||||
|
return new SlowLogParams(regionName, params);
|
||||||
|
} else if (message instanceof ClientProtos.MutateRequest) {
|
||||||
|
ClientProtos.MutateRequest mutateRequest = (ClientProtos.MutateRequest) message;
|
||||||
|
String regionName = getStringForByteString(mutateRequest.getRegion().getValue());
|
||||||
|
String params = "region= " + regionName;
|
||||||
|
return new SlowLogParams(regionName, params);
|
||||||
|
} else if (message instanceof CoprocessorServiceRequest) {
|
||||||
|
CoprocessorServiceRequest coprocessorServiceRequest = (CoprocessorServiceRequest) message;
|
||||||
|
String params =
|
||||||
|
"coprocessorService= " + coprocessorServiceRequest.getCall().getServiceName() + ":"
|
||||||
|
+ coprocessorServiceRequest.getCall().getMethodName();
|
||||||
|
return new SlowLogParams(params);
|
||||||
|
}
|
||||||
|
String params = message.getClass().toString();
|
||||||
|
return new SlowLogParams(params);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Print out some subset of a MutationProto rather than all of it and its data
|
* Print out some subset of a MutationProto rather than all of it and its data
|
||||||
* @param proto Protobuf to print out
|
* @param proto Protobuf to print out
|
||||||
|
@ -3663,6 +3728,121 @@ public final class ProtobufUtil {
|
||||||
return Collections.emptySet();
|
return Collections.emptySet();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert Protobuf class
|
||||||
|
* {@link TooSlowLog.SlowLogPayload}
|
||||||
|
* To client SlowLog Payload class {@link OnlineLogRecord}
|
||||||
|
*
|
||||||
|
* @param slowLogPayload SlowLog Payload protobuf instance
|
||||||
|
* @return SlowLog Payload for client usecase
|
||||||
|
*/
|
||||||
|
private static LogEntry getSlowLogRecord(
|
||||||
|
final TooSlowLog.SlowLogPayload slowLogPayload) {
|
||||||
|
OnlineLogRecord onlineLogRecord = new OnlineLogRecord.OnlineLogRecordBuilder()
|
||||||
|
.setCallDetails(slowLogPayload.getCallDetails())
|
||||||
|
.setClientAddress(slowLogPayload.getClientAddress())
|
||||||
|
.setMethodName(slowLogPayload.getMethodName())
|
||||||
|
.setMultiGetsCount(slowLogPayload.getMultiGets())
|
||||||
|
.setMultiMutationsCount(slowLogPayload.getMultiMutations())
|
||||||
|
.setMultiServiceCalls(slowLogPayload.getMultiServiceCalls())
|
||||||
|
.setParam(slowLogPayload.getParam())
|
||||||
|
.setProcessingTime(slowLogPayload.getProcessingTime())
|
||||||
|
.setQueueTime(slowLogPayload.getQueueTime())
|
||||||
|
.setRegionName(slowLogPayload.getRegionName())
|
||||||
|
.setResponseSize(slowLogPayload.getResponseSize())
|
||||||
|
.setServerClass(slowLogPayload.getServerClass())
|
||||||
|
.setStartTime(slowLogPayload.getStartTime())
|
||||||
|
.setUserName(slowLogPayload.getUserName())
|
||||||
|
.build();
|
||||||
|
return onlineLogRecord;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert AdminProtos#SlowLogResponses to list of {@link OnlineLogRecord}
|
||||||
|
*
|
||||||
|
* @param logEntry slowlog response protobuf instance
|
||||||
|
* @return list of SlowLog payloads for client usecase
|
||||||
|
*/
|
||||||
|
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")) {
|
||||||
|
SlowLogResponses slowLogResponses = (SlowLogResponses) method
|
||||||
|
.invoke(null, logEntry.getLogMessage());
|
||||||
|
List<LogEntry> logEntries = new ArrayList<>();
|
||||||
|
for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogResponses.getSlowLogPayloadsList()) {
|
||||||
|
logEntries.add(ProtobufUtil.getSlowLogRecord(slowLogPayload));
|
||||||
|
}
|
||||||
|
return logEntries;
|
||||||
|
}
|
||||||
|
} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
|
||||||
|
| InvocationTargetException e) {
|
||||||
|
throw new RuntimeException("Error while retrieving response from server");
|
||||||
|
}
|
||||||
|
throw new RuntimeException("Invalid response from server");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert {@link ClearSlowLogResponses} to boolean
|
||||||
|
*
|
||||||
|
* @param clearSlowLogResponses Clear slowlog response protobuf instance
|
||||||
|
* @return boolean representing clear slowlog response
|
||||||
|
*/
|
||||||
|
public static boolean toClearSlowLogPayload(final ClearSlowLogResponses clearSlowLogResponses) {
|
||||||
|
return clearSlowLogResponses.getIsCleaned();
|
||||||
|
}
|
||||||
|
|
||||||
|
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")) {
|
||||||
|
MasterProtos.BalancerDecisionsResponse response =
|
||||||
|
(MasterProtos.BalancerDecisionsResponse) method
|
||||||
|
.invoke(null, logEntry.getLogMessage());
|
||||||
|
return getBalancerDecisionEntries(response);
|
||||||
|
}
|
||||||
|
} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
|
||||||
|
| InvocationTargetException e) {
|
||||||
|
throw new RuntimeException("Error while retrieving response from server");
|
||||||
|
}
|
||||||
|
throw new RuntimeException("Invalid response from server");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static List<LogEntry> getBalancerDecisionEntries(
|
||||||
|
MasterProtos.BalancerDecisionsResponse response) {
|
||||||
|
List<RecentLogs.BalancerDecision> balancerDecisions = response.getBalancerDecisionList();
|
||||||
|
if (CollectionUtils.isEmpty(balancerDecisions)) {
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
List<LogEntry> logEntries = new ArrayList<>();
|
||||||
|
for (RecentLogs.BalancerDecision balancerDecision : balancerDecisions) {
|
||||||
|
BalancerDecision bd =
|
||||||
|
new BalancerDecision.Builder().setInitTotalCost(balancerDecision.getInitTotalCost())
|
||||||
|
.setInitialFunctionCosts(balancerDecision.getInitialFunctionCosts())
|
||||||
|
.setComputedTotalCost(balancerDecision.getComputedTotalCost())
|
||||||
|
.setFinalFunctionCosts(balancerDecision.getFinalFunctionCosts())
|
||||||
|
.setComputedSteps(balancerDecision.getComputedSteps())
|
||||||
|
.setRegionPlans(balancerDecision.getRegionPlansList()).build();
|
||||||
|
logEntries.add(bd);
|
||||||
|
}
|
||||||
|
return logEntries;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HBaseProtos.LogRequest toBalancerDecisionRequest(int limit) {
|
||||||
|
MasterProtos.BalancerDecisionsRequest balancerDecisionsRequest =
|
||||||
|
MasterProtos.BalancerDecisionsRequest.newBuilder().setLimit(limit).build();
|
||||||
|
return HBaseProtos.LogRequest.newBuilder()
|
||||||
|
.setLogClassName(balancerDecisionsRequest.getClass().getName())
|
||||||
|
.setLogMessage(balancerDecisionsRequest.toByteString())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the Meta region state from the passed data bytes. Can handle both old and new style
|
* Get the Meta region state from the passed data bytes. Can handle both old and new style
|
||||||
* server names.
|
* server names.
|
||||||
|
|
|
@ -19,8 +19,11 @@ package org.apache.hadoop.hbase.protobuf;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
import org.apache.commons.collections.MapUtils;
|
||||||
|
import org.apache.commons.lang.StringUtils;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||||
|
|
||||||
|
@ -48,6 +51,7 @@ import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
|
||||||
|
@ -59,6 +63,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsReques
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
|
||||||
|
@ -77,6 +82,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Col
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
||||||
|
@ -1895,4 +1901,77 @@ public final class RequestConverter {
|
||||||
return IsSnapshotCleanupEnabledRequest.newBuilder().build();
|
return IsSnapshotCleanupEnabledRequest.newBuilder().build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build RPC request payload for getLogEntries
|
||||||
|
*
|
||||||
|
* @param filterParams map of filter params
|
||||||
|
* @param limit limit for no of records that server returns
|
||||||
|
* @param logType type of the log records
|
||||||
|
* @return request payload HBaseProtos.LogRequest
|
||||||
|
*/
|
||||||
|
public static HBaseProtos.LogRequest buildSlowLogResponseRequest(
|
||||||
|
final Map<String, Object> filterParams, final int limit, final String logType) {
|
||||||
|
SlowLogResponseRequest.Builder builder = SlowLogResponseRequest.newBuilder();
|
||||||
|
builder.setLimit(limit);
|
||||||
|
if (logType.equals("SLOW_LOG")) {
|
||||||
|
builder.setLogType(SlowLogResponseRequest.LogType.SLOW_LOG);
|
||||||
|
} else if (logType.equals("LARGE_LOG")) {
|
||||||
|
builder.setLogType(SlowLogResponseRequest.LogType.LARGE_LOG);
|
||||||
|
}
|
||||||
|
boolean filterByAnd = false;
|
||||||
|
if (MapUtils.isNotEmpty(filterParams)) {
|
||||||
|
if (filterParams.containsKey("clientAddress")) {
|
||||||
|
final String clientAddress = (String) filterParams.get("clientAddress");
|
||||||
|
if (StringUtils.isNotEmpty(clientAddress)) {
|
||||||
|
builder.setClientAddress(clientAddress);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (filterParams.containsKey("regionName")) {
|
||||||
|
final String regionName = (String) filterParams.get("regionName");
|
||||||
|
if (StringUtils.isNotEmpty(regionName)) {
|
||||||
|
builder.setRegionName(regionName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (filterParams.containsKey("tableName")) {
|
||||||
|
final String tableName = (String) filterParams.get("tableName");
|
||||||
|
if (StringUtils.isNotEmpty(tableName)) {
|
||||||
|
builder.setTableName(tableName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (filterParams.containsKey("userName")) {
|
||||||
|
final String userName = (String) filterParams.get("userName");
|
||||||
|
if (StringUtils.isNotEmpty(userName)) {
|
||||||
|
builder.setUserName(userName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (filterParams.containsKey("filterByOperator")) {
|
||||||
|
final String filterByOperator = (String) filterParams.get("filterByOperator");
|
||||||
|
if (StringUtils.isNotEmpty(filterByOperator)) {
|
||||||
|
if (filterByOperator.toUpperCase().equals("AND")) {
|
||||||
|
filterByAnd = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (filterByAnd) {
|
||||||
|
builder.setFilterByOperator(SlowLogResponseRequest.FilterByOperator.AND);
|
||||||
|
} else {
|
||||||
|
builder.setFilterByOperator(SlowLogResponseRequest.FilterByOperator.OR);
|
||||||
|
}
|
||||||
|
SlowLogResponseRequest slowLogResponseRequest = builder.build();
|
||||||
|
return HBaseProtos.LogRequest.newBuilder()
|
||||||
|
.setLogClassName(slowLogResponseRequest.getClass().getName())
|
||||||
|
.setLogMessage(slowLogResponseRequest.toByteString())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a protocol buffer {@link ClearSlowLogResponseRequest}
|
||||||
|
*
|
||||||
|
* @return a protocol buffer ClearSlowLogResponseRequest
|
||||||
|
*/
|
||||||
|
public static ClearSlowLogResponseRequest buildClearSlowLogResponseRequest() {
|
||||||
|
return ClearSlowLogResponseRequest.newBuilder().build();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,154 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.slowlog;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Random;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
|
import org.apache.hadoop.hbase.client.Durability;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Slowlog Accessor to record slow/large RPC log identified at each RegionServer RpcServer level.
|
||||||
|
* This can be done only optionally to record the entire history of slow/large rpc calls
|
||||||
|
* since RingBuffer can handle only limited latest records.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public final class SlowLogTableAccessor {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(SlowLogTableAccessor.class);
|
||||||
|
|
||||||
|
private static final Random RANDOM = new Random();
|
||||||
|
|
||||||
|
private static Connection connection;
|
||||||
|
|
||||||
|
private SlowLogTableAccessor() {
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* hbase:slowlog table name - can be enabled
|
||||||
|
* with config - hbase.regionserver.slowlog.systable.enabled
|
||||||
|
*/
|
||||||
|
public static final TableName SLOW_LOG_TABLE_NAME =
|
||||||
|
TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "slowlog");
|
||||||
|
|
||||||
|
private static void doPut(final Connection connection, final List<Put> puts)
|
||||||
|
throws IOException {
|
||||||
|
try (Table table = connection.getTable(SLOW_LOG_TABLE_NAME)) {
|
||||||
|
table.put(puts);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add slow/large log records to hbase:slowlog table
|
||||||
|
* @param slowLogPayloads List of SlowLogPayload to process
|
||||||
|
* @param configuration Configuration to use for connection
|
||||||
|
*/
|
||||||
|
public static void addSlowLogRecords(final List<TooSlowLog.SlowLogPayload> slowLogPayloads,
|
||||||
|
final Configuration configuration) {
|
||||||
|
List<Put> puts = new ArrayList<>(slowLogPayloads.size());
|
||||||
|
for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
|
||||||
|
final byte[] rowKey = getRowKey(slowLogPayload);
|
||||||
|
final Put put = new Put(rowKey);
|
||||||
|
put.setDurability(Durability.SKIP_WAL);
|
||||||
|
put.setPriority(HConstants.NORMAL_QOS);
|
||||||
|
put.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("call_details"),
|
||||||
|
Bytes.toBytes(slowLogPayload.getCallDetails()))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("client_address"),
|
||||||
|
Bytes.toBytes(slowLogPayload.getClientAddress()))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("method_name"),
|
||||||
|
Bytes.toBytes(slowLogPayload.getMethodName()))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("param"),
|
||||||
|
Bytes.toBytes(slowLogPayload.getParam()))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("processing_time"),
|
||||||
|
Bytes.toBytes(Integer.toString(slowLogPayload.getProcessingTime())))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
|
||||||
|
Bytes.toBytes(Integer.toString(slowLogPayload.getQueueTime())))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("region_name"),
|
||||||
|
Bytes.toBytes(slowLogPayload.getRegionName()))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("response_size"),
|
||||||
|
Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize())))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("server_class"),
|
||||||
|
Bytes.toBytes(slowLogPayload.getServerClass()))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),
|
||||||
|
Bytes.toBytes(Long.toString(slowLogPayload.getStartTime())))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("type"),
|
||||||
|
Bytes.toBytes(slowLogPayload.getType().name()))
|
||||||
|
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("username"),
|
||||||
|
Bytes.toBytes(slowLogPayload.getUserName()));
|
||||||
|
puts.add(put);
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
if (connection == null) {
|
||||||
|
createConnection(configuration);
|
||||||
|
}
|
||||||
|
doPut(connection, puts);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.warn("Failed to add slow/large log records to hbase:slowlog table.", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static synchronized void createConnection(Configuration configuration)
|
||||||
|
throws IOException {
|
||||||
|
Configuration conf = new Configuration(configuration);
|
||||||
|
// rpc timeout: 20s
|
||||||
|
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 20000);
|
||||||
|
// retry count: 5
|
||||||
|
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
|
||||||
|
conf.setInt("hbase.client.serverside.retries.multiplier", 1);
|
||||||
|
connection = ConnectionFactory.createConnection(conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create rowKey: currentTimeMillis APPEND slowLogPayload.hashcode
|
||||||
|
* Scan on slowlog table should keep records with sorted order of time, however records
|
||||||
|
* added at the very same time (currentTimeMillis) could be in random order.
|
||||||
|
*
|
||||||
|
* @param slowLogPayload SlowLogPayload to process
|
||||||
|
* @return rowKey byte[]
|
||||||
|
*/
|
||||||
|
private static byte[] getRowKey(final TooSlowLog.SlowLogPayload slowLogPayload) {
|
||||||
|
String hashcode = String.valueOf(slowLogPayload.hashCode());
|
||||||
|
String lastFiveDig =
|
||||||
|
hashcode.substring((hashcode.length() > 5) ? (hashcode.length() - 5) : 0);
|
||||||
|
if (lastFiveDig.startsWith("-")) {
|
||||||
|
lastFiveDig = String.valueOf(RANDOM.nextInt(99999));
|
||||||
|
}
|
||||||
|
final long currentTimeMillis = EnvironmentEdgeManager.currentTime();
|
||||||
|
final String timeAndHashcode = currentTimeMillis + lastFiveDig;
|
||||||
|
final long rowKeyLong = Long.parseLong(timeAndHashcode);
|
||||||
|
return Bytes.toBytes(rowKeyLong);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -1401,6 +1401,22 @@ public final class HConstants {
|
||||||
*/
|
*/
|
||||||
public static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
|
public static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
|
||||||
|
|
||||||
|
public static final int DEFAULT_SLOW_LOG_RING_BUFFER_SIZE = 256;
|
||||||
|
|
||||||
|
public static final String SLOW_LOG_BUFFER_ENABLED_KEY =
|
||||||
|
"hbase.regionserver.slowlog.buffer.enabled";
|
||||||
|
public static final boolean DEFAULT_ONLINE_LOG_PROVIDER_ENABLED = false;
|
||||||
|
|
||||||
|
/** The slowlog info family as a string*/
|
||||||
|
private static final String SLOWLOG_INFO_FAMILY_STR = "info";
|
||||||
|
|
||||||
|
/** The slowlog info family */
|
||||||
|
public static final byte [] SLOWLOG_INFO_FAMILY = Bytes.toBytes(SLOWLOG_INFO_FAMILY_STR);
|
||||||
|
|
||||||
|
public static final String SLOW_LOG_SYS_TABLE_ENABLED_KEY =
|
||||||
|
"hbase.regionserver.slowlog.systable.enabled";
|
||||||
|
public static final boolean DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY = false;
|
||||||
|
|
||||||
private HConstants() {
|
private HConstants() {
|
||||||
// Can't be instantiated with this ctor.
|
// Can't be instantiated with this ctor.
|
||||||
}
|
}
|
||||||
|
|
|
@ -1690,4 +1690,60 @@ possible configurations would overwhelm and obscure the important.
|
||||||
too large batch request.
|
too large batch request.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.regionserver.slowlog.ringbuffer.size</name>
|
||||||
|
<value>256</value>
|
||||||
|
<description>
|
||||||
|
Default size of ringbuffer to be maintained by each RegionServer in order
|
||||||
|
to store online slowlog responses. This is an in-memory ring buffer of
|
||||||
|
requests that were judged to be too slow in addition to the responseTooSlow
|
||||||
|
logging. The in-memory representation would be complete.
|
||||||
|
For more details, please look into Doc Section:
|
||||||
|
Get Slow Response Log from shell
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.regionserver.slowlog.buffer.enabled</name>
|
||||||
|
<value>false</value>
|
||||||
|
<description>
|
||||||
|
Indicates whether RegionServers have ring buffer running for storing
|
||||||
|
Online Slow logs in FIFO manner with limited entries. The size of
|
||||||
|
the ring buffer is indicated by config: hbase.regionserver.slowlog.ringbuffer.size
|
||||||
|
The default value is false, turn this on and get latest slowlog
|
||||||
|
responses with complete data.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.regionserver.slowlog.systable.enabled</name>
|
||||||
|
<value>false</value>
|
||||||
|
<description>
|
||||||
|
Should be enabled only if hbase.regionserver.slowlog.buffer.enabled is enabled. If enabled
|
||||||
|
(true), all slow/large RPC logs would be persisted to system table hbase:slowlog (in addition
|
||||||
|
to in-memory ring buffer at each RegionServer). The records are stored in increasing
|
||||||
|
order of time. Operators can scan the table with various combination of ColumnValueFilter.
|
||||||
|
More details are provided in the doc section:
|
||||||
|
"Get Slow/Large Response Logs from System table hbase:slowlog"
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.namedqueue.provider.classes</name>
|
||||||
|
<value>org.apache.hadoop.hbase.namequeues.impl.SlowLogQueueService,org.apache.hadoop.hbase.namequeues.impl.BalancerDecisionQueueService</value>
|
||||||
|
<description>
|
||||||
|
Default values for NamedQueueService implementors. This comma separated full class names
|
||||||
|
represent all implementors of NamedQueueService that we would like to be invoked by
|
||||||
|
LogEvent handler service. One example of NamedQueue service is SlowLogQueueService which
|
||||||
|
is used to store slow/large RPC logs in ringbuffer at each RegionServer.
|
||||||
|
All implementors of NamedQueueService should be found under package:
|
||||||
|
"org.apache.hadoop.hbase.namequeues.impl"
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.master.balancer.decision.buffer.enabled</name>
|
||||||
|
<value>false</value>
|
||||||
|
<description>
|
||||||
|
Indicates whether active HMaster has ring buffer running for storing
|
||||||
|
balancer decisions in FIFO manner with limited entries. The size of
|
||||||
|
the ring buffer is indicated by config: hbase.master.balancer.decision.queue.size
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
</configuration>
|
</configuration>
|
||||||
|
|
|
@ -191,6 +191,7 @@
|
||||||
<include>Procedure.proto</include>
|
<include>Procedure.proto</include>
|
||||||
<include>Quota.proto</include>
|
<include>Quota.proto</include>
|
||||||
<include>RPC.proto</include>
|
<include>RPC.proto</include>
|
||||||
|
<include>RecentLogs.proto</include>
|
||||||
<include>RegionNormalizer.proto</include>
|
<include>RegionNormalizer.proto</include>
|
||||||
<include>RegionServerStatus.proto</include>
|
<include>RegionServerStatus.proto</include>
|
||||||
<include>RowProcessor.proto</include>
|
<include>RowProcessor.proto</include>
|
||||||
|
@ -201,6 +202,7 @@
|
||||||
<include>Snapshot.proto</include>
|
<include>Snapshot.proto</include>
|
||||||
<include>SnapshotCleanup.proto</include>
|
<include>SnapshotCleanup.proto</include>
|
||||||
<include>Table.proto</include>
|
<include>Table.proto</include>
|
||||||
|
<include>TooSlowLog.proto</include>
|
||||||
<include>Tracing.proto</include>
|
<include>Tracing.proto</include>
|
||||||
<include>VisibilityLabels.proto</include>
|
<include>VisibilityLabels.proto</include>
|
||||||
<include>WAL.proto</include>
|
<include>WAL.proto</include>
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -28,6 +28,7 @@ option optimize_for = SPEED;
|
||||||
import "Client.proto";
|
import "Client.proto";
|
||||||
import "HBase.proto";
|
import "HBase.proto";
|
||||||
import "WAL.proto";
|
import "WAL.proto";
|
||||||
|
import "TooSlowLog.proto";
|
||||||
|
|
||||||
message GetRegionInfoRequest {
|
message GetRegionInfoRequest {
|
||||||
required RegionSpecifier region = 1;
|
required RegionSpecifier region = 1;
|
||||||
|
@ -264,6 +265,52 @@ message UpdateConfigurationRequest {
|
||||||
message UpdateConfigurationResponse {
|
message UpdateConfigurationResponse {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Slow/Large log (LogRequest) use-case specific RPC request. This request payload will be
|
||||||
|
* converted in bytes and sent to generic RPC API: GetLogEntries
|
||||||
|
* LogRequest message has two params:
|
||||||
|
* 1. log_class_name: SlowLogResponseRequest (for Slow/Large log use-case)
|
||||||
|
* 2. log_message: SlowLogResponseRequest converted in bytes (for Slow/Large log use-case)
|
||||||
|
*/
|
||||||
|
message SlowLogResponseRequest {
|
||||||
|
enum FilterByOperator {
|
||||||
|
AND = 0;
|
||||||
|
OR = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
enum LogType {
|
||||||
|
SLOW_LOG = 0;
|
||||||
|
LARGE_LOG = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
optional string region_name = 1;
|
||||||
|
optional string table_name = 2;
|
||||||
|
optional string client_address = 3;
|
||||||
|
optional string user_name = 4;
|
||||||
|
optional uint32 limit = 5 [default = 10];
|
||||||
|
optional FilterByOperator filter_by_operator = 6 [default = OR];
|
||||||
|
optional LogType log_type = 7;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Slow/Large log (LogEntry) use-case specific RPC response. This response payload will be
|
||||||
|
* converted in bytes by servers and sent as response to generic RPC API: GetLogEntries
|
||||||
|
* LogEntry message has two params:
|
||||||
|
* 1. log_class_name: SlowLogResponses (for Slow/Large log use-case)
|
||||||
|
* 2. log_message: SlowLogResponses converted in bytes (for Slow/Large log use-case)
|
||||||
|
*/
|
||||||
|
message SlowLogResponses {
|
||||||
|
repeated SlowLogPayload slow_log_payloads = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
message ClearSlowLogResponseRequest {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
message ClearSlowLogResponses {
|
||||||
|
required bool is_cleaned = 1;
|
||||||
|
}
|
||||||
|
|
||||||
service AdminService {
|
service AdminService {
|
||||||
rpc GetRegionInfo(GetRegionInfoRequest)
|
rpc GetRegionInfo(GetRegionInfoRequest)
|
||||||
returns(GetRegionInfoResponse);
|
returns(GetRegionInfoResponse);
|
||||||
|
@ -318,4 +365,11 @@ service AdminService {
|
||||||
|
|
||||||
rpc UpdateConfiguration(UpdateConfigurationRequest)
|
rpc UpdateConfiguration(UpdateConfigurationRequest)
|
||||||
returns(UpdateConfigurationResponse);
|
returns(UpdateConfigurationResponse);
|
||||||
|
|
||||||
|
rpc ClearSlowLogsResponses(ClearSlowLogResponseRequest)
|
||||||
|
returns(ClearSlowLogResponses);
|
||||||
|
|
||||||
|
rpc GetLogEntries(LogRequest)
|
||||||
|
returns(LogEntry);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -256,6 +256,16 @@ message SnapshotDescription {
|
||||||
optional int64 ttl = 8 [default = 0];
|
optional int64 ttl = 8 [default = 0];
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message LogRequest {
|
||||||
|
required string log_class_name = 1;
|
||||||
|
required bytes log_message = 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
message LogEntry {
|
||||||
|
required string log_class_name = 1;
|
||||||
|
required bytes log_message = 2;
|
||||||
|
}
|
||||||
|
|
||||||
message RegionLocation {
|
message RegionLocation {
|
||||||
required RegionInfo region_info = 1;
|
required RegionInfo region_info = 1;
|
||||||
optional ServerName server_name = 2;
|
optional ServerName server_name = 2;
|
||||||
|
|
|
@ -34,6 +34,7 @@ import "ErrorHandling.proto";
|
||||||
import "Procedure.proto";
|
import "Procedure.proto";
|
||||||
import "Quota.proto";
|
import "Quota.proto";
|
||||||
import "Snapshot.proto";
|
import "Snapshot.proto";
|
||||||
|
import "RecentLogs.proto";
|
||||||
|
|
||||||
/* Column-level protobufs */
|
/* Column-level protobufs */
|
||||||
|
|
||||||
|
@ -595,6 +596,28 @@ message IsSnapshotCleanupEnabledResponse {
|
||||||
required bool enabled = 1;
|
required bool enabled = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* BalancerDecision (LogRequest) use-case specific RPC request. This request payload will be
|
||||||
|
* converted in bytes and sent to generic RPC API: GetLogEntries
|
||||||
|
* LogRequest message has two params:
|
||||||
|
* 1. log_class_name: BalancerDecisionsRequest (for BalancerDecision use-case)
|
||||||
|
* 2. log_message: BalancerDecisionsRequest converted in bytes (for BalancerDecision use-case)
|
||||||
|
*/
|
||||||
|
message BalancerDecisionsRequest {
|
||||||
|
optional uint32 limit = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* BalancerDecision (LogEntry) use-case specific RPC response. This response payload will be
|
||||||
|
* converted in bytes by servers and sent as response to generic RPC API: GetLogEntries
|
||||||
|
* LogEntry message has two params:
|
||||||
|
* 1. log_class_name: BalancerDecisionsResponse (for BalancerDecision use-case)
|
||||||
|
* 2. log_message: BalancerDecisionsResponse converted in bytes (for BalancerDecision use-case)
|
||||||
|
*/
|
||||||
|
message BalancerDecisionsResponse {
|
||||||
|
repeated BalancerDecision balancer_decision = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
service MasterService {
|
service MasterService {
|
||||||
/** Used by the client to get the number of regions that have received the updated schema */
|
/** Used by the client to get the number of regions that have received the updated schema */
|
||||||
|
@ -909,6 +932,9 @@ service MasterService {
|
||||||
rpc IsSnapshotCleanupEnabled (IsSnapshotCleanupEnabledRequest)
|
rpc IsSnapshotCleanupEnabled (IsSnapshotCleanupEnabledRequest)
|
||||||
returns (IsSnapshotCleanupEnabledResponse);
|
returns (IsSnapshotCleanupEnabledResponse);
|
||||||
|
|
||||||
|
rpc GetLogEntries(LogRequest)
|
||||||
|
returns(LogEntry);
|
||||||
|
|
||||||
/** returns table state */
|
/** returns table state */
|
||||||
rpc GetTableState(GetTableStateRequest)
|
rpc GetTableState(GetTableStateRequest)
|
||||||
returns(GetTableStateResponse);
|
returns(GetTableStateResponse);
|
||||||
|
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
// This file contains protocol buffers that are used for Online BalancerDecision history
|
||||||
|
// To be used as Ring Buffer payload
|
||||||
|
package hbase.pb;
|
||||||
|
|
||||||
|
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||||
|
option java_outer_classname = "RecentLogs";
|
||||||
|
option java_generate_equals_and_hash = true;
|
||||||
|
option optimize_for = SPEED;
|
||||||
|
|
||||||
|
message BalancerDecision {
|
||||||
|
|
||||||
|
required string initial_function_costs = 1;
|
||||||
|
required string final_function_costs = 2;
|
||||||
|
required double init_total_cost = 3;
|
||||||
|
required double computed_total_cost = 4;
|
||||||
|
required uint64 computed_steps = 5;
|
||||||
|
repeated string region_plans = 6;
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,54 @@
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
// This file contains protocol buffers that are used for Online TooSlowLogs
|
||||||
|
// To be used as Ring Buffer payload
|
||||||
|
package hbase.pb;
|
||||||
|
|
||||||
|
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||||
|
option java_outer_classname = "TooSlowLog";
|
||||||
|
option java_generate_equals_and_hash = true;
|
||||||
|
option optimize_for = SPEED;
|
||||||
|
|
||||||
|
message SlowLogPayload {
|
||||||
|
required int64 start_time = 1;
|
||||||
|
required int32 processing_time = 2;
|
||||||
|
required int32 queue_time = 3;
|
||||||
|
required int64 response_size = 4;
|
||||||
|
required string client_address = 5;
|
||||||
|
required string server_class = 6;
|
||||||
|
required string method_name = 7;
|
||||||
|
required string call_details = 8;
|
||||||
|
optional string param = 9;
|
||||||
|
required string user_name = 10;
|
||||||
|
optional string region_name = 11;
|
||||||
|
optional int32 multi_gets = 12 [default = 0];
|
||||||
|
optional int32 multi_mutations = 13 [default = 0];
|
||||||
|
optional int32 multi_service_calls = 14 [default = 0];
|
||||||
|
required Type type = 15;
|
||||||
|
|
||||||
|
// SLOW_LOG is RPC call slow in nature whereas LARGE_LOG is RPC call quite large.
|
||||||
|
// Majority of times, slow logs are also large logs and hence, ALL is combination of
|
||||||
|
// both
|
||||||
|
enum Type {
|
||||||
|
SLOW_LOG = 0;
|
||||||
|
LARGE_LOG = 1;
|
||||||
|
ALL = 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -97,6 +97,8 @@ import org.apache.hadoop.hbase.io.ByteBufferInputStream;
|
||||||
import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
|
import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
|
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
|
||||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.VersionInfo;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.VersionInfo;
|
||||||
|
@ -168,6 +170,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
||||||
= new CallQueueTooBigException();
|
= new CallQueueTooBigException();
|
||||||
|
|
||||||
private final boolean authorize;
|
private final boolean authorize;
|
||||||
|
private final boolean isOnlineLogProviderEnabled;
|
||||||
private boolean isSecurityEnabled;
|
private boolean isSecurityEnabled;
|
||||||
|
|
||||||
public static final byte CURRENT_VERSION = 0;
|
public static final byte CURRENT_VERSION = 0;
|
||||||
|
@ -309,6 +312,12 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
||||||
*/
|
*/
|
||||||
private RSRpcServices rsRpcServices;
|
private RSRpcServices rsRpcServices;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Use to add online slowlog responses
|
||||||
|
*/
|
||||||
|
private NamedQueueRecorder namedQueueRecorder;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Datastructure that holds all necessary to a method invocation and then afterward, carries
|
* Datastructure that holds all necessary to a method invocation and then afterward, carries
|
||||||
* the result.
|
* the result.
|
||||||
|
@ -2254,6 +2263,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
||||||
}
|
}
|
||||||
initReconfigurable(conf);
|
initReconfigurable(conf);
|
||||||
|
|
||||||
|
this.isOnlineLogProviderEnabled = conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
|
||||||
|
HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
|
||||||
this.scheduler = scheduler;
|
this.scheduler = scheduler;
|
||||||
this.scheduler.init(new RpcSchedulerContext(this));
|
this.scheduler.init(new RpcSchedulerContext(this));
|
||||||
}
|
}
|
||||||
|
@ -2433,13 +2444,19 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
||||||
boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
|
boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
|
||||||
boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
|
boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
|
||||||
if (tooSlow || tooLarge) {
|
if (tooSlow || tooLarge) {
|
||||||
|
final String userName = call.getRequestUserName() != null ? call.getRequestUserName() : "";
|
||||||
// when tagging, we let TooLarge trump TooSmall to keep output simple
|
// when tagging, we let TooLarge trump TooSmall to keep output simple
|
||||||
// note that large responses will often also be slow.
|
// note that large responses will often also be slow.
|
||||||
logResponse(param,
|
logResponse(param, md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
|
||||||
md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
|
tooLarge, tooSlow, status.getClient(), startTime, processingTime, qTime, responseSize);
|
||||||
(tooLarge ? "TooLarge" : "TooSlow"),
|
if (this.namedQueueRecorder != null && this.isOnlineLogProviderEnabled) {
|
||||||
status.getClient(), startTime, processingTime, qTime,
|
// send logs to ring buffer owned by slowLogRecorder
|
||||||
responseSize);
|
final String className =
|
||||||
|
server == null ? "" : server.getClass().getSimpleName();
|
||||||
|
this.namedQueueRecorder.addRecord(
|
||||||
|
new RpcLogDetails(md, param, status.getClient(), responseSize, className, tooSlow,
|
||||||
|
tooLarge, receiveTime, startTime, userName));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return new Pair<Message, CellScanner>(result, controller.cellScanner());
|
return new Pair<Message, CellScanner>(result, controller.cellScanner());
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
|
@ -2470,18 +2487,18 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
||||||
* @param param The parameters received in the call.
|
* @param param The parameters received in the call.
|
||||||
* @param methodName The name of the method invoked
|
* @param methodName The name of the method invoked
|
||||||
* @param call The string representation of the call
|
* @param call The string representation of the call
|
||||||
* @param tag The tag that will be used to indicate this event in the log.
|
* @param tooLarge To indicate if the event is tooLarge
|
||||||
* @param clientAddress The address of the client who made this call.
|
* @param tooSlow To indicate if the event is tooSlow
|
||||||
* @param startTime The time that the call was initiated, in ms.
|
* @param clientAddress The address of the client who made this call.
|
||||||
* @param processingTime The duration that the call took to run, in ms.
|
* @param startTime The time that the call was initiated, in ms.
|
||||||
* @param qTime The duration that the call spent on the queue
|
* @param processingTime The duration that the call took to run, in ms.
|
||||||
* prior to being initiated, in ms.
|
* @param qTime The duration that the call spent on the queue
|
||||||
* @param responseSize The size in bytes of the response buffer.
|
* prior to being initiated, in ms.
|
||||||
|
* @param responseSize The size in bytes of the response buffer.
|
||||||
*/
|
*/
|
||||||
void logResponse(Message param, String methodName, String call, String tag,
|
void logResponse(Message param, String methodName, String call, boolean tooLarge, boolean tooSlow,
|
||||||
String clientAddress, long startTime, int processingTime, int qTime,
|
String clientAddress, long startTime, int processingTime, int qTime, long responseSize)
|
||||||
long responseSize)
|
throws IOException {
|
||||||
throws IOException {
|
|
||||||
// base information that is reported regardless of type of call
|
// base information that is reported regardless of type of call
|
||||||
Map<String, Object> responseInfo = new HashMap<String, Object>();
|
Map<String, Object> responseInfo = new HashMap<String, Object>();
|
||||||
responseInfo.put("starttimems", startTime);
|
responseInfo.put("starttimems", startTime);
|
||||||
|
@ -2534,6 +2551,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
||||||
responseInfo.put("multi.mutations", numMutations);
|
responseInfo.put("multi.mutations", numMutations);
|
||||||
responseInfo.put("multi.servicecalls", numServiceCalls);
|
responseInfo.put("multi.servicecalls", numServiceCalls);
|
||||||
}
|
}
|
||||||
|
final String tag = (tooLarge && tooSlow) ? "TooLarge & TooSlow"
|
||||||
|
: (tooSlow ? "TooSlow" : "TooLarge");
|
||||||
LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
|
LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2881,4 +2900,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
||||||
public void setRsRpcServices(RSRpcServices rsRpcServices) {
|
public void setRsRpcServices(RSRpcServices rsRpcServices) {
|
||||||
this.rsRpcServices = rsRpcServices;
|
this.rsRpcServices = rsRpcServices;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setNamedQueueRecorder(NamedQueueRecorder namedQueueRecorder) {
|
||||||
|
this.namedQueueRecorder = namedQueueRecorder;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hbase.CellScanner;
|
import org.apache.hadoop.hbase.CellScanner;
|
||||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
|
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
|
||||||
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.security.authorize.PolicyProvider;
|
import org.apache.hadoop.security.authorize.PolicyProvider;
|
||||||
|
@ -88,4 +89,12 @@ public interface RpcServerInterface {
|
||||||
RpcScheduler getScheduler();
|
RpcScheduler getScheduler();
|
||||||
|
|
||||||
void setRsRpcServices(RSRpcServices rsRpcServices);
|
void setRsRpcServices(RSRpcServices rsRpcServices);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set Online SlowLog Provider
|
||||||
|
*
|
||||||
|
* @param namedQueueRecorder instance of {@link NamedQueueRecorder}
|
||||||
|
*/
|
||||||
|
void setNamedQueueRecorder(final NamedQueueRecorder namedQueueRecorder);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -128,6 +128,7 @@ import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
|
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
|
||||||
import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
|
import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
|
||||||
|
import org.apache.hadoop.hbase.master.slowlog.SlowLogMasterService;
|
||||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||||
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
|
@ -945,6 +946,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
status.setStatus("Starting quota manager");
|
status.setStatus("Starting quota manager");
|
||||||
initQuotaManager();
|
initQuotaManager();
|
||||||
|
|
||||||
|
final SlowLogMasterService slowLogMasterService = new SlowLogMasterService(conf, this);
|
||||||
|
slowLogMasterService.init();
|
||||||
|
|
||||||
// assign the meta replicas
|
// assign the meta replicas
|
||||||
Set<ServerName> EMPTY_SET = new HashSet<ServerName>();
|
Set<ServerName> EMPTY_SET = new HashSet<ServerName>();
|
||||||
int numReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
|
int numReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
package org.apache.hadoop.hbase.master;
|
package org.apache.hadoop.hbase.master;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.lang.reflect.InvocationTargetException;
|
||||||
|
import java.lang.reflect.Method;
|
||||||
import java.net.InetAddress;
|
import java.net.InetAddress;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
@ -53,6 +55,10 @@ import org.apache.hadoop.hbase.ipc.QosPriority;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
|
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.BalancerDecisionDetails;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
|
import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
|
||||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
|
@ -288,6 +294,51 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HBaseProtos.LogEntry getLogEntries(RpcController controller,
|
||||||
|
HBaseProtos.LogRequest request) throws ServiceException {
|
||||||
|
try {
|
||||||
|
final String logClassName = request.getLogClassName();
|
||||||
|
Class<?> logClass = Class.forName(logClassName)
|
||||||
|
.asSubclass(Message.class);
|
||||||
|
Method method = logClass.getMethod("parseFrom", ByteString.class);
|
||||||
|
if (logClassName.contains("BalancerDecisionsRequest")) {
|
||||||
|
MasterProtos.BalancerDecisionsRequest balancerDecisionsRequest =
|
||||||
|
(MasterProtos.BalancerDecisionsRequest) method
|
||||||
|
.invoke(null, request.getLogMessage());
|
||||||
|
MasterProtos.BalancerDecisionsResponse balancerDecisionsResponse =
|
||||||
|
getBalancerDecisions(balancerDecisionsRequest);
|
||||||
|
return HBaseProtos.LogEntry.newBuilder()
|
||||||
|
.setLogClassName(balancerDecisionsResponse.getClass().getName())
|
||||||
|
.setLogMessage(balancerDecisionsResponse.toByteString())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
|
||||||
|
| InvocationTargetException e) {
|
||||||
|
LOG.error("Error while retrieving log entries.", e);
|
||||||
|
throw new ServiceException(e);
|
||||||
|
}
|
||||||
|
throw new ServiceException("Invalid request params");
|
||||||
|
}
|
||||||
|
|
||||||
|
private MasterProtos.BalancerDecisionsResponse getBalancerDecisions(
|
||||||
|
MasterProtos.BalancerDecisionsRequest request) {
|
||||||
|
final NamedQueueRecorder namedQueueRecorder = this.regionServer.getNamedQueueRecorder();
|
||||||
|
if (namedQueueRecorder == null) {
|
||||||
|
return MasterProtos.BalancerDecisionsResponse.newBuilder()
|
||||||
|
.addAllBalancerDecision(new ArrayList<RecentLogs.BalancerDecision>()).build();
|
||||||
|
}
|
||||||
|
final NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
|
||||||
|
namedQueueGetRequest.setNamedQueueEvent(BalancerDecisionDetails.BALANCER_DECISION_EVENT);
|
||||||
|
namedQueueGetRequest.setBalancerDecisionsRequest(request);
|
||||||
|
NamedQueueGetResponse namedQueueGetResponse =
|
||||||
|
namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
|
||||||
|
List<RecentLogs.BalancerDecision> balancerDecisions =
|
||||||
|
namedQueueGetResponse.getBalancerDecisions();
|
||||||
|
return MasterProtos.BalancerDecisionsResponse.newBuilder()
|
||||||
|
.addAllBalancerDecision(balancerDecisions).build();
|
||||||
|
}
|
||||||
|
|
||||||
enum BalanceSwitchMode {
|
enum BalanceSwitchMode {
|
||||||
SYNC,
|
SYNC,
|
||||||
ASYNC
|
ASYNC
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
|
||||||
import org.apache.hadoop.hbase.master.RackManager;
|
import org.apache.hadoop.hbase.master.RackManager;
|
||||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||||
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action.Type;
|
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action.Type;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
@ -67,6 +68,11 @@ import com.google.common.collect.Sets;
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
public abstract class BaseLoadBalancer implements LoadBalancer {
|
public abstract class BaseLoadBalancer implements LoadBalancer {
|
||||||
|
|
||||||
|
public static final String BALANCER_DECISION_BUFFER_ENABLED =
|
||||||
|
"hbase.master.balancer.decision.buffer.enabled";
|
||||||
|
public static final boolean DEFAULT_BALANCER_DECISION_BUFFER_ENABLED = false;
|
||||||
|
|
||||||
protected static final int MIN_SERVER_BALANCE = 2;
|
protected static final int MIN_SERVER_BALANCE = 2;
|
||||||
private volatile boolean stopped = false;
|
private volatile boolean stopped = false;
|
||||||
|
|
||||||
|
@ -75,6 +81,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
|
||||||
protected RegionLocationFinder regionFinder;
|
protected RegionLocationFinder regionFinder;
|
||||||
protected boolean useRegionFinder;
|
protected boolean useRegionFinder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Use to add balancer decision history to ring-buffer
|
||||||
|
*/
|
||||||
|
protected NamedQueueRecorder namedQueueRecorder;
|
||||||
|
|
||||||
private static class DefaultRackManager extends RackManager {
|
private static class DefaultRackManager extends RackManager {
|
||||||
@Override
|
@Override
|
||||||
public String getRack(ServerName server) {
|
public String getRack(ServerName server) {
|
||||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.RegionLoad;
|
||||||
import org.apache.hadoop.hbase.ServerLoad;
|
import org.apache.hadoop.hbase.ServerLoad;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.client.BalancerDecision;
|
||||||
import org.apache.hadoop.hbase.master.MasterServices;
|
import org.apache.hadoop.hbase.master.MasterServices;
|
||||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||||
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
|
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
|
||||||
|
@ -51,6 +52,8 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.AssignRe
|
||||||
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.LocalityType;
|
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.LocalityType;
|
||||||
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.MoveRegionAction;
|
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.MoveRegionAction;
|
||||||
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegionsAction;
|
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegionsAction;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.BalancerDecisionDetails;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
|
||||||
import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
|
import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||||
|
@ -232,6 +235,13 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
|
||||||
curFunctionCosts = new Double[costFunctions.size()];
|
curFunctionCosts = new Double[costFunctions.size()];
|
||||||
tempFunctionCosts = new Double[costFunctions.size()];
|
tempFunctionCosts = new Double[costFunctions.size()];
|
||||||
|
|
||||||
|
boolean isBalancerDecisionRecording = getConf()
|
||||||
|
.getBoolean(BaseLoadBalancer.BALANCER_DECISION_BUFFER_ENABLED,
|
||||||
|
BaseLoadBalancer.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED);
|
||||||
|
if (this.namedQueueRecorder == null && isBalancerDecisionRecording) {
|
||||||
|
this.namedQueueRecorder = NamedQueueRecorder.getInstance(getConf());
|
||||||
|
}
|
||||||
|
|
||||||
LOG.info("Loaded config; maxSteps=" + maxSteps + ", stepsPerRegion=" + stepsPerRegion +
|
LOG.info("Loaded config; maxSteps=" + maxSteps + ", stepsPerRegion=" + stepsPerRegion +
|
||||||
", maxRunningTime=" + maxRunningTime + ", isByTable=" + isByTable + ", etc." +
|
", maxRunningTime=" + maxRunningTime + ", isByTable=" + isByTable + ", etc." +
|
||||||
", maxRunningTime=" + maxRunningTime + ", isByTable=" + isByTable + ", CostFunctions=" +
|
", maxRunningTime=" + maxRunningTime + ", isByTable=" + isByTable + ", CostFunctions=" +
|
||||||
|
@ -430,7 +440,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
|
||||||
System.arraycopy(tempFunctionCosts, 0, curFunctionCosts, 0, this.curFunctionCosts.length);
|
System.arraycopy(tempFunctionCosts, 0, curFunctionCosts, 0, this.curFunctionCosts.length);
|
||||||
|
|
||||||
double initCost = currentCost;
|
double initCost = currentCost;
|
||||||
double newCost = currentCost;
|
double newCost;
|
||||||
|
|
||||||
long computedMaxSteps = 0;
|
long computedMaxSteps = 0;
|
||||||
if (runMaxSteps) {
|
if (runMaxSteps) {
|
||||||
|
@ -452,6 +462,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
|
||||||
LOG.info("start StochasticLoadBalancer.balancer, initCost=" + currentCost + ", functionCost="
|
LOG.info("start StochasticLoadBalancer.balancer, initCost=" + currentCost + ", functionCost="
|
||||||
+ functionCost() + " computedMaxSteps: " + computedMaxSteps);
|
+ functionCost() + " computedMaxSteps: " + computedMaxSteps);
|
||||||
|
|
||||||
|
final String initFunctionTotalCosts = totalCostsPerFunc();
|
||||||
// Perform a stochastic walk to see if we can get a good fit.
|
// Perform a stochastic walk to see if we can get a good fit.
|
||||||
long step;
|
long step;
|
||||||
|
|
||||||
|
@ -502,7 +513,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
|
||||||
+ plans.size() + " regions; Going from a computed cost of "
|
+ plans.size() + " regions; Going from a computed cost of "
|
||||||
+ initCost + " to a new cost of " + currentCost);
|
+ initCost + " to a new cost of " + currentCost);
|
||||||
}
|
}
|
||||||
|
sendRegionPlansToRingBuffer(plans, currentCost, initCost, initFunctionTotalCosts, step);
|
||||||
return plans;
|
return plans;
|
||||||
}
|
}
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
|
@ -513,6 +524,27 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void sendRegionPlansToRingBuffer(List<RegionPlan> plans, double currentCost,
|
||||||
|
double initCost, String initFunctionTotalCosts, long step) {
|
||||||
|
if (this.namedQueueRecorder != null) {
|
||||||
|
List<String> regionPlans = new ArrayList<>();
|
||||||
|
for (RegionPlan plan : plans) {
|
||||||
|
regionPlans.add(
|
||||||
|
"table: " + plan.getRegionInfo().getTable() + " , region: " + plan.getRegionName()
|
||||||
|
+ " , source: " + plan.getSource() + " , destination: " + plan.getDestination());
|
||||||
|
}
|
||||||
|
BalancerDecision balancerDecision =
|
||||||
|
new BalancerDecision.Builder()
|
||||||
|
.setInitTotalCost(initCost)
|
||||||
|
.setInitialFunctionCosts(initFunctionTotalCosts)
|
||||||
|
.setComputedTotalCost(currentCost)
|
||||||
|
.setFinalFunctionCosts(totalCostsPerFunc())
|
||||||
|
.setComputedSteps(step)
|
||||||
|
.setRegionPlans(regionPlans).build();
|
||||||
|
namedQueueRecorder.addRecord(new BalancerDecisionDetails(balancerDecision));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* update costs to JMX
|
* update costs to JMX
|
||||||
*/
|
*/
|
||||||
|
@ -557,6 +589,23 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
|
||||||
return builder.toString();
|
return builder.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private String totalCostsPerFunc() {
|
||||||
|
StringBuilder builder = new StringBuilder();
|
||||||
|
for (CostFunction c : costFunctions) {
|
||||||
|
if (c.getMultiplier() * c.cost() > 0.0) {
|
||||||
|
builder.append(" ");
|
||||||
|
builder.append(c.getClass().getSimpleName());
|
||||||
|
builder.append(" : ");
|
||||||
|
builder.append(c.getMultiplier() * c.cost());
|
||||||
|
builder.append(";");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (builder.length() > 0) {
|
||||||
|
builder.deleteCharAt(builder.length() - 1);
|
||||||
|
}
|
||||||
|
return builder.toString();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create all of the RegionPlan's needed to move from the initial cluster state to the desired
|
* Create all of the RegionPlan's needed to move from the initial cluster state to the desired
|
||||||
* state.
|
* state.
|
||||||
|
|
|
@ -0,0 +1,73 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.master.slowlog;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.master.MasterServices;
|
||||||
|
import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Slowlog Master services - Table creation to be used by HMaster
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class SlowLogMasterService {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(SlowLogMasterService.class);
|
||||||
|
|
||||||
|
private final boolean slowlogTableEnabled;
|
||||||
|
private final MasterServices masterServices;
|
||||||
|
|
||||||
|
private static final HTableDescriptor TABLE_DESCRIPTOR_BUILDER =
|
||||||
|
new HTableDescriptor(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME).setRegionReplication(1);
|
||||||
|
|
||||||
|
static {
|
||||||
|
TABLE_DESCRIPTOR_BUILDER.addFamily(new HColumnDescriptor(HConstants.SLOWLOG_INFO_FAMILY)
|
||||||
|
.setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBlockCacheEnabled(false).setMaxVersions(1));
|
||||||
|
}
|
||||||
|
|
||||||
|
public SlowLogMasterService(final Configuration configuration,
|
||||||
|
final MasterServices masterServices) {
|
||||||
|
slowlogTableEnabled = configuration.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,
|
||||||
|
HConstants.DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY);
|
||||||
|
this.masterServices = masterServices;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void init() throws IOException {
|
||||||
|
if (!slowlogTableEnabled) {
|
||||||
|
LOG.info("Slow/Large requests logging to system table hbase:slowlog is disabled. Quitting.");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (!MetaTableAccessor
|
||||||
|
.tableExists(masterServices.getConnection(), SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) {
|
||||||
|
LOG.info("slowlog table not found. Creating.");
|
||||||
|
this.masterServices.createSystemTable(TABLE_DESCRIPTOR_BUILDER);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,51 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.lang.builder.ToStringBuilder;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.client.BalancerDecision;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Balancer decision details that would be passed on to ring buffer for history
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class BalancerDecisionDetails extends NamedQueuePayload {
|
||||||
|
|
||||||
|
public static final int BALANCER_DECISION_EVENT = 1;
|
||||||
|
|
||||||
|
private final BalancerDecision balancerDecision;
|
||||||
|
|
||||||
|
public BalancerDecisionDetails(BalancerDecision balancerDecision) {
|
||||||
|
super(BALANCER_DECISION_EVENT);
|
||||||
|
this.balancerDecision = balancerDecision;
|
||||||
|
}
|
||||||
|
|
||||||
|
public BalancerDecision getBalancerDecision() {
|
||||||
|
return balancerDecision;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new ToStringBuilder(this)
|
||||||
|
.append("balancerDecision", balancerDecision)
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,50 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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 com.lmax.disruptor.ExceptionHandler;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Exception Handler for Online Slow Log Ring Buffer
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class DisruptorExceptionHandler implements ExceptionHandler {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(DisruptorExceptionHandler.class);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void handleEventException(Throwable e, long sequence, Object event) {
|
||||||
|
LOG.error("Sequence={}, event={}", sequence, event, e);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void handleOnStartException(Throwable e) {
|
||||||
|
LOG.error("Disruptor onStartException: ", e);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void handleOnShutdownException(Throwable e) {
|
||||||
|
LOG.error("Disruptor onShutdownException: ", e);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,128 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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 com.lmax.disruptor.EventHandler;
|
||||||
|
import com.lmax.disruptor.RingBuffer;
|
||||||
|
import java.lang.reflect.InvocationTargetException;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Event Handler run by disruptor ringbuffer consumer.
|
||||||
|
* Although this is generic implementation for namedQueue, it can have individual queue specific
|
||||||
|
* logic.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class LogEventHandler implements EventHandler<RingBufferEnvelope> {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(LogEventHandler.class);
|
||||||
|
|
||||||
|
// Map that binds namedQueues to corresponding queue service implementation.
|
||||||
|
// If NamedQueue of specific type is enabled, corresponding service will be used to
|
||||||
|
// insert and retrieve records.
|
||||||
|
// Individual queue sizes should be determined based on their individual configs within
|
||||||
|
// each service.
|
||||||
|
private final Map<NamedQueuePayload.NamedQueueEvent, NamedQueueService> namedQueueServices =
|
||||||
|
new HashMap<>();
|
||||||
|
|
||||||
|
private static final String NAMED_QUEUE_PROVIDER_CLASSES = "hbase.namedqueue.provider.classes";
|
||||||
|
|
||||||
|
LogEventHandler(final Configuration conf) {
|
||||||
|
for (String implName : conf.getStringCollection(NAMED_QUEUE_PROVIDER_CLASSES)) {
|
||||||
|
Class<?> clz;
|
||||||
|
try {
|
||||||
|
clz = Class.forName(implName);
|
||||||
|
} catch (ClassNotFoundException e) {
|
||||||
|
LOG.warn("Failed to find NamedQueueService implementor class {}", implName, e);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!NamedQueueService.class.isAssignableFrom(clz)) {
|
||||||
|
LOG.warn("Class {} is not implementor of NamedQueueService.", clz);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
// add all service mappings here
|
||||||
|
try {
|
||||||
|
NamedQueueService namedQueueService =
|
||||||
|
(NamedQueueService) clz.getConstructor(Configuration.class).newInstance(conf);
|
||||||
|
namedQueueServices.put(namedQueueService.getEvent(), namedQueueService);
|
||||||
|
} catch (InstantiationException | IllegalAccessException | NoSuchMethodException
|
||||||
|
| InvocationTargetException e) {
|
||||||
|
LOG.warn("Unable to instantiate/add NamedQueueService implementor {} to service map.",
|
||||||
|
clz);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Called when a publisher has published an event to the {@link RingBuffer}.
|
||||||
|
* This is generic consumer of disruptor ringbuffer and for each new namedQueue that we
|
||||||
|
* add, we should also provide specific consumer logic here.
|
||||||
|
*
|
||||||
|
* @param event published to the {@link RingBuffer}
|
||||||
|
* @param sequence of the event being processed
|
||||||
|
* @param endOfBatch flag to indicate if this is the last event in a batch from
|
||||||
|
* the {@link RingBuffer}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void onEvent(RingBufferEnvelope event, long sequence, boolean endOfBatch) {
|
||||||
|
final NamedQueuePayload namedQueuePayload = event.getPayload();
|
||||||
|
// consume ringbuffer payload based on event type
|
||||||
|
namedQueueServices.get(namedQueuePayload.getNamedQueueEvent())
|
||||||
|
.consumeEventFromDisruptor(namedQueuePayload);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Cleans up queues maintained by services.
|
||||||
|
*
|
||||||
|
* @param namedQueueEvent type of queue to clear
|
||||||
|
* @return true if queue is cleaned up, false otherwise
|
||||||
|
*/
|
||||||
|
boolean clearNamedQueue(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
|
||||||
|
return namedQueueServices.get(namedQueueEvent).clearNamedQueue();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add all in memory queue records to system table. The implementors can use system table
|
||||||
|
* or direct HDFS file or ZK as persistence system.
|
||||||
|
*/
|
||||||
|
void persistAll(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
|
||||||
|
namedQueueServices.get(namedQueueEvent).persistAll();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve in memory queue records from ringbuffer
|
||||||
|
*
|
||||||
|
* @param request namedQueue request with event type
|
||||||
|
* @return queue records from ringbuffer after filter (if applied)
|
||||||
|
*/
|
||||||
|
NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
|
||||||
|
return namedQueueServices.get(request.getNamedQueueEvent()).getNamedQueueRecords(request);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,107 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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 java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.commons.lang.StringUtils;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Event Handler utility class
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public final class LogHandlerUtils {
|
||||||
|
|
||||||
|
private LogHandlerUtils() {
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int getTotalFiltersCount(AdminProtos.SlowLogResponseRequest request) {
|
||||||
|
int totalFilters = 0;
|
||||||
|
if (StringUtils.isNotEmpty(request.getRegionName())) {
|
||||||
|
totalFilters++;
|
||||||
|
}
|
||||||
|
if (StringUtils.isNotEmpty(request.getTableName())) {
|
||||||
|
totalFilters++;
|
||||||
|
}
|
||||||
|
if (StringUtils.isNotEmpty(request.getClientAddress())) {
|
||||||
|
totalFilters++;
|
||||||
|
}
|
||||||
|
if (StringUtils.isNotEmpty(request.getUserName())) {
|
||||||
|
totalFilters++;
|
||||||
|
}
|
||||||
|
return totalFilters;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static List<TooSlowLog.SlowLogPayload> filterLogs(
|
||||||
|
AdminProtos.SlowLogResponseRequest request,
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloadList, int totalFilters) {
|
||||||
|
List<TooSlowLog.SlowLogPayload> filteredSlowLogPayloads = new ArrayList<>();
|
||||||
|
final String regionName =
|
||||||
|
StringUtils.isNotEmpty(request.getRegionName()) ? request.getRegionName() : null;
|
||||||
|
final String tableName =
|
||||||
|
StringUtils.isNotEmpty(request.getTableName()) ? request.getTableName() : null;
|
||||||
|
final String clientAddress =
|
||||||
|
StringUtils.isNotEmpty(request.getClientAddress()) ? request.getClientAddress() : null;
|
||||||
|
final String userName =
|
||||||
|
StringUtils.isNotEmpty(request.getUserName()) ? request.getUserName() : null;
|
||||||
|
for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloadList) {
|
||||||
|
int totalFilterMatches = 0;
|
||||||
|
if (slowLogPayload.getRegionName().equals(regionName)) {
|
||||||
|
totalFilterMatches++;
|
||||||
|
}
|
||||||
|
if (tableName != null && slowLogPayload.getRegionName().startsWith(tableName)) {
|
||||||
|
totalFilterMatches++;
|
||||||
|
}
|
||||||
|
if (slowLogPayload.getClientAddress().equals(clientAddress)) {
|
||||||
|
totalFilterMatches++;
|
||||||
|
}
|
||||||
|
if (slowLogPayload.getUserName().equals(userName)) {
|
||||||
|
totalFilterMatches++;
|
||||||
|
}
|
||||||
|
if (request.hasFilterByOperator() && request.getFilterByOperator()
|
||||||
|
.equals(AdminProtos.SlowLogResponseRequest.FilterByOperator.AND)) {
|
||||||
|
// Filter by AND operator
|
||||||
|
if (totalFilterMatches == totalFilters) {
|
||||||
|
filteredSlowLogPayloads.add(slowLogPayload);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Filter by OR operator
|
||||||
|
if (totalFilterMatches > 0) {
|
||||||
|
filteredSlowLogPayloads.add(slowLogPayload);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return filteredSlowLogPayloads;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static List<TooSlowLog.SlowLogPayload> getFilteredLogs(
|
||||||
|
AdminProtos.SlowLogResponseRequest request, List<TooSlowLog.SlowLogPayload> logPayloadList) {
|
||||||
|
int totalFilters = getTotalFiltersCount(request);
|
||||||
|
if (totalFilters > 0) {
|
||||||
|
logPayloadList = filterLogs(request, logPayloadList, totalFilters);
|
||||||
|
}
|
||||||
|
int limit = Math.min(request.getLimit(), logPayloadList.size());
|
||||||
|
return logPayloadList.subList(0, limit);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,72 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base payload to be prepared by client to send various namedQueue events for in-memory
|
||||||
|
* ring buffer storage in either HMaster or RegionServer.
|
||||||
|
* e.g slowLog responses
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class NamedQueuePayload {
|
||||||
|
|
||||||
|
public enum NamedQueueEvent {
|
||||||
|
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(
|
||||||
|
"NamedQueue event with ordinal " + value + " not defined");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getValue() {
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private final NamedQueueEvent namedQueueEvent;
|
||||||
|
|
||||||
|
public NamedQueuePayload(int eventOrdinal) {
|
||||||
|
this.namedQueueEvent = NamedQueueEvent.getEventByOrdinal(eventOrdinal);
|
||||||
|
}
|
||||||
|
|
||||||
|
public NamedQueueEvent getNamedQueueEvent() {
|
||||||
|
return namedQueueEvent;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,164 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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 com.google.common.base.Preconditions;
|
||||||
|
import com.lmax.disruptor.BlockingWaitStrategy;
|
||||||
|
import com.lmax.disruptor.EventFactory;
|
||||||
|
import com.lmax.disruptor.RingBuffer;
|
||||||
|
import com.lmax.disruptor.dsl.Disruptor;
|
||||||
|
import com.lmax.disruptor.dsl.ProducerType;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* NamedQueue recorder that maintains various named queues.
|
||||||
|
* The service uses LMAX Disruptor to save queue records which are then consumed by
|
||||||
|
* a queue and based on the ring buffer size, the available records are then fetched
|
||||||
|
* from the queue in thread-safe manner.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
public final class NamedQueueRecorder {
|
||||||
|
|
||||||
|
private final Disruptor<RingBufferEnvelope> disruptor;
|
||||||
|
private final LogEventHandler logEventHandler;
|
||||||
|
|
||||||
|
private static NamedQueueRecorder namedQueueRecorder;
|
||||||
|
private static boolean isInit = false;
|
||||||
|
private static final Object LOCK = new Object();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initialize disruptor with configurable ringbuffer size
|
||||||
|
*/
|
||||||
|
private NamedQueueRecorder(Configuration conf) {
|
||||||
|
|
||||||
|
// This is the 'writer' -- a single threaded executor. This single thread consumes what is
|
||||||
|
// put on the ringbuffer.
|
||||||
|
final String hostingThreadName = Thread.currentThread().getName();
|
||||||
|
|
||||||
|
int eventCount = conf.getInt("hbase.namedqueue.ringbuffer.size", 1024);
|
||||||
|
|
||||||
|
// disruptor initialization with BlockingWaitStrategy
|
||||||
|
this.disruptor = new Disruptor<>(getEventFactory(), getEventCount(eventCount), Executors.
|
||||||
|
newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName
|
||||||
|
+ ".slowlog.append-pool")),
|
||||||
|
ProducerType.MULTI, new BlockingWaitStrategy());
|
||||||
|
this.disruptor.handleExceptionsWith(new DisruptorExceptionHandler());
|
||||||
|
|
||||||
|
// initialize ringbuffer event handler
|
||||||
|
this.logEventHandler = new LogEventHandler(conf);
|
||||||
|
this.disruptor.handleEventsWith(new LogEventHandler[]{this.logEventHandler});
|
||||||
|
this.disruptor.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
private EventFactory<RingBufferEnvelope> getEventFactory() {
|
||||||
|
return new EventFactory<RingBufferEnvelope>() {
|
||||||
|
@Override
|
||||||
|
public RingBufferEnvelope newInstance() {
|
||||||
|
return new RingBufferEnvelope();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public static NamedQueueRecorder getInstance(Configuration conf) {
|
||||||
|
if (namedQueueRecorder != null) {
|
||||||
|
return namedQueueRecorder;
|
||||||
|
}
|
||||||
|
synchronized (LOCK) {
|
||||||
|
if (!isInit) {
|
||||||
|
namedQueueRecorder = new NamedQueueRecorder(conf);
|
||||||
|
isInit = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return namedQueueRecorder;
|
||||||
|
}
|
||||||
|
|
||||||
|
// must be power of 2 for disruptor ringbuffer
|
||||||
|
private int getEventCount(int eventCount) {
|
||||||
|
Preconditions.checkArgument(eventCount >= 0, "hbase.namedqueue.ringbuffer.size must be > 0");
|
||||||
|
int floor = Integer.highestOneBit(eventCount);
|
||||||
|
if (floor == eventCount) {
|
||||||
|
return floor;
|
||||||
|
}
|
||||||
|
// max capacity is 1 << 30
|
||||||
|
if (floor >= 1 << 29) {
|
||||||
|
return 1 << 30;
|
||||||
|
}
|
||||||
|
return floor << 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve in memory queue records from ringbuffer
|
||||||
|
*
|
||||||
|
* @param request namedQueue request with event type
|
||||||
|
* @return queue records from ringbuffer after filter (if applied)
|
||||||
|
*/
|
||||||
|
public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
|
||||||
|
return this.logEventHandler.getNamedQueueRecords(request);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* clears queue records from ringbuffer
|
||||||
|
*
|
||||||
|
* @param namedQueueEvent type of queue to clear
|
||||||
|
* @return true if slow log payloads are cleaned up or
|
||||||
|
* hbase.regionserver.slowlog.buffer.enabled is not set to true, false if failed to
|
||||||
|
* clean up slow logs
|
||||||
|
*/
|
||||||
|
public boolean clearNamedQueue(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
|
||||||
|
return this.logEventHandler.clearNamedQueue(namedQueueEvent);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add various NamedQueue records to ringbuffer. Based on the type of the event (e.g slowLog),
|
||||||
|
* consumer of disruptor ringbuffer will have specific logic.
|
||||||
|
* This method is producer of disruptor ringbuffer which is initialized in NamedQueueRecorder
|
||||||
|
* constructor.
|
||||||
|
*
|
||||||
|
* @param namedQueuePayload namedQueue payload sent by client of ring buffer
|
||||||
|
* service
|
||||||
|
*/
|
||||||
|
public void addRecord(NamedQueuePayload namedQueuePayload) {
|
||||||
|
RingBuffer<RingBufferEnvelope> ringBuffer = this.disruptor.getRingBuffer();
|
||||||
|
long seqId = ringBuffer.next();
|
||||||
|
try {
|
||||||
|
ringBuffer.get(seqId).load(namedQueuePayload);
|
||||||
|
} finally {
|
||||||
|
ringBuffer.publish(seqId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add all in memory queue records to system table. The implementors can use system table
|
||||||
|
* or direct HDFS file or ZK as persistence system.
|
||||||
|
*/
|
||||||
|
public void persistAll(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
|
||||||
|
if (this.logEventHandler != null) {
|
||||||
|
this.logEventHandler.persistAll(namedQueueEvent);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,69 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* In-memory Queue service provider for multiple use-cases. Implementers should be
|
||||||
|
* registered in LogEventHandler
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public interface NamedQueueService {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve event type for NamedQueueService implementation.
|
||||||
|
*
|
||||||
|
* @return {@link NamedQueuePayload.NamedQueueEvent}
|
||||||
|
*/
|
||||||
|
NamedQueuePayload.NamedQueueEvent getEvent();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This implementation is generic for consuming records from LMAX
|
||||||
|
* disruptor and inserts records to EvictingQueue which is maintained by each
|
||||||
|
* ringbuffer provider.
|
||||||
|
*
|
||||||
|
* @param namedQueuePayload namedQueue payload from disruptor ring buffer
|
||||||
|
*/
|
||||||
|
void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Cleans up queues maintained by services.
|
||||||
|
*
|
||||||
|
* @return true if slow log payloads are cleaned up, false otherwise
|
||||||
|
*/
|
||||||
|
boolean clearNamedQueue();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve in memory queue records from ringbuffer
|
||||||
|
*
|
||||||
|
* @param request namedQueue request with event type
|
||||||
|
* @return queue records from ringbuffer after filter (if applied)
|
||||||
|
*/
|
||||||
|
NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add all in memory queue records to system table. The implementors can use system table
|
||||||
|
* or direct HDFS file or ZK as persistence system.
|
||||||
|
*/
|
||||||
|
void persistAll();
|
||||||
|
}
|
|
@ -0,0 +1,56 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An envelope to carry payload in the ring buffer that serves as online buffer
|
||||||
|
* to provide latest events
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
final class RingBufferEnvelope {
|
||||||
|
|
||||||
|
private NamedQueuePayload namedQueuePayload;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Load the Envelope with NamedQueuePayload
|
||||||
|
*
|
||||||
|
* @param namedQueuePayload all details of rpc call that would be useful for ring buffer
|
||||||
|
* consumers
|
||||||
|
*/
|
||||||
|
public void load(NamedQueuePayload namedQueuePayload) {
|
||||||
|
this.namedQueuePayload = namedQueuePayload;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve current namedQueue payload {@link NamedQueuePayload} available on Envelope and
|
||||||
|
* free up the Envelope
|
||||||
|
*
|
||||||
|
* @return Retrieve rpc log details
|
||||||
|
*/
|
||||||
|
public NamedQueuePayload getPayload() {
|
||||||
|
final NamedQueuePayload namedQueuePayload = this.namedQueuePayload;
|
||||||
|
this.namedQueuePayload = null;
|
||||||
|
return namedQueuePayload;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,117 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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 com.google.protobuf.Descriptors;
|
||||||
|
import com.google.protobuf.Message;
|
||||||
|
import org.apache.commons.lang.builder.ToStringBuilder;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RpcCall details that would be passed on to ring buffer of slow log responses
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class RpcLogDetails extends NamedQueuePayload {
|
||||||
|
|
||||||
|
public static final int SLOW_LOG_EVENT = 0;
|
||||||
|
|
||||||
|
private final Descriptors.MethodDescriptor methodDescriptor;
|
||||||
|
private final Message param;
|
||||||
|
private final String clientAddress;
|
||||||
|
private final long responseSize;
|
||||||
|
private final String className;
|
||||||
|
private final boolean isSlowLog;
|
||||||
|
private final boolean isLargeLog;
|
||||||
|
private final long receiveTime;
|
||||||
|
private final long startTime;
|
||||||
|
private final String userName;
|
||||||
|
|
||||||
|
public RpcLogDetails(Descriptors.MethodDescriptor methodDescriptor, Message param,
|
||||||
|
String clientAddress, long responseSize, String className, boolean isSlowLog,
|
||||||
|
boolean isLargeLog, long receiveTime, long startTime, String userName) {
|
||||||
|
super(SLOW_LOG_EVENT);
|
||||||
|
this.methodDescriptor = methodDescriptor;
|
||||||
|
this.param = param;
|
||||||
|
this.clientAddress = clientAddress;
|
||||||
|
this.responseSize = responseSize;
|
||||||
|
this.className = className;
|
||||||
|
this.isSlowLog = isSlowLog;
|
||||||
|
this.isLargeLog = isLargeLog;
|
||||||
|
this.receiveTime = receiveTime;
|
||||||
|
this.startTime = startTime;
|
||||||
|
this.userName = userName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Descriptors.MethodDescriptor getMethodDescriptor() {
|
||||||
|
return methodDescriptor;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getClientAddress() {
|
||||||
|
return clientAddress;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getResponseSize() {
|
||||||
|
return responseSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getClassName() {
|
||||||
|
return className;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isSlowLog() {
|
||||||
|
return isSlowLog;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isLargeLog() {
|
||||||
|
return isLargeLog;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Message getParam() {
|
||||||
|
return param;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getReceiveTime() {
|
||||||
|
return receiveTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getStartTime() {
|
||||||
|
return startTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getUserName() {
|
||||||
|
return userName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new ToStringBuilder(this)
|
||||||
|
.append("methodDescriptor", methodDescriptor)
|
||||||
|
.append("param", param)
|
||||||
|
.append("clientAddress", clientAddress)
|
||||||
|
.append("responseSize", responseSize)
|
||||||
|
.append("className", className)
|
||||||
|
.append("isSlowLog", isSlowLog)
|
||||||
|
.append("isLargeLog", isLargeLog)
|
||||||
|
.append("receiveTime", receiveTime)
|
||||||
|
.append("startTime", startTime)
|
||||||
|
.append("userName", userName)
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,94 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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 java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.queue.EvictingQueue;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
|
||||||
|
import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Persistent service provider for Slow/LargeLog events
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class SlowLogPersistentService {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(SlowLogPersistentService.class);
|
||||||
|
|
||||||
|
private static final ReentrantLock LOCK = new ReentrantLock();
|
||||||
|
private static final String SYS_TABLE_QUEUE_SIZE =
|
||||||
|
"hbase.regionserver.slowlog.systable.queue.size";
|
||||||
|
private static final int DEFAULT_SYS_TABLE_QUEUE_SIZE = 1000;
|
||||||
|
private static final int SYSTABLE_PUT_BATCH_SIZE = 100;
|
||||||
|
|
||||||
|
private final EvictingQueue<TooSlowLog.SlowLogPayload> queueForSysTable;
|
||||||
|
|
||||||
|
private final Configuration configuration;
|
||||||
|
|
||||||
|
public SlowLogPersistentService(final Configuration configuration) {
|
||||||
|
this.configuration = configuration;
|
||||||
|
int sysTableQueueSize =
|
||||||
|
configuration.getInt(SYS_TABLE_QUEUE_SIZE, DEFAULT_SYS_TABLE_QUEUE_SIZE);
|
||||||
|
queueForSysTable = EvictingQueue.create(sysTableQueueSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void addToQueueForSysTable(TooSlowLog.SlowLogPayload slowLogPayload) {
|
||||||
|
queueForSysTable.add(slowLogPayload);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Poll from queueForSysTable and insert 100 records in hbase:slowlog table in single batch
|
||||||
|
*/
|
||||||
|
public void addAllLogsToSysTable() {
|
||||||
|
if (queueForSysTable == null) {
|
||||||
|
LOG.trace("hbase.regionserver.slowlog.systable.enabled is turned off. Exiting.");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (LOCK.isLocked()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
LOCK.lock();
|
||||||
|
try {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloads = new ArrayList<>();
|
||||||
|
int i = 0;
|
||||||
|
while (!queueForSysTable.isEmpty()) {
|
||||||
|
slowLogPayloads.add(queueForSysTable.poll());
|
||||||
|
i++;
|
||||||
|
if (i == SYSTABLE_PUT_BATCH_SIZE) {
|
||||||
|
SlowLogTableAccessor.addSlowLogRecords(slowLogPayloads, this.configuration);
|
||||||
|
slowLogPayloads.clear();
|
||||||
|
i = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (slowLogPayloads.size() > 0) {
|
||||||
|
SlowLogTableAccessor.addSlowLogRecords(slowLogPayloads, this.configuration);
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
LOCK.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,63 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.hadoop.hbase.ScheduledChore;
|
||||||
|
import org.apache.hadoop.hbase.Stoppable;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Chore to insert multiple accumulated slow/large logs to hbase:slowlog system table
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class SlowLogTableOpsChore extends ScheduledChore {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(SlowLogTableOpsChore.class);
|
||||||
|
|
||||||
|
private final NamedQueueRecorder namedQueueRecorder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Chore Constructor
|
||||||
|
*
|
||||||
|
* @param stopper The stopper - When {@link Stoppable#isStopped()} is true, this chore will
|
||||||
|
* cancel and cleanup
|
||||||
|
* @param period Period in millis with which this Chore repeats execution when scheduled
|
||||||
|
* @param namedQueueRecorder {@link NamedQueueRecorder} instance
|
||||||
|
*/
|
||||||
|
public SlowLogTableOpsChore(final Stoppable stopper, final int period,
|
||||||
|
final NamedQueueRecorder namedQueueRecorder) {
|
||||||
|
super("SlowLogTableOpsChore", stopper, period);
|
||||||
|
this.namedQueueRecorder = namedQueueRecorder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void chore() {
|
||||||
|
if (LOG.isTraceEnabled()) {
|
||||||
|
LOG.trace("SlowLog Table Ops Chore is starting up.");
|
||||||
|
}
|
||||||
|
namedQueueRecorder.persistAll(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
|
||||||
|
if (LOG.isTraceEnabled()) {
|
||||||
|
LOG.trace("SlowLog Table Ops Chore is closing.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,144 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.impl;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.client.BalancerDecision;
|
||||||
|
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.BalancerDecisionDetails;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueuePayload;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueService;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.queue.EvictingQueue;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.RecentLogs;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* In-memory Queue service provider for Balancer Decision events
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class BalancerDecisionQueueService implements NamedQueueService {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(BalancerDecisionQueueService.class);
|
||||||
|
|
||||||
|
private final boolean isBalancerDecisionRecording;
|
||||||
|
|
||||||
|
private static final String BALANCER_DECISION_QUEUE_SIZE =
|
||||||
|
"hbase.master.balancer.decision.queue.size";
|
||||||
|
private static final int DEFAULT_BALANCER_DECISION_QUEUE_SIZE = 250;
|
||||||
|
|
||||||
|
private static final int REGION_PLANS_THRESHOLD_PER_BALANCER = 15;
|
||||||
|
|
||||||
|
private final EvictingQueue<RecentLogs.BalancerDecision> balancerDecisionQueue;
|
||||||
|
|
||||||
|
public BalancerDecisionQueueService(Configuration conf) {
|
||||||
|
isBalancerDecisionRecording = conf.getBoolean(BaseLoadBalancer.BALANCER_DECISION_BUFFER_ENABLED,
|
||||||
|
BaseLoadBalancer.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED);
|
||||||
|
if (!isBalancerDecisionRecording) {
|
||||||
|
balancerDecisionQueue = null;
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
final int queueSize =
|
||||||
|
conf.getInt(BALANCER_DECISION_QUEUE_SIZE, DEFAULT_BALANCER_DECISION_QUEUE_SIZE);
|
||||||
|
balancerDecisionQueue = EvictingQueue.create(queueSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NamedQueuePayload.NamedQueueEvent getEvent() {
|
||||||
|
return NamedQueuePayload.NamedQueueEvent.BALANCE_DECISION;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) {
|
||||||
|
if (!isBalancerDecisionRecording) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (!(namedQueuePayload instanceof BalancerDecisionDetails)) {
|
||||||
|
LOG.warn(
|
||||||
|
"BalancerDecisionQueueService: NamedQueuePayload is not of type BalancerDecisionDetails.");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
BalancerDecisionDetails balancerDecisionDetails = (BalancerDecisionDetails) namedQueuePayload;
|
||||||
|
BalancerDecision balancerDecisionRecords =
|
||||||
|
balancerDecisionDetails.getBalancerDecision();
|
||||||
|
List<String> regionPlans = balancerDecisionRecords.getRegionPlans();
|
||||||
|
List<List<String>> regionPlansList;
|
||||||
|
if (regionPlans.size() > REGION_PLANS_THRESHOLD_PER_BALANCER) {
|
||||||
|
regionPlansList = Lists.partition(regionPlans, REGION_PLANS_THRESHOLD_PER_BALANCER);
|
||||||
|
} else {
|
||||||
|
regionPlansList = Collections.singletonList(regionPlans);
|
||||||
|
}
|
||||||
|
for (List<String> regionPlansPerBalancer : regionPlansList) {
|
||||||
|
RecentLogs.BalancerDecision balancerDecision = RecentLogs.BalancerDecision.newBuilder()
|
||||||
|
.setInitTotalCost(balancerDecisionRecords.getInitTotalCost())
|
||||||
|
.setInitialFunctionCosts(balancerDecisionRecords.getInitialFunctionCosts())
|
||||||
|
.setComputedTotalCost(balancerDecisionRecords.getComputedTotalCost())
|
||||||
|
.setFinalFunctionCosts(balancerDecisionRecords.getFinalFunctionCosts())
|
||||||
|
.setComputedSteps(balancerDecisionRecords.getComputedSteps())
|
||||||
|
.addAllRegionPlans(regionPlansPerBalancer)
|
||||||
|
.build();
|
||||||
|
balancerDecisionQueue.add(balancerDecision);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean clearNamedQueue() {
|
||||||
|
if (!isBalancerDecisionRecording) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
LOG.debug("Received request to clean up balancer decision queue.");
|
||||||
|
balancerDecisionQueue.clear();
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
|
||||||
|
if (!isBalancerDecisionRecording) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
List<RecentLogs.BalancerDecision> balancerDecisions =
|
||||||
|
Arrays.asList(balancerDecisionQueue.toArray(new RecentLogs.BalancerDecision[0]));
|
||||||
|
// latest records should be displayed first, hence reverse order sorting
|
||||||
|
Collections.reverse(balancerDecisions);
|
||||||
|
int limit = balancerDecisions.size();
|
||||||
|
if (request.getBalancerDecisionsRequest().hasLimit()) {
|
||||||
|
limit = Math.min(request.getBalancerDecisionsRequest().getLimit(), balancerDecisions.size());
|
||||||
|
}
|
||||||
|
// filter limit if provided
|
||||||
|
balancerDecisions = balancerDecisions.subList(0, limit);
|
||||||
|
final NamedQueueGetResponse namedQueueGetResponse = new NamedQueueGetResponse();
|
||||||
|
namedQueueGetResponse.setNamedQueueEvent(BalancerDecisionDetails.BALANCER_DECISION_EVENT);
|
||||||
|
namedQueueGetResponse.setBalancerDecisions(balancerDecisions);
|
||||||
|
return namedQueueGetResponse;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void persistAll() {
|
||||||
|
// no-op for now
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,268 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.impl;
|
||||||
|
|
||||||
|
import com.google.protobuf.Descriptors;
|
||||||
|
import com.google.protobuf.Message;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.commons.lang.StringUtils;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.client.SlowLogParams;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.LogHandlerUtils;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueuePayload;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueService;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.SlowLogPersistentService;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.queue.EvictingQueue;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* In-memory Queue service provider for Slow/LargeLog events
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class SlowLogQueueService implements NamedQueueService {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(SlowLogQueueService.class);
|
||||||
|
|
||||||
|
private static final String SLOW_LOG_RING_BUFFER_SIZE =
|
||||||
|
"hbase.regionserver.slowlog.ringbuffer.size";
|
||||||
|
|
||||||
|
private final boolean isOnlineLogProviderEnabled;
|
||||||
|
private final boolean isSlowLogTableEnabled;
|
||||||
|
private final SlowLogPersistentService slowLogPersistentService;
|
||||||
|
private final EvictingQueue<TooSlowLog.SlowLogPayload> slowLogQueue;
|
||||||
|
|
||||||
|
public SlowLogQueueService(Configuration conf) {
|
||||||
|
this.isOnlineLogProviderEnabled = conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
|
||||||
|
HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
|
||||||
|
|
||||||
|
if (!isOnlineLogProviderEnabled) {
|
||||||
|
this.isSlowLogTableEnabled = false;
|
||||||
|
this.slowLogPersistentService = null;
|
||||||
|
this.slowLogQueue = null;
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Initialize SlowLog Queue
|
||||||
|
int slowLogQueueSize =
|
||||||
|
conf.getInt(SLOW_LOG_RING_BUFFER_SIZE, HConstants.DEFAULT_SLOW_LOG_RING_BUFFER_SIZE);
|
||||||
|
|
||||||
|
slowLogQueue = EvictingQueue.create(slowLogQueueSize);
|
||||||
|
this.isSlowLogTableEnabled = conf.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,
|
||||||
|
HConstants.DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY);
|
||||||
|
if (isSlowLogTableEnabled) {
|
||||||
|
slowLogPersistentService = new SlowLogPersistentService(conf);
|
||||||
|
} else {
|
||||||
|
slowLogPersistentService = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NamedQueuePayload.NamedQueueEvent getEvent() {
|
||||||
|
return NamedQueuePayload.NamedQueueEvent.SLOW_LOG;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This implementation is specific to slowLog event. This consumes slowLog event from
|
||||||
|
* disruptor and inserts records to EvictingQueue.
|
||||||
|
*
|
||||||
|
* @param namedQueuePayload namedQueue payload from disruptor ring buffer
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) {
|
||||||
|
if (!isOnlineLogProviderEnabled) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (!(namedQueuePayload instanceof RpcLogDetails)) {
|
||||||
|
LOG.warn("SlowLogQueueService: NamedQueuePayload is not of type RpcLogDetails.");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
final RpcLogDetails rpcLogDetails = (RpcLogDetails) namedQueuePayload;
|
||||||
|
final Descriptors.MethodDescriptor methodDescriptor = rpcLogDetails.getMethodDescriptor();
|
||||||
|
final String clientAddress = rpcLogDetails.getClientAddress();
|
||||||
|
final long responseSize = rpcLogDetails.getResponseSize();
|
||||||
|
final String className = rpcLogDetails.getClassName();
|
||||||
|
final TooSlowLog.SlowLogPayload.Type type = getLogType(rpcLogDetails);
|
||||||
|
if (type == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
Message param = rpcLogDetails.getParam();
|
||||||
|
long receiveTime = rpcLogDetails.getReceiveTime();
|
||||||
|
long startTime = rpcLogDetails.getStartTime();
|
||||||
|
long endTime = System.currentTimeMillis();
|
||||||
|
int processingTime = (int) (endTime - startTime);
|
||||||
|
int qTime = (int) (startTime - receiveTime);
|
||||||
|
final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
|
||||||
|
int numGets = 0;
|
||||||
|
int numMutations = 0;
|
||||||
|
int numServiceCalls = 0;
|
||||||
|
if (param instanceof ClientProtos.MultiRequest) {
|
||||||
|
ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
|
||||||
|
for (ClientProtos.RegionAction regionAction : multi.getRegionActionList()) {
|
||||||
|
for (ClientProtos.Action action : regionAction.getActionList()) {
|
||||||
|
if (action.hasMutation()) {
|
||||||
|
numMutations++;
|
||||||
|
}
|
||||||
|
if (action.hasGet()) {
|
||||||
|
numGets++;
|
||||||
|
}
|
||||||
|
if (action.hasServiceCall()) {
|
||||||
|
numServiceCalls++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
final String userName =
|
||||||
|
rpcLogDetails.getUserName() != null ? rpcLogDetails.getUserName() : StringUtils.EMPTY;
|
||||||
|
final String methodDescriptorName =
|
||||||
|
methodDescriptor != null ? methodDescriptor.getName() : StringUtils.EMPTY;
|
||||||
|
TooSlowLog.SlowLogPayload slowLogPayload = TooSlowLog.SlowLogPayload.newBuilder()
|
||||||
|
.setCallDetails(methodDescriptorName + "(" + param.getClass().getName() + ")")
|
||||||
|
.setClientAddress(clientAddress)
|
||||||
|
.setMethodName(methodDescriptorName)
|
||||||
|
.setMultiGets(numGets)
|
||||||
|
.setMultiMutations(numMutations)
|
||||||
|
.setMultiServiceCalls(numServiceCalls)
|
||||||
|
.setParam(slowLogParams != null ? slowLogParams.getParams() : StringUtils.EMPTY)
|
||||||
|
.setProcessingTime(processingTime)
|
||||||
|
.setQueueTime(qTime)
|
||||||
|
.setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : StringUtils.EMPTY)
|
||||||
|
.setResponseSize(responseSize)
|
||||||
|
.setServerClass(className)
|
||||||
|
.setStartTime(startTime)
|
||||||
|
.setType(type)
|
||||||
|
.setUserName(userName)
|
||||||
|
.build();
|
||||||
|
slowLogQueue.add(slowLogPayload);
|
||||||
|
if (isSlowLogTableEnabled) {
|
||||||
|
if (!slowLogPayload.getRegionName().startsWith("hbase:slowlog")) {
|
||||||
|
slowLogPersistentService.addToQueueForSysTable(slowLogPayload);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean clearNamedQueue() {
|
||||||
|
if (!isOnlineLogProviderEnabled) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
LOG.debug("Received request to clean up online slowlog buffer.");
|
||||||
|
slowLogQueue.clear();
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
|
||||||
|
if (!isOnlineLogProviderEnabled) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final AdminProtos.SlowLogResponseRequest slowLogResponseRequest =
|
||||||
|
request.getSlowLogResponseRequest();
|
||||||
|
final List<TooSlowLog.SlowLogPayload> slowLogPayloads;
|
||||||
|
if (AdminProtos.SlowLogResponseRequest.LogType.LARGE_LOG
|
||||||
|
.equals(slowLogResponseRequest.getLogType())) {
|
||||||
|
slowLogPayloads = getLargeLogPayloads(slowLogResponseRequest);
|
||||||
|
} else {
|
||||||
|
slowLogPayloads = getSlowLogPayloads(slowLogResponseRequest);
|
||||||
|
}
|
||||||
|
NamedQueueGetResponse response = new NamedQueueGetResponse();
|
||||||
|
response.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT);
|
||||||
|
response.setSlowLogPayloads(slowLogPayloads);
|
||||||
|
return response;
|
||||||
|
}
|
||||||
|
|
||||||
|
private TooSlowLog.SlowLogPayload.Type getLogType(RpcLogDetails rpcCallDetails) {
|
||||||
|
final boolean isSlowLog = rpcCallDetails.isSlowLog();
|
||||||
|
final boolean isLargeLog = rpcCallDetails.isLargeLog();
|
||||||
|
final TooSlowLog.SlowLogPayload.Type type;
|
||||||
|
if (!isSlowLog && !isLargeLog) {
|
||||||
|
LOG.error("slowLog and largeLog both are false. Ignoring the event. rpcCallDetails: {}",
|
||||||
|
rpcCallDetails);
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
if (isSlowLog && isLargeLog) {
|
||||||
|
type = TooSlowLog.SlowLogPayload.Type.ALL;
|
||||||
|
} else if (isSlowLog) {
|
||||||
|
type = TooSlowLog.SlowLogPayload.Type.SLOW_LOG;
|
||||||
|
} else {
|
||||||
|
type = TooSlowLog.SlowLogPayload.Type.LARGE_LOG;
|
||||||
|
}
|
||||||
|
return type;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add all slowLog events to system table. This is only for slowLog event's persistence on
|
||||||
|
* system table.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void persistAll() {
|
||||||
|
if (!isOnlineLogProviderEnabled) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (slowLogPersistentService != null) {
|
||||||
|
slowLogPersistentService.addAllLogsToSysTable();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<TooSlowLog.SlowLogPayload> getSlowLogPayloads(
|
||||||
|
final AdminProtos.SlowLogResponseRequest request) {
|
||||||
|
TooSlowLog.SlowLogPayload[] slowLogPayloads =
|
||||||
|
slowLogQueue.toArray(new TooSlowLog.SlowLogPayload[0]);
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloadList = new ArrayList<>();
|
||||||
|
for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
|
||||||
|
if (slowLogPayload.getType() == TooSlowLog.SlowLogPayload.Type.ALL
|
||||||
|
|| slowLogPayload.getType() == TooSlowLog.SlowLogPayload.Type.SLOW_LOG) {
|
||||||
|
slowLogPayloadList.add(slowLogPayload);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// latest slow logs first, operator is interested in latest records from in-memory buffer
|
||||||
|
Collections.reverse(slowLogPayloadList);
|
||||||
|
return LogHandlerUtils.getFilteredLogs(request, slowLogPayloadList);
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<TooSlowLog.SlowLogPayload> getLargeLogPayloads(
|
||||||
|
final AdminProtos.SlowLogResponseRequest request) {
|
||||||
|
TooSlowLog.SlowLogPayload[] slowLogPayloads =
|
||||||
|
slowLogQueue.toArray(new TooSlowLog.SlowLogPayload[0]);
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloadList = new ArrayList<>();
|
||||||
|
for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
|
||||||
|
if (slowLogPayload.getType() == TooSlowLog.SlowLogPayload.Type.ALL
|
||||||
|
|| slowLogPayload.getType() == TooSlowLog.SlowLogPayload.Type.LARGE_LOG) {
|
||||||
|
slowLogPayloadList.add(slowLogPayload);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// latest large logs first, operator is interested in latest records from in-memory buffer
|
||||||
|
Collections.reverse(slowLogPayloadList);
|
||||||
|
return LogHandlerUtils.getFilteredLogs(request, slowLogPayloadList);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,85 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.queue;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.Queue;
|
||||||
|
import java.util.concurrent.ArrayBlockingQueue;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public final class EvictingQueue<E> implements Serializable {
|
||||||
|
|
||||||
|
private final Queue<E> delegate;
|
||||||
|
final int maxSize;
|
||||||
|
private static final long serialVersionUID = 0L;
|
||||||
|
|
||||||
|
private EvictingQueue(int maxSize) {
|
||||||
|
Preconditions.checkArgument(maxSize >= 0, "maxSize (%s) must >= 0", maxSize);
|
||||||
|
this.delegate = new ArrayBlockingQueue<>(maxSize);
|
||||||
|
this.maxSize = maxSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <E> EvictingQueue<E> create(int maxSize) {
|
||||||
|
return new EvictingQueue<>(maxSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
public int remainingCapacity() {
|
||||||
|
return this.maxSize - this.delegate.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Queue<E> delegate() {
|
||||||
|
return this.delegate;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean offer(E e) {
|
||||||
|
return this.add(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean add(E e) {
|
||||||
|
Preconditions.checkNotNull(e);
|
||||||
|
if (this.maxSize == 0) {
|
||||||
|
return true;
|
||||||
|
} else {
|
||||||
|
if (this.delegate().size() == this.maxSize) {
|
||||||
|
this.delegate.remove();
|
||||||
|
}
|
||||||
|
this.delegate.add(e);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public <T> T[] toArray(T[] array) {
|
||||||
|
return this.delegate().toArray(array);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void clear() {
|
||||||
|
this.delegate().clear();
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isEmpty() {
|
||||||
|
return this.delegate().isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
|
public E poll() {
|
||||||
|
return this.delegate().poll();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,77 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.request;
|
||||||
|
|
||||||
|
import org.apache.commons.lang.builder.ToStringBuilder;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueuePayload;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Request object to be used by ring buffer use-cases. Clients get records by sending
|
||||||
|
* this request object.
|
||||||
|
* For each ring buffer use-case, add request payload to this class, client should set
|
||||||
|
* namedQueueEvent based on use-case.
|
||||||
|
* Protobuf does not support inheritance, hence we need to work with
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class NamedQueueGetRequest {
|
||||||
|
|
||||||
|
private AdminProtos.SlowLogResponseRequest slowLogResponseRequest;
|
||||||
|
private NamedQueuePayload.NamedQueueEvent namedQueueEvent;
|
||||||
|
private MasterProtos.BalancerDecisionsRequest balancerDecisionsRequest;
|
||||||
|
|
||||||
|
public AdminProtos.SlowLogResponseRequest getSlowLogResponseRequest() {
|
||||||
|
return slowLogResponseRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSlowLogResponseRequest(
|
||||||
|
AdminProtos.SlowLogResponseRequest slowLogResponseRequest) {
|
||||||
|
this.slowLogResponseRequest = slowLogResponseRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
public MasterProtos.BalancerDecisionsRequest getBalancerDecisionsRequest() {
|
||||||
|
return balancerDecisionsRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setBalancerDecisionsRequest(
|
||||||
|
MasterProtos.BalancerDecisionsRequest balancerDecisionsRequest) {
|
||||||
|
this.balancerDecisionsRequest = balancerDecisionsRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
public NamedQueuePayload.NamedQueueEvent getNamedQueueEvent() {
|
||||||
|
return 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("balancerDecisionsRequest", balancerDecisionsRequest)
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,72 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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.response;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.commons.lang.builder.ToStringBuilder;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueuePayload;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.RecentLogs;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Response object to be sent by namedQueue service back to caller
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class NamedQueueGetResponse {
|
||||||
|
|
||||||
|
private List<TooSlowLog.SlowLogPayload> slowLogPayloads;
|
||||||
|
private List<RecentLogs.BalancerDecision> balancerDecisions;
|
||||||
|
private NamedQueuePayload.NamedQueueEvent namedQueueEvent;
|
||||||
|
|
||||||
|
public List<TooSlowLog.SlowLogPayload> getSlowLogPayloads() {
|
||||||
|
return slowLogPayloads;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSlowLogPayloads(List<TooSlowLog.SlowLogPayload> slowLogPayloads) {
|
||||||
|
this.slowLogPayloads = slowLogPayloads;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<RecentLogs.BalancerDecision> getBalancerDecisions() {
|
||||||
|
return balancerDecisions;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setBalancerDecisions(List<RecentLogs.BalancerDecision> balancerDecisions) {
|
||||||
|
this.balancerDecisions = balancerDecisions;
|
||||||
|
}
|
||||||
|
|
||||||
|
public NamedQueuePayload.NamedQueueEvent getNamedQueueEvent() {
|
||||||
|
return namedQueueEvent;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setNamedQueueEvent(int eventOrdinal) {
|
||||||
|
this.namedQueueEvent = NamedQueuePayload.NamedQueueEvent.getEventByOrdinal(eventOrdinal);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new ToStringBuilder(this)
|
||||||
|
.append("slowLogPayloads", slowLogPayloads)
|
||||||
|
.append("balancerDecisions", balancerDecisions)
|
||||||
|
.append("namedQueueEvent", namedQueueEvent)
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -125,6 +125,8 @@ import org.apache.hadoop.hbase.master.HMaster;
|
||||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||||
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
|
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.SlowLogTableOpsChore;
|
||||||
import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
|
import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||||
|
@ -417,6 +419,8 @@ public class HRegionServer extends HasThread implements
|
||||||
private final int operationTimeout;
|
private final int operationTimeout;
|
||||||
private final int shortOperationTimeout;
|
private final int shortOperationTimeout;
|
||||||
|
|
||||||
|
private SlowLogTableOpsChore slowLogTableOpsChore = null;
|
||||||
|
|
||||||
private final RegionServerAccounting regionServerAccounting;
|
private final RegionServerAccounting regionServerAccounting;
|
||||||
|
|
||||||
// Cache configuration and block cache reference
|
// Cache configuration and block cache reference
|
||||||
|
@ -528,6 +532,11 @@ public class HRegionServer extends HasThread implements
|
||||||
|
|
||||||
private volatile ThroughputController flushThroughputController;
|
private volatile ThroughputController flushThroughputController;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provide online slow log responses from ringbuffer
|
||||||
|
*/
|
||||||
|
private NamedQueueRecorder namedQueueRecorder = null;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Starts a HRegionServer at the default location.
|
* Starts a HRegionServer at the default location.
|
||||||
*/
|
*/
|
||||||
|
@ -581,6 +590,8 @@ public class HRegionServer extends HasThread implements
|
||||||
this.abortRequested = new AtomicBoolean(false);
|
this.abortRequested = new AtomicBoolean(false);
|
||||||
this.stopped = false;
|
this.stopped = false;
|
||||||
|
|
||||||
|
initNamedQueueRecorder(conf);
|
||||||
|
|
||||||
rpcServices = createRpcServices();
|
rpcServices = createRpcServices();
|
||||||
if (this instanceof HMaster) {
|
if (this instanceof HMaster) {
|
||||||
useThisHostnameInstead = conf.get(MASTER_HOSTNAME_KEY);
|
useThisHostnameInstead = conf.get(MASTER_HOSTNAME_KEY);
|
||||||
|
@ -697,6 +708,24 @@ public class HRegionServer extends HasThread implements
|
||||||
this.initLatch = latch;
|
this.initLatch = latch;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void initNamedQueueRecorder(Configuration conf) {
|
||||||
|
if (!(this instanceof HMaster)) {
|
||||||
|
final boolean isOnlineLogProviderEnabled = conf.getBoolean(
|
||||||
|
HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
|
||||||
|
HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
|
||||||
|
if (isOnlineLogProviderEnabled) {
|
||||||
|
this.namedQueueRecorder = NamedQueueRecorder.getInstance(this.conf);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
final boolean isBalancerDecisionRecording = conf
|
||||||
|
.getBoolean(BaseLoadBalancer.BALANCER_DECISION_BUFFER_ENABLED,
|
||||||
|
BaseLoadBalancer.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED);
|
||||||
|
if (isBalancerDecisionRecording) {
|
||||||
|
this.namedQueueRecorder = NamedQueueRecorder.getInstance(this.conf);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Returns true if configured hostname should be used
|
* Returns true if configured hostname should be used
|
||||||
*/
|
*/
|
||||||
|
@ -927,6 +956,14 @@ public class HRegionServer extends HasThread implements
|
||||||
this.periodicFlusher = new PeriodicMemstoreFlusher(this.flushCheckFrequency, this);
|
this.periodicFlusher = new PeriodicMemstoreFlusher(this.flushCheckFrequency, this);
|
||||||
this.leases = new Leases(this.threadWakeFrequency);
|
this.leases = new Leases(this.threadWakeFrequency);
|
||||||
|
|
||||||
|
final boolean isSlowLogTableEnabled = conf.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,
|
||||||
|
HConstants.DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY);
|
||||||
|
if (isSlowLogTableEnabled) {
|
||||||
|
// default chore duration: 10 min
|
||||||
|
final int duration = conf.getInt("hbase.slowlog.systable.chore.duration", 10 * 60 * 1000);
|
||||||
|
slowLogTableOpsChore = new SlowLogTableOpsChore(this, duration, this.namedQueueRecorder);
|
||||||
|
}
|
||||||
|
|
||||||
// Create the thread to clean the moved regions list
|
// Create the thread to clean the moved regions list
|
||||||
movedRegionsCleaner = MovedRegionsCleaner.create(this);
|
movedRegionsCleaner = MovedRegionsCleaner.create(this);
|
||||||
|
|
||||||
|
@ -1436,6 +1473,15 @@ public class HRegionServer extends HasThread implements
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* get NamedQueue Provider to add different logs to ringbuffer
|
||||||
|
*
|
||||||
|
* @return NamedQueueRecorder
|
||||||
|
*/
|
||||||
|
public NamedQueueRecorder getNamedQueueRecorder() {
|
||||||
|
return this.namedQueueRecorder;
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Run init. Sets up wal and starts up all server threads.
|
* Run init. Sets up wal and starts up all server threads.
|
||||||
*
|
*
|
||||||
|
@ -1904,6 +1950,9 @@ public class HRegionServer extends HasThread implements
|
||||||
if (this.movedRegionsCleaner != null) {
|
if (this.movedRegionsCleaner != null) {
|
||||||
choreService.scheduleChore(movedRegionsCleaner);
|
choreService.scheduleChore(movedRegionsCleaner);
|
||||||
}
|
}
|
||||||
|
if (this.slowLogTableOpsChore != null) {
|
||||||
|
choreService.scheduleChore(slowLogTableOpsChore);
|
||||||
|
}
|
||||||
|
|
||||||
// Leases is not a Thread. Internally it runs a daemon thread. If it gets
|
// Leases is not a Thread. Internally it runs a daemon thread. If it gets
|
||||||
// an unhandled exception, it will just exit.
|
// an unhandled exception, it will just exit.
|
||||||
|
@ -2357,6 +2406,7 @@ public class HRegionServer extends HasThread implements
|
||||||
choreService.cancelChore(executorStatusChore);
|
choreService.cancelChore(executorStatusChore);
|
||||||
choreService.cancelChore(storefileRefresher);
|
choreService.cancelChore(storefileRefresher);
|
||||||
choreService.cancelChore(movedRegionsCleaner);
|
choreService.cancelChore(movedRegionsCleaner);
|
||||||
|
choreService.cancelChore(slowLogTableOpsChore);
|
||||||
// clean up the remaining scheduled chores (in case we missed out any)
|
// clean up the remaining scheduled chores (in case we missed out any)
|
||||||
choreService.shutdown();
|
choreService.shutdown();
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,6 +30,8 @@ import com.google.protobuf.TextFormat;
|
||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InterruptedIOException;
|
import java.io.InterruptedIOException;
|
||||||
|
import java.lang.reflect.InvocationTargetException;
|
||||||
|
import java.lang.reflect.Method;
|
||||||
import java.net.BindException;
|
import java.net.BindException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.net.UnknownHostException;
|
import java.net.UnknownHostException;
|
||||||
|
@ -103,11 +105,19 @@ import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||||
|
import org.apache.hadoop.hbase.master.HMaster;
|
||||||
import org.apache.hadoop.hbase.master.MasterRpcServices;
|
import org.apache.hadoop.hbase.master.MasterRpcServices;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueuePayload;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ClearSlowLogResponses;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||||
|
@ -134,6 +144,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryR
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponseRequest;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SlowLogResponses;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
|
||||||
|
@ -168,6 +180,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResul
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
|
||||||
|
@ -175,6 +188,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetrics;
|
import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetrics;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
|
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog.SlowLogPayload;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
||||||
|
@ -263,7 +277,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
final RpcServerInterface rpcServer;
|
final RpcServerInterface rpcServer;
|
||||||
final InetSocketAddress isa;
|
final InetSocketAddress isa;
|
||||||
|
|
||||||
private final HRegionServer regionServer;
|
protected final HRegionServer regionServer;
|
||||||
private final long maxScannerResultSize;
|
private final long maxScannerResultSize;
|
||||||
|
|
||||||
// The reference to the priority extraction function
|
// The reference to the priority extraction function
|
||||||
|
@ -1172,6 +1186,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
"' configuration property.", be.getCause() != null ? be.getCause() : be);
|
"' configuration property.", be.getCause() != null ? be.getCause() : be);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (!(rs instanceof HMaster)) {
|
||||||
|
rpcServer.setNamedQueueRecorder(rs.getNamedQueueRecorder());
|
||||||
|
}
|
||||||
scannerLeaseTimeoutPeriod = rs.conf.getInt(
|
scannerLeaseTimeoutPeriod = rs.conf.getInt(
|
||||||
HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
|
HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
|
||||||
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
|
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
|
||||||
|
@ -3326,4 +3343,68 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
}
|
}
|
||||||
return UpdateConfigurationResponse.getDefaultInstance();
|
return UpdateConfigurationResponse.getDefaultInstance();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private List<SlowLogPayload> getSlowLogPayloads(SlowLogResponseRequest request,
|
||||||
|
NamedQueueRecorder namedQueueRecorder) {
|
||||||
|
if (namedQueueRecorder == null) {
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
List<SlowLogPayload> slowLogPayloads;
|
||||||
|
NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
|
||||||
|
namedQueueGetRequest.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT);
|
||||||
|
namedQueueGetRequest.setSlowLogResponseRequest(request);
|
||||||
|
NamedQueueGetResponse namedQueueGetResponse =
|
||||||
|
namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
|
||||||
|
slowLogPayloads = namedQueueGetResponse != null ?
|
||||||
|
namedQueueGetResponse.getSlowLogPayloads() : new ArrayList<SlowLogPayload>();
|
||||||
|
return slowLogPayloads;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@QosPriority(priority=HConstants.ADMIN_QOS)
|
||||||
|
public ClearSlowLogResponses clearSlowLogsResponses(RpcController controller,
|
||||||
|
ClearSlowLogResponseRequest request) throws ServiceException {
|
||||||
|
final NamedQueueRecorder namedQueueRecorder =
|
||||||
|
this.regionServer.getNamedQueueRecorder();
|
||||||
|
boolean slowLogsCleaned = false;
|
||||||
|
if (namedQueueRecorder != null) {
|
||||||
|
namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
|
||||||
|
slowLogsCleaned = true;
|
||||||
|
}
|
||||||
|
ClearSlowLogResponses clearSlowLogResponses = ClearSlowLogResponses.newBuilder()
|
||||||
|
.setIsCleaned(slowLogsCleaned)
|
||||||
|
.build();
|
||||||
|
return clearSlowLogResponses;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@QosPriority(priority = HConstants.ADMIN_QOS)
|
||||||
|
public HBaseProtos.LogEntry getLogEntries(RpcController controller,
|
||||||
|
HBaseProtos.LogRequest request) throws ServiceException {
|
||||||
|
try {
|
||||||
|
final String logClassName = request.getLogClassName();
|
||||||
|
Class<?> logClass = Class.forName(logClassName)
|
||||||
|
.asSubclass(Message.class);
|
||||||
|
Method method = logClass.getMethod("parseFrom", ByteString.class);
|
||||||
|
if (logClassName.contains("SlowLogResponseRequest")) {
|
||||||
|
SlowLogResponseRequest slowLogResponseRequest =
|
||||||
|
(SlowLogResponseRequest) method.invoke(null, request.getLogMessage());
|
||||||
|
final NamedQueueRecorder namedQueueRecorder =
|
||||||
|
this.regionServer.getNamedQueueRecorder();
|
||||||
|
final List<SlowLogPayload> slowLogPayloads =
|
||||||
|
getSlowLogPayloads(slowLogResponseRequest, namedQueueRecorder);
|
||||||
|
SlowLogResponses slowLogResponses = SlowLogResponses.newBuilder()
|
||||||
|
.addAllSlowLogPayloads(slowLogPayloads)
|
||||||
|
.build();
|
||||||
|
return HBaseProtos.LogEntry.newBuilder()
|
||||||
|
.setLogClassName(slowLogResponses.getClass().getName())
|
||||||
|
.setLogMessage(slowLogResponses.toByteString()).build();
|
||||||
|
}
|
||||||
|
} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
|
||||||
|
| InvocationTargetException e) {
|
||||||
|
LOG.error("Error while retrieving log entries.", e);
|
||||||
|
throw new ServiceException(e);
|
||||||
|
}
|
||||||
|
throw new ServiceException("Invalid request params");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||||
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
|
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
|
||||||
import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
|
import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
|
||||||
|
@ -654,6 +655,18 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AdminProtos.ClearSlowLogResponses clearSlowLogsResponses(RpcController controller,
|
||||||
|
AdminProtos.ClearSlowLogResponseRequest request) throws ServiceException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HBaseProtos.LogEntry getLogEntries(RpcController controller,
|
||||||
|
HBaseProtos.LogRequest request) throws ServiceException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public HeapMemoryManager getHeapMemoryManager() {
|
public HeapMemoryManager getHeapMemoryManager() {
|
||||||
return null;
|
return null;
|
||||||
|
|
|
@ -0,0 +1,102 @@
|
||||||
|
/*
|
||||||
|
* 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.master.balancer;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
|
import org.apache.hadoop.hbase.client.LogEntry;
|
||||||
|
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.BalancerDecisionDetails;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.RecentLogs;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test BalancerDecision ring buffer using namedQueue interface
|
||||||
|
*/
|
||||||
|
@Category({ MasterTests.class, MediumTests.class })
|
||||||
|
public class TestBalancerDecision extends BalancerTestBase {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testBalancerDecisions() {
|
||||||
|
conf.setBoolean("hbase.master.balancer.decision.buffer.enabled", true);
|
||||||
|
loadBalancer.setConf(conf);
|
||||||
|
float minCost = conf.getFloat("hbase.master.balancer.stochastic.minCostNeedBalance", 0.05f);
|
||||||
|
conf.setFloat("hbase.master.balancer.stochastic.minCostNeedBalance", 1.0f);
|
||||||
|
try {
|
||||||
|
// Test with/without per table balancer.
|
||||||
|
boolean[] perTableBalancerConfigs = {true, false};
|
||||||
|
for (boolean isByTable : perTableBalancerConfigs) {
|
||||||
|
conf.setBoolean(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, isByTable);
|
||||||
|
loadBalancer.setConf(conf);
|
||||||
|
for (int[] mockCluster : clusterStateMocks) {
|
||||||
|
Map<ServerName, List<HRegionInfo>> servers = mockClusterServers(mockCluster);
|
||||||
|
List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
|
||||||
|
boolean emptyPlans = plans == null || plans.isEmpty();
|
||||||
|
Assert.assertTrue(emptyPlans || needsBalanceIdleRegion(mockCluster));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
final NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
|
||||||
|
namedQueueGetRequest.setNamedQueueEvent(BalancerDecisionDetails.BALANCER_DECISION_EVENT);
|
||||||
|
namedQueueGetRequest
|
||||||
|
.setBalancerDecisionsRequest(MasterProtos.BalancerDecisionsRequest.getDefaultInstance());
|
||||||
|
NamedQueueGetResponse namedQueueGetResponse =
|
||||||
|
loadBalancer.namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
|
||||||
|
List<RecentLogs.BalancerDecision> balancerDecisions =
|
||||||
|
namedQueueGetResponse.getBalancerDecisions();
|
||||||
|
MasterProtos.BalancerDecisionsResponse response =
|
||||||
|
MasterProtos.BalancerDecisionsResponse.newBuilder()
|
||||||
|
.addAllBalancerDecision(balancerDecisions)
|
||||||
|
.build();
|
||||||
|
List<LogEntry> balancerDecisionRecords =
|
||||||
|
ProtobufUtil.getBalancerDecisionEntries(response);
|
||||||
|
Assert.assertTrue(balancerDecisionRecords.size() > 160);
|
||||||
|
} finally {
|
||||||
|
// reset config
|
||||||
|
conf.unset(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE);
|
||||||
|
conf.setFloat("hbase.master.balancer.stochastic.minCostNeedBalance", minCost);
|
||||||
|
loadBalancer.setConf(conf);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean needsBalanceIdleRegion(int[] clusters) {
|
||||||
|
boolean b1 = false;
|
||||||
|
boolean b2 = false;
|
||||||
|
for (int cluster : clusters) {
|
||||||
|
if (cluster > 1) {
|
||||||
|
b1 = true;
|
||||||
|
} else {
|
||||||
|
b2 = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return b1 && b2;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,716 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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 com.google.protobuf.ByteString;
|
||||||
|
import com.google.protobuf.Message;
|
||||||
|
import java.lang.reflect.Constructor;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.Waiter;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests for Online SlowLog Provider Service
|
||||||
|
*/
|
||||||
|
@Category({MasterTests.class, MediumTests.class})
|
||||||
|
public class TestNamedQueueRecorder {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(TestNamedQueueRecorder.class);
|
||||||
|
|
||||||
|
private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new HBaseTestingUtility();
|
||||||
|
|
||||||
|
private NamedQueueRecorder namedQueueRecorder;
|
||||||
|
|
||||||
|
private static int i = 0;
|
||||||
|
|
||||||
|
private static Configuration applySlowLogRecorderConf(int eventSize) {
|
||||||
|
Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
|
||||||
|
conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
|
||||||
|
conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", eventSize);
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* confirm that for a ringbuffer of slow logs, payload on given index of buffer
|
||||||
|
* has expected elements
|
||||||
|
*
|
||||||
|
* @param i index of ringbuffer logs
|
||||||
|
* @param j data value that was put on index i
|
||||||
|
* @param slowLogPayloads list of payload retrieved from {@link NamedQueueRecorder}
|
||||||
|
* @return if actual values are as per expectations
|
||||||
|
*/
|
||||||
|
private boolean confirmPayloadParams(int i, int j,
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloads) {
|
||||||
|
boolean isClientExpected = slowLogPayloads.get(i).getClientAddress().equals("client_" + j);
|
||||||
|
boolean isUserExpected = slowLogPayloads.get(i).getUserName().equals("userName_" + j);
|
||||||
|
boolean isClassExpected = slowLogPayloads.get(i).getServerClass().equals("class_" + j);
|
||||||
|
return isClassExpected && isClientExpected && isUserExpected;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOnlieSlowLogConsumption() throws Exception{
|
||||||
|
Configuration conf = applySlowLogRecorderConf(8);
|
||||||
|
Constructor<NamedQueueRecorder> constructor =
|
||||||
|
NamedQueueRecorder.class.getDeclaredConstructor(Configuration.class);
|
||||||
|
constructor.setAccessible(true);
|
||||||
|
namedQueueRecorder = constructor.newInstance(conf);
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
|
||||||
|
namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
LOG.debug("Initially ringbuffer of Slow Log records is empty");
|
||||||
|
int i = 0;
|
||||||
|
// add 5 records initially
|
||||||
|
for (; i < 5; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
Assert.assertNotEquals(-1,
|
||||||
|
HBASE_TESTING_UTILITY.waitFor(3000, new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() == 5;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloads = getSlowLogPayloads(request);
|
||||||
|
Assert.assertTrue(confirmPayloadParams(0, 5, slowLogPayloads));
|
||||||
|
Assert.assertTrue(confirmPayloadParams(1, 4, slowLogPayloads));
|
||||||
|
Assert.assertTrue(confirmPayloadParams(2, 3, slowLogPayloads));
|
||||||
|
Assert.assertTrue(confirmPayloadParams(3, 2, slowLogPayloads));
|
||||||
|
Assert.assertTrue(confirmPayloadParams(4, 1, slowLogPayloads));
|
||||||
|
for (; i < 7; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() == 7;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloadsList =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(request);
|
||||||
|
return slowLogPayloadsList.size() == 7 && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(0, 7, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(5, 2, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(6, 1, slowLogPayloadsList);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
// add 3 more records
|
||||||
|
for (; i < 10; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() == 8;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloadsList =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(request);
|
||||||
|
// confirm ringbuffer is full
|
||||||
|
return slowLogPayloadsList.size() == 8 && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(7, 3, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(0, 10, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(1, 9, slowLogPayloadsList);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
// add 4 more records
|
||||||
|
for (; i < 14; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() == 8;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloadsList =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(request);
|
||||||
|
return slowLogPayloadsList.size() == 8 && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(0, 14, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(1, 13, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(2, 12, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(3, 11, slowLogPayloadsList);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
final AdminProtos.SlowLogResponseRequest largeLogRequest =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.setLogType(AdminProtos.SlowLogResponseRequest.LogType.LARGE_LOG)
|
||||||
|
.build();
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloadsList =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(largeLogRequest);
|
||||||
|
return slowLogPayloadsList.size() == 8 && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(0, 14, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(1, 13, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(2, 12, slowLogPayloadsList) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(3, 11, slowLogPayloadsList);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
boolean isRingBufferCleaned =
|
||||||
|
namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
|
||||||
|
LOG.debug("cleared the ringbuffer of Online Slow Log records");
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloadsList =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(request);
|
||||||
|
return slowLogPayloadsList.size() == 0 && isRingBufferCleaned;
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<TooSlowLog.SlowLogPayload> getSlowLogPayloads(
|
||||||
|
AdminProtos.SlowLogResponseRequest request) {
|
||||||
|
NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
|
||||||
|
namedQueueGetRequest.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT);
|
||||||
|
namedQueueGetRequest.setSlowLogResponseRequest(request);
|
||||||
|
NamedQueueGetResponse namedQueueGetResponse =
|
||||||
|
namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
|
||||||
|
return namedQueueGetResponse == null ? new ArrayList<TooSlowLog.SlowLogPayload>()
|
||||||
|
: namedQueueGetResponse.getSlowLogPayloads();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOnlineSlowLogWithHighRecords() throws Exception {
|
||||||
|
|
||||||
|
Configuration conf = applySlowLogRecorderConf(14);
|
||||||
|
Constructor<NamedQueueRecorder> constructor =
|
||||||
|
NamedQueueRecorder.class.getDeclaredConstructor(Configuration.class);
|
||||||
|
constructor.setAccessible(true);
|
||||||
|
namedQueueRecorder = constructor.newInstance(conf);
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(14 * 11).build();
|
||||||
|
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
LOG.debug("Initially ringbuffer of Slow Log records is empty");
|
||||||
|
|
||||||
|
for (int i = 0; i < 14 * 11; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
LOG.debug("Added 14 * 11 records, ringbuffer should only provide latest 14 records");
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() == 14;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloads =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(request);
|
||||||
|
|
||||||
|
// confirm strict order of slow log payloads
|
||||||
|
return slowLogPayloads.size() == 14 && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(0, 154, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(1, 153, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(2, 152, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(3, 151, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(4, 150, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(5, 149, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(6, 148, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(7, 147, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(8, 146, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(9, 145, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(10, 144, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(11, 143, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(12, 142, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(13, 141, slowLogPayloads);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
|
||||||
|
boolean isRingBufferCleaned = namedQueueRecorder.clearNamedQueue(
|
||||||
|
NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
|
||||||
|
Assert.assertTrue(isRingBufferCleaned);
|
||||||
|
LOG.debug("cleared the ringbuffer of Online Slow Log records");
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloads = getSlowLogPayloads(request);
|
||||||
|
|
||||||
|
// confirm ringbuffer is empty
|
||||||
|
Assert.assertEquals(slowLogPayloads.size(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOnlineSlowLogWithDefaultDisableConfig() throws Exception {
|
||||||
|
Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
|
||||||
|
conf.unset(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY);
|
||||||
|
|
||||||
|
Constructor<NamedQueueRecorder> constructor =
|
||||||
|
NamedQueueRecorder.class.getDeclaredConstructor(Configuration.class);
|
||||||
|
constructor.setAccessible(true);
|
||||||
|
namedQueueRecorder = constructor.newInstance(conf);
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder().build();
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
LOG.debug("Initially ringbuffer of Slow Log records is empty");
|
||||||
|
for (int i = 0; i < 300; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloads =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(request);
|
||||||
|
return slowLogPayloads.size() == 0;
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOnlineSlowLogWithDisableConfig() throws Exception {
|
||||||
|
Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
|
||||||
|
conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, false);
|
||||||
|
Constructor<NamedQueueRecorder> constructor =
|
||||||
|
NamedQueueRecorder.class.getDeclaredConstructor(Configuration.class);
|
||||||
|
constructor.setAccessible(true);
|
||||||
|
namedQueueRecorder = constructor.newInstance(conf);
|
||||||
|
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder().build();
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
LOG.debug("Initially ringbuffer of Slow Log records is empty");
|
||||||
|
for (int i = 0; i < 300; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloads =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(request);
|
||||||
|
return slowLogPayloads.size() == 0;
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSlowLogFilters() throws Exception {
|
||||||
|
|
||||||
|
Configuration conf = applySlowLogRecorderConf(30);
|
||||||
|
Constructor<NamedQueueRecorder> constructor =
|
||||||
|
NamedQueueRecorder.class.getDeclaredConstructor(Configuration.class);
|
||||||
|
constructor.setAccessible(true);
|
||||||
|
namedQueueRecorder = constructor.newInstance(conf);
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.setUserName("userName_87")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
|
||||||
|
LOG.debug("Initially ringbuffer of Slow Log records is empty");
|
||||||
|
|
||||||
|
for (int i = 0; i < 100; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
LOG.debug("Added 100 records, ringbuffer should only 1 record with matching filter");
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() == 1;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
|
||||||
|
final AdminProtos.SlowLogResponseRequest requestClient =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.setClientAddress("client_85")
|
||||||
|
.build();
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(requestClient).size() == 1;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
|
||||||
|
final AdminProtos.SlowLogResponseRequest requestSlowLog =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.build();
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(requestSlowLog).size() == 15;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConcurrentSlowLogEvents() throws Exception {
|
||||||
|
|
||||||
|
Configuration conf = applySlowLogRecorderConf(50000);
|
||||||
|
Constructor<NamedQueueRecorder> constructor =
|
||||||
|
NamedQueueRecorder.class.getDeclaredConstructor(Configuration.class);
|
||||||
|
constructor.setAccessible(true);
|
||||||
|
namedQueueRecorder = constructor.newInstance(conf);
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(500000).build();
|
||||||
|
final AdminProtos.SlowLogResponseRequest largeLogRequest =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(500000)
|
||||||
|
.setLogType(AdminProtos.SlowLogResponseRequest.LogType.LARGE_LOG)
|
||||||
|
.build();
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
LOG.debug("Initially ringbuffer of Slow Log records is empty");
|
||||||
|
|
||||||
|
for (int j = 0; j < 1000; j++) {
|
||||||
|
|
||||||
|
new Thread(new Runnable() {
|
||||||
|
@Override public void run() {
|
||||||
|
for (int i = 0; i < 3500; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}).start();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
Thread.sleep(500);
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(
|
||||||
|
7000, new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() > 10000;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(
|
||||||
|
7000, new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(largeLogRequest).size() > 10000;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSlowLargeLogEvents() throws Exception {
|
||||||
|
Configuration conf = applySlowLogRecorderConf(28);
|
||||||
|
Constructor<NamedQueueRecorder> constructor =
|
||||||
|
NamedQueueRecorder.class.getDeclaredConstructor(Configuration.class);
|
||||||
|
constructor.setAccessible(true);
|
||||||
|
namedQueueRecorder = constructor.newInstance(conf);
|
||||||
|
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(14 * 11).build();
|
||||||
|
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
LOG.debug("Initially ringbuffer of Slow Log records is empty");
|
||||||
|
|
||||||
|
boolean isSlowLog;
|
||||||
|
boolean isLargeLog;
|
||||||
|
for (int i = 0; i < 14 * 11; i++) {
|
||||||
|
if (i % 2 == 0) {
|
||||||
|
isSlowLog = true;
|
||||||
|
isLargeLog = false;
|
||||||
|
} else {
|
||||||
|
isSlowLog = false;
|
||||||
|
isLargeLog = true;
|
||||||
|
}
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1),
|
||||||
|
isSlowLog, isLargeLog);
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
LOG.debug("Added 14 * 11 records, ringbuffer should only provide latest 14 records");
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() == 14;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> slowLogPayloads =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(request);
|
||||||
|
|
||||||
|
// confirm strict order of slow log payloads
|
||||||
|
return slowLogPayloads.size() == 14 && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(0, 153, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(1, 151, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(2, 149, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(3, 147, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(4, 145, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(5, 143, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(6, 141, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(7, 139, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(8, 137, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(9, 135, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(10, 133, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(11, 131, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(12, 129, slowLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(13, 127, slowLogPayloads);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
|
||||||
|
final AdminProtos.SlowLogResponseRequest largeLogRequest =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(14 * 11)
|
||||||
|
.setLogType(AdminProtos.SlowLogResponseRequest.LogType.LARGE_LOG)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(largeLogRequest).size() == 14;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
List<TooSlowLog.SlowLogPayload> largeLogPayloads =
|
||||||
|
TestNamedQueueRecorder.this.getSlowLogPayloads(largeLogRequest);
|
||||||
|
|
||||||
|
// confirm strict order of slow log payloads
|
||||||
|
return largeLogPayloads.size() == 14 && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(0, 154, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(1, 152, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(2, 150, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(3, 148, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(4, 146, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(5, 144, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(6, 142, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(7, 140, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(8, 138, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(9, 136, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(10, 134, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(11, 132, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(12, 130, largeLogPayloads) && TestNamedQueueRecorder.this
|
||||||
|
.confirmPayloadParams(13, 128, largeLogPayloads);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSlowLogMixedFilters() throws Exception {
|
||||||
|
|
||||||
|
Configuration conf = applySlowLogRecorderConf(30);
|
||||||
|
Constructor<NamedQueueRecorder> constructor =
|
||||||
|
NamedQueueRecorder.class.getDeclaredConstructor(Configuration.class);
|
||||||
|
constructor.setAccessible(true);
|
||||||
|
namedQueueRecorder = constructor.newInstance(conf);
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.setUserName("userName_87")
|
||||||
|
.setClientAddress("client_88")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
|
||||||
|
for (int i = 0; i < 100; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails =
|
||||||
|
getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(request).size() == 2;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
|
||||||
|
AdminProtos.SlowLogResponseRequest request2 = AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.setUserName("userName_1")
|
||||||
|
.setClientAddress("client_2")
|
||||||
|
.build();
|
||||||
|
Assert.assertEquals(0, getSlowLogPayloads(request2).size());
|
||||||
|
|
||||||
|
AdminProtos.SlowLogResponseRequest request3 =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.setUserName("userName_87")
|
||||||
|
.setClientAddress("client_88")
|
||||||
|
.setFilterByOperator(AdminProtos.SlowLogResponseRequest.FilterByOperator.AND)
|
||||||
|
.build();
|
||||||
|
Assert.assertEquals(0, getSlowLogPayloads(request3).size());
|
||||||
|
|
||||||
|
AdminProtos.SlowLogResponseRequest request4 =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.setUserName("userName_87")
|
||||||
|
.setClientAddress("client_87")
|
||||||
|
.setFilterByOperator(AdminProtos.SlowLogResponseRequest.FilterByOperator.AND)
|
||||||
|
.build();
|
||||||
|
Assert.assertEquals(1, getSlowLogPayloads(request4).size());
|
||||||
|
|
||||||
|
AdminProtos.SlowLogResponseRequest request5 =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.setUserName("userName_88")
|
||||||
|
.setClientAddress("client_89")
|
||||||
|
.setFilterByOperator(AdminProtos.SlowLogResponseRequest.FilterByOperator.OR)
|
||||||
|
.build();
|
||||||
|
Assert.assertEquals(2, getSlowLogPayloads(request5).size());
|
||||||
|
|
||||||
|
final AdminProtos.SlowLogResponseRequest requestSlowLog =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder()
|
||||||
|
.setLimit(15)
|
||||||
|
.build();
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestNamedQueueRecorder.this.getSlowLogPayloads(requestSlowLog).size() == 15;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
static RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className) {
|
||||||
|
return new RpcLogDetails(null, getMessage(), clientAddress, 0, className, true, true, 0, 0,
|
||||||
|
userName);
|
||||||
|
}
|
||||||
|
|
||||||
|
private RpcLogDetails getRpcLogDetails(String userName, String clientAddress,
|
||||||
|
String className, boolean isSlowLog, boolean isLargeLog) {
|
||||||
|
return new RpcLogDetails(null, getMessage(), clientAddress, 0, className,
|
||||||
|
isSlowLog, isLargeLog, 0, 0, userName);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Message getMessage() {
|
||||||
|
i = (i + 1) % 3;
|
||||||
|
Message message = null;
|
||||||
|
switch (i) {
|
||||||
|
case 0: {
|
||||||
|
message = ClientProtos.ScanRequest.newBuilder()
|
||||||
|
.setRegion(HBaseProtos.RegionSpecifier.newBuilder()
|
||||||
|
.setValue(ByteString.copyFromUtf8("region1"))
|
||||||
|
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 1: {
|
||||||
|
message = ClientProtos.MutateRequest.newBuilder()
|
||||||
|
.setRegion(HBaseProtos.RegionSpecifier.newBuilder()
|
||||||
|
.setValue(ByteString.copyFromUtf8("region2"))
|
||||||
|
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME))
|
||||||
|
.setMutation(ClientProtos.MutationProto.newBuilder()
|
||||||
|
.setRow(ByteString.copyFromUtf8("row123"))
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 2: {
|
||||||
|
message = ClientProtos.GetRequest.newBuilder()
|
||||||
|
.setRegion(HBaseProtos.RegionSpecifier.newBuilder()
|
||||||
|
.setValue(ByteString.copyFromUtf8("region2"))
|
||||||
|
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME))
|
||||||
|
.setGet(ClientProtos.Get.newBuilder()
|
||||||
|
.setRow(ByteString.copyFromUtf8("row123"))
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
default:
|
||||||
|
throw new RuntimeException("Not supposed to get here?");
|
||||||
|
}
|
||||||
|
|
||||||
|
return message;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,237 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* 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 java.io.IOException;
|
||||||
|
import java.lang.reflect.Field;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
|
import org.apache.hadoop.hbase.Waiter;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
|
||||||
|
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
|
import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests for SlowLog System Table
|
||||||
|
*/
|
||||||
|
@Category({ MasterTests.class, MediumTests.class })
|
||||||
|
public class TestSlowLogAccessor {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(TestNamedQueueRecorder.class);
|
||||||
|
|
||||||
|
private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new HBaseTestingUtility();
|
||||||
|
|
||||||
|
private NamedQueueRecorder namedQueueRecorder;
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void setup() throws Exception {
|
||||||
|
try {
|
||||||
|
HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.debug("No worries.");
|
||||||
|
}
|
||||||
|
Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
|
||||||
|
conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
|
||||||
|
conf.setBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY, true);
|
||||||
|
conf.setInt("hbase.slowlog.systable.chore.duration", 900);
|
||||||
|
conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", 50000);
|
||||||
|
HBASE_TESTING_UTILITY.startMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void teardown() throws Exception {
|
||||||
|
HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
HRegionServer hRegionServer = HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0);
|
||||||
|
Field slowLogRecorder = HRegionServer.class.getDeclaredField("namedQueueRecorder");
|
||||||
|
slowLogRecorder.setAccessible(true);
|
||||||
|
this.namedQueueRecorder = (NamedQueueRecorder) slowLogRecorder.get(hRegionServer);
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<TooSlowLog.SlowLogPayload> getSlowLogPayloads(
|
||||||
|
AdminProtos.SlowLogResponseRequest request) {
|
||||||
|
NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
|
||||||
|
namedQueueGetRequest.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT);
|
||||||
|
namedQueueGetRequest.setSlowLogResponseRequest(request);
|
||||||
|
NamedQueueGetResponse namedQueueGetResponse =
|
||||||
|
namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
|
||||||
|
return namedQueueGetResponse.getSlowLogPayloads();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSlowLogRecords() throws Exception {
|
||||||
|
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
|
||||||
|
namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
|
||||||
|
int i = 0;
|
||||||
|
|
||||||
|
final Connection connection = waitForSlowLogTableCreation();
|
||||||
|
// add 5 records initially
|
||||||
|
for (; i < 5; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
|
||||||
|
.getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
|
||||||
|
// add 2 more records
|
||||||
|
for (; i < 7; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
|
||||||
|
.getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
|
||||||
|
// add 3 more records
|
||||||
|
for (; i < 10; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
|
||||||
|
.getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
|
||||||
|
// add 4 more records
|
||||||
|
for (; i < 14; i++) {
|
||||||
|
RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
|
||||||
|
.getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY
|
||||||
|
.waitFor(3000, new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestSlowLogAccessor.this.getSlowLogPayloads(request).size() == 14;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1,
|
||||||
|
HBASE_TESTING_UTILITY.waitFor(3000, new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
return TestSlowLogAccessor.this.getTableCount(connection) == 14;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
private int getTableCount(Connection connection) {
|
||||||
|
try (Table table = connection.getTable(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) {
|
||||||
|
ResultScanner resultScanner = table.getScanner(new Scan().setReadType(Scan.ReadType.STREAM));
|
||||||
|
int count = 0;
|
||||||
|
for (Result result : resultScanner) {
|
||||||
|
++count;
|
||||||
|
}
|
||||||
|
return count;
|
||||||
|
} catch (Exception e) {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Connection waitForSlowLogTableCreation() throws Exception {
|
||||||
|
final Connection connection =
|
||||||
|
HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0).getConnection();
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(2000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override public boolean evaluate() throws Exception {
|
||||||
|
try {
|
||||||
|
return MetaTableAccessor
|
||||||
|
.tableExists(connection, SlowLogTableAccessor.SLOW_LOG_TABLE_NAME);
|
||||||
|
} catch (IOException e) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
return connection;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testHigherSlowLogs() throws Exception {
|
||||||
|
final Connection connection = waitForSlowLogTableCreation();
|
||||||
|
|
||||||
|
namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
|
||||||
|
final AdminProtos.SlowLogResponseRequest request =
|
||||||
|
AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(500000).build();
|
||||||
|
Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
|
||||||
|
|
||||||
|
for (int j = 0; j < 100; j++) {
|
||||||
|
new Thread(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
for (int i = 0; i < 350; i++) {
|
||||||
|
if (i == 300) {
|
||||||
|
try {
|
||||||
|
Thread.sleep(500);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
LOG.warn("Interrupted.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
|
||||||
|
.getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
|
||||||
|
namedQueueRecorder.addRecord(rpcLogDetails);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}).start();
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(8000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() throws Exception {
|
||||||
|
int count = TestSlowLogAccessor.this.getSlowLogPayloads(request).size();
|
||||||
|
LOG.debug("RingBuffer records count: {}", count);
|
||||||
|
return count > 1500;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
|
||||||
|
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(11000,
|
||||||
|
new Waiter.Predicate<Exception>() {
|
||||||
|
@Override public boolean evaluate() throws Exception {
|
||||||
|
int count = TestSlowLogAccessor.this.getTableCount(connection);
|
||||||
|
LOG.debug("SlowLog Table records count: {}", count);
|
||||||
|
return count > 1500;
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.*;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.*;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||||
import org.apache.hadoop.hbase.replication.TestReplicationBase;
|
import org.apache.hadoop.hbase.replication.TestReplicationBase;
|
||||||
import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint;
|
import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint;
|
||||||
|
@ -397,6 +398,18 @@ public class TestReplicator extends TestReplicationBase {
|
||||||
UpdateConfigurationRequest request) throws ServiceException {
|
UpdateConfigurationRequest request) throws ServiceException {
|
||||||
return delegate.updateConfiguration(controller, request);
|
return delegate.updateConfiguration(controller, request);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClearSlowLogResponses clearSlowLogsResponses(RpcController controller,
|
||||||
|
ClearSlowLogResponseRequest request) throws ServiceException {
|
||||||
|
return delegate.clearSlowLogsResponses(controller, request);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HBaseProtos.LogEntry getLogEntries(RpcController controller,
|
||||||
|
HBaseProtos.LogRequest request) throws ServiceException {
|
||||||
|
return delegate.getLogEntries(controller, request);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public class FailureInjectingReplicatorForTest extends ReplicatorForTest {
|
public class FailureInjectingReplicatorForTest extends ReplicatorForTest {
|
||||||
|
|
|
@ -1175,6 +1175,104 @@ module Hbase
|
||||||
@admin.deleteNamespace(namespace_name)
|
@admin.deleteNamespace(namespace_name)
|
||||||
end
|
end
|
||||||
|
|
||||||
|
#----------------------------------------------------------------------------------------------
|
||||||
|
# Retrieve SlowLog Responses from RegionServers
|
||||||
|
def get_slowlog_responses(server_names, args, is_large_log = false)
|
||||||
|
unless server_names.is_a?(Array) || server_names.is_a?(String)
|
||||||
|
raise(ArgumentError,
|
||||||
|
"#{server_names.class} of #{server_names.inspect} is not of Array/String type")
|
||||||
|
end
|
||||||
|
if server_names == '*'
|
||||||
|
server_names = getServerNames([])
|
||||||
|
else
|
||||||
|
server_names_list = to_server_names(server_names)
|
||||||
|
server_names = getServerNames(server_names_list)
|
||||||
|
end
|
||||||
|
filter_params = get_filter_params(args)
|
||||||
|
(args.key? 'LIMIT') ? limit = args['LIMIT'] : limit = 10
|
||||||
|
is_large_log ? log_type = 'LARGE_LOG' : log_type = 'SLOW_LOG'
|
||||||
|
log_dest = org.apache.hadoop.hbase.client.ServerType::REGION_SERVER
|
||||||
|
server_names_set = java.util.HashSet.new(server_names)
|
||||||
|
slow_log_responses = @admin.getLogEntries(server_names_set, log_type, log_dest, limit,
|
||||||
|
filter_params)
|
||||||
|
slow_log_responses_arr = []
|
||||||
|
slow_log_responses.each do |slow_log_response|
|
||||||
|
slow_log_responses_arr << slow_log_response.toJsonPrettyPrint
|
||||||
|
end
|
||||||
|
slow_log_responses_arr
|
||||||
|
end
|
||||||
|
|
||||||
|
def get_filter_params(args)
|
||||||
|
filter_params = java.util.HashMap.new
|
||||||
|
if args.key? 'REGION_NAME'
|
||||||
|
region_name = args['REGION_NAME']
|
||||||
|
filter_params.put('regionName', region_name)
|
||||||
|
end
|
||||||
|
if args.key? 'TABLE_NAME'
|
||||||
|
table_name = args['TABLE_NAME']
|
||||||
|
filter_params.put('tableName', table_name)
|
||||||
|
end
|
||||||
|
if args.key? 'CLIENT_IP'
|
||||||
|
client_address = args['CLIENT_IP']
|
||||||
|
filter_params.put('clientAddress', client_address)
|
||||||
|
end
|
||||||
|
if args.key? 'USER'
|
||||||
|
user = args['USER']
|
||||||
|
filter_params.put('userName', user)
|
||||||
|
end
|
||||||
|
if args.key? 'FILTER_BY_OP'
|
||||||
|
filter_by_op = args['FILTER_BY_OP']
|
||||||
|
if filter_by_op != 'OR' && filter_by_op != 'AND'
|
||||||
|
raise(ArgumentError, 'FILTER_BY_OP should be either OR / AND')
|
||||||
|
end
|
||||||
|
|
||||||
|
filter_params.put('filterByOperator', 'AND') if filter_by_op == 'AND'
|
||||||
|
end
|
||||||
|
filter_params
|
||||||
|
end
|
||||||
|
|
||||||
|
#----------------------------------------------------------------------------------------------
|
||||||
|
# Clears SlowLog Responses from RegionServers
|
||||||
|
def clear_slowlog_responses(server_names)
|
||||||
|
unless server_names.nil? || server_names.is_a?(Array) || server_names.is_a?(String)
|
||||||
|
raise(ArgumentError,
|
||||||
|
"#{server_names.class} of #{server_names.inspect} is not of correct type")
|
||||||
|
end
|
||||||
|
if server_names.nil?
|
||||||
|
server_names = getServerNames([])
|
||||||
|
else
|
||||||
|
server_names_list = to_server_names(server_names)
|
||||||
|
server_names = getServerNames(server_names_list)
|
||||||
|
end
|
||||||
|
clear_log_responses = @admin.clearSlowLogResponses(java.util.HashSet.new(server_names))
|
||||||
|
clear_log_success_count = 0
|
||||||
|
clear_log_responses.each do |response|
|
||||||
|
if response
|
||||||
|
clear_log_success_count += 1
|
||||||
|
end
|
||||||
|
end
|
||||||
|
puts 'Cleared Slowlog responses from ' \
|
||||||
|
"#{clear_log_success_count}/#{clear_log_responses.size} RegionServers"
|
||||||
|
end
|
||||||
|
|
||||||
|
#----------------------------------------------------------------------------------------------
|
||||||
|
# Retrieve latest balancer decisions made by LoadBalancers
|
||||||
|
def get_balancer_decisions(args)
|
||||||
|
if args.key? 'LIMIT'
|
||||||
|
limit = args['LIMIT']
|
||||||
|
else
|
||||||
|
limit = 250
|
||||||
|
end
|
||||||
|
log_type = 'BALANCER_DECISION'
|
||||||
|
log_dest = org.apache.hadoop.hbase.client.ServerType::MASTER
|
||||||
|
balancer_decisions_responses = @admin.getLogEntries(nil, log_type, log_dest, limit, nil)
|
||||||
|
balancer_decisions_resp_arr = []
|
||||||
|
balancer_decisions_responses.each do |balancer_dec_resp|
|
||||||
|
balancer_decisions_resp_arr << balancer_dec_resp.toJsonPrettyPrint
|
||||||
|
end
|
||||||
|
balancer_decisions_resp_arr
|
||||||
|
end
|
||||||
|
|
||||||
#----------------------------------------------------------------------------------------------
|
#----------------------------------------------------------------------------------------------
|
||||||
# Get security capabilities
|
# Get security capabilities
|
||||||
def get_security_capabilities
|
def get_security_capabilities
|
||||||
|
@ -1238,6 +1336,16 @@ module Hbase
|
||||||
@admin.listDeadServers.to_a
|
@admin.listDeadServers.to_a
|
||||||
end
|
end
|
||||||
|
|
||||||
|
#----------------------------------------------------------------------------------------------
|
||||||
|
# Get list of server names
|
||||||
|
def to_server_names(server_names)
|
||||||
|
if server_names.is_a?(Array)
|
||||||
|
server_names
|
||||||
|
else
|
||||||
|
java.util.Arrays.asList(server_names)
|
||||||
|
end
|
||||||
|
end
|
||||||
|
|
||||||
#----------------------------------------------------------------------------------------------
|
#----------------------------------------------------------------------------------------------
|
||||||
# clear dead region servers
|
# clear dead region servers
|
||||||
def clear_deadservers(dead_servers)
|
def clear_deadservers(dead_servers)
|
||||||
|
|
|
@ -330,10 +330,14 @@ Shell.load_command_group(
|
||||||
normalizer_switch
|
normalizer_switch
|
||||||
normalizer_enabled
|
normalizer_enabled
|
||||||
is_in_maintenance_mode
|
is_in_maintenance_mode
|
||||||
|
clear_slowlog_responses
|
||||||
close_region
|
close_region
|
||||||
compact
|
compact
|
||||||
compaction_switch
|
compaction_switch
|
||||||
flush
|
flush
|
||||||
|
get_balancer_decisions
|
||||||
|
get_slowlog_responses
|
||||||
|
get_largelog_responses
|
||||||
major_compact
|
major_compact
|
||||||
move
|
move
|
||||||
split
|
split
|
||||||
|
|
|
@ -0,0 +1,47 @@
|
||||||
|
#
|
||||||
|
#
|
||||||
|
# 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.
|
||||||
|
|
||||||
|
# Clear slowlog responses maintained in memory by RegionServers
|
||||||
|
|
||||||
|
module Shell
|
||||||
|
module Commands
|
||||||
|
# Clear slowlog responses
|
||||||
|
class ClearSlowlogResponses < Command
|
||||||
|
def help
|
||||||
|
<<-EOF
|
||||||
|
Clears SlowLog Responses maintained by each or specific RegionServers.
|
||||||
|
Specify array of server names for specific RS. A server name is
|
||||||
|
the host, port plus startcode of a RegionServer.
|
||||||
|
e.g.: host187.example.com,60020,1289493121758 (find servername in
|
||||||
|
master ui or when you do detailed status in shell)
|
||||||
|
|
||||||
|
Examples:
|
||||||
|
|
||||||
|
hbase> clear_slowlog_responses => clears slowlog responses from all RS
|
||||||
|
hbase> clear_slowlog_responses ['SERVER_NAME1', 'SERVER_NAME2'] => clears slowlog responses from SERVER_NAME1,
|
||||||
|
SERVER_NAME2
|
||||||
|
|
||||||
|
|
||||||
|
EOF
|
||||||
|
end
|
||||||
|
|
||||||
|
def command(server_names = nil)
|
||||||
|
admin.clear_slowlog_responses(server_names)
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
|
@ -0,0 +1,47 @@
|
||||||
|
#
|
||||||
|
#
|
||||||
|
# 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.
|
||||||
|
|
||||||
|
# Retrieve latest balancer decisions maintained in memory by HMaster
|
||||||
|
|
||||||
|
module Shell
|
||||||
|
module Commands
|
||||||
|
# Retrieve latest large log responses
|
||||||
|
class GetBalancerDecisions < Command
|
||||||
|
def help
|
||||||
|
<<-EOF
|
||||||
|
Retrieve latest balancer decisions made by LoadBalancers.
|
||||||
|
|
||||||
|
Examples:
|
||||||
|
|
||||||
|
hbase> get_balancer_decisions => Retrieve recent balancer decisions with
|
||||||
|
region plans
|
||||||
|
hbase> get_balancer_decisions LIMIT => 10 => Retrieve 10 most recent balancer decisions
|
||||||
|
with region plans
|
||||||
|
|
||||||
|
EOF
|
||||||
|
end
|
||||||
|
|
||||||
|
def command(args = {})
|
||||||
|
raise 'Filter parameters are not Hash' unless args.is_a? Hash
|
||||||
|
|
||||||
|
balancer_decisions_resp_arr = admin.get_balancer_decisions(args)
|
||||||
|
puts 'Retrieved BalancerDecision Responses'
|
||||||
|
puts balancer_decisions_resp_arr
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
|
@ -0,0 +1,98 @@
|
||||||
|
#
|
||||||
|
#
|
||||||
|
# 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.
|
||||||
|
|
||||||
|
# Retrieve latest large log responses maintained in memory by RegionServers
|
||||||
|
|
||||||
|
module Shell
|
||||||
|
module Commands
|
||||||
|
# Retrieve latest large log responses
|
||||||
|
class GetLargelogResponses < Command
|
||||||
|
def help
|
||||||
|
<<-EOF
|
||||||
|
Retrieve latest LargeLog Responses maintained by each or specific RegionServers.
|
||||||
|
Specify '*' to include all RS otherwise array of server names for specific
|
||||||
|
RS. A server name is the host, port plus startcode of a RegionServer.
|
||||||
|
e.g.: host187.example.com,60020,1289493121758 (find servername in
|
||||||
|
master ui or when you do detailed status in shell)
|
||||||
|
|
||||||
|
Provide optional filter parameters as Hash.
|
||||||
|
Default Limit of each server for providing no of large log records is 10. User can specify
|
||||||
|
more limit by 'LIMIT' param in case more than 10 records should be retrieved.
|
||||||
|
|
||||||
|
Examples:
|
||||||
|
|
||||||
|
hbase> get_largelog_responses '*' => get largelog responses from all RS
|
||||||
|
hbase> get_largelog_responses '*', {'LIMIT' => 50} => get largelog responses from all RS
|
||||||
|
with 50 records limit (default limit: 10)
|
||||||
|
hbase> get_largelog_responses ['SERVER_NAME1', 'SERVER_NAME2'] => get largelog responses from SERVER_NAME1,
|
||||||
|
SERVER_NAME2
|
||||||
|
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1'}
|
||||||
|
=> get largelog responses only related to meta
|
||||||
|
region
|
||||||
|
hbase> get_largelog_responses '*', {'TABLE_NAME' => 't1'} => get largelog responses only related to t1 table
|
||||||
|
hbase> get_largelog_responses '*', {'CLIENT_IP' => '192.162.1.40:60225', 'LIMIT' => 100}
|
||||||
|
=> get largelog responses with given client
|
||||||
|
IP address and get 100 records limit
|
||||||
|
(default limit: 10)
|
||||||
|
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1'}
|
||||||
|
=> get largelog responses with given region name
|
||||||
|
or table name
|
||||||
|
hbase> get_largelog_responses '*', {'USER' => 'user_name', 'CLIENT_IP' => '192.162.1.40:60225'}
|
||||||
|
=> get largelog responses that match either
|
||||||
|
provided client IP address or user name
|
||||||
|
|
||||||
|
All of above queries with filters have default OR operation applied i.e. all
|
||||||
|
records with any of the provided filters applied will be returned. However,
|
||||||
|
we can also apply AND operator i.e. all records that match all (not any) of
|
||||||
|
the provided filters should be returned.
|
||||||
|
|
||||||
|
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1', 'FILTER_BY_OP' => 'AND'}
|
||||||
|
=> get largelog responses with given region name
|
||||||
|
and table name, both should match
|
||||||
|
|
||||||
|
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1', 'FILTER_BY_OP' => 'OR'}
|
||||||
|
=> get largelog responses with given region name
|
||||||
|
or table name, any one can match
|
||||||
|
|
||||||
|
hbase> get_largelog_responses '*', {'TABLE_NAME' => 't1', 'CLIENT_IP' => '192.163.41.53:52781', 'FILTER_BY_OP' => 'AND'}
|
||||||
|
=> get largelog responses with given region name
|
||||||
|
and client IP address, both should match
|
||||||
|
|
||||||
|
Since OR is the default filter operator, without providing 'FILTER_BY_OP', query will have
|
||||||
|
same result as providing 'FILTER_BY_OP' => 'OR'.
|
||||||
|
|
||||||
|
Sometimes output can be long pretty printed json for user to scroll in
|
||||||
|
a single screen and hence user might prefer
|
||||||
|
redirecting output of get_largelog_responses to a file.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
echo "get_largelog_responses '*'" | hbase shell > xyz.out 2>&1
|
||||||
|
|
||||||
|
EOF
|
||||||
|
end
|
||||||
|
|
||||||
|
def command(server_names, args = {})
|
||||||
|
raise 'Filter parameters are not Hash' unless args.is_a? Hash
|
||||||
|
|
||||||
|
large_log_responses_arr = admin.get_slowlog_responses(server_names, args, true)
|
||||||
|
puts 'Retrieved LargeLog Responses from RegionServers'
|
||||||
|
puts large_log_responses_arr
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
|
@ -0,0 +1,98 @@
|
||||||
|
#
|
||||||
|
#
|
||||||
|
# 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.
|
||||||
|
|
||||||
|
# Retrieve latest slowlog responses maintained in memory by RegionServers
|
||||||
|
|
||||||
|
module Shell
|
||||||
|
module Commands
|
||||||
|
# Retrieve latest slowlog responses
|
||||||
|
class GetSlowlogResponses < Command
|
||||||
|
def help
|
||||||
|
<<-EOF
|
||||||
|
Retrieve latest SlowLog Responses maintained by each or specific RegionServers.
|
||||||
|
Specify '*' to include all RS otherwise array of server names for specific
|
||||||
|
RS. A server name is the host, port plus startcode of a RegionServer.
|
||||||
|
e.g.: host187.example.com,60020,1289493121758 (find servername in
|
||||||
|
master ui or when you do detailed status in shell)
|
||||||
|
|
||||||
|
Provide optional filter parameters as Hash.
|
||||||
|
Default Limit of each server for providing no of slow log records is 10. User can specify
|
||||||
|
more limit by 'LIMIT' param in case more than 10 records should be retrieved.
|
||||||
|
|
||||||
|
Examples:
|
||||||
|
|
||||||
|
hbase> get_slowlog_responses '*' => get slowlog responses from all RS
|
||||||
|
hbase> get_slowlog_responses '*', {'LIMIT' => 50} => get slowlog responses from all RS
|
||||||
|
with 50 records limit (default limit: 10)
|
||||||
|
hbase> get_slowlog_responses ['SERVER_NAME1', 'SERVER_NAME2'] => get slowlog responses from SERVER_NAME1,
|
||||||
|
SERVER_NAME2
|
||||||
|
hbase> get_slowlog_responses '*', {'REGION_NAME' => 'hbase:meta,,1'}
|
||||||
|
=> get slowlog responses only related to meta
|
||||||
|
region
|
||||||
|
hbase> get_slowlog_responses '*', {'TABLE_NAME' => 't1'} => get slowlog responses only related to t1 table
|
||||||
|
hbase> get_slowlog_responses '*', {'CLIENT_IP' => '192.162.1.40:60225', 'LIMIT' => 100}
|
||||||
|
=> get slowlog responses with given client
|
||||||
|
IP address and get 100 records limit
|
||||||
|
(default limit: 10)
|
||||||
|
hbase> get_slowlog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1'}
|
||||||
|
=> get slowlog responses with given region name
|
||||||
|
or table name
|
||||||
|
hbase> get_slowlog_responses '*', {'USER' => 'user_name', 'CLIENT_IP' => '192.162.1.40:60225'}
|
||||||
|
=> get slowlog responses that match either
|
||||||
|
provided client IP address or user name
|
||||||
|
|
||||||
|
All of above queries with filters have default OR operation applied i.e. all
|
||||||
|
records with any of the provided filters applied will be returned. However,
|
||||||
|
we can also apply AND operator i.e. all records that match all (not any) of
|
||||||
|
the provided filters should be returned.
|
||||||
|
|
||||||
|
hbase> get_slowlog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1', 'FILTER_BY_OP' => 'AND'}
|
||||||
|
=> get slowlog responses with given region name
|
||||||
|
and table name, both should match
|
||||||
|
|
||||||
|
hbase> get_slowlog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1', 'FILTER_BY_OP' => 'OR'}
|
||||||
|
=> get slowlog responses with given region name
|
||||||
|
or table name, any one can match
|
||||||
|
|
||||||
|
hbase> get_slowlog_responses '*', {'TABLE_NAME' => 't1', 'CLIENT_IP' => '192.163.41.53:52781', 'FILTER_BY_OP' => 'AND'}
|
||||||
|
=> get slowlog responses with given region name
|
||||||
|
and client IP address, both should match
|
||||||
|
|
||||||
|
Since OR is the default filter operator, without providing 'FILTER_BY_OP', query will have
|
||||||
|
same result as providing 'FILTER_BY_OP' => 'OR'.
|
||||||
|
|
||||||
|
Sometimes output can be long pretty printed json for user to scroll in
|
||||||
|
a single screen and hence user might prefer
|
||||||
|
redirecting output of get_slowlog_responses to a file.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
echo "get_slowlog_responses '*'" | hbase shell > xyz.out 2>&1
|
||||||
|
|
||||||
|
EOF
|
||||||
|
end
|
||||||
|
|
||||||
|
def command(server_names, args = {})
|
||||||
|
raise 'Filter parameters are not Hash' unless args.is_a? Hash
|
||||||
|
|
||||||
|
slow_log_responses_arr = admin.get_slowlog_responses(server_names, args)
|
||||||
|
puts 'Retrieved SlowLog Responses from RegionServers'
|
||||||
|
puts slow_log_responses_arr
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
|
@ -2273,3 +2273,75 @@ The percent of region server RPC threads failed to abort RS.
|
||||||
+
|
+
|
||||||
.Default
|
.Default
|
||||||
`false`
|
`false`
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
[[hbase.regionserver.slowlog.ringbuffer.size]]
|
||||||
|
*`hbase.regionserver.slowlog.ringbuffer.size`*::
|
||||||
|
+
|
||||||
|
.Description
|
||||||
|
|
||||||
|
Default size of ringbuffer to be maintained by each RegionServer in order
|
||||||
|
to store online slowlog responses. This is an in-memory ring buffer of
|
||||||
|
requests that were judged to be too slow in addition to the responseTooSlow
|
||||||
|
logging. The in-memory representation would be complete.
|
||||||
|
For more details, please look into Doc Section:
|
||||||
|
<<slow_log_responses, slow_log_responses>>
|
||||||
|
|
||||||
|
|
||||||
|
+
|
||||||
|
.Default
|
||||||
|
`256`
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
[[hbase.regionserver.slowlog.buffer.enabled]]
|
||||||
|
*`hbase.regionserver.slowlog.buffer.enabled`*::
|
||||||
|
+
|
||||||
|
.Description
|
||||||
|
|
||||||
|
Indicates whether RegionServers have ring buffer running for storing
|
||||||
|
Online Slow logs in FIFO manner with limited entries. The size of
|
||||||
|
the ring buffer is indicated by config: hbase.regionserver.slowlog.ringbuffer.size
|
||||||
|
The default value is false, turn this on and get latest slowlog
|
||||||
|
responses with complete data.
|
||||||
|
For more details, please look into Doc Section:
|
||||||
|
<<slow_log_responses, slow_log_responses>>
|
||||||
|
|
||||||
|
|
||||||
|
+
|
||||||
|
.Default
|
||||||
|
`false`
|
||||||
|
|
||||||
|
|
||||||
|
[[hbase.regionserver.slowlog.systable.enabled]]
|
||||||
|
*`hbase.regionserver.slowlog.systable.enabled`*::
|
||||||
|
+
|
||||||
|
.Description
|
||||||
|
|
||||||
|
Should be enabled only if hbase.regionserver.slowlog.buffer.enabled is enabled.
|
||||||
|
If enabled (true), all slow/large RPC logs would be persisted to system table
|
||||||
|
hbase:slowlog (in addition to in-memory ring buffer at each RegionServer).
|
||||||
|
The records are stored in increasing order of time.
|
||||||
|
Operators can scan the table with various combination of ColumnValueFilter and
|
||||||
|
time range.
|
||||||
|
More details are provided in the doc section:
|
||||||
|
"Get Slow/Large Response Logs from System table hbase:slowlog"
|
||||||
|
|
||||||
|
+
|
||||||
|
.Default
|
||||||
|
`false`
|
||||||
|
|
||||||
|
[[hbase.master.balancer.decision.buffer.enabled]]
|
||||||
|
*`hbase.master.balancer.decision.buffer.enabled`*::
|
||||||
|
+
|
||||||
|
.Description
|
||||||
|
|
||||||
|
Indicates whether active HMaster has ring buffer running for storing
|
||||||
|
balancer decisions in FIFO manner with limited entries. The size of
|
||||||
|
the ring buffer is indicated by config:
|
||||||
|
hbase.master.balancer.decision.queue.size
|
||||||
|
|
||||||
|
+
|
||||||
|
.Default
|
||||||
|
`false`
|
||||||
|
|
|
@ -1472,6 +1472,187 @@ In the case that the call is not a client operation, that detailed fingerprint i
|
||||||
|
|
||||||
This particular example, for example, would indicate that the likely cause of slowness is simply a very large (on the order of 100MB) multiput, as we can tell by the "vlen," or value length, fields of each put in the multiPut.
|
This particular example, for example, would indicate that the likely cause of slowness is simply a very large (on the order of 100MB) multiput, as we can tell by the "vlen," or value length, fields of each put in the multiPut.
|
||||||
|
|
||||||
|
|
||||||
|
[[slow_log_responses]]
|
||||||
|
==== Get Slow Response Log from shell
|
||||||
|
When an individual RPC exceeds a configurable time bound we log a complaint
|
||||||
|
by way of the logging subsystem
|
||||||
|
|
||||||
|
e.g.
|
||||||
|
|
||||||
|
----
|
||||||
|
2019-10-02 10:10:22,195 WARN [,queue=15,port=60020] ipc.RpcServer - (responseTooSlow):
|
||||||
|
{"call":"Scan(org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ScanRequest)",
|
||||||
|
"starttimems":1567203007549,
|
||||||
|
"responsesize":6819737,
|
||||||
|
"method":"Scan",
|
||||||
|
"param":"region { type: REGION_NAME value: \"t1,\\000\\000\\215\\f)o\\\\\\024\\302\\220\\000\\000\\000\\000\\000\\001\\000\\000\\000\\000\\000\\006\\000\\000\\000\\000\\000\\005\\000\\000<TRUNCATED>",
|
||||||
|
"processingtimems":28646,
|
||||||
|
"client":"10.253.196.215:41116",
|
||||||
|
"queuetimems":22453,
|
||||||
|
"class":"HRegionServer"}
|
||||||
|
----
|
||||||
|
|
||||||
|
Unfortunately often the request parameters are truncated as per above Example.
|
||||||
|
The truncation is unfortunate because it eliminates much of the utility of
|
||||||
|
the warnings. For example, the region name, the start and end keys, and the
|
||||||
|
filter hierarchy are all important clues for debugging performance problems
|
||||||
|
caused by moderate to low selectivity queries or queries made at a high rate.
|
||||||
|
|
||||||
|
HBASE-22978 introduces maintaining an in-memory ring buffer of requests that were judged to
|
||||||
|
be too slow in addition to the responseTooSlow logging. The in-memory representation can be
|
||||||
|
complete. There is some chance a high rate of requests will cause information on other
|
||||||
|
interesting requests to be overwritten before it can be read. This is an acceptable trade off.
|
||||||
|
|
||||||
|
In order to enable the in-memory ring buffer at RegionServers, we need to enable
|
||||||
|
config:
|
||||||
|
----
|
||||||
|
hbase.regionserver.slowlog.buffer.enabled
|
||||||
|
----
|
||||||
|
|
||||||
|
One more config determines the size of the ring buffer:
|
||||||
|
----
|
||||||
|
hbase.regionserver.slowlog.ringbuffer.size
|
||||||
|
----
|
||||||
|
|
||||||
|
Check the config section for the detailed description.
|
||||||
|
|
||||||
|
This config would be disabled by default. Turn it on and these shell commands
|
||||||
|
would provide expected results from the ring-buffers.
|
||||||
|
|
||||||
|
|
||||||
|
shell commands to retrieve slowlog responses from RegionServers:
|
||||||
|
|
||||||
|
----
|
||||||
|
Retrieve latest SlowLog Responses maintained by each or specific RegionServers.
|
||||||
|
Specify '*' to include all RS otherwise array of server names for specific
|
||||||
|
RS. A server name is the host, port plus startcode of a RegionServer.
|
||||||
|
e.g.: host187.example.com,60020,1289493121758 (find servername in
|
||||||
|
master ui or when you do detailed status in shell)
|
||||||
|
|
||||||
|
Provide optional filter parameters as Hash.
|
||||||
|
Default Limit of each server for providing no of slow log records is 10. User can specify
|
||||||
|
more limit by 'LIMIT' param in case more than 10 records should be retrieved.
|
||||||
|
|
||||||
|
Examples:
|
||||||
|
|
||||||
|
hbase> get_slowlog_responses '*' => get slowlog responses from all RS
|
||||||
|
hbase> get_slowlog_responses '*', {'LIMIT' => 50} => get slowlog responses from all RS
|
||||||
|
with 50 records limit (default limit: 10)
|
||||||
|
hbase> get_slowlog_responses ['SERVER_NAME1', 'SERVER_NAME2'] => get slowlog responses from SERVER_NAME1,
|
||||||
|
SERVER_NAME2
|
||||||
|
hbase> get_slowlog_responses '*', {'REGION_NAME' => 'hbase:meta,,1'}
|
||||||
|
=> get slowlog responses only related to meta
|
||||||
|
region
|
||||||
|
hbase> get_slowlog_responses '*', {'TABLE_NAME' => 't1'} => get slowlog responses only related to t1 table
|
||||||
|
hbase> get_slowlog_responses '*', {'CLIENT_IP' => '192.162.1.40:60225', 'LIMIT' => 100}
|
||||||
|
=> get slowlog responses with given client
|
||||||
|
IP address and get 100 records limit
|
||||||
|
(default limit: 10)
|
||||||
|
hbase> get_slowlog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1'}
|
||||||
|
=> get slowlog responses with given region name
|
||||||
|
or table name
|
||||||
|
hbase> get_slowlog_responses '*', {'USER' => 'user_name', 'CLIENT_IP' => '192.162.1.40:60225'}
|
||||||
|
=> get slowlog responses that match either
|
||||||
|
provided client IP address or user name
|
||||||
|
|
||||||
|
|
||||||
|
----
|
||||||
|
|
||||||
|
All of above queries with filters have default OR operation applied i.e. all
|
||||||
|
records with any of the provided filters applied will be returned. However,
|
||||||
|
we can also apply AND operator i.e. all records that match all (not any) of
|
||||||
|
the provided filters should be returned.
|
||||||
|
|
||||||
|
----
|
||||||
|
hbase> get_slowlog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1', 'FILTER_BY_OP' => 'AND'}
|
||||||
|
=> get slowlog responses with given region name
|
||||||
|
and table name, both should match
|
||||||
|
|
||||||
|
hbase> get_slowlog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1', 'FILTER_BY_OP' => 'OR'}
|
||||||
|
=> get slowlog responses with given region name
|
||||||
|
or table name, any one can match
|
||||||
|
|
||||||
|
hbase> get_slowlog_responses '*', {'TABLE_NAME' => 't1', 'CLIENT_IP' => '192.163.41.53:52781', 'FILTER_BY_OP' => 'AND'}
|
||||||
|
=> get slowlog responses with given region name
|
||||||
|
and client IP address, both should match
|
||||||
|
|
||||||
|
----
|
||||||
|
Since OR is the default filter operator, without providing 'FILTER_BY_OP', query will have
|
||||||
|
same result as providing 'FILTER_BY_OP' => 'OR'.
|
||||||
|
|
||||||
|
|
||||||
|
Sometimes output can be long pretty printed json for user to scroll in
|
||||||
|
a single screen and hence user might prefer
|
||||||
|
redirecting output of get_slowlog_responses to a file.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
----
|
||||||
|
echo "get_slowlog_responses '*'" | hbase shell > xyz.out 2>&1
|
||||||
|
----
|
||||||
|
|
||||||
|
Similar to slow RPC logs, client can also retrieve large RPC logs.
|
||||||
|
Sometimes, slow logs important to debug perf issues turn out to be
|
||||||
|
larger in size.
|
||||||
|
|
||||||
|
----
|
||||||
|
|
||||||
|
hbase> get_largelog_responses '*' => get largelog responses from all RS
|
||||||
|
hbase> get_largelog_responses '*', {'LIMIT' => 50} => get largelog responses from all RS
|
||||||
|
with 50 records limit (default limit: 10)
|
||||||
|
hbase> get_largelog_responses ['SERVER_NAME1', 'SERVER_NAME2'] => get largelog responses from SERVER_NAME1,
|
||||||
|
SERVER_NAME2
|
||||||
|
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1'}
|
||||||
|
=> get largelog responses only related to meta
|
||||||
|
region
|
||||||
|
hbase> get_largelog_responses '*', {'TABLE_NAME' => 't1'} => get largelog responses only related to t1 table
|
||||||
|
hbase> get_largelog_responses '*', {'CLIENT_IP' => '192.162.1.40:60225', 'LIMIT' => 100}
|
||||||
|
=> get largelog responses with given client
|
||||||
|
IP address and get 100 records limit
|
||||||
|
(default limit: 10)
|
||||||
|
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1'}
|
||||||
|
=> get largelog responses with given region name
|
||||||
|
or table name
|
||||||
|
hbase> get_largelog_responses '*', {'USER' => 'user_name', 'CLIENT_IP' => '192.162.1.40:60225'}
|
||||||
|
=> get largelog responses that match either
|
||||||
|
provided client IP address or user name
|
||||||
|
|
||||||
|
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1', 'FILTER_BY_OP' => 'AND'}
|
||||||
|
=> get largelog responses with given region name
|
||||||
|
and table name, both should match
|
||||||
|
|
||||||
|
hbase> get_largelog_responses '*', {'REGION_NAME' => 'hbase:meta,,1', 'TABLE_NAME' => 't1', 'FILTER_BY_OP' => 'OR'}
|
||||||
|
=> get largelog responses with given region name
|
||||||
|
or table name, any one can match
|
||||||
|
|
||||||
|
hbase> get_largelog_responses '*', {'TABLE_NAME' => 't1', 'CLIENT_IP' => '192.163.41.53:52781', 'FILTER_BY_OP' => 'AND'}
|
||||||
|
=> get largelog responses with given region name
|
||||||
|
and client IP address, both should match
|
||||||
|
|
||||||
|
----
|
||||||
|
|
||||||
|
|
||||||
|
shell command to clear slow/largelog responses from RegionServer:
|
||||||
|
|
||||||
|
----
|
||||||
|
Clears SlowLog Responses maintained by each or specific RegionServers.
|
||||||
|
Specify array of server names for specific RS. A server name is
|
||||||
|
the host, port plus startcode of a RegionServer.
|
||||||
|
e.g.: host187.example.com,60020,1289493121758 (find servername in
|
||||||
|
master ui or when you do detailed status in shell)
|
||||||
|
|
||||||
|
Examples:
|
||||||
|
|
||||||
|
hbase> clear_slowlog_responses => clears slowlog responses from all RS
|
||||||
|
hbase> clear_slowlog_responses ['SERVER_NAME1', 'SERVER_NAME2'] => clears slowlog responses from SERVER_NAME1,
|
||||||
|
SERVER_NAME2
|
||||||
|
|
||||||
|
|
||||||
|
----
|
||||||
|
|
||||||
|
include::slow_log_responses_from_systable.adoc[]
|
||||||
|
|
||||||
|
|
||||||
=== Block Cache Monitoring
|
=== Block Cache Monitoring
|
||||||
|
|
||||||
Starting with HBase 0.98, the HBase Web UI includes the ability to monitor and report on the performance of the block cache.
|
Starting with HBase 0.98, the HBase Web UI includes the ability to monitor and report on the performance of the block cache.
|
||||||
|
|
|
@ -0,0 +1,118 @@
|
||||||
|
////
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
////
|
||||||
|
|
||||||
|
[[slow_log_responses_from_systable]]
|
||||||
|
==== Get Slow/Large Response Logs from System table hbase:slowlog
|
||||||
|
|
||||||
|
The above section provides details about Admin APIs:
|
||||||
|
|
||||||
|
* get_slowlog_responses
|
||||||
|
* get_largelog_responses
|
||||||
|
* clear_slowlog_responses
|
||||||
|
|
||||||
|
All of the above APIs access online in-memory ring buffers from
|
||||||
|
individual RegionServers and accumulate logs from ring buffers to display
|
||||||
|
to end user. However, since the logs are stored in memory, after RegionServer is
|
||||||
|
restarted, all the objects held in memory of that RegionServer will be cleaned up
|
||||||
|
and previous logs are lost. What if we want to persist all these logs forever?
|
||||||
|
What if we want to store them in such a manner that operator can get all historical
|
||||||
|
records with some filters? e.g get me all large/slow RPC logs that are triggered by
|
||||||
|
user1 and are related to region:
|
||||||
|
cluster_test,cccccccc,1589635796466.aa45e1571d533f5ed0bb31cdccaaf9cf. ?
|
||||||
|
|
||||||
|
If we have a system table that stores such logs in increasing (not so strictly though)
|
||||||
|
order of time, it can definitely help operators debug some historical events
|
||||||
|
(scan, get, put, compaction, flush etc) with detailed inputs.
|
||||||
|
|
||||||
|
Config which enabled system table to be created and store all log events is
|
||||||
|
`hbase.regionserver.slowlog.systable.enabled`.
|
||||||
|
|
||||||
|
The default value for this config is `false`. If provided `true`
|
||||||
|
(Note: `hbase.regionserver.slowlog.buffer.enabled` should also be `true`),
|
||||||
|
a cron job running in every RegionServer will persist the slow/large logs into
|
||||||
|
table hbase:slowlog. By default cron job runs every 10 min. Duration can be configured
|
||||||
|
with key: `hbase.slowlog.systable.chore.duration`. By default, RegionServer will
|
||||||
|
store upto 1000(config key: `hbase.regionserver.slowlog.systable.queue.size`)
|
||||||
|
slow/large logs in an internal queue and the chore will retrieve these logs
|
||||||
|
from the queue and perform batch insertion in hbase:slowlog.
|
||||||
|
|
||||||
|
hbase:slowlog has single ColumnFamily: `info`
|
||||||
|
`info` contains multiple qualifiers which are the same attributes present as
|
||||||
|
part of `get_slowlog_responses` API response.
|
||||||
|
|
||||||
|
* info:call_details
|
||||||
|
* info:client_address
|
||||||
|
* info:method_name
|
||||||
|
* info:param
|
||||||
|
* info:processing_time
|
||||||
|
* info:queue_time
|
||||||
|
* info:region_name
|
||||||
|
* info:response_size
|
||||||
|
* info:server_class
|
||||||
|
* info:start_time
|
||||||
|
* info:type
|
||||||
|
* info:username
|
||||||
|
|
||||||
|
And example of 2 rows from hbase:slowlog scan result:
|
||||||
|
[source]
|
||||||
|
----
|
||||||
|
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:call_details, timestamp=2020-05-16T14:58:14.211Z, value=Scan(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ScanRequest)
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:client_address, timestamp=2020-05-16T14:58:14.211Z, value=172.20.10.2:57347
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:method_name, timestamp=2020-05-16T14:58:14.211Z, value=Scan
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:param, timestamp=2020-05-16T14:58:14.211Z, value=region { type: REGION_NAME value: "hbase:meta,,1" } scan { column { family: "info" } attribute { name: "_isolationle
|
||||||
|
vel_" value: "\x5C000" } start_row: "cluster_test,33333333,99999999999999" stop_row: "cluster_test,," time_range { from: 0 to: 9223372036854775807 } max_versions: 1 cache_blocks
|
||||||
|
: true max_result_size: 2097152 reversed: true caching: 10 include_stop_row: true readType: PREAD } number_of_rows: 10 close_scanner: false client_handles_partials: true client_
|
||||||
|
handles_heartbeats: true track_scan_metrics: false
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:processing_time, timestamp=2020-05-16T14:58:14.211Z, value=18
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:queue_time, timestamp=2020-05-16T14:58:14.211Z, value=0
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:region_name, timestamp=2020-05-16T14:58:14.211Z, value=hbase:meta,,1
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:response_size, timestamp=2020-05-16T14:58:14.211Z, value=1575
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:server_class, timestamp=2020-05-16T14:58:14.211Z, value=HRegionServer
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:start_time, timestamp=2020-05-16T14:58:14.211Z, value=1589640743732
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:type, timestamp=2020-05-16T14:58:14.211Z, value=ALL
|
||||||
|
\x024\xC1\x03\xE9\x04\xF5@ column=info:username, timestamp=2020-05-16T14:58:14.211Z, value=user2
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:call_details, timestamp=2020-05-16T14:59:58.764Z, value=Scan(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ScanRequest)
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:client_address, timestamp=2020-05-16T14:59:58.764Z, value=172.20.10.2:57348
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:method_name, timestamp=2020-05-16T14:59:58.764Z, value=Scan
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:param, timestamp=2020-05-16T14:59:58.764Z, value=region { type: REGION_NAME value: "cluster_test,cccccccc,1589635796466.aa45e1571d533f5ed0bb31cdccaaf9cf." } scan { a
|
||||||
|
ttribute { name: "_isolationlevel_" value: "\x5C000" } start_row: "cccccccc" time_range { from: 0 to: 9223372036854775807 } max_versions: 1 cache_blocks: true max_result_size: 2
|
||||||
|
097152 caching: 2147483647 include_stop_row: false } number_of_rows: 2147483647 close_scanner: false client_handles_partials: true client_handles_heartbeats: true track_scan_met
|
||||||
|
rics: false
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:processing_time, timestamp=2020-05-16T14:59:58.764Z, value=24
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:queue_time, timestamp=2020-05-16T14:59:58.764Z, value=0
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:region_name, timestamp=2020-05-16T14:59:58.764Z, value=cluster_test,cccccccc,1589635796466.aa45e1571d533f5ed0bb31cdccaaf9cf.
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:response_size, timestamp=2020-05-16T14:59:58.764Z, value=211227
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:server_class, timestamp=2020-05-16T14:59:58.764Z, value=HRegionServer
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:start_time, timestamp=2020-05-16T14:59:58.764Z, value=1589640743932
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:type, timestamp=2020-05-16T14:59:58.764Z, value=ALL
|
||||||
|
\x024\xC1\x06X\x81\xF6\xEC column=info:username, timestamp=2020-05-16T14:59:58.764Z, value=user1
|
||||||
|
----
|
||||||
|
|
||||||
|
Operator can use ColumnValueFilter to filter records based on region_name, username,
|
||||||
|
client_address etc.
|
||||||
|
|
||||||
|
Time range based queries will also be very useful.
|
||||||
|
Example:
|
||||||
|
[source]
|
||||||
|
----
|
||||||
|
scan 'hbase:slowlog', { TIMERANGE => [1589621394000, 1589637999999] }
|
||||||
|
----
|
Loading…
Reference in New Issue