Skip to content

HBASE-23065 [hbtop] Top-N heavy hitter user and client drill downs #722

New issue

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

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

Already on GitHub? Sign in to your account

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,11 @@ default String getVersion() {
*/
Map<byte[], RegionMetrics> getRegionMetrics();

/**
* @return metrics per user
*/
Map<byte[], UserMetrics> getUserMetrics();

/**
* Return the RegionServer-level and Region-level coprocessors
* @return string set of loaded RegionServer-level and Region-level coprocessors
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,8 @@ public static ServerMetrics toServerMetrics(ServerName serverName, int versionNu
.map(HBaseProtos.Coprocessor::getName).collect(Collectors.toList()))
.setRegionMetrics(serverLoadPB.getRegionLoadsList().stream()
.map(RegionMetricsBuilder::toRegionMetrics).collect(Collectors.toList()))
.setUserMetrics(serverLoadPB.getUserLoadsList().stream()
.map(UserMetricsBuilder::toUserMetrics).collect(Collectors.toList()))
.setReplicationLoadSources(serverLoadPB.getReplLoadSourceList().stream()
.map(ProtobufUtil::toReplicationLoadSource).collect(Collectors.toList()))
.setReplicationLoadSink(serverLoadPB.hasReplLoadSink()
Expand All @@ -100,19 +102,19 @@ public static ClusterStatusProtos.ServerLoad toServerLoad(ServerMetrics metrics)
.setInfoServerPort(metrics.getInfoServerPort())
.setMaxHeapMB((int) metrics.getMaxHeapSize().get(Size.Unit.MEGABYTE))
.setUsedHeapMB((int) metrics.getUsedHeapSize().get(Size.Unit.MEGABYTE))
.addAllCoprocessors(toCoprocessor(metrics.getCoprocessorNames()))
.addAllRegionLoads(metrics.getRegionMetrics().values().stream()
.map(RegionMetricsBuilder::toRegionLoad)
.collect(Collectors.toList()))
.addAllReplLoadSource(metrics.getReplicationLoadSourceList().stream()
.map(ProtobufUtil::toReplicationLoadSource)
.collect(Collectors.toList()))
.addAllCoprocessors(toCoprocessor(metrics.getCoprocessorNames())).addAllRegionLoads(
metrics.getRegionMetrics().values().stream().map(RegionMetricsBuilder::toRegionLoad)
.collect(Collectors.toList())).addAllUserLoads(
metrics.getUserMetrics().values().stream().map(UserMetricsBuilder::toUserMetrics)
.collect(Collectors.toList())).addAllReplLoadSource(
metrics.getReplicationLoadSourceList().stream()
.map(ProtobufUtil::toReplicationLoadSource).collect(Collectors.toList()))
.setReportStartTime(metrics.getLastReportTimestamp())
.setReportEndTime(metrics.getReportTimestamp());
if (metrics.getReplicationLoadSink() != null) {
builder.setReplLoadSink(ProtobufUtil.toReplicationLoadSink(
metrics.getReplicationLoadSink()));
builder.setReplLoadSink(ProtobufUtil.toReplicationLoadSink(metrics.getReplicationLoadSink()));
}

return builder.build();
}

Expand All @@ -132,6 +134,7 @@ public static ServerMetricsBuilder newBuilder(ServerName sn) {
@Nullable
private ReplicationLoadSink sink = null;
private final Map<byte[], RegionMetrics> regionStatus = new TreeMap<>(Bytes.BYTES_COMPARATOR);
private final Map<byte[], UserMetrics> userMetrics = new TreeMap<>(Bytes.BYTES_COMPARATOR);
private final Set<String> coprocessorNames = new TreeSet<>();
private long reportTimestamp = System.currentTimeMillis();
private long lastReportTimestamp = 0;
Expand Down Expand Up @@ -189,6 +192,11 @@ public ServerMetricsBuilder setRegionMetrics(List<RegionMetrics> value) {
return this;
}

public ServerMetricsBuilder setUserMetrics(List<UserMetrics> value) {
value.forEach(v -> this.userMetrics.put(v.getUserName(), v));
return this;
}

public ServerMetricsBuilder setCoprocessorNames(List<String> value) {
coprocessorNames.addAll(value);
return this;
Expand Down Expand Up @@ -219,7 +227,8 @@ public ServerMetrics build() {
regionStatus,
coprocessorNames,
reportTimestamp,
lastReportTimestamp);
lastReportTimestamp,
userMetrics);
}

private static class ServerMetricsImpl implements ServerMetrics {
Expand All @@ -238,12 +247,13 @@ private static class ServerMetricsImpl implements ServerMetrics {
private final Set<String> coprocessorNames;
private final long reportTimestamp;
private final long lastReportTimestamp;
private final Map<byte[], UserMetrics> userMetrics;

ServerMetricsImpl(ServerName serverName, int versionNumber, String version,
long requestCountPerSecond, long requestCount, Size usedHeapSize, Size maxHeapSize,
int infoServerPort, List<ReplicationLoadSource> sources, ReplicationLoadSink sink,
Map<byte[], RegionMetrics> regionStatus, Set<String> coprocessorNames, long reportTimestamp,
long lastReportTimestamp) {
long lastReportTimestamp, Map<byte[], UserMetrics> userMetrics) {
this.serverName = Preconditions.checkNotNull(serverName);
this.versionNumber = versionNumber;
this.version = version;
Expand All @@ -255,6 +265,7 @@ private static class ServerMetricsImpl implements ServerMetrics {
this.sources = Preconditions.checkNotNull(sources);
this.sink = sink;
this.regionStatus = Preconditions.checkNotNull(regionStatus);
this.userMetrics = Preconditions.checkNotNull(userMetrics);
this.coprocessorNames =Preconditions.checkNotNull(coprocessorNames);
this.reportTimestamp = reportTimestamp;
this.lastReportTimestamp = lastReportTimestamp;
Expand Down Expand Up @@ -324,6 +335,11 @@ public Map<byte[], RegionMetrics> getRegionMetrics() {
return Collections.unmodifiableMap(regionStatus);
}

@Override
public Map<byte[], UserMetrics> getUserMetrics() {
return Collections.unmodifiableMap(userMetrics);
}

@Override
public Set<String> getCoprocessorNames() {
return Collections.unmodifiableSet(coprocessorNames);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/**
* Copyright The Apache Software Foundation
*
* 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;

import java.util.Map;

import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;

/**
* Encapsulates per-user load metrics.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface UserMetrics {

interface ClientMetrics {

String getHostName();

long getReadRequestsCount();

long getWriteRequestsCount();

long getFilteredReadRequestsCount();
}

/**
* @return the user name
*/
byte[] getUserName();

/**
* @return the number of read requests made by user
*/
long getReadRequestCount();

/**
* @return the number of write requests made by user
*/
long getWriteRequestCount();

/**
* @return the number of write requests and read requests and coprocessor
* service requests made by the user
*/
default long getRequestCount() {
return getReadRequestCount() + getWriteRequestCount();
}

/**
* @return the user name as a string
*/
default String getNameAsString() {
return Bytes.toStringBinary(getUserName());
}

/**
* @return metrics per client(hostname)
*/
Map<String, ClientMetrics> getClientMetrics();

/**
* @return count of filtered read requests for a user
*/
long getFilteredReadRequests();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
/**
* Copyright The Apache Software Foundation
*
* 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;

import java.util.HashMap;
import java.util.Map;

import org.apache.hadoop.hbase.util.Strings;
import org.apache.yetus.audience.InterfaceAudience;

import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;

import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;

@InterfaceAudience.Private
public final class UserMetricsBuilder {

public static UserMetrics toUserMetrics(ClusterStatusProtos.UserLoad userLoad) {
UserMetricsBuilder builder = UserMetricsBuilder.newBuilder(userLoad.getUserName().getBytes());
userLoad.getClientMetricsList().stream().map(
clientMetrics -> new ClientMetricsImpl(clientMetrics.getHostName(),
clientMetrics.getReadRequestsCount(), clientMetrics.getWriteRequestsCount(),
clientMetrics.getFilteredRequestsCount())).forEach(builder::addClientMetris);
return builder.build();
}

public static ClusterStatusProtos.UserLoad toUserMetrics(UserMetrics userMetrics) {
ClusterStatusProtos.UserLoad.Builder builder =
ClusterStatusProtos.UserLoad.newBuilder().setUserName(userMetrics.getNameAsString());
userMetrics.getClientMetrics().values().stream().map(
clientMetrics -> ClusterStatusProtos.ClientMetrics.newBuilder()
.setHostName(clientMetrics.getHostName())
.setWriteRequestsCount(clientMetrics.getWriteRequestsCount())
.setReadRequestsCount(clientMetrics.getReadRequestsCount())
.setFilteredRequestsCount(clientMetrics.getFilteredReadRequestsCount()).build())
.forEach(builder::addClientMetrics);
return builder.build();
}

public static UserMetricsBuilder newBuilder(byte[] name) {
return new UserMetricsBuilder(name);
}


private final byte[] name;
private Map<String, UserMetrics.ClientMetrics> clientMetricsMap = new HashMap<>();
private UserMetricsBuilder(byte[] name) {
this.name = name;
}

public UserMetricsBuilder addClientMetris(UserMetrics.ClientMetrics clientMetrics) {
clientMetricsMap.put(clientMetrics.getHostName(), clientMetrics);
return this;
}

public UserMetrics build() {
return new UserMetricsImpl(name, clientMetricsMap);
}

public static class ClientMetricsImpl implements UserMetrics.ClientMetrics {
private final long filteredReadRequestsCount;
private final String hostName;
private final long readRequestCount;
private final long writeRequestCount;

public ClientMetricsImpl(String hostName, long readRequest, long writeRequest,
long filteredReadRequestsCount) {
this.hostName = hostName;
this.readRequestCount = readRequest;
this.writeRequestCount = writeRequest;
this.filteredReadRequestsCount = filteredReadRequestsCount;
}

@Override public String getHostName() {
return hostName;
}

@Override public long getReadRequestsCount() {
return readRequestCount;
}

@Override public long getWriteRequestsCount() {
return writeRequestCount;
}

@Override public long getFilteredReadRequestsCount() {
return filteredReadRequestsCount;
}
}

private static class UserMetricsImpl implements UserMetrics {
private final byte[] name;
private final Map<String, ClientMetrics> clientMetricsMap;

UserMetricsImpl(byte[] name, Map<String, ClientMetrics> clientMetricsMap) {
this.name = Preconditions.checkNotNull(name);
this.clientMetricsMap = clientMetricsMap;
}

@Override public byte[] getUserName() {
return name;
}

@Override public long getReadRequestCount() {
return clientMetricsMap.values().stream().map(c -> c.getReadRequestsCount())
.reduce(0L, Long::sum);
}

@Override public long getWriteRequestCount() {
return clientMetricsMap.values().stream().map(c -> c.getWriteRequestsCount())
.reduce(0L, Long::sum);
}

@Override public Map<String, ClientMetrics> getClientMetrics() {
return this.clientMetricsMap;
}

@Override public long getFilteredReadRequests() {
return clientMetricsMap.values().stream().map(c -> c.getFilteredReadRequestsCount())
.reduce(0L, Long::sum);
}

@Override
public String toString() {
StringBuilder sb = Strings
.appendKeyValue(new StringBuilder(), "readRequestCount", this.getReadRequestCount());
Strings.appendKeyValue(sb, "writeRequestCount", this.getWriteRequestCount());
Strings.appendKeyValue(sb, "filteredReadRequestCount", this.getFilteredReadRequests());
return sb.toString();
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@

package org.apache.hadoop.hbase.regionserver;

import java.util.Map;

import org.apache.hadoop.hbase.metrics.BaseSource;
import org.apache.yetus.audience.InterfaceAudience;

Expand Down Expand Up @@ -59,4 +61,6 @@ public interface MetricsUserAggregateSource extends BaseSource {
MetricsUserSource getOrCreateMetricsUser(String user);

void deregister(MetricsUserSource toRemove);

Map<String, MetricsUserSource> getUserSources();
}
Loading