HBASE-19496 Reusing the ByteBuffer in rpc layer corrupt the ServerLoad and RegionLoad

This commit is contained in:
Chia-Ping Tsai 2017-12-21 20:46:57 +08:00
parent 7ff1feeb85
commit d4af099e9e
66 changed files with 3396 additions and 903 deletions

View File

@ -0,0 +1,198 @@
/**
*
* 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 edu.umd.cs.findbugs.annotations.Nullable;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Metrics information on the HBase cluster.
* <p>
* <tt>ClusterMetrics</tt> provides clients with information such as:
* <ul>
* <li>The count and names of region servers in the cluster.</li>
* <li>The count and names of dead region servers in the cluster.</li>
* <li>The name of the active master for the cluster.</li>
* <li>The name(s) of the backup master(s) for the cluster, if they exist.</li>
* <li>The average cluster load.</li>
* <li>The number of regions deployed on the cluster.</li>
* <li>The number of requests since last report.</li>
* <li>Detailed region server loading and resource usage information,
* per server and per region.</li>
* <li>Regions in transition at master</li>
* <li>The unique cluster ID</li>
* </ul>
* <tt>{@link Option}</tt> provides a way to get desired ClusterStatus information.
* The following codes will get all the cluster information.
* <pre>
* {@code
* // Original version still works
* Admin admin = connection.getAdmin();
* ClusterMetrics metrics = admin.getClusterStatus();
* // or below, a new version which has the same effects
* ClusterMetrics metrics = admin.getClusterStatus(EnumSet.allOf(Option.class));
* }
* </pre>
* If information about live servers is the only wanted.
* then codes in the following way:
* <pre>
* {@code
* Admin admin = connection.getAdmin();
* ClusterMetrics metrics = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
* }
* </pre>
*/
@InterfaceAudience.Public
public interface ClusterMetrics {
/**
* @return the HBase version string as reported by the HMaster
*/
@Nullable
String getHBaseVersion();
/**
* @return the names of region servers on the dead list
*/
List<ServerName> getDeadServerNames();
/**
* @return the names of region servers on the live list
*/
Map<ServerName, ServerMetrics> getLiveServerMetrics();
/**
* @return the number of regions deployed on the cluster
*/
default int getRegionCount() {
return getLiveServerMetrics().entrySet().stream()
.mapToInt(v -> v.getValue().getRegionMetrics().size()).sum();
}
/**
* @return the number of requests since last report
*/
default long getRequestCount() {
return getLiveServerMetrics().entrySet().stream()
.flatMap(v -> v.getValue().getRegionMetrics().values().stream())
.mapToLong(RegionMetrics::getRequestCount).sum();
}
/**
* Returns detailed information about the current master {@link ServerName}.
* @return current master information if it exists
*/
@Nullable
ServerName getMasterName();
/**
* @return the names of backup masters
*/
List<ServerName> getBackupMasterNames();
@InterfaceAudience.Private
List<RegionState> getRegionStatesInTransition();
@Nullable
String getClusterId();
List<String> getMasterCoprocessorNames();
default long getLastMajorCompactionTimestamp(TableName table) {
return getLiveServerMetrics().values().stream()
.flatMap(s -> s.getRegionMetrics().values().stream())
.filter(r -> RegionInfo.getTable(r.getRegionName()).equals(table))
.mapToLong(RegionMetrics::getLastMajorCompactionTimestamp).min().orElse(0);
}
default long getLastMajorCompactionTimestamp(byte[] regionName) {
return getLiveServerMetrics().values().stream()
.filter(s -> s.getRegionMetrics().containsKey(regionName))
.findAny()
.map(s -> s.getRegionMetrics().get(regionName).getLastMajorCompactionTimestamp())
.orElse(0L);
}
@Nullable
Boolean getBalancerOn();
int getMasterInfoPort();
/**
* @return the average cluster load
*/
default double getAverageLoad() {
int serverSize = getLiveServerMetrics().size();
if (serverSize == 0) {
return 0;
}
return (double)getRegionCount() / (double)serverSize;
}
/**
* Kinds of ClusterMetrics
*/
enum Option {
/**
* metrics about hbase version
*/
HBASE_VERSION,
/**
* metrics about cluster id
*/
CLUSTER_ID,
/**
* metrics about balancer is on or not
*/
BALANCER_ON,
/**
* metrics about live region servers
*/
LIVE_SERVERS,
/**
* metrics about dead region servers
*/
DEAD_SERVERS,
/**
* metrics about master name
*/
MASTER,
/**
* metrics about backup masters name
*/
BACKUP_MASTERS,
/**
* metrics about master coprocessors
*/
MASTER_COPROCESSORS,
/**
* metrics about regions in transition
*/
REGIONS_IN_TRANSITION,
/**
* metrics info port
*/
MASTER_INFO_PORT
}
}

View File

@ -0,0 +1,397 @@
/**
*
* 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 edu.umd.cs.findbugs.annotations.Nullable;
import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@InterfaceAudience.Private
public final class ClusterMetricsBuilder {
public static ClusterStatusProtos.ClusterStatus toClusterStatus(ClusterMetrics metrics) {
ClusterStatusProtos.ClusterStatus.Builder builder
= ClusterStatusProtos.ClusterStatus.newBuilder()
.addAllBackupMasters(metrics.getBackupMasterNames().stream()
.map(ProtobufUtil::toServerName).collect(Collectors.toList()))
.addAllDeadServers(metrics.getDeadServerNames().stream()
.map(ProtobufUtil::toServerName).collect(Collectors.toList()))
.addAllLiveServers(metrics.getLiveServerMetrics().entrySet().stream()
.map(s -> ClusterStatusProtos.LiveServerInfo
.newBuilder()
.setServer(ProtobufUtil.toServerName(s.getKey()))
.setServerLoad(ServerMetricsBuilder.toServerLoad(s.getValue()))
.build())
.collect(Collectors.toList()))
.addAllMasterCoprocessors(metrics.getMasterCoprocessorNames().stream()
.map(n -> HBaseProtos.Coprocessor.newBuilder().setName(n).build())
.collect(Collectors.toList()))
.addAllRegionsInTransition(metrics.getRegionStatesInTransition().stream()
.map(r -> ClusterStatusProtos.RegionInTransition
.newBuilder()
.setSpec(HBaseProtos.RegionSpecifier
.newBuilder()
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
.setValue(UnsafeByteOperations.unsafeWrap(r.getRegion().getRegionName()))
.build())
.setRegionState(r.convert())
.build())
.collect(Collectors.toList()))
.setMasterInfoPort(metrics.getMasterInfoPort());
if (metrics.getMasterName() != null) {
builder.setMaster(ProtobufUtil.toServerName((metrics.getMasterName())));
}
if (metrics.getBalancerOn() != null) {
builder.setBalancerOn(metrics.getBalancerOn());
}
if (metrics.getClusterId() != null) {
builder.setClusterId(new ClusterId(metrics.getClusterId()).convert());
}
if (metrics.getHBaseVersion() != null) {
builder.setHbaseVersion(
FSProtos.HBaseVersionFileContent.newBuilder()
.setVersion(metrics.getHBaseVersion()));
}
return builder.build();
}
public static ClusterMetrics toClusterMetrics(
ClusterStatusProtos.ClusterStatus proto) {
ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
builder.setLiveServerMetrics(proto.getLiveServersList().stream()
.collect(Collectors.toMap(e -> ProtobufUtil.toServerName(e.getServer()),
ServerMetricsBuilder::toServerMetrics)))
.setDeadServerNames(proto.getDeadServersList().stream()
.map(ProtobufUtil::toServerName)
.collect(Collectors.toList()))
.setBackerMasterNames(proto.getBackupMastersList().stream()
.map(ProtobufUtil::toServerName)
.collect(Collectors.toList()))
.setRegionsInTransition(proto.getRegionsInTransitionList().stream()
.map(ClusterStatusProtos.RegionInTransition::getRegionState)
.map(RegionState::convert)
.collect(Collectors.toList()))
.setMasterCoprocessorNames(proto.getMasterCoprocessorsList().stream()
.map(HBaseProtos.Coprocessor::getName)
.collect(Collectors.toList()));
if (proto.hasClusterId()) {
builder.setClusterId(ClusterId.convert(proto.getClusterId()).toString());
}
if (proto.hasHbaseVersion()) {
builder.setHBaseVersion(proto.getHbaseVersion().getVersion());
}
if (proto.hasMaster()) {
builder.setMasterName(ProtobufUtil.toServerName(proto.getMaster()));
}
if (proto.hasBalancerOn()) {
builder.setBalancerOn(proto.getBalancerOn());
}
if (proto.hasMasterInfoPort()) {
builder.setMasterInfoPort(proto.getMasterInfoPort());
}
return builder.build();
}
/**
* Convert ClusterStatusProtos.Option to ClusterMetrics.Option
* @param option a ClusterStatusProtos.Option
* @return converted ClusterMetrics.Option
*/
public static ClusterMetrics.Option toOption(ClusterStatusProtos.Option option) {
switch (option) {
case HBASE_VERSION: return ClusterMetrics.Option.HBASE_VERSION;
case LIVE_SERVERS: return ClusterMetrics.Option.LIVE_SERVERS;
case DEAD_SERVERS: return ClusterMetrics.Option.DEAD_SERVERS;
case REGIONS_IN_TRANSITION: return ClusterMetrics.Option.REGIONS_IN_TRANSITION;
case CLUSTER_ID: return ClusterMetrics.Option.CLUSTER_ID;
case MASTER_COPROCESSORS: return ClusterMetrics.Option.MASTER_COPROCESSORS;
case MASTER: return ClusterMetrics.Option.MASTER;
case BACKUP_MASTERS: return ClusterMetrics.Option.BACKUP_MASTERS;
case BALANCER_ON: return ClusterMetrics.Option.BALANCER_ON;
case MASTER_INFO_PORT: return ClusterMetrics.Option.MASTER_INFO_PORT;
// should not reach here
default: throw new IllegalArgumentException("Invalid option: " + option);
}
}
/**
* Convert ClusterMetrics.Option to ClusterStatusProtos.Option
* @param option a ClusterMetrics.Option
* @return converted ClusterStatusProtos.Option
*/
public static ClusterStatusProtos.Option toOption(ClusterMetrics.Option option) {
switch (option) {
case HBASE_VERSION: return ClusterStatusProtos.Option.HBASE_VERSION;
case LIVE_SERVERS: return ClusterStatusProtos.Option.LIVE_SERVERS;
case DEAD_SERVERS: return ClusterStatusProtos.Option.DEAD_SERVERS;
case REGIONS_IN_TRANSITION: return ClusterStatusProtos.Option.REGIONS_IN_TRANSITION;
case CLUSTER_ID: return ClusterStatusProtos.Option.CLUSTER_ID;
case MASTER_COPROCESSORS: return ClusterStatusProtos.Option.MASTER_COPROCESSORS;
case MASTER: return ClusterStatusProtos.Option.MASTER;
case BACKUP_MASTERS: return ClusterStatusProtos.Option.BACKUP_MASTERS;
case BALANCER_ON: return ClusterStatusProtos.Option.BALANCER_ON;
case MASTER_INFO_PORT: return ClusterStatusProtos.Option.MASTER_INFO_PORT;
// should not reach here
default: throw new IllegalArgumentException("Invalid option: " + option);
}
}
/**
* Convert a list of ClusterStatusProtos.Option to an enum set of ClusterMetrics.Option
* @param options the pb options
* @return an enum set of ClusterMetrics.Option
*/
public static EnumSet<ClusterMetrics.Option> toOptions(List<ClusterStatusProtos.Option> options) {
return options.stream().map(ClusterMetricsBuilder::toOption)
.collect(Collectors.toCollection(() -> EnumSet.noneOf(ClusterMetrics.Option.class)));
}
/**
* Convert an enum set of ClusterMetrics.Option to a list of ClusterStatusProtos.Option
* @param options the ClusterMetrics options
* @return a list of ClusterStatusProtos.Option
*/
public static List<ClusterStatusProtos.Option> toOptions(EnumSet<ClusterMetrics.Option> options) {
return options.stream().map(ClusterMetricsBuilder::toOption).collect(Collectors.toList());
}
public static ClusterMetricsBuilder newBuilder() {
return new ClusterMetricsBuilder();
}
@Nullable
private String hbaseVersion;
private List<ServerName> deadServerNames = Collections.emptyList();
private Map<ServerName, ServerMetrics> liveServerMetrics = new TreeMap<>();
@Nullable
private ServerName masterName;
private List<ServerName> backupMasterNames = Collections.emptyList();
private List<RegionState> regionsInTransition = Collections.emptyList();
@Nullable
private String clusterId;
private List<String> masterCoprocessorNames = Collections.emptyList();
@Nullable
private Boolean balancerOn;
private int masterInfoPort;
private ClusterMetricsBuilder() {
}
public ClusterMetricsBuilder setHBaseVersion(String value) {
this.hbaseVersion = value;
return this;
}
public ClusterMetricsBuilder setDeadServerNames(List<ServerName> value) {
this.deadServerNames = value;
return this;
}
public ClusterMetricsBuilder setLiveServerMetrics(Map<ServerName, ServerMetrics> value) {
liveServerMetrics.putAll(value);
return this;
}
public ClusterMetricsBuilder setMasterName(ServerName value) {
this.masterName = value;
return this;
}
public ClusterMetricsBuilder setBackerMasterNames(List<ServerName> value) {
this.backupMasterNames = value;
return this;
}
public ClusterMetricsBuilder setRegionsInTransition(List<RegionState> value) {
this.regionsInTransition = value;
return this;
}
public ClusterMetricsBuilder setClusterId(String value) {
this.clusterId = value;
return this;
}
public ClusterMetricsBuilder setMasterCoprocessorNames(List<String> value) {
this.masterCoprocessorNames = value;
return this;
}
public ClusterMetricsBuilder setBalancerOn(@Nullable Boolean value) {
this.balancerOn = value;
return this;
}
public ClusterMetricsBuilder setMasterInfoPort(int value) {
this.masterInfoPort = value;
return this;
}
public ClusterMetrics build() {
return new ClusterMetricsImpl(
hbaseVersion,
deadServerNames,
liveServerMetrics,
masterName,
backupMasterNames,
regionsInTransition,
clusterId,
masterCoprocessorNames,
balancerOn,
masterInfoPort);
}
private static class ClusterMetricsImpl implements ClusterMetrics {
@Nullable
private final String hbaseVersion;
private final List<ServerName> deadServerNames;
private final Map<ServerName, ServerMetrics> liveServerMetrics;
@Nullable
private final ServerName masterName;
private final List<ServerName> backupMasterNames;
private final List<RegionState> regionsInTransition;
@Nullable
private final String clusterId;
private final List<String> masterCoprocessorNames;
@Nullable
private final Boolean balancerOn;
private final int masterInfoPort;
ClusterMetricsImpl(String hbaseVersion, List<ServerName> deadServerNames,
Map<ServerName, ServerMetrics> liveServerMetrics,
ServerName masterName,
List<ServerName> backupMasterNames,
List<RegionState> regionsInTransition,
String clusterId,
List<String> masterCoprocessorNames,
Boolean balancerOn,
int masterInfoPort) {
this.hbaseVersion = hbaseVersion;
this.deadServerNames = Preconditions.checkNotNull(deadServerNames);
this.liveServerMetrics = Preconditions.checkNotNull(liveServerMetrics);
this.masterName = masterName;
this.backupMasterNames = Preconditions.checkNotNull(backupMasterNames);
this.regionsInTransition = Preconditions.checkNotNull(regionsInTransition);
this.clusterId = clusterId;
this.masterCoprocessorNames = Preconditions.checkNotNull(masterCoprocessorNames);
this.balancerOn = balancerOn;
this.masterInfoPort = masterInfoPort;
}
@Override
public String getHBaseVersion() {
return hbaseVersion;
}
@Override
public List<ServerName> getDeadServerNames() {
return Collections.unmodifiableList(deadServerNames);
}
@Override
public Map<ServerName, ServerMetrics> getLiveServerMetrics() {
return Collections.unmodifiableMap(liveServerMetrics);
}
@Override
public ServerName getMasterName() {
return masterName;
}
@Override
public List<ServerName> getBackupMasterNames() {
return Collections.unmodifiableList(backupMasterNames);
}
@Override
public List<RegionState> getRegionStatesInTransition() {
return Collections.unmodifiableList(regionsInTransition);
}
@Override
public String getClusterId() {
return clusterId;
}
@Override
public List<String> getMasterCoprocessorNames() {
return Collections.unmodifiableList(masterCoprocessorNames);
}
@Override
public Boolean getBalancerOn() {
return balancerOn;
}
@Override
public int getMasterInfoPort() {
return masterInfoPort;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(1024);
sb.append("Master: " + getMasterName());
int backupMastersSize = getBackupMasterNames().size();
sb.append("\nNumber of backup masters: " + backupMastersSize);
if (backupMastersSize > 0) {
for (ServerName serverName: getBackupMasterNames()) {
sb.append("\n " + serverName);
}
}
int serversSize = getLiveServerMetrics().size();
sb.append("\nNumber of live region servers: " + serversSize);
if (serversSize > 0) {
for (ServerName serverName : getLiveServerMetrics().keySet()) {
sb.append("\n " + serverName.getServerName());
}
}
int deadServerSize = getDeadServerNames().size();
sb.append("\nNumber of dead region servers: " + deadServerSize);
if (deadServerSize > 0) {
for (ServerName serverName : getDeadServerNames()) {
sb.append("\n " + serverName);
}
}
sb.append("\nAverage load: " + getAverageLoad());
sb.append("\nNumber of requests: " + getRequestCount());
sb.append("\nNumber of regions: " + getRegionCount());
int ritSize = getRegionStatesInTransition().size();
sb.append("\nNumber of regions in transition: " + ritSize);
if (ritSize > 0) {
for (RegionState state : getRegionStatesInTransition()) {
sb.append("\n " + state.toDescriptiveString());
}
}
return sb.toString();
}
}
}

View File

@ -19,16 +19,17 @@
package org.apache.hadoop.hbase;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Objects;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.master.RegionState;
/**
* Status information on the HBase cluster.
@ -47,7 +48,7 @@ import org.apache.hadoop.hbase.master.RegionState;
* <li>Regions in transition at master</li>
* <li>The unique cluster ID</li>
* </ul>
* <tt>{@link Option}</tt> provides a way to get desired ClusterStatus information.
* <tt>{@link ClusterMetrics.Option}</tt> provides a way to get desired ClusterStatus information.
* The following codes will get all the cluster information.
* <pre>
* {@code
@ -66,28 +67,20 @@ import org.apache.hadoop.hbase.master.RegionState;
* ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
* }
* </pre>
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link ClusterMetrics} instead.
*/
@InterfaceAudience.Public
public class ClusterStatus {
@Deprecated
public class ClusterStatus implements ClusterMetrics {
// TODO: remove this in 3.0
private static final byte VERSION = 2;
private String hbaseVersion;
private Map<ServerName, ServerLoad> liveServers;
private List<ServerName> deadServers;
private ServerName master;
private List<ServerName> backupMasters;
private List<RegionState> intransition;
private String clusterId;
private String[] masterCoprocessors;
private Boolean balancerOn;
private int masterInfoPort;
private final ClusterMetrics metrics;
/**
* Use {@link ClusterStatus.Builder} to construct a ClusterStatus instead.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* (<a href="https://issues.apache.org/jira/browse/HBASE-15511">HBASE-15511</a>).
*/
@Deprecated
public ClusterStatus(final String hbaseVersion, final String clusterid,
@ -100,61 +93,52 @@ public class ClusterStatus {
final Boolean balancerOn,
final int masterInfoPort) {
// TODO: make this constructor private
this.hbaseVersion = hbaseVersion;
this.liveServers = servers;
this.deadServers = new ArrayList<>(deadServers);
this.master = master;
this.backupMasters = new ArrayList<>(backupMasters);
this.intransition = rit;
this.clusterId = clusterid;
this.masterCoprocessors = masterCoprocessors;
this.balancerOn = balancerOn;
this.masterInfoPort = masterInfoPort;
this(ClusterMetricsBuilder.newBuilder().setHBaseVersion(hbaseVersion)
.setDeadServerNames(new ArrayList<>(deadServers))
.setLiveServerMetrics(servers.entrySet().stream()
.collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())))
.setBackerMasterNames(new ArrayList<>(backupMasters)).setBalancerOn(balancerOn)
.setClusterId(clusterid)
.setMasterCoprocessorNames(Arrays.asList(masterCoprocessors))
.setMasterName(master)
.setMasterInfoPort(masterInfoPort)
.setRegionsInTransition(rit)
.build());
}
private ClusterStatus(final String hbaseVersion, final String clusterid,
final Map<ServerName, ServerLoad> servers,
final List<ServerName> deadServers,
final ServerName master,
final List<ServerName> backupMasters,
final List<RegionState> rit,
final String[] masterCoprocessors,
final Boolean balancerOn,
final int masterInfoPort) {
this.hbaseVersion = hbaseVersion;
this.liveServers = servers;
this.deadServers = deadServers;
this.master = master;
this.backupMasters = backupMasters;
this.intransition = rit;
this.clusterId = clusterid;
this.masterCoprocessors = masterCoprocessors;
this.balancerOn = balancerOn;
this.masterInfoPort = masterInfoPort;
@InterfaceAudience.Private
public ClusterStatus(ClusterMetrics metrics) {
this.metrics = metrics;
}
/**
* @return the names of region servers on the dead list
*/
@Override
public List<ServerName> getDeadServerNames() {
if (deadServers == null) {
return Collections.emptyList();
}
return Collections.unmodifiableList(deadServers);
return metrics.getDeadServerNames();
}
@Override
public Map<ServerName, ServerMetrics> getLiveServerMetrics() {
return metrics.getLiveServerMetrics();
}
/**
* @return the number of region servers in the cluster
*/
* @return the number of region servers in the cluster
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getLiveServerMetrics()}.
*/
@Deprecated
public int getServersSize() {
return liveServers != null ? liveServers.size() : 0;
return metrics.getLiveServerMetrics().size();
}
/**
* @return the number of dead region servers in the cluster
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* (<a href="https://issues.apache.org/jira/browse/HBASE-13656">HBASE-13656</a>).
* Use {@link #getDeadServersSize()}.
* Use {@link #getDeadServerNames()}.
*/
@Deprecated
public int getDeadServers() {
@ -163,32 +147,22 @@ public class ClusterStatus {
/**
* @return the number of dead region servers in the cluster
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getDeadServerNames()}.
*/
@Deprecated
public int getDeadServersSize() {
return deadServers != null ? deadServers.size() : 0;
}
/**
* @return the average cluster load
*/
public double getAverageLoad() {
int load = getRegionsCount();
int serverSize = getServersSize();
return serverSize != 0 ? (double)load / (double)serverSize : 0.0;
return metrics.getDeadServerNames().size();
}
/**
* @return the number of regions deployed on the cluster
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionCount()}.
*/
@Deprecated
public int getRegionsCount() {
int count = 0;
if (liveServers != null && !liveServers.isEmpty()) {
for (Map.Entry<ServerName, ServerLoad> e: this.liveServers.entrySet()) {
count = count + e.getValue().getNumberOfRegions();
}
}
return count;
return getRegionCount();
}
/**
@ -201,20 +175,32 @@ public class ClusterStatus {
return (int) getRequestCount();
}
public long getRequestCount() {
if (liveServers == null) {
return 0L;
}
return liveServers.values().stream()
.mapToLong(ServerLoad::getNumberOfRequests)
.sum();
@Nullable
@Override
public ServerName getMasterName() {
return metrics.getMasterName();
}
@Override
public List<ServerName> getBackupMasterNames() {
return metrics.getBackupMasterNames();
}
@Override
public List<RegionState> getRegionStatesInTransition() {
return metrics.getRegionStatesInTransition();
}
/**
* @return the HBase version string as reported by the HMaster
*/
public String getHBaseVersion() {
return hbaseVersion;
return metrics.getHBaseVersion();
}
private Map<ServerName, ServerLoad> getLiveServerLoads() {
return metrics.getLiveServerMetrics().entrySet().stream()
.collect(Collectors.toMap(e -> e.getKey(), e -> new ServerLoad(e.getValue())));
}
@Override
@ -227,7 +213,7 @@ public class ClusterStatus {
}
ClusterStatus other = (ClusterStatus) o;
return Objects.equal(getHBaseVersion(), other.getHBaseVersion()) &&
Objects.equal(this.liveServers, other.liveServers) &&
Objects.equal(getLiveServerLoads(), other.getLiveServerLoads()) &&
getDeadServerNames().containsAll(other.getDeadServerNames()) &&
Arrays.equals(getMasterCoprocessors(), other.getMasterCoprocessors()) &&
Objects.equal(getMaster(), other.getMaster()) &&
@ -238,8 +224,7 @@ public class ClusterStatus {
@Override
public int hashCode() {
return Objects.hashCode(hbaseVersion, liveServers, deadServers, master, backupMasters,
clusterId, masterInfoPort);
return metrics.hashCode();
}
/**
@ -251,111 +236,123 @@ public class ClusterStatus {
return VERSION;
}
//
// Getters
//
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getLiveServerMetrics()} instead.
*/
@Deprecated
public Collection<ServerName> getServers() {
if (liveServers == null) {
return Collections.emptyList();
}
return Collections.unmodifiableCollection(this.liveServers.keySet());
return metrics.getLiveServerMetrics().keySet();
}
/**
* Returns detailed information about the current master {@link ServerName}.
* @return current master information if it exists
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getMasterName} instead.
*/
@Deprecated
public ServerName getMaster() {
return this.master;
return metrics.getMasterName();
}
/**
* @return the number of backup masters in the cluster
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getBackupMasterNames} instead.
*/
@Deprecated
public int getBackupMastersSize() {
return backupMasters != null ? backupMasters.size() : 0;
return metrics.getBackupMasterNames().size();
}
/**
* @return the names of backup masters
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getBackupMasterNames} instead.
*/
@Deprecated
public List<ServerName> getBackupMasters() {
if (backupMasters == null) {
return Collections.emptyList();
}
return Collections.unmodifiableList(this.backupMasters);
return metrics.getBackupMasterNames();
}
/**
* @param sn
* @return Server's load or null if not found.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getLiveServerMetrics} instead.
*/
@Deprecated
public ServerLoad getLoad(final ServerName sn) {
return liveServers != null ? liveServers.get(sn) : null;
}
@InterfaceAudience.Private
public List<RegionState> getRegionsInTransition() {
if (intransition == null) {
return Collections.emptyList();
}
return Collections.unmodifiableList(intransition);
ServerMetrics serverMetrics = metrics.getLiveServerMetrics().get(sn);
return serverMetrics == null ? null : new ServerLoad(serverMetrics);
}
public String getClusterId() {
return clusterId;
return metrics.getClusterId();
}
@Override
public List<String> getMasterCoprocessorNames() {
return metrics.getMasterCoprocessorNames();
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getMasterCoprocessorNames} instead.
*/
@Deprecated
public String[] getMasterCoprocessors() {
return masterCoprocessors;
List<String> rval = metrics.getMasterCoprocessorNames();
return rval.toArray(new String[rval.size()]);
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getLastMajorCompactionTimestamp(TableName)} instead.
*/
@Deprecated
public long getLastMajorCompactionTsForTable(TableName table) {
long result = Long.MAX_VALUE;
for (ServerName server : getServers()) {
ServerLoad load = getLoad(server);
for (RegionLoad rl : load.getRegionsLoad().values()) {
if (table.equals(HRegionInfo.getTable(rl.getName()))) {
result = Math.min(result, rl.getLastMajorCompactionTs());
}
}
}
return result == Long.MAX_VALUE ? 0 : result;
return metrics.getLastMajorCompactionTimestamp(table);
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getLastMajorCompactionTimestamp(byte[])} instead.
*/
@Deprecated
public long getLastMajorCompactionTsForRegion(final byte[] region) {
for (ServerName server : getServers()) {
ServerLoad load = getLoad(server);
RegionLoad rl = load.getRegionsLoad().get(region);
if (rl != null) {
return rl.getLastMajorCompactionTs();
}
}
return 0;
return metrics.getLastMajorCompactionTimestamp(region);
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* No flag in 2.0
*/
@Deprecated
public boolean isBalancerOn() {
return balancerOn != null && balancerOn;
return metrics.getBalancerOn() != null && metrics.getBalancerOn();
}
@Override
public Boolean getBalancerOn() {
return balancerOn;
return metrics.getBalancerOn();
}
@Override
public int getMasterInfoPort() {
return masterInfoPort;
return metrics.getMasterInfoPort();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(1024);
sb.append("Master: " + master);
sb.append("Master: " + metrics.getMasterName());
int backupMastersSize = getBackupMastersSize();
sb.append("\nNumber of backup masters: " + backupMastersSize);
if (backupMastersSize > 0) {
for (ServerName serverName: backupMasters) {
for (ServerName serverName: metrics.getBackupMasterNames()) {
sb.append("\n " + serverName);
}
}
@ -363,15 +360,15 @@ public class ClusterStatus {
int serversSize = getServersSize();
sb.append("\nNumber of live region servers: " + serversSize);
if (serversSize > 0) {
for (ServerName serverName: liveServers.keySet()) {
for (ServerName serverName : metrics.getLiveServerMetrics().keySet()) {
sb.append("\n " + serverName.getServerName());
}
}
int deadServerSize = getDeadServersSize();
int deadServerSize = metrics.getDeadServerNames().size();
sb.append("\nNumber of dead region servers: " + deadServerSize);
if (deadServerSize > 0) {
for (ServerName serverName: deadServers) {
for (ServerName serverName : metrics.getDeadServerNames()) {
sb.append("\n " + serverName);
}
}
@ -380,109 +377,13 @@ public class ClusterStatus {
sb.append("\nNumber of requests: " + getRequestCount());
sb.append("\nNumber of regions: " + getRegionsCount());
int ritSize = (intransition != null) ? intransition.size() : 0;
int ritSize = metrics.getRegionStatesInTransition().size();
sb.append("\nNumber of regions in transition: " + ritSize);
if (ritSize > 0) {
for (RegionState state: intransition) {
for (RegionState state: metrics.getRegionStatesInTransition()) {
sb.append("\n " + state.toDescriptiveString());
}
}
return sb.toString();
}
@InterfaceAudience.Private
public static Builder newBuilder() {
return new Builder();
}
/**
* Builder for construct a ClusterStatus.
*/
@InterfaceAudience.Private
public static class Builder {
private String hbaseVersion = null;
private Map<ServerName, ServerLoad> liveServers = null;
private List<ServerName> deadServers = null;
private ServerName master = null;
private List<ServerName> backupMasters = null;
private List<RegionState> intransition = null;
private String clusterId = null;
private String[] masterCoprocessors = null;
private Boolean balancerOn = null;
private int masterInfoPort = -1;
private Builder() {}
public Builder setHBaseVersion(String hbaseVersion) {
this.hbaseVersion = hbaseVersion;
return this;
}
public Builder setLiveServers(Map<ServerName, ServerLoad> liveServers) {
this.liveServers = liveServers;
return this;
}
public Builder setDeadServers(List<ServerName> deadServers) {
this.deadServers = deadServers;
return this;
}
public Builder setMaster(ServerName master) {
this.master = master;
return this;
}
public Builder setBackupMasters(List<ServerName> backupMasters) {
this.backupMasters = backupMasters;
return this;
}
public Builder setRegionState(List<RegionState> intransition) {
this.intransition = intransition;
return this;
}
public Builder setClusterId(String clusterId) {
this.clusterId = clusterId;
return this;
}
public Builder setMasterCoprocessors(String[] masterCoprocessors) {
this.masterCoprocessors = masterCoprocessors;
return this;
}
public Builder setBalancerOn(Boolean balancerOn) {
this.balancerOn = balancerOn;
return this;
}
public Builder setMasterInfoPort(int masterInfoPort) {
this.masterInfoPort = masterInfoPort;
return this;
}
public ClusterStatus build() {
return new ClusterStatus(hbaseVersion, clusterId, liveServers,
deadServers, master, backupMasters, intransition, masterCoprocessors,
balancerOn, masterInfoPort);
}
}
/**
* Kinds of ClusterStatus
*/
public enum Option {
HBASE_VERSION, /** status about hbase version */
CLUSTER_ID, /** status about cluster id */
BALANCER_ON, /** status about balancer is on or not */
LIVE_SERVERS, /** status about live region servers */
DEAD_SERVERS, /** status about dead region servers */
MASTER, /** status about master */
BACKUP_MASTERS, /** status about backup masters */
MASTER_COPROCESSORS, /** status about master coprocessors */
REGIONS_IN_TRANSITION, /** status about regions in transition */
MASTER_INFO_PORT /** master info port **/
}
}

View File

@ -21,184 +21,341 @@
package org.apache.hadoop.hbase;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
/**
* Encapsulates per-region load metrics.
*/
* Encapsulates per-region load metrics.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link RegionMetrics} instead.
*/
@InterfaceAudience.Public
public class RegionLoad {
@Deprecated
public class RegionLoad implements RegionMetrics {
// DONT use this pb object since the byte array backed may be modified in rpc layer
// we keep this pb object for BC.
protected ClusterStatusProtos.RegionLoad regionLoadPB;
private final RegionMetrics metrics;
@InterfaceAudience.Private
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
public RegionLoad(ClusterStatusProtos.RegionLoad regionLoadPB) {
this.regionLoadPB = regionLoadPB;
this.metrics = RegionMetricsBuilder.toRegionMetrics(regionLoadPB);
}
RegionLoad(RegionMetrics metrics) {
this.metrics = metrics;
this.regionLoadPB = RegionMetricsBuilder.toRegionLoad(metrics);
}
/**
* @return the region name
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionName} instead.
*/
@Deprecated
public byte[] getName() {
return regionLoadPB.getRegionSpecifier().getValue().toByteArray();
return metrics.getRegionName();
}
/**
* @return the region name as a string
*/
public String getNameAsString() {
return Bytes.toStringBinary(getName());
@Override
public byte[] getRegionName() {
return metrics.getRegionName();
}
@Override
public int getStoreCount() {
return metrics.getStoreCount();
}
@Override
public int getStoreFileCount() {
return metrics.getStoreFileCount();
}
@Override
public Size getStoreFileSize() {
return metrics.getStoreFileSize();
}
@Override
public Size getMemStoreSize() {
return metrics.getMemStoreSize();
}
@Override
public long getReadRequestCount() {
return metrics.getReadRequestCount();
}
@Override
public long getFilteredReadRequestCount() {
return metrics.getFilteredReadRequestCount();
}
@Override
public Size getStoreFileIndexSize() {
return metrics.getStoreFileIndexSize();
}
@Override
public long getWriteRequestCount() {
return metrics.getWriteRequestCount();
}
@Override
public Size getStoreFileRootLevelIndexSize() {
return metrics.getStoreFileRootLevelIndexSize();
}
@Override
public Size getStoreFileUncompressedDataIndexSize() {
return metrics.getStoreFileUncompressedDataIndexSize();
}
@Override
public Size getBloomFilterSize() {
return metrics.getBloomFilterSize();
}
@Override
public long getCompactingCellCount() {
return metrics.getCompactingCellCount();
}
@Override
public long getCompactedCellCount() {
return metrics.getCompactedCellCount();
}
@Override
public long getCompletedSequenceId() {
return metrics.getCompletedSequenceId();
}
@Override
public Map<byte[], Long> getStoreSequenceId() {
return metrics.getStoreSequenceId();
}
@Override
public Size getUncompressedStoreFileSize() {
return metrics.getUncompressedStoreFileSize();
}
/**
* @return the number of stores
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStoreCount} instead.
*/
@Deprecated
public int getStores() {
return regionLoadPB.getStores();
return metrics.getStoreCount();
}
/**
* @return the number of storefiles
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStoreFileCount} instead.
*/
@Deprecated
public int getStorefiles() {
return regionLoadPB.getStorefiles();
return metrics.getStoreFileCount();
}
/**
* @return the total size of the storefiles, in MB
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStoreFileSize} instead.
*/
@Deprecated
public int getStorefileSizeMB() {
return regionLoadPB.getStorefileSizeMB();
return (int) metrics.getStoreFileSize().get(Size.Unit.MEGABYTE);
}
/**
* @return the memstore size, in MB
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getMemStoreSize} instead.
*/
@Deprecated
public int getMemStoreSizeMB() {
return regionLoadPB.getMemStoreSizeMB();
return (int) metrics.getMemStoreSize().get(Size.Unit.MEGABYTE);
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* ((<a href="https://issues.apache.org/jira/browse/HBASE-3935">HBASE-3935</a>)).
* Use {@link #getStorefileIndexSizeKB()} instead.
* Use {@link #getStoreFileRootLevelIndexSize} instead.
*/
@Deprecated
public int getStorefileIndexSizeMB() {
// Return value divided by 1024
return (int) (regionLoadPB.getStorefileIndexSizeKB() >> 10);
return (getRootIndexSizeKB() >> 10);
}
public long getStorefileIndexSizeKB() {
return regionLoadPB.getStorefileIndexSizeKB();
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStoreFileRootLevelIndexSize()} instead.
*/
@Deprecated
public int getStorefileIndexSizeKB() {
return getRootIndexSizeKB();
}
/**
* @return the number of requests made to region
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRequestCount()} instead.
*/
@Deprecated
public long getRequestsCount() {
return getReadRequestsCount() + getWriteRequestsCount();
return metrics.getRequestCount();
}
/**
* @return the number of read requests made to region
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getReadRequestCount} instead.
*/
@Deprecated
public long getReadRequestsCount() {
return regionLoadPB.getReadRequestsCount();
return metrics.getReadRequestCount();
}
/**
* @return the number of filtered read requests made to region
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getFilteredReadRequestCount} instead.
*/
@Deprecated
public long getFilteredReadRequestsCount() {
return regionLoadPB.getFilteredReadRequestsCount();
return metrics.getFilteredReadRequestCount();
}
/**
* @return the number of write requests made to region
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getWriteRequestCount} instead.
*/
@Deprecated
public long getWriteRequestsCount() {
return regionLoadPB.getWriteRequestsCount();
return metrics.getWriteRequestCount();
}
/**
* @return The current total size of root-level indexes for the region, in KB.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStoreFileRootLevelIndexSize} instead.
*/
@Deprecated
public int getRootIndexSizeKB() {
return regionLoadPB.getRootIndexSizeKB();
return (int) metrics.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
}
/**
* @return The total size of all index blocks, not just the root level, in KB.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStoreFileUncompressedDataIndexSize} instead.
*/
@Deprecated
public int getTotalStaticIndexSizeKB() {
return regionLoadPB.getTotalStaticIndexSizeKB();
return (int) metrics.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
}
/**
* @return The total size of all Bloom filter blocks, not just loaded into the
* block cache, in KB.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getBloomFilterSize} instead.
*/
@Deprecated
public int getTotalStaticBloomSizeKB() {
return regionLoadPB.getTotalStaticBloomSizeKB();
return (int) metrics.getBloomFilterSize().get(Size.Unit.KILOBYTE);
}
/**
* @return the total number of kvs in current compaction
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getCompactingCellCount} instead.
*/
@Deprecated
public long getTotalCompactingKVs() {
return regionLoadPB.getTotalCompactingKVs();
return metrics.getCompactingCellCount();
}
/**
* @return the number of already compacted kvs in current compaction
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getCompactedCellCount} instead.
*/
@Deprecated
public long getCurrentCompactedKVs() {
return regionLoadPB.getCurrentCompactedKVs();
return metrics.getCompactedCellCount();
}
/**
* This does not really belong inside RegionLoad but its being done in the name of expediency.
* @return the completed sequence Id for the region
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getCompletedSequenceId} instead.
*/
@Deprecated
public long getCompleteSequenceId() {
return regionLoadPB.getCompleteSequenceId();
return metrics.getCompletedSequenceId();
}
/**
* @return completed sequence id per store.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStoreSequenceId} instead.
*/
public List<StoreSequenceId> getStoreCompleteSequenceId() {
return regionLoadPB.getStoreCompleteSequenceIdList();
@Deprecated
public List<ClusterStatusProtos.StoreSequenceId> getStoreCompleteSequenceId() {
return metrics.getStoreSequenceId().entrySet().stream()
.map(s -> ClusterStatusProtos.StoreSequenceId.newBuilder()
.setFamilyName(UnsafeByteOperations.unsafeWrap(s.getKey()))
.setSequenceId(s.getValue())
.build())
.collect(Collectors.toList());
}
/**
* @return the uncompressed size of the storefiles in MB.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getUncompressedStoreFileSize} instead.
*/
@Deprecated
public int getStoreUncompressedSizeMB() {
return regionLoadPB.getStoreUncompressedSizeMB();
return (int) metrics.getUncompressedStoreFileSize().get(Size.Unit.KILOBYTE);
}
/**
* @return the data locality of region in the regionserver.
*/
@Override
public float getDataLocality() {
if (regionLoadPB.hasDataLocality()) {
return regionLoadPB.getDataLocality();
}
return 0.0f;
return metrics.getDataLocality();
}
@Override
public long getLastMajorCompactionTimestamp() {
return metrics.getLastMajorCompactionTimestamp();
}
/**
* @return the timestamp of the oldest hfile for any store of this region.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getLastMajorCompactionTimestamp} instead.
*/
@Deprecated
public long getLastMajorCompactionTs() {
return regionLoadPB.getLastMajorCompactionTs();
return metrics.getLastMajorCompactionTimestamp();
}
/**
@ -208,47 +365,43 @@ public class RegionLoad {
public String toString() {
StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "numberOfStores",
this.getStores());
sb = Strings.appendKeyValue(sb, "numberOfStorefiles",
this.getStorefiles());
sb = Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
this.getStoreUncompressedSizeMB());
sb = Strings.appendKeyValue(sb, "lastMajorCompactionTimestamp",
this.getLastMajorCompactionTs());
sb = Strings.appendKeyValue(sb, "storefileSizeMB",
this.getStorefileSizeMB());
Strings.appendKeyValue(sb, "numberOfStorefiles", this.getStorefiles());
Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
this.getStoreUncompressedSizeMB());
Strings.appendKeyValue(sb, "lastMajorCompactionTimestamp",
this.getLastMajorCompactionTs());
Strings.appendKeyValue(sb, "storefileSizeMB", this.getStorefileSizeMB());
if (this.getStoreUncompressedSizeMB() != 0) {
sb = Strings.appendKeyValue(sb, "compressionRatio",
Strings.appendKeyValue(sb, "compressionRatio",
String.format("%.4f", (float) this.getStorefileSizeMB() /
(float) this.getStoreUncompressedSizeMB()));
}
sb = Strings.appendKeyValue(sb, "memstoreSizeMB",
Strings.appendKeyValue(sb, "memstoreSizeMB",
this.getMemStoreSizeMB());
sb = Strings.appendKeyValue(sb, "storefileIndexSizeKB",
this.getStorefileIndexSizeKB());
sb = Strings.appendKeyValue(sb, "readRequestsCount",
Strings.appendKeyValue(sb, "readRequestsCount",
this.getReadRequestsCount());
sb = Strings.appendKeyValue(sb, "writeRequestsCount",
Strings.appendKeyValue(sb, "writeRequestsCount",
this.getWriteRequestsCount());
sb = Strings.appendKeyValue(sb, "rootIndexSizeKB",
Strings.appendKeyValue(sb, "rootIndexSizeKB",
this.getRootIndexSizeKB());
sb = Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
this.getTotalStaticIndexSizeKB());
sb = Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
this.getTotalStaticBloomSizeKB());
sb = Strings.appendKeyValue(sb, "totalCompactingKVs",
Strings.appendKeyValue(sb, "totalCompactingKVs",
this.getTotalCompactingKVs());
sb = Strings.appendKeyValue(sb, "currentCompactedKVs",
Strings.appendKeyValue(sb, "currentCompactedKVs",
this.getCurrentCompactedKVs());
float compactionProgressPct = Float.NaN;
if (this.getTotalCompactingKVs() > 0) {
compactionProgressPct = ((float) this.getCurrentCompactedKVs() /
(float) this.getTotalCompactingKVs());
}
sb = Strings.appendKeyValue(sb, "compactionProgressPct",
Strings.appendKeyValue(sb, "compactionProgressPct",
compactionProgressPct);
sb = Strings.appendKeyValue(sb, "completeSequenceId",
Strings.appendKeyValue(sb, "completeSequenceId",
this.getCompleteSequenceId());
sb = Strings.appendKeyValue(sb, "dataLocality",
Strings.appendKeyValue(sb, "dataLocality",
this.getDataLocality());
return sb.toString();
}

View File

@ -0,0 +1,147 @@
/**
* 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;
/**
* Encapsulates per-region load metrics.
*/
@InterfaceAudience.Public
public interface RegionMetrics {
/**
* @return the region name
*/
byte[] getRegionName();
/**
* @return the number of stores
*/
int getStoreCount();
/**
* @return the number of storefiles
*/
int getStoreFileCount();
/**
* @return the total size of the storefiles
*/
Size getStoreFileSize();
/**
* @return the memstore size
*/
Size getMemStoreSize();
/**
* @return the number of read requests made to region
*/
long getReadRequestCount();
/**
* @return the number of write requests made to region
*/
long getWriteRequestCount();
/**
* @return the number of write requests and read requests made to region
*/
default long getRequestCount() {
return getReadRequestCount() + getWriteRequestCount();
}
/**
* @return the region name as a string
*/
default String getNameAsString() {
return Bytes.toStringBinary(getRegionName());
}
/**
* @return the number of filtered read requests made to region
*/
long getFilteredReadRequestCount();
/**
* TODO: why we pass the same value to different counters? Currently, the value from
* getStoreFileIndexSize() is same with getStoreFileRootLevelIndexSize()
* see HRegionServer#createRegionLoad.
* @return The current total size of root-level indexes for the region
*/
Size getStoreFileIndexSize();
/**
* @return The current total size of root-level indexes for the region
*/
Size getStoreFileRootLevelIndexSize();
/**
* @return The total size of all index blocks, not just the root level
*/
Size getStoreFileUncompressedDataIndexSize();
/**
* @return The total size of all Bloom filter blocks, not just loaded into the block cache
*/
Size getBloomFilterSize();
/**
* @return the total number of cells in current compaction
*/
long getCompactingCellCount();
/**
* @return the number of already compacted kvs in current compaction
*/
long getCompactedCellCount();
/**
* This does not really belong inside RegionLoad but its being done in the name of expediency.
* @return the completed sequence Id for the region
*/
long getCompletedSequenceId();
/**
* @return completed sequence id per store.
*/
Map<byte[], Long> getStoreSequenceId();
/**
* @return the uncompressed size of the storefiles
*/
Size getUncompressedStoreFileSize();
/**
* @return the data locality of region in the regionserver.
*/
float getDataLocality();
/**
* @return the timestamp of the oldest hfile for any store of this region.
*/
long getLastMajorCompactionTimestamp();
}

View File

@ -0,0 +1,437 @@
/**
* 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.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@InterfaceAudience.Private
public final class RegionMetricsBuilder {
public static RegionMetrics toRegionMetrics(ClusterStatusProtos.RegionLoad regionLoadPB) {
return RegionMetricsBuilder
.newBuilder(regionLoadPB.getRegionSpecifier().getValue().toByteArray())
.setBloomFilterSize(new Size(regionLoadPB.getTotalStaticBloomSizeKB(), Size.Unit.KILOBYTE))
.setCompactedCellCount(regionLoadPB.getCurrentCompactedKVs())
.setCompactingCellCount(regionLoadPB.getTotalCompactingKVs())
.setCompletedSequenceId(regionLoadPB.getCompleteSequenceId())
.setDataLocality(regionLoadPB.hasDataLocality() ? regionLoadPB.getDataLocality() : 0.0f)
.setFilteredReadRequestCount(regionLoadPB.getFilteredReadRequestsCount())
.setStoreFileUncompressedDataIndexSize(new Size(regionLoadPB.getTotalStaticIndexSizeKB(),
Size.Unit.KILOBYTE))
.setLastMajorCompactionTimestamp(regionLoadPB.getLastMajorCompactionTs())
.setMemStoreSize(new Size(regionLoadPB.getMemStoreSizeMB(), Size.Unit.MEGABYTE))
.setReadRequestCount(regionLoadPB.getReadRequestsCount())
.setWriteRequestCount(regionLoadPB.getWriteRequestsCount())
.setStoreFileIndexSize(new Size(regionLoadPB.getStorefileIndexSizeKB(),
Size.Unit.KILOBYTE))
.setStoreFileRootLevelIndexSize(new Size(regionLoadPB.getRootIndexSizeKB(),
Size.Unit.KILOBYTE))
.setStoreCount(regionLoadPB.getStores())
.setStoreFileCount(regionLoadPB.getStorefiles())
.setStoreFileSize(new Size(regionLoadPB.getStorefileSizeMB(), Size.Unit.MEGABYTE))
.setStoreSequenceIds(regionLoadPB.getStoreCompleteSequenceIdList().stream()
.collect(Collectors.toMap(
(ClusterStatusProtos.StoreSequenceId s) -> s.getFamilyName().toByteArray(),
ClusterStatusProtos.StoreSequenceId::getSequenceId)))
.setUncompressedStoreFileSize(
new Size(regionLoadPB.getStoreUncompressedSizeMB(),Size.Unit.MEGABYTE))
.build();
}
private static List<ClusterStatusProtos.StoreSequenceId> toStoreSequenceId(
Map<byte[], Long> ids) {
return ids.entrySet().stream()
.map(e -> ClusterStatusProtos.StoreSequenceId.newBuilder()
.setFamilyName(UnsafeByteOperations.unsafeWrap(e.getKey()))
.setSequenceId(e.getValue())
.build())
.collect(Collectors.toList());
}
public static ClusterStatusProtos.RegionLoad toRegionLoad(RegionMetrics regionMetrics) {
return ClusterStatusProtos.RegionLoad.newBuilder()
.setRegionSpecifier(HBaseProtos.RegionSpecifier
.newBuilder().setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
.setValue(UnsafeByteOperations.unsafeWrap(regionMetrics.getRegionName()))
.build())
.setTotalStaticBloomSizeKB((int) regionMetrics.getBloomFilterSize()
.get(Size.Unit.KILOBYTE))
.setCurrentCompactedKVs(regionMetrics.getCompactedCellCount())
.setTotalCompactingKVs(regionMetrics.getCompactingCellCount())
.setCompleteSequenceId(regionMetrics.getCompletedSequenceId())
.setDataLocality(regionMetrics.getDataLocality())
.setFilteredReadRequestsCount(regionMetrics.getFilteredReadRequestCount())
.setTotalStaticIndexSizeKB((int) regionMetrics.getStoreFileUncompressedDataIndexSize()
.get(Size.Unit.KILOBYTE))
.setLastMajorCompactionTs(regionMetrics.getLastMajorCompactionTimestamp())
.setMemStoreSizeMB((int) regionMetrics.getMemStoreSize().get(Size.Unit.MEGABYTE))
.setReadRequestsCount(regionMetrics.getReadRequestCount())
.setWriteRequestsCount(regionMetrics.getWriteRequestCount())
.setStorefileIndexSizeKB((long) regionMetrics.getStoreFileIndexSize()
.get(Size.Unit.KILOBYTE))
.setRootIndexSizeKB((int) regionMetrics.getStoreFileRootLevelIndexSize()
.get(Size.Unit.KILOBYTE))
.setStores(regionMetrics.getStoreCount())
.setStorefiles(regionMetrics.getStoreCount())
.setStorefileSizeMB((int) regionMetrics.getStoreFileSize().get(Size.Unit.MEGABYTE))
.addAllStoreCompleteSequenceId(toStoreSequenceId(regionMetrics.getStoreSequenceId()))
.setStoreUncompressedSizeMB(
(int) regionMetrics.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE))
.build();
}
public static RegionMetricsBuilder newBuilder(byte[] name) {
return new RegionMetricsBuilder(name);
}
private final byte[] name;
private int storeCount;
private int storeFileCount;
private long compactingCellCount;
private long compactedCellCount;
private Size storeFileSize = Size.ZERO;
private Size memStoreSize = Size.ZERO;
private Size indexSize = Size.ZERO;
private Size rootLevelIndexSize = Size.ZERO;
private Size uncompressedDataIndexSize = Size.ZERO;
private Size bloomFilterSize = Size.ZERO;
private Size uncompressedStoreFileSize = Size.ZERO;
private long writeRequestCount;
private long readRequestCount;
private long filteredReadRequestCount;
private long completedSequenceId;
private Map<byte[], Long> storeSequenceIds = Collections.emptyMap();
private float dataLocality;
private long lastMajorCompactionTimestamp;
private RegionMetricsBuilder(byte[] name) {
this.name = name;
}
public RegionMetricsBuilder setStoreCount(int value) {
this.storeCount = value;
return this;
}
public RegionMetricsBuilder setStoreFileCount(int value) {
this.storeFileCount = value;
return this;
}
public RegionMetricsBuilder setCompactingCellCount(long value) {
this.compactingCellCount = value;
return this;
}
public RegionMetricsBuilder setCompactedCellCount(long value) {
this.compactedCellCount = value;
return this;
}
public RegionMetricsBuilder setStoreFileSize(Size value) {
this.storeFileSize = value;
return this;
}
public RegionMetricsBuilder setMemStoreSize(Size value) {
this.memStoreSize = value;
return this;
}
public RegionMetricsBuilder setStoreFileIndexSize(Size value) {
this.indexSize = value;
return this;
}
public RegionMetricsBuilder setStoreFileRootLevelIndexSize(Size value) {
this.rootLevelIndexSize = value;
return this;
}
public RegionMetricsBuilder setStoreFileUncompressedDataIndexSize(Size value) {
this.uncompressedDataIndexSize = value;
return this;
}
public RegionMetricsBuilder setBloomFilterSize(Size value) {
this.bloomFilterSize = value;
return this;
}
public RegionMetricsBuilder setUncompressedStoreFileSize(Size value) {
this.uncompressedStoreFileSize = value;
return this;
}
public RegionMetricsBuilder setWriteRequestCount(long value) {
this.writeRequestCount = value;
return this;
}
public RegionMetricsBuilder setReadRequestCount(long value) {
this.readRequestCount = value;
return this;
}
public RegionMetricsBuilder setFilteredReadRequestCount(long value) {
this.filteredReadRequestCount = value;
return this;
}
public RegionMetricsBuilder setCompletedSequenceId(long value) {
this.completedSequenceId = value;
return this;
}
public RegionMetricsBuilder setStoreSequenceIds(Map<byte[], Long> value) {
this.storeSequenceIds = value;
return this;
}
public RegionMetricsBuilder setDataLocality(float value) {
this.dataLocality = value;
return this;
}
public RegionMetricsBuilder setLastMajorCompactionTimestamp(long value) {
this.lastMajorCompactionTimestamp = value;
return this;
}
public RegionMetrics build() {
return new RegionMetricsImpl(name,
storeCount,
storeFileCount,
compactingCellCount,
compactedCellCount,
storeFileSize,
memStoreSize,
indexSize,
rootLevelIndexSize,
uncompressedDataIndexSize,
bloomFilterSize,
uncompressedStoreFileSize,
writeRequestCount,
readRequestCount,
filteredReadRequestCount,
completedSequenceId,
storeSequenceIds,
dataLocality,
lastMajorCompactionTimestamp);
}
private static class RegionMetricsImpl implements RegionMetrics {
private final byte[] name;
private final int storeCount;
private final int storeFileCount;
private final long compactingCellCount;
private final long compactedCellCount;
private final Size storeFileSize;
private final Size memStoreSize;
private final Size indexSize;
private final Size rootLevelIndexSize;
private final Size uncompressedDataIndexSize;
private final Size bloomFilterSize;
private final Size uncompressedStoreFileSize;
private final long writeRequestCount;
private final long readRequestCount;
private final long filteredReadRequestCount;
private final long completedSequenceId;
private final Map<byte[], Long> storeSequenceIds;
private final float dataLocality;
private final long lastMajorCompactionTimestamp;
RegionMetricsImpl(byte[] name,
int storeCount,
int storeFileCount,
final long compactingCellCount,
long compactedCellCount,
Size storeFileSize,
Size memStoreSize,
Size indexSize,
Size rootLevelIndexSize,
Size uncompressedDataIndexSize,
Size bloomFilterSize,
Size uncompressedStoreFileSize,
long writeRequestCount,
long readRequestCount,
long filteredReadRequestCount,
long completedSequenceId,
Map<byte[], Long> storeSequenceIds,
float dataLocality,
long lastMajorCompactionTimestamp) {
this.name = Preconditions.checkNotNull(name);
this.storeCount = storeCount;
this.storeFileCount = storeFileCount;
this.compactingCellCount = compactingCellCount;
this.compactedCellCount = compactedCellCount;
this.storeFileSize = Preconditions.checkNotNull(storeFileSize);
this.memStoreSize = Preconditions.checkNotNull(memStoreSize);
this.indexSize = Preconditions.checkNotNull(indexSize);
this.rootLevelIndexSize = Preconditions.checkNotNull(rootLevelIndexSize);
this.uncompressedDataIndexSize = Preconditions.checkNotNull(uncompressedDataIndexSize);
this.bloomFilterSize = Preconditions.checkNotNull(bloomFilterSize);
this.uncompressedStoreFileSize = Preconditions.checkNotNull(uncompressedStoreFileSize);
this.writeRequestCount = writeRequestCount;
this.readRequestCount = readRequestCount;
this.filteredReadRequestCount = filteredReadRequestCount;
this.completedSequenceId = completedSequenceId;
this.storeSequenceIds = Preconditions.checkNotNull(storeSequenceIds);
this.dataLocality = dataLocality;
this.lastMajorCompactionTimestamp = lastMajorCompactionTimestamp;
}
@Override
public byte[] getRegionName() {
return name;
}
@Override
public int getStoreCount() {
return storeCount;
}
@Override
public int getStoreFileCount() {
return storeFileCount;
}
@Override
public Size getStoreFileSize() {
return storeFileSize;
}
@Override
public Size getMemStoreSize() {
return memStoreSize;
}
@Override
public long getReadRequestCount() {
return readRequestCount;
}
@Override
public long getFilteredReadRequestCount() {
return filteredReadRequestCount;
}
@Override
public long getWriteRequestCount() {
return writeRequestCount;
}
@Override
public Size getStoreFileIndexSize() {
return indexSize;
}
@Override
public Size getStoreFileRootLevelIndexSize() {
return rootLevelIndexSize;
}
@Override
public Size getStoreFileUncompressedDataIndexSize() {
return uncompressedDataIndexSize;
}
@Override
public Size getBloomFilterSize() {
return bloomFilterSize;
}
@Override
public long getCompactingCellCount() {
return compactingCellCount;
}
@Override
public long getCompactedCellCount() {
return compactedCellCount;
}
@Override
public long getCompletedSequenceId() {
return completedSequenceId;
}
@Override
public Map<byte[], Long> getStoreSequenceId() {
return Collections.unmodifiableMap(storeSequenceIds);
}
@Override
public Size getUncompressedStoreFileSize() {
return uncompressedStoreFileSize;
}
@Override
public float getDataLocality() {
return dataLocality;
}
@Override
public long getLastMajorCompactionTimestamp() {
return lastMajorCompactionTimestamp;
}
@Override
public String toString() {
StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "storeCount",
this.getStoreCount());
Strings.appendKeyValue(sb, "storeFileCount",
this.getStoreFileCount());
Strings.appendKeyValue(sb, "uncompressedStoreFileSize",
this.getUncompressedStoreFileSize());
Strings.appendKeyValue(sb, "lastMajorCompactionTimestamp",
this.getLastMajorCompactionTimestamp());
Strings.appendKeyValue(sb, "storeFileSize",
this.getStoreFileSize());
if (this.getUncompressedStoreFileSize().get() != 0) {
Strings.appendKeyValue(sb, "compressionRatio",
String.format("%.4f",
(float) this.getStoreFileSize().get(Size.Unit.MEGABYTE) /
(float) this.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE)));
}
Strings.appendKeyValue(sb, "memStoreSize",
this.getMemStoreSize());
Strings.appendKeyValue(sb, "readRequestCount",
this.getReadRequestCount());
Strings.appendKeyValue(sb, "writeRequestCount",
this.getWriteRequestCount());
Strings.appendKeyValue(sb, "rootLevelIndexSize",
this.getStoreFileRootLevelIndexSize());
Strings.appendKeyValue(sb, "uncompressedDataIndexSize",
this.getStoreFileUncompressedDataIndexSize());
Strings.appendKeyValue(sb, "bloomFilterSize",
this.getBloomFilterSize());
Strings.appendKeyValue(sb, "compactingCellCount",
this.getCompactingCellCount());
Strings.appendKeyValue(sb, "compactedCellCount",
this.getCompactedCellCount());
float compactionProgressPct = Float.NaN;
if (this.getCompactingCellCount() > 0) {
compactionProgressPct = ((float) this.getCompactedCellCount() /
(float) this.getCompactingCellCount());
}
Strings.appendKeyValue(sb, "compactionProgressPct",
compactionProgressPct);
Strings.appendKeyValue(sb, "completedSequenceId",
this.getCompletedSequenceId());
Strings.appendKeyValue(sb, "dataLocality",
this.getDataLocality());
return sb.toString();
}
}
}

View File

@ -24,23 +24,26 @@ import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Objects;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Objects;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
/**
* This class is used for exporting current state of load on a RegionServer.
*
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link ServerMetrics} instead.
*/
@InterfaceAudience.Public
public class ServerLoad {
@Deprecated
public class ServerLoad implements ServerMetrics {
private final ServerMetrics metrics;
private int stores = 0;
private int storefiles = 0;
private int storeUncompressedSizeMB = 0;
@ -55,113 +58,200 @@ public class ServerLoad {
private int totalStaticBloomSizeKB = 0;
private long totalCompactingKVs = 0;
private long currentCompactedKVs = 0;
private long reportTime = 0;
/**
* DONT USE this construction. It make a fake server name;
*/
@InterfaceAudience.Private
public ServerLoad(ClusterStatusProtos.ServerLoad serverLoad) {
this(ServerName.valueOf("localhost,1,1"), serverLoad);
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
@InterfaceAudience.Private
public ServerLoad(ServerName name, ClusterStatusProtos.ServerLoad serverLoad) {
this(ServerMetricsBuilder.toServerMetrics(name, serverLoad));
this.serverLoad = serverLoad;
this.reportTime = System.currentTimeMillis();
for (ClusterStatusProtos.RegionLoad rl: serverLoad.getRegionLoadsList()) {
stores += rl.getStores();
storefiles += rl.getStorefiles();
storeUncompressedSizeMB += rl.getStoreUncompressedSizeMB();
storefileSizeMB += rl.getStorefileSizeMB();
memstoreSizeMB += rl.getMemStoreSizeMB();
storefileIndexSizeKB += rl.getStorefileIndexSizeKB();
readRequestsCount += rl.getReadRequestsCount();
filteredReadRequestsCount += rl.getFilteredReadRequestsCount();
writeRequestsCount += rl.getWriteRequestsCount();
rootIndexSizeKB += rl.getRootIndexSizeKB();
totalStaticIndexSizeKB += rl.getTotalStaticIndexSizeKB();
totalStaticBloomSizeKB += rl.getTotalStaticBloomSizeKB();
totalCompactingKVs += rl.getTotalCompactingKVs();
currentCompactedKVs += rl.getCurrentCompactedKVs();
}
@InterfaceAudience.Private
public ServerLoad(ServerMetrics metrics) {
this.metrics = metrics;
this.serverLoad = ServerMetricsBuilder.toServerLoad(metrics);
for (RegionMetrics rl : metrics.getRegionMetrics().values()) {
stores += rl.getStoreCount();
storefiles += rl.getStoreFileCount();
storeUncompressedSizeMB += rl.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE);
storefileSizeMB += rl.getStoreFileSize().get(Size.Unit.MEGABYTE);
memstoreSizeMB += rl.getMemStoreSize().get(Size.Unit.MEGABYTE);
readRequestsCount += rl.getReadRequestCount();
filteredReadRequestsCount += rl.getFilteredReadRequestCount();
writeRequestsCount += rl.getWriteRequestCount();
storefileIndexSizeKB += rl.getStoreFileIndexSize().get(Size.Unit.KILOBYTE);
rootIndexSizeKB += rl.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
totalStaticIndexSizeKB += rl.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
totalStaticBloomSizeKB += rl.getBloomFilterSize().get(Size.Unit.KILOBYTE);
totalCompactingKVs += rl.getCompactingCellCount();
currentCompactedKVs += rl.getCompactedCellCount();
}
}
// NOTE: Function name cannot start with "get" because then an OpenDataException is thrown because
// HBaseProtos.ServerLoad cannot be converted to an open data type(see HBASE-5967).
/* @return the underlying ServerLoad protobuf object */
/**
* NOTE: Function name cannot start with "get" because then an OpenDataException is thrown because
* HBaseProtos.ServerLoad cannot be converted to an open data type(see HBASE-5967).
* @return the underlying ServerLoad protobuf object
* @deprecated DONT use this pb object since the byte array backed may be modified in rpc layer
*/
@InterfaceAudience.Private
@Deprecated
public ClusterStatusProtos.ServerLoad obtainServerLoadPB() {
return serverLoad;
}
protected ClusterStatusProtos.ServerLoad serverLoad;
/* @return number of requests since last report. */
/**
* @return number of requests since last report.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #getRequestCountPerSecond} instead.
*/
@Deprecated
public long getNumberOfRequests() {
return serverLoad.getNumberOfRequests();
return getRequestCountPerSecond();
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* No flag in 2.0
*/
@Deprecated
public boolean hasNumberOfRequests() {
return serverLoad.hasNumberOfRequests();
return true;
}
/* @return total Number of requests from the start of the region server. */
/**
* @return total Number of requests from the start of the region server.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #getRequestCount} instead.
*/
@Deprecated
public long getTotalNumberOfRequests() {
return serverLoad.getTotalNumberOfRequests();
return getRequestCount();
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* No flag in 2.0
*/
@Deprecated
public boolean hasTotalNumberOfRequests() {
return serverLoad.hasTotalNumberOfRequests();
return true;
}
/* @return the amount of used heap, in MB. */
/**
* @return the amount of used heap, in MB.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #getUsedHeapSize} instead.
*/
@Deprecated
public int getUsedHeapMB() {
return serverLoad.getUsedHeapMB();
return (int) getUsedHeapSize().get(Size.Unit.MEGABYTE);
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* No flag in 2.0
*/
@Deprecated
public boolean hasUsedHeapMB() {
return serverLoad.hasUsedHeapMB();
return true;
}
/* @return the maximum allowable size of the heap, in MB. */
/**
* @return the maximum allowable size of the heap, in MB.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getMaxHeapSize} instead.
*/
@Deprecated
public int getMaxHeapMB() {
return serverLoad.getMaxHeapMB();
}
public boolean hasMaxHeapMB() {
return serverLoad.hasMaxHeapMB();
return (int) getMaxHeapSize().get(Size.Unit.MEGABYTE);
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* No flag in 2.0
*/
@Deprecated
public boolean hasMaxHeapMB() {
return true;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getStores() {
return stores;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getStorefiles() {
return storefiles;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getStoreUncompressedSizeMB() {
return storeUncompressedSizeMB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStorefileSizeMB()} instead.
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getStorefileSizeInMB() {
return storefileSizeMB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getStorefileSizeMB() {
return storefileSizeMB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getMemStoreSizeMB()} instead.
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getMemstoreSizeInMB() {
return memstoreSizeMB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getMemStoreSizeMB() {
return memstoreSizeMB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getStorefileIndexSizeKB()} instead.
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getStorefileIndexSizeInMB() {
@ -169,71 +259,162 @@ public class ServerLoad {
return (int) (getStorefileIndexSizeKB() >> 10);
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public long getStorefileIndexSizeKB() {
return storefileIndexSizeKB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public long getReadRequestsCount() {
return readRequestsCount;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public long getFilteredReadRequestsCount() {
return filteredReadRequestsCount;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public long getWriteRequestsCount() {
return writeRequestsCount;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getRootIndexSizeKB() {
return rootIndexSizeKB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getTotalStaticIndexSizeKB() {
return totalStaticIndexSizeKB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getTotalStaticBloomSizeKB() {
return totalStaticBloomSizeKB;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public long getTotalCompactingKVs() {
return totalCompactingKVs;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public long getCurrentCompactedKVs() {
return currentCompactedKVs;
}
/**
* @return the number of regions
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public int getNumberOfRegions() {
return serverLoad.getRegionLoadsCount();
return metrics.getRegionMetrics().size();
}
@Override
public ServerName getServerName() {
return metrics.getServerName();
}
@Override
public long getRequestCountPerSecond() {
return metrics.getRequestCountPerSecond();
}
@Override
public long getRequestCount() {
return metrics.getRequestCount();
}
@Override
public Size getUsedHeapSize() {
return metrics.getUsedHeapSize();
}
@Override
public Size getMaxHeapSize() {
return metrics.getMaxHeapSize();
}
@Override
public int getInfoServerPort() {
return serverLoad.getInfoServerPort();
return metrics.getInfoServerPort();
}
/**
* Call directly from client such as hbase shell
* @return the list of ReplicationLoadSource
*/
@Override
public List<ReplicationLoadSource> getReplicationLoadSourceList() {
return ProtobufUtil.toReplicationLoadSourceList(serverLoad.getReplLoadSourceList());
return metrics.getReplicationLoadSourceList();
}
/**
* Call directly from client such as hbase shell
* @return ReplicationLoadSink
*/
@Override
public ReplicationLoadSink getReplicationLoadSink() {
if (serverLoad.hasReplLoadSink()) {
return ProtobufUtil.toReplicationLoadSink(serverLoad.getReplLoadSink());
} else {
return null;
}
return metrics.getReplicationLoadSink();
}
@Override
public Map<byte[], RegionMetrics> getRegionMetrics() {
return metrics.getRegionMetrics();
}
@Override
public List<String> getCoprocessorNames() {
return metrics.getCoprocessorNames();
}
@Override
public long getReportTimestamp() {
return metrics.getReportTimestamp();
}
@Override
public long getLastReportTimestamp() {
return metrics.getLastReportTimestamp();
}
/**
@ -243,8 +424,11 @@ public class ServerLoad {
* interim, until we can figure out how to make rebalancing use all the info
* available, we're just going to make load purely the number of regions.
*
* @return load factor for this server
* @return load factor for this server.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getNumberOfRegions} instead.
*/
@Deprecated
public int getLoad() {
// See above comment
// int load = numberOfRequests == 0 ? 1 : numberOfRequests;
@ -254,53 +438,43 @@ public class ServerLoad {
}
/**
* @return region load metrics
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegionMetrics} instead.
*/
@Deprecated
public Map<byte[], RegionLoad> getRegionsLoad() {
Map<byte[], RegionLoad> regionLoads =
new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (ClusterStatusProtos.RegionLoad rl : serverLoad.getRegionLoadsList()) {
RegionLoad regionLoad = new RegionLoad(rl);
regionLoads.put(regionLoad.getName(), regionLoad);
}
return regionLoads;
return getRegionMetrics().entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> new RegionLoad(e.getValue()),
(v1, v2) -> {
throw new RuntimeException("key collisions?");
}, () -> new TreeMap(Bytes.BYTES_COMPARATOR)));
}
/**
* Return the RegionServer-level coprocessors
* @return string array of loaded RegionServer-level coprocessors
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getCoprocessorNames} instead.
*/
@Deprecated
public String[] getRegionServerCoprocessors() {
List<Coprocessor> list = obtainServerLoadPB().getCoprocessorsList();
String [] ret = new String[list.size()];
int i = 0;
for (Coprocessor elem : list) {
ret[i++] = elem.getName();
}
return ret;
return getCoprocessorNames().toArray(new String[getCoprocessorNames().size()]);
}
/**
* Return the RegionServer-level and Region-level coprocessors
* @return string array of loaded RegionServer-level and
* Region-level coprocessors
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getCoprocessorNames} instead.
*/
@Deprecated
public String[] getRsCoprocessors() {
// Need a set to remove duplicates, but since generated Coprocessor class
// is not Comparable, make it a Set<String> instead of Set<Coprocessor>
TreeSet<String> coprocessSet = new TreeSet<>();
for (Coprocessor coprocessor : obtainServerLoadPB().getCoprocessorsList()) {
coprocessSet.add(coprocessor.getName());
}
return coprocessSet.toArray(new String[coprocessSet.size()]);
return getRegionServerCoprocessors();
}
/**
* @return number of requests per second received since the last report
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRequestCountPerSecond} instead.
*/
@Deprecated
public double getRequestsPerSecond() {
return getNumberOfRequests();
return getRequestCountPerSecond();
}
/**
@ -308,70 +482,73 @@ public class ServerLoad {
*/
@Override
public String toString() {
StringBuilder sb =
Strings.appendKeyValue(new StringBuilder(), "requestsPerSecond",
Double.valueOf(getRequestsPerSecond()));
StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "requestsPerSecond",
Double.valueOf(getRequestsPerSecond()));
Strings.appendKeyValue(sb, "numberOfOnlineRegions", Integer.valueOf(getNumberOfRegions()));
sb = Strings.appendKeyValue(sb, "usedHeapMB", Integer.valueOf(this.getUsedHeapMB()));
sb = Strings.appendKeyValue(sb, "maxHeapMB", Integer.valueOf(getMaxHeapMB()));
sb = Strings.appendKeyValue(sb, "numberOfStores", Integer.valueOf(this.stores));
sb = Strings.appendKeyValue(sb, "numberOfStorefiles", Integer.valueOf(this.storefiles));
sb =
Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
Integer.valueOf(this.storeUncompressedSizeMB));
sb = Strings.appendKeyValue(sb, "storefileSizeMB", Integer.valueOf(this.storefileSizeMB));
Strings.appendKeyValue(sb, "usedHeapMB", Integer.valueOf(this.getUsedHeapMB()));
Strings.appendKeyValue(sb, "maxHeapMB", Integer.valueOf(getMaxHeapMB()));
Strings.appendKeyValue(sb, "numberOfStores", Integer.valueOf(this.stores));
Strings.appendKeyValue(sb, "numberOfStorefiles", Integer.valueOf(this.storefiles));
Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
Integer.valueOf(this.storeUncompressedSizeMB));
Strings.appendKeyValue(sb, "storefileSizeMB", Integer.valueOf(this.storefileSizeMB));
if (this.storeUncompressedSizeMB != 0) {
sb =
Strings.appendKeyValue(
sb,
"compressionRatio",
String.format("%.4f", (float) this.storefileSizeMB
/ (float) this.storeUncompressedSizeMB));
Strings.appendKeyValue(sb, "compressionRatio", String.format("%.4f",
(float) this.storefileSizeMB / (float) this.storeUncompressedSizeMB));
}
sb = Strings.appendKeyValue(sb, "memstoreSizeMB", Integer.valueOf(this.memstoreSizeMB));
sb =
Strings.appendKeyValue(sb, "storefileIndexSizeKB",
Long.valueOf(this.storefileIndexSizeKB));
sb = Strings.appendKeyValue(sb, "readRequestsCount", Long.valueOf(this.readRequestsCount));
sb = Strings.appendKeyValue(sb, "filteredReadRequestsCount",
Long.valueOf(this.filteredReadRequestsCount));
sb = Strings.appendKeyValue(sb, "writeRequestsCount", Long.valueOf(this.writeRequestsCount));
sb = Strings.appendKeyValue(sb, "rootIndexSizeKB", Integer.valueOf(this.rootIndexSizeKB));
sb =
Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
Integer.valueOf(this.totalStaticIndexSizeKB));
sb =
Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
Integer.valueOf(this.totalStaticBloomSizeKB));
sb = Strings.appendKeyValue(sb, "totalCompactingKVs", Long.valueOf(this.totalCompactingKVs));
sb = Strings.appendKeyValue(sb, "currentCompactedKVs", Long.valueOf(this.currentCompactedKVs));
Strings.appendKeyValue(sb, "memstoreSizeMB", Integer.valueOf(this.memstoreSizeMB));
Strings.appendKeyValue(sb, "storefileIndexSizeKB",
Long.valueOf(this.storefileIndexSizeKB));
Strings.appendKeyValue(sb, "readRequestsCount", Long.valueOf(this.readRequestsCount));
Strings.appendKeyValue(sb, "filteredReadRequestsCount",
Long.valueOf(this.filteredReadRequestsCount));
Strings.appendKeyValue(sb, "writeRequestsCount", Long.valueOf(this.writeRequestsCount));
Strings.appendKeyValue(sb, "rootIndexSizeKB", Integer.valueOf(this.rootIndexSizeKB));
Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
Integer.valueOf(this.totalStaticIndexSizeKB));
Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
Integer.valueOf(this.totalStaticBloomSizeKB));
Strings.appendKeyValue(sb, "totalCompactingKVs", Long.valueOf(this.totalCompactingKVs));
Strings.appendKeyValue(sb, "currentCompactedKVs", Long.valueOf(this.currentCompactedKVs));
float compactionProgressPct = Float.NaN;
if (this.totalCompactingKVs > 0) {
compactionProgressPct =
Float.valueOf((float) this.currentCompactedKVs / this.totalCompactingKVs);
}
sb = Strings.appendKeyValue(sb, "compactionProgressPct", compactionProgressPct);
Strings.appendKeyValue(sb, "compactionProgressPct", compactionProgressPct);
String[] coprocessorStrings = getRsCoprocessors();
if (coprocessorStrings != null) {
sb = Strings.appendKeyValue(sb, "coprocessors", Arrays.toString(coprocessorStrings));
Strings.appendKeyValue(sb, "coprocessors", Arrays.toString(coprocessorStrings));
}
return sb.toString();
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link ServerMetricsBuilder#of(ServerName)} instead.
*/
@Deprecated
public static final ServerLoad EMPTY_SERVERLOAD =
new ServerLoad(ClusterStatusProtos.ServerLoad.newBuilder().build());
new ServerLoad(ServerName.valueOf("localhost,1,1"),
ClusterStatusProtos.ServerLoad.newBuilder().build());
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getReportTimestamp} instead.
*/
@Deprecated
public long getReportTime() {
return reportTime;
return getReportTimestamp();
}
@Override
public int hashCode() {
return Objects.hashCode(stores, storefiles, storeUncompressedSizeMB,
storefileSizeMB, memstoreSizeMB, storefileIndexSizeKB, readRequestsCount,
filteredReadRequestsCount, writeRequestsCount, rootIndexSizeKB, totalStaticIndexSizeKB,
totalStaticBloomSizeKB, totalCompactingKVs, currentCompactedKVs);
return Objects
.hashCode(stores, storefiles, storeUncompressedSizeMB, storefileSizeMB, memstoreSizeMB,
storefileIndexSizeKB, readRequestsCount, filteredReadRequestsCount, writeRequestsCount,
rootIndexSizeKB, totalStaticIndexSizeKB, totalStaticBloomSizeKB, totalCompactingKVs,
currentCompactedKVs);
}
@Override
@ -379,16 +556,17 @@ public class ServerLoad {
if (other == this) return true;
if (other instanceof ServerLoad) {
ServerLoad sl = ((ServerLoad) other);
return stores == sl.stores && storefiles == sl.storefiles &&
storeUncompressedSizeMB == sl.storeUncompressedSizeMB &&
storefileSizeMB == sl.storefileSizeMB && memstoreSizeMB == sl.memstoreSizeMB &&
storefileIndexSizeKB == sl.storefileIndexSizeKB && readRequestsCount == sl.readRequestsCount &&
filteredReadRequestsCount == sl.filteredReadRequestsCount &&
writeRequestsCount == sl.writeRequestsCount && rootIndexSizeKB == sl.rootIndexSizeKB &&
totalStaticIndexSizeKB == sl.totalStaticIndexSizeKB &&
totalStaticBloomSizeKB == sl.totalStaticBloomSizeKB &&
totalCompactingKVs == sl.totalCompactingKVs &&
currentCompactedKVs == sl.currentCompactedKVs;
return stores == sl.stores && storefiles == sl.storefiles
&& storeUncompressedSizeMB == sl.storeUncompressedSizeMB
&& storefileSizeMB == sl.storefileSizeMB && memstoreSizeMB == sl.memstoreSizeMB
&& storefileIndexSizeKB == sl.storefileIndexSizeKB
&& readRequestsCount == sl.readRequestsCount
&& filteredReadRequestsCount == sl.filteredReadRequestsCount
&& writeRequestsCount == sl.writeRequestsCount && rootIndexSizeKB == sl.rootIndexSizeKB
&& totalStaticIndexSizeKB == sl.totalStaticIndexSizeKB
&& totalStaticBloomSizeKB == sl.totalStaticBloomSizeKB
&& totalCompactingKVs == sl.totalCompactingKVs
&& currentCompactedKVs == sl.currentCompactedKVs;
}
return false;
}

View File

@ -0,0 +1,90 @@
/**
* 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 edu.umd.cs.findbugs.annotations.Nullable;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This class is used for exporting current state of load on a RegionServer.
*/
@InterfaceAudience.Public
public interface ServerMetrics {
ServerName getServerName();
/**
* @return the number of requests per second.
*/
long getRequestCountPerSecond();
/**
* @return total Number of requests from the start of the region server.
*/
long getRequestCount();
/**
* @return the amount of used heap
*/
Size getUsedHeapSize();
/**
* @return the maximum allowable size of the heap
*/
Size getMaxHeapSize();
int getInfoServerPort();
/**
* Call directly from client such as hbase shell
* @return the list of ReplicationLoadSource
*/
List<ReplicationLoadSource> getReplicationLoadSourceList();
/**
* Call directly from client such as hbase shell
* @return ReplicationLoadSink
*/
@Nullable
ReplicationLoadSink getReplicationLoadSink();
/**
* @return region load metrics
*/
Map<byte[], RegionMetrics> getRegionMetrics();
/**
* Return the RegionServer-level and Region-level coprocessors
* @return string list of loaded RegionServer-level and Region-level coprocessors
*/
List<String> getCoprocessorNames();
/**
* @return the timestamp (server side) of generating this metrics
*/
long getReportTimestamp();
/**
* @return the last timestamp (server side) of generating this metrics
*/
long getLastReportTimestamp();
}

View File

@ -0,0 +1,352 @@
/**
* 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 edu.umd.cs.findbugs.annotations.Nullable;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@InterfaceAudience.Private
public final class ServerMetricsBuilder {
/**
* @param sn the server name
* @return a empty metrics
*/
public static ServerMetrics of(ServerName sn) {
return newBuilder(sn).build();
}
public static ServerMetrics toServerMetrics(ClusterStatusProtos.LiveServerInfo serverInfo) {
return toServerMetrics(ProtobufUtil.toServerName(serverInfo.getServer()),
serverInfo.getServerLoad());
}
public static ServerMetrics toServerMetrics(ServerName serverName,
ClusterStatusProtos.ServerLoad serverLoadPB) {
return ServerMetricsBuilder.newBuilder(serverName)
.setRequestCountPerSecond(serverLoadPB.getNumberOfRequests())
.setRequestCount(serverLoadPB.getTotalNumberOfRequests())
.setInfoServerPort(serverLoadPB.getInfoServerPort())
.setMaxHeapSize(new Size(serverLoadPB.getMaxHeapMB(), Size.Unit.MEGABYTE))
.setUsedHeapSize(new Size(serverLoadPB.getUsedHeapMB(), Size.Unit.MEGABYTE))
.setCoprocessorNames(serverLoadPB.getCoprocessorsList().stream()
.map(HBaseProtos.Coprocessor::getName).collect(Collectors.toList()))
.setRegionMetrics(serverLoadPB.getRegionLoadsList().stream()
.map(RegionMetricsBuilder::toRegionMetrics)
.collect(Collectors.toList()))
.setReplicationLoadSources(serverLoadPB.getReplLoadSourceList().stream()
.map(ProtobufUtil::toReplicationLoadSource)
.collect(Collectors.toList()))
.setReplicationLoadSink(serverLoadPB.hasReplLoadSink() ?
ProtobufUtil.toReplicationLoadSink(serverLoadPB.getReplLoadSink()) : null)
.setReportTimestamp(serverLoadPB.getReportEndTime())
.setLastReportTimestamp(serverLoadPB.getReportStartTime())
.build();
}
public static List<HBaseProtos.Coprocessor> toCoprocessor(List<String> names) {
return names.stream()
.map(n -> HBaseProtos.Coprocessor.newBuilder().setName(n).build())
.collect(Collectors.toList());
}
public static ClusterStatusProtos.ServerLoad toServerLoad(ServerMetrics metrics) {
ClusterStatusProtos.ServerLoad.Builder builder = ClusterStatusProtos.ServerLoad.newBuilder()
.setNumberOfRequests(metrics.getRequestCountPerSecond())
.setTotalNumberOfRequests(metrics.getRequestCount())
.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()))
.setReportStartTime(metrics.getLastReportTimestamp())
.setReportEndTime(metrics.getReportTimestamp());
if (metrics.getReplicationLoadSink() != null) {
builder.setReplLoadSink(ProtobufUtil.toReplicationLoadSink(
metrics.getReplicationLoadSink()));
}
return builder.build();
}
public static ServerMetricsBuilder newBuilder(ServerName sn) {
return new ServerMetricsBuilder(sn);
}
private final ServerName serverName;
private long requestCountPerSecond;
private long requestCount;
private Size usedHeapSize = Size.ZERO;
private Size maxHeapSize = Size.ZERO;
private int infoServerPort;
private List<ReplicationLoadSource> sources = Collections.emptyList();
@Nullable
private ReplicationLoadSink sink = null;
private final Map<byte[], RegionMetrics> regionStatus = new TreeMap<>(Bytes.BYTES_COMPARATOR);
private List<String> coprocessorNames = Collections.emptyList();
private long reportTimestamp = System.currentTimeMillis();
private long lastReportTimestamp = 0;
private ServerMetricsBuilder(ServerName serverName) {
this.serverName = serverName;
}
public ServerMetricsBuilder setRequestCountPerSecond(long value) {
this.requestCountPerSecond = value;
return this;
}
public ServerMetricsBuilder setRequestCount(long value) {
this.requestCount = value;
return this;
}
public ServerMetricsBuilder setUsedHeapSize(Size value) {
this.usedHeapSize = value;
return this;
}
public ServerMetricsBuilder setMaxHeapSize(Size value) {
this.maxHeapSize = value;
return this;
}
public ServerMetricsBuilder setInfoServerPort(int value) {
this.infoServerPort = value;
return this;
}
public ServerMetricsBuilder setReplicationLoadSources(List<ReplicationLoadSource> value) {
this.sources = value;
return this;
}
public ServerMetricsBuilder setReplicationLoadSink(ReplicationLoadSink value) {
this.sink = value;
return this;
}
public ServerMetricsBuilder setRegionMetrics(List<RegionMetrics> value) {
value.forEach(v -> this.regionStatus.put(v.getRegionName(), v));
return this;
}
public ServerMetricsBuilder setCoprocessorNames(List<String> value) {
this.coprocessorNames = value;
return this;
}
public ServerMetricsBuilder setReportTimestamp(long value) {
this.reportTimestamp = value;
return this;
}
public ServerMetricsBuilder setLastReportTimestamp(long value) {
this.lastReportTimestamp = value;
return this;
}
public ServerMetrics build() {
return new ServerMetricsImpl(
serverName,
requestCountPerSecond,
requestCount,
usedHeapSize,
maxHeapSize,
infoServerPort,
sources,
sink,
regionStatus,
coprocessorNames,
reportTimestamp,
lastReportTimestamp);
}
private static class ServerMetricsImpl implements ServerMetrics {
private final ServerName serverName;
private final long requestCountPerSecond;
private final long requestCount;
private final Size usedHeapSize;
private final Size maxHeapSize;
private final int infoServerPort;
private final List<ReplicationLoadSource> sources;
@Nullable
private final ReplicationLoadSink sink;
private final Map<byte[], RegionMetrics> regionStatus;
private final List<String> coprocessorNames;
private final long reportTimestamp;
private final long lastReportTimestamp;
ServerMetricsImpl(ServerName serverName, long requestCountPerSecond, long requestCount,
Size usedHeapSize, Size maxHeapSize, int infoServerPort, List<ReplicationLoadSource> sources,
ReplicationLoadSink sink, Map<byte[], RegionMetrics> regionStatus,
List<String> coprocessorNames, long reportTimestamp, long lastReportTimestamp) {
this.serverName = Preconditions.checkNotNull(serverName);
this.requestCountPerSecond = requestCountPerSecond;
this.requestCount = requestCount;
this.usedHeapSize = Preconditions.checkNotNull(usedHeapSize);
this.maxHeapSize = Preconditions.checkNotNull(maxHeapSize);
this.infoServerPort = infoServerPort;
this.sources = Preconditions.checkNotNull(sources);
this.sink = sink;
this.regionStatus = Preconditions.checkNotNull(regionStatus);
this.coprocessorNames =Preconditions.checkNotNull(coprocessorNames);
this.reportTimestamp = reportTimestamp;
this.lastReportTimestamp = lastReportTimestamp;
}
@Override
public ServerName getServerName() {
return serverName;
}
@Override
public long getRequestCountPerSecond() {
return requestCountPerSecond;
}
@Override
public long getRequestCount() {
return requestCount;
}
@Override
public Size getUsedHeapSize() {
return usedHeapSize;
}
@Override
public Size getMaxHeapSize() {
return maxHeapSize;
}
@Override
public int getInfoServerPort() {
return infoServerPort;
}
@Override
public List<ReplicationLoadSource> getReplicationLoadSourceList() {
return Collections.unmodifiableList(sources);
}
@Override
public ReplicationLoadSink getReplicationLoadSink() {
return sink;
}
@Override
public Map<byte[], RegionMetrics> getRegionMetrics() {
return Collections.unmodifiableMap(regionStatus);
}
@Override
public List<String> getCoprocessorNames() {
return Collections.unmodifiableList(coprocessorNames);
}
@Override
public long getReportTimestamp() {
return reportTimestamp;
}
@Override
public long getLastReportTimestamp() {
return lastReportTimestamp;
}
@Override
public String toString() {
int storeCount = 0;
int storeFileCount = 0;
long uncompressedStoreFileSizeMB = 0;
long storeFileSizeMB = 0;
long memStoreSizeMB = 0;
long storefileIndexSizeKB = 0;
long rootLevelIndexSizeKB = 0;
long readRequestsCount = 0;
long writeRequestsCount = 0;
long filteredReadRequestsCount = 0;
long bloomFilterSizeMB = 0;
long compactingCellCount = 0;
long compactedCellCount = 0;
for (RegionMetrics r : getRegionMetrics().values()) {
storeCount += r.getStoreCount();
storeFileCount += r.getStoreFileCount();
uncompressedStoreFileSizeMB += r.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE);
storeFileSizeMB += r.getStoreFileSize().get(Size.Unit.MEGABYTE);
memStoreSizeMB += r.getMemStoreSize().get(Size.Unit.MEGABYTE);
storefileIndexSizeKB += r.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
readRequestsCount += r.getReadRequestCount();
writeRequestsCount += r.getWriteRequestCount();
filteredReadRequestsCount += r.getFilteredReadRequestCount();
rootLevelIndexSizeKB += r.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
bloomFilterSizeMB += r.getBloomFilterSize().get(Size.Unit.MEGABYTE);
compactedCellCount += r.getCompactedCellCount();
compactingCellCount += r.getCompactingCellCount();
}
StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "requestsPerSecond",
Double.valueOf(getRequestCountPerSecond()));
Strings.appendKeyValue(sb, "numberOfOnlineRegions",
Integer.valueOf(getRegionMetrics().size()));
Strings.appendKeyValue(sb, "usedHeapMB", getUsedHeapSize());
Strings.appendKeyValue(sb, "maxHeapMB", getMaxHeapSize());
Strings.appendKeyValue(sb, "numberOfStores", storeCount);
Strings.appendKeyValue(sb, "numberOfStorefiles", storeFileCount);
Strings.appendKeyValue(sb, "storefileUncompressedSizeMB", uncompressedStoreFileSizeMB);
Strings.appendKeyValue(sb, "storefileSizeMB", storeFileSizeMB);
if (uncompressedStoreFileSizeMB != 0) {
Strings.appendKeyValue(sb, "compressionRatio", String.format("%.4f",
(float) storeFileSizeMB / (float) uncompressedStoreFileSizeMB));
}
Strings.appendKeyValue(sb, "memstoreSizeMB", memStoreSizeMB);
Strings.appendKeyValue(sb, "readRequestsCount", readRequestsCount);
Strings.appendKeyValue(sb, "filteredReadRequestsCount", filteredReadRequestsCount);
Strings.appendKeyValue(sb, "writeRequestsCount", writeRequestsCount);
Strings.appendKeyValue(sb, "rootIndexSizeKB", rootLevelIndexSizeKB);
Strings.appendKeyValue(sb, "totalStaticIndexSizeKB", storefileIndexSizeKB);
Strings.appendKeyValue(sb, "totalStaticBloomSizeKB", bloomFilterSizeMB);
Strings.appendKeyValue(sb, "totalCompactingKVs", compactingCellCount);
Strings.appendKeyValue(sb, "currentCompactedKVs", compactedCellCount);
float compactionProgressPct = Float.NaN;
if (compactingCellCount > 0) {
compactionProgressPct =
Float.valueOf((float) compactedCellCount / compactingCellCount);
}
Strings.appendKeyValue(sb, "compactionProgressPct", compactionProgressPct);
Strings.appendKeyValue(sb, "coprocessors", getCoprocessorNames());
return sb.toString();
}
}
}

View File

@ -0,0 +1,158 @@
/**
* 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.math.BigDecimal;
import java.util.Objects;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
/**
* It is used to represent the size with different units.
* This class doesn't serve for the precise computation.
*/
@InterfaceAudience.Public
public final class Size implements Comparable<Size> {
public static final Size ZERO = new Size(0, Unit.KILOBYTE);
private static final BigDecimal SCALE_BASE = BigDecimal.valueOf(1024D);
public enum Unit {
// keep the room to add more units for HBase 10.x
PETABYTE(100, "PB"),
TERABYTE(99, "TB"),
GIGABYTE(98, "GB"),
MEGABYTE(97, "MB"),
KILOBYTE(96, "KB"),
BYTE(95, "B");
private final int orderOfSize;
private final String simpleName;
Unit(int orderOfSize, String simpleName) {
this.orderOfSize = orderOfSize;
this.simpleName = simpleName;
}
public int getOrderOfSize() {
return orderOfSize;
}
public String getSimpleName() {
return simpleName;
}
}
private final double value;
private final Unit unit;
public Size(double value, Unit unit) {
if (value < 0) {
throw new IllegalArgumentException("The value:" + value + " can't be negative");
}
this.value = value;
this.unit = Preconditions.checkNotNull(unit);
}
/**
* @return size unit
*/
public Unit getUnit() {
return unit;
}
/**
* get the value
*/
public long getLongValue() {
return (long) value;
}
/**
* get the value
*/
public double get() {
return value;
}
/**
* get the value which is converted to specified unit.
*
* @param unit size unit
* @return the converted value
*/
public double get(Unit unit) {
if (value == 0) {
return value;
}
int diff = this.unit.getOrderOfSize() - unit.getOrderOfSize();
if (diff == 0) {
return value;
}
BigDecimal rval = BigDecimal.valueOf(value);
for (int i = 0; i != Math.abs(diff); ++i) {
rval = diff > 0 ? rval.multiply(SCALE_BASE) : rval.divide(SCALE_BASE);
}
return rval.doubleValue();
}
@Override
public int compareTo(Size other) {
int diff = unit.getOrderOfSize() - other.unit.getOrderOfSize();
if (diff == 0) {
return Double.compare(value, other.value);
}
BigDecimal thisValue = BigDecimal.valueOf(value);
BigDecimal otherValue = BigDecimal.valueOf(other.value);
if (diff > 0) {
for (int i = 0; i != Math.abs(diff); ++i) {
thisValue = thisValue.multiply(SCALE_BASE);
}
} else {
for (int i = 0; i != Math.abs(diff); ++i) {
otherValue = otherValue.multiply(SCALE_BASE);
}
}
return thisValue.compareTo(otherValue);
}
@Override
public String toString() {
return value + unit.getSimpleName();
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if (obj instanceof Size) {
return compareTo((Size)obj) == 0;
}
return false;
}
@Override
public int hashCode() {
return Objects.hash(value, unit);
}
}

View File

@ -28,12 +28,11 @@ import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.Future;
import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CacheEvictionStats;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@ -43,7 +42,6 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@ -60,6 +58,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The administrative API for HBase. Obtain an instance from {@link Connection#getAdmin()} and

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.client;
import com.google.protobuf.RpcChannel;
import java.util.Arrays;
import java.util.Collection;
import java.util.EnumSet;
@ -29,9 +28,8 @@ import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import java.util.regex.Pattern;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.client;
import com.google.protobuf.RpcChannel;
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
@ -29,9 +28,8 @@ import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.function.Function;
import java.util.regex.Pattern;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;

View File

@ -264,7 +264,7 @@ class ClusterStatusListener implements Closeable {
ByteBufInputStream bis = new ByteBufInputStream(dp.content());
try {
ClusterStatusProtos.ClusterStatus csp = ClusterStatusProtos.ClusterStatus.parseFrom(bis);
ClusterStatus ncs = ProtobufUtil.convert(csp);
ClusterStatus ncs = ProtobufUtil.toClusterStatus(csp);
receive(ncs);
} finally {
bis.close();

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
@ -45,13 +44,12 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CacheEvictionStats;
import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
@ -103,6 +101,7 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@ -2086,7 +2085,7 @@ public class HBaseAdmin implements Admin {
@Override
protected ClusterStatus rpcCall() throws Exception {
GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest(options);
return ProtobufUtil.convert(
return ProtobufUtil.toClusterStatus(
master.getClusterStatus(getRpcController(), req).getClusterStatus());
}
});

View File

@ -23,7 +23,6 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.protobuf.Message;
import com.google.protobuf.RpcChannel;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -43,12 +42,11 @@ import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
@ -87,6 +85,7 @@ import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
@ -2617,7 +2616,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
.<GetClusterStatusRequest, GetClusterStatusResponse, ClusterStatus> call(controller,
stub, RequestConverter.buildGetClusterStatusRequest(options),
(s, c, req, done) -> s.getClusterStatus(c, req, done),
resp -> ProtobufUtil.convert(resp.getClusterStatus()))).call();
resp -> ProtobufUtil.toClusterStatus(resp.getClusterStatus()))).call();
}
@Override

View File

@ -15,11 +15,13 @@ import org.apache.yetus.audience.InterfaceAudience;
/**
* A HBase ReplicationLoad to present MetricsSink information
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
public class ReplicationLoadSink {
private long ageOfLastAppliedOp;
private long timeStampsOfLastAppliedOp;
private final long ageOfLastAppliedOp;
private final long timeStampsOfLastAppliedOp;
// TODO: add the builder for this class
@InterfaceAudience.Private
public ReplicationLoadSink(long age, long timeStamp) {
this.ageOfLastAppliedOp = age;
this.timeStampsOfLastAppliedOp = timeStamp;

View File

@ -15,14 +15,16 @@ import org.apache.yetus.audience.InterfaceAudience;
/**
* A HBase ReplicationLoad to present MetricsSource information
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
public class ReplicationLoadSource {
private String peerID;
private long ageOfLastShippedOp;
private int sizeOfLogQueue;
private long timeStampOfLastShippedOp;
private long replicationLag;
private final String peerID;
private final long ageOfLastShippedOp;
private final int sizeOfLogQueue;
private final long timeStampOfLastShippedOp;
private final long replicationLag;
// TODO: add the builder for this class
@InterfaceAudience.Private
public ReplicationLoadSource(String id, long age, int size, long timeStamp, long lag) {
this.peerID = id;
this.ageOfLastShippedOp = age;

View File

@ -25,7 +25,6 @@ import java.lang.reflect.Method;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
@ -49,9 +48,8 @@ import org.apache.hadoop.hbase.Cell.DataType;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.ExtendedCellBuilder;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
@ -60,7 +58,6 @@ import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Append;
@ -92,7 +89,6 @@ import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.LimitInputStream;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter;
import org.apache.hadoop.hbase.quotas.QuotaScope;
@ -150,11 +146,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationPr
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair;
@ -2731,23 +2724,14 @@ public final class ProtobufUtil {
}
public static ReplicationLoadSink toReplicationLoadSink(
ClusterStatusProtos.ReplicationLoadSink cls) {
return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());
ClusterStatusProtos.ReplicationLoadSink rls) {
return new ReplicationLoadSink(rls.getAgeOfLastAppliedOp(), rls.getTimeStampsOfLastAppliedOp());
}
public static ReplicationLoadSource toReplicationLoadSource(
ClusterStatusProtos.ReplicationLoadSource cls) {
return new ReplicationLoadSource(cls.getPeerID(), cls.getAgeOfLastShippedOp(),
cls.getSizeOfLogQueue(), cls.getTimeStampOfLastShippedOp(), cls.getReplicationLag());
}
public static List<ReplicationLoadSource> toReplicationLoadSourceList(
List<ClusterStatusProtos.ReplicationLoadSource> clsList) {
ArrayList<ReplicationLoadSource> rlsList = new ArrayList<>(clsList.size());
for (ClusterStatusProtos.ReplicationLoadSource cls : clsList) {
rlsList.add(toReplicationLoadSource(cls));
}
return rlsList;
ClusterStatusProtos.ReplicationLoadSource rls) {
return new ReplicationLoadSource(rls.getPeerID(), rls.getAgeOfLastShippedOp(),
rls.getSizeOfLogQueue(), rls.getTimeStampOfLastShippedOp(), rls.getReplicationLag());
}
/**
@ -2991,213 +2975,8 @@ public final class ProtobufUtil {
* @param proto the protobuf ClusterStatus
* @return the converted ClusterStatus
*/
public static ClusterStatus convert(ClusterStatusProtos.ClusterStatus proto) {
ClusterStatus.Builder builder = ClusterStatus.newBuilder();
Map<ServerName, ServerLoad> servers = null;
servers = new HashMap<>(proto.getLiveServersList().size());
for (LiveServerInfo lsi : proto.getLiveServersList()) {
servers.put(ProtobufUtil.toServerName(
lsi.getServer()), new ServerLoad(lsi.getServerLoad()));
}
List<ServerName> deadServers = new ArrayList<>(proto.getDeadServersList().size());
for (HBaseProtos.ServerName sn : proto.getDeadServersList()) {
deadServers.add(ProtobufUtil.toServerName(sn));
}
List<ServerName> backupMasters = new ArrayList<>(proto.getBackupMastersList().size());
for (HBaseProtos.ServerName sn : proto.getBackupMastersList()) {
backupMasters.add(ProtobufUtil.toServerName(sn));
}
List<RegionState> rit =
new ArrayList<>(proto.getRegionsInTransitionList().size());
for (RegionInTransition region : proto.getRegionsInTransitionList()) {
RegionState value = RegionState.convert(region.getRegionState());
rit.add(value);
}
String[] masterCoprocessors = null;
final int numMasterCoprocessors = proto.getMasterCoprocessorsCount();
masterCoprocessors = new String[numMasterCoprocessors];
for (int i = 0; i < numMasterCoprocessors; i++) {
masterCoprocessors[i] = proto.getMasterCoprocessors(i).getName();
}
String clusterId = null;
if (proto.hasClusterId()) {
clusterId = ClusterId.convert(proto.getClusterId()).toString();
}
String hbaseVersion = null;
if (proto.hasHbaseVersion()) {
hbaseVersion = proto.getHbaseVersion().getVersion();
}
ServerName master = null;
if (proto.hasMaster()) {
master = ProtobufUtil.toServerName(proto.getMaster());
}
Boolean balancerOn = null;
if (proto.hasBalancerOn()) {
balancerOn = proto.getBalancerOn();
}
builder.setHBaseVersion(hbaseVersion)
.setClusterId(clusterId)
.setLiveServers(servers)
.setDeadServers(deadServers)
.setMaster(master)
.setBackupMasters(backupMasters)
.setRegionState(rit)
.setMasterCoprocessors(masterCoprocessors)
.setBalancerOn(balancerOn);
if (proto.hasMasterInfoPort()) {
builder.setMasterInfoPort(proto.getMasterInfoPort());
}
return builder.build();
}
/**
* Convert ClusterStatusProtos.Option to ClusterStatus.Option
* @param option a ClusterStatusProtos.Option
* @return converted ClusterStatus.Option
*/
public static ClusterStatus.Option toOption(ClusterStatusProtos.Option option) {
switch (option) {
case HBASE_VERSION: return ClusterStatus.Option.HBASE_VERSION;
case LIVE_SERVERS: return ClusterStatus.Option.LIVE_SERVERS;
case DEAD_SERVERS: return ClusterStatus.Option.DEAD_SERVERS;
case REGIONS_IN_TRANSITION: return ClusterStatus.Option.REGIONS_IN_TRANSITION;
case CLUSTER_ID: return ClusterStatus.Option.CLUSTER_ID;
case MASTER_COPROCESSORS: return ClusterStatus.Option.MASTER_COPROCESSORS;
case MASTER: return ClusterStatus.Option.MASTER;
case BACKUP_MASTERS: return ClusterStatus.Option.BACKUP_MASTERS;
case BALANCER_ON: return ClusterStatus.Option.BALANCER_ON;
case MASTER_INFO_PORT: return ClusterStatus.Option.MASTER_INFO_PORT;
// should not reach here
default: throw new IllegalArgumentException("Invalid option: " + option);
}
}
/**
* Convert ClusterStatus.Option to ClusterStatusProtos.Option
* @param option a ClusterStatus.Option
* @return converted ClusterStatusProtos.Option
*/
public static ClusterStatusProtos.Option toOption(ClusterStatus.Option option) {
switch (option) {
case HBASE_VERSION: return ClusterStatusProtos.Option.HBASE_VERSION;
case LIVE_SERVERS: return ClusterStatusProtos.Option.LIVE_SERVERS;
case DEAD_SERVERS: return ClusterStatusProtos.Option.DEAD_SERVERS;
case REGIONS_IN_TRANSITION: return ClusterStatusProtos.Option.REGIONS_IN_TRANSITION;
case CLUSTER_ID: return ClusterStatusProtos.Option.CLUSTER_ID;
case MASTER_COPROCESSORS: return ClusterStatusProtos.Option.MASTER_COPROCESSORS;
case MASTER: return ClusterStatusProtos.Option.MASTER;
case BACKUP_MASTERS: return ClusterStatusProtos.Option.BACKUP_MASTERS;
case BALANCER_ON: return ClusterStatusProtos.Option.BALANCER_ON;
case MASTER_INFO_PORT: return ClusterStatusProtos.Option.MASTER_INFO_PORT;
// should not reach here
default: throw new IllegalArgumentException("Invalid option: " + option);
}
}
/**
* Convert a list of ClusterStatusProtos.Option to an enum set of ClusterStatus.Option
* @param options
* @return an enum set of ClusterStatus.Option
*/
public static EnumSet<Option> toOptions(List<ClusterStatusProtos.Option> options) {
EnumSet<Option> result = EnumSet.noneOf(Option.class);
for (ClusterStatusProtos.Option opt : options) {
result.add(toOption(opt));
}
return result;
}
/**
* Convert an enum set of ClusterStatus.Option to a list of ClusterStatusProtos.Option
* @param options
* @return a list of ClusterStatusProtos.Option
*/
public static List<ClusterStatusProtos.Option> toOptions(EnumSet<Option> options) {
List<ClusterStatusProtos.Option> result = new ArrayList<>(options.size());
for (ClusterStatus.Option opt : options) {
result.add(toOption(opt));
}
return result;
}
/**
* Convert a ClusterStatus to a protobuf ClusterStatus
*
* @return the protobuf ClusterStatus
*/
public static ClusterStatusProtos.ClusterStatus convert(ClusterStatus status) {
ClusterStatusProtos.ClusterStatus.Builder builder =
ClusterStatusProtos.ClusterStatus.newBuilder();
if (status.getHBaseVersion() != null) {
builder.setHbaseVersion(
HBaseVersionFileContent.newBuilder()
.setVersion(status.getHBaseVersion()));
}
if (status.getServers() != null) {
for (ServerName serverName : status.getServers()) {
LiveServerInfo.Builder lsi =
LiveServerInfo.newBuilder().setServer(ProtobufUtil.toServerName(serverName));
lsi.setServerLoad(status.getLoad(serverName).obtainServerLoadPB());
builder.addLiveServers(lsi.build());
}
}
if (status.getDeadServerNames() != null) {
for (ServerName deadServer : status.getDeadServerNames()) {
builder.addDeadServers(ProtobufUtil.toServerName(deadServer));
}
}
if (status.getRegionsInTransition() != null) {
for (RegionState rit : status.getRegionsInTransition()) {
ClusterStatusProtos.RegionState rs = rit.convert();
RegionSpecifier.Builder spec =
RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
spec.setValue(UnsafeByteOperations.unsafeWrap(rit.getRegion().getRegionName()));
RegionInTransition pbRIT =
RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();
builder.addRegionsInTransition(pbRIT);
}
}
if (status.getClusterId() != null) {
builder.setClusterId(new ClusterId(status.getClusterId()).convert());
}
if (status.getMasterCoprocessors() != null) {
for (String coprocessor : status.getMasterCoprocessors()) {
builder.addMasterCoprocessors(HBaseProtos.Coprocessor.newBuilder().setName(coprocessor));
}
}
if (status.getMaster() != null) {
builder.setMaster(ProtobufUtil.toServerName(status.getMaster()));
}
if (status.getBackupMasters() != null) {
for (ServerName backup : status.getBackupMasters()) {
builder.addBackupMasters(ProtobufUtil.toServerName(backup));
}
}
if (status.getBalancerOn() != null) {
builder.setBalancerOn(status.getBalancerOn());
}
builder.setMasterInfoPort(status.getMasterInfoPort());
return builder.build();
public static ClusterStatus toClusterStatus(ClusterStatusProtos.ClusterStatus proto) {
return new ClusterStatus(ClusterMetricsBuilder.toClusterMetrics(proto));
}
public static RegionLoadStats createRegionLoadStats(ClientProtos.RegionLoadStats stats) {
@ -3459,4 +3238,23 @@ public final class ProtobufUtil {
.setMaxCacheSize(cacheEvictionStats.getMaxCacheSize())
.build();
}
public static ClusterStatusProtos.ReplicationLoadSource toReplicationLoadSource(
ReplicationLoadSource rls) {
return ClusterStatusProtos.ReplicationLoadSource.newBuilder()
.setPeerID(rls.getPeerID())
.setAgeOfLastShippedOp(rls.getAgeOfLastShippedOp())
.setSizeOfLogQueue((int) rls.getSizeOfLogQueue())
.setTimeStampOfLastShippedOp(rls.getTimeStampOfLastShippedOp())
.setReplicationLag(rls.getReplicationLag())
.build();
}
public static ClusterStatusProtos.ReplicationLoadSink toReplicationLoadSink(
ReplicationLoadSink rls) {
return ClusterStatusProtos.ReplicationLoadSink.newBuilder()
.setAgeOfLastAppliedOp(rls.getAgeOfLastAppliedOp())
.setTimeStampsOfLastAppliedOp(rls.getTimeStampsOfLastAppliedOp())
.build();
}
}

View File

@ -27,7 +27,8 @@ import java.util.Set;
import java.util.regex.Pattern;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@ -1523,7 +1524,7 @@ public final class RequestConverter {
*/
public static GetClusterStatusRequest buildGetClusterStatusRequest(EnumSet<Option> options) {
return GetClusterStatusRequest.newBuilder()
.addAllOptions(ProtobufUtil.toOptions(options))
.addAllOptions(ClusterMetricsBuilder.toOptions(options))
.build();
}

View File

@ -22,12 +22,11 @@ import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.util.Bytes;

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.IntegrationTestBase;

View File

@ -49,7 +49,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.rest;
import java.io.IOException;
import java.util.EnumSet;
import javax.ws.rs.GET;
import javax.ws.rs.Produces;
import javax.ws.rs.core.CacheControl;
@ -29,16 +28,15 @@ import javax.ws.rs.core.Context;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.UriInfo;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
@InterfaceAudience.Private
public class StorageClusterStatusResource extends ResourceBase {

View File

@ -21,20 +21,18 @@ package org.apache.hadoop.hbase.rest;
import java.io.IOException;
import java.util.EnumSet;
import javax.ws.rs.GET;
import javax.ws.rs.Produces;
import javax.ws.rs.core.CacheControl;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriInfo;
import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.UriInfo;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel;
@InterfaceAudience.Private
public class StorageClusterVersionResource extends ResourceBase {

View File

@ -33,9 +33,8 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseCluster;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
@ -59,6 +58,7 @@ import org.junit.Test;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@ -525,7 +525,7 @@ public abstract class TestRSGroupsBase {
getTableRegionMap().get(tableName) != null &&
getTableRegionMap().get(tableName).size() == 6 &&
admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
.getRegionsInTransition().size() < 1;
.getRegionStatesInTransition().size() < 1;
}
});
@ -609,7 +609,7 @@ public abstract class TestRSGroupsBase {
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return cluster.getClusterStatus().getRegionsInTransition().isEmpty();
return cluster.getClusterStatus().getRegionStatesInTransition().isEmpty();
}
});
Set<Address> newServers = Sets.newHashSet();
@ -626,7 +626,7 @@ public abstract class TestRSGroupsBase {
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return cluster.getClusterStatus().getRegionsInTransition().isEmpty();
return cluster.getClusterStatus().getRegionStatesInTransition().isEmpty();
}
});
@ -830,7 +830,7 @@ public abstract class TestRSGroupsBase {
getTableRegionMap().get(tableName).size() == 5 &&
getTableServerRegionMap().get(tableName).size() == 1 &&
admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
.getRegionsInTransition().size() < 1;
.getRegionStatesInTransition().size() < 1;
}
});

View File

@ -22,7 +22,7 @@ HMaster master;
<%import>
java.util.*;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.ClusterStatus;
org.apache.hadoop.hbase.ClusterMetrics;
org.apache.hadoop.hbase.master.HMaster;
org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
</%import>
@ -55,7 +55,7 @@ MasterAddressTracker masterAddressTracker = master.getMasterAddressTracker();
</tr>
<%java>
Collection<ServerName> backup_masters = master.getClusterStatusWithoutCoprocessor(
EnumSet.of(ClusterStatus.Option.BACKUP_MASTERS)).getBackupMasters();
EnumSet.of(ClusterMetrics.Option.BACKUP_MASTERS)).getBackupMasters();
ServerName [] backupServerNames = backup_masters.toArray(new ServerName[backup_masters.size()]);
Arrays.sort(backupServerNames);
for (ServerName serverName : backupServerNames) {

View File

@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.master;
import java.io.Closeable;
import java.io.IOException;
import java.net.Inet6Address;
@ -35,8 +34,8 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
@ -66,7 +65,6 @@ import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.InternetProtocolFa
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioDatagramChannel;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.MessageToMessageEncoder;
import org.apache.hadoop.hbase.shaded.io.netty.util.internal.StringUtil;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
@ -161,12 +159,12 @@ public class ClusterStatusPublisher extends ScheduledChore {
// We're reusing an existing protobuf message, but we don't send everything.
// This could be extended in the future, for example if we want to send stuff like the
// hbase:meta server name.
ClusterStatus.Builder csBuilder = ClusterStatus.newBuilder();
csBuilder.setHBaseVersion(VersionInfo.getVersion())
.setClusterId(master.getMasterFileSystem().getClusterId().toString())
.setMaster(master.getServerName())
.setDeadServers(sns);
publisher.publish(csBuilder.build());
publisher.publish(new ClusterStatus(ClusterMetricsBuilder.newBuilder()
.setHBaseVersion(VersionInfo.getVersion())
.setClusterId(master.getMasterFileSystem().getClusterId().toString())
.setMasterName(master.getServerName())
.setDeadServerNames(sns)
.build()));
}
protected void cleanup() {
@ -340,7 +338,8 @@ public class ClusterStatusPublisher extends ScheduledChore {
@Override
protected void encode(ChannelHandlerContext channelHandlerContext,
ClusterStatus clusterStatus, List<Object> objects) {
ClusterStatusProtos.ClusterStatus csp = ProtobufUtil.convert(clusterStatus);
ClusterStatusProtos.ClusterStatus csp
= ClusterMetricsBuilder.toClusterStatus(clusterStatus);
objects.add(new DatagramPacket(Unpooled.wrappedBuffer(csp.toByteArray()), isa));
}
}

View File

@ -18,10 +18,8 @@
*/
package org.apache.hadoop.hbase.master;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Service;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Constructor;
@ -30,6 +28,7 @@ import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
@ -50,11 +49,16 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
@ -66,6 +70,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerMetricsBuilder;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
@ -180,8 +185,8 @@ import org.apache.hadoop.hbase.zookeeper.MasterMaintenanceModeTracker;
import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.eclipse.jetty.server.Server;
@ -190,6 +195,7 @@ import org.eclipse.jetty.servlet.ServletHolder;
import org.eclipse.jetty.webapp.WebAppContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@ -201,9 +207,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolat
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Service;
/**
* HMaster is the "master server" for HBase. An HBase cluster has one active
* master. If many masters are started, all compete. Whichever wins goes on to
@ -1021,7 +1024,7 @@ public class HMaster extends HRegionServer implements MasterServices {
for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
// The isServerOnline check is opportunistic, correctness is handled inside
if (!this.serverManager.isServerOnline(sn) &&
serverManager.checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
serverManager.checkAndRecordNewServer(sn, new ServerLoad(ServerMetricsBuilder.of(sn)))) {
LOG.info("Registered server found up in zk but who has not yet reported in: " + sn);
}
}
@ -2412,7 +2415,7 @@ public class HMaster extends HRegionServer implements MasterServices {
public ClusterStatus getClusterStatusWithoutCoprocessor(EnumSet<Option> options)
throws InterruptedIOException {
ClusterStatus.Builder builder = ClusterStatus.newBuilder();
ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
// given that hbase1 can't submit the request with Option,
// we return all information to client if the list of Option is empty.
if (options.isEmpty()) {
@ -2423,30 +2426,32 @@ public class HMaster extends HRegionServer implements MasterServices {
switch (opt) {
case HBASE_VERSION: builder.setHBaseVersion(VersionInfo.getVersion()); break;
case CLUSTER_ID: builder.setClusterId(getClusterId()); break;
case MASTER: builder.setMaster(getServerName()); break;
case BACKUP_MASTERS: builder.setBackupMasters(getBackupMasters()); break;
case MASTER: builder.setMasterName(getServerName()); break;
case BACKUP_MASTERS: builder.setBackerMasterNames(getBackupMasters()); break;
case LIVE_SERVERS: {
if (serverManager != null) {
builder.setLiveServers(serverManager.getOnlineServers());
builder.setLiveServerMetrics(serverManager.getOnlineServers().entrySet().stream()
.collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())));
}
break;
}
case DEAD_SERVERS: {
if (serverManager != null) {
builder.setDeadServers(new ArrayList<>(
builder.setDeadServerNames(new ArrayList<>(
serverManager.getDeadServers().copyServerNames()));
}
break;
}
case MASTER_COPROCESSORS: {
if (cpHost != null) {
builder.setMasterCoprocessors(getMasterCoprocessors());
builder.setMasterCoprocessorNames(Arrays.asList(getMasterCoprocessors()));
}
break;
}
case REGIONS_IN_TRANSITION: {
if (assignmentManager != null) {
builder.setRegionState(assignmentManager.getRegionStates().getRegionsStateInTransition());
builder.setRegionsInTransition(assignmentManager.getRegionStates()
.getRegionsStateInTransition());
}
break;
}
@ -2464,7 +2469,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
}
return builder.build();
return new ClusterStatus(builder.build());
}
/**
@ -2496,7 +2501,7 @@ public class HMaster extends HRegionServer implements MasterServices {
backupMasterStrings = null;
}
List<ServerName> backupMasters = null;
List<ServerName> backupMasters = Collections.emptyList();
if (backupMasterStrings != null && !backupMasterStrings.isEmpty()) {
backupMasters = new ArrayList<>(backupMasterStrings.size());
for (String s: backupMasterStrings) {

View File

@ -32,6 +32,7 @@ import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor;
@ -446,7 +447,7 @@ public class MasterRpcServices extends RSRpcServices
ClusterStatusProtos.ServerLoad sl = request.getLoad();
ServerName serverName = ProtobufUtil.toServerName(request.getServer());
ServerLoad oldLoad = master.getServerManager().getLoad(serverName);
ServerLoad newLoad = new ServerLoad(sl);
ServerLoad newLoad = new ServerLoad(serverName, sl);
master.getServerManager().regionServerReport(serverName, newLoad);
int version = VersionInfoUtil.getCurrentClientVersionNumber();
master.getAssignmentManager().reportOnlineRegions(serverName,
@ -902,8 +903,8 @@ public class MasterRpcServices extends RSRpcServices
GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
try {
master.checkInitialized();
response.setClusterStatus(ProtobufUtil.convert(
master.getClusterStatus(ProtobufUtil.toOptions(req.getOptionsList()))));
response.setClusterStatus(ClusterMetricsBuilder.toClusterStatus(
master.getClusterStatus(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
} catch (IOException e) {
throw new ServiceException(e);
}

View File

@ -31,7 +31,6 @@ import java.util.Random;
import java.util.Scanner;
import java.util.Set;
import java.util.TreeMap;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter;
@ -40,7 +39,7 @@ import org.apache.commons.cli.ParseException;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
@ -58,6 +57,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerMetricsBuilder;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@ -244,7 +245,7 @@ public class ServerManager {
request.getServerStartCode());
checkClockSkew(sn, request.getServerCurrentTime());
checkIsDead(sn, "STARTUP");
if (!checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
if (!checkAndRecordNewServer(sn, new ServerLoad(ServerMetricsBuilder.of(sn)))) {
LOG.warn("THIS SHOULD NOT HAPPEN, RegionServerStartup"
+ " could not record the server: " + sn);
}

View File

@ -1614,7 +1614,7 @@ public class HRegionServer extends HasThread implements
int storefileSizeMB = 0;
int memstoreSizeMB = (int) (r.getMemStoreSize() / 1024 / 1024);
long storefileIndexSizeKB = 0;
int rootIndexSizeKB = 0;
int rootLevelIndexSizeKB = 0;
int totalStaticIndexSizeKB = 0;
int totalStaticBloomSizeKB = 0;
long totalCompactingKVs = 0;
@ -1625,13 +1625,14 @@ public class HRegionServer extends HasThread implements
storefiles += store.getStorefilesCount();
storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() / 1024 / 1024);
storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
storefileIndexSizeKB += store.getStorefilesIndexSize() / 1024;
//TODO: storefileIndexSizeKB is same with rootLevelIndexSizeKB?
storefileIndexSizeKB += store.getStorefilesRootLevelIndexSize() / 1024;
CompactionProgress progress = store.getCompactionProgress();
if (progress != null) {
totalCompactingKVs += progress.totalCompactingKVs;
currentCompactedKVs += progress.currentCompactedKVs;
}
rootIndexSizeKB += (int) (store.getStorefilesIndexSize() / 1024);
rootLevelIndexSizeKB += (int) (store.getStorefilesRootLevelIndexSize() / 1024);
totalStaticIndexSizeKB += (int) (store.getTotalStaticIndexSize() / 1024);
totalStaticBloomSizeKB += (int) (store.getTotalStaticBloomSize() / 1024);
}
@ -1653,7 +1654,7 @@ public class HRegionServer extends HasThread implements
.setStorefileSizeMB(storefileSizeMB)
.setMemStoreSizeMB(memstoreSizeMB)
.setStorefileIndexSizeKB(storefileIndexSizeKB)
.setRootIndexSizeKB(rootIndexSizeKB)
.setRootIndexSizeKB(rootLevelIndexSizeKB)
.setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
.setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
.setReadRequestsCount(r.getReadRequestsCount())

View File

@ -2084,7 +2084,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
}
@Override
public long getStorefilesIndexSize() {
public long getStorefilesRootLevelIndexSize() {
return getStoreFileFieldSize(StoreFileReader::indexSize);
}

View File

@ -791,7 +791,7 @@ class MetricsRegionServerWrapperImpl
avgAgeNumerator += storeAvgStoreFileAge.getAsDouble() * storeHFiles;
}
tempStorefileIndexSize += store.getStorefilesIndexSize();
tempStorefileIndexSize += store.getStorefilesRootLevelIndexSize();
tempTotalStaticBloomSize += store.getTotalStaticBloomSize();
tempTotalStaticIndexSize += store.getTotalStaticIndexSize();
tempFlushedCellsCount += store.getFlushedCellsCount();

View File

@ -172,9 +172,9 @@ public interface Store {
long getHFilesSize();
/**
* @return The size of the store file indexes, in bytes.
* @return The size of the store file root-level indexes, in bytes.
*/
long getStorefilesIndexSize();
long getStorefilesRootLevelIndexSize();
/**
* Returns the total size of all index blocks in the data block indexes, including the root level,

View File

@ -47,13 +47,12 @@ import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.lang3.time.StopWatch;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.AuthUtil;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
@ -98,6 +97,7 @@ import org.apache.zookeeper.client.ConnectStringParser;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
/**

View File

@ -58,7 +58,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.lang3.StringUtils;
@ -73,8 +72,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
@ -569,7 +568,7 @@ public class HBaseFsck extends Configured implements Closeable {
errors.print("Number of requests: " + status.getRequestCount());
errors.print("Number of regions: " + status.getRegionsCount());
List<RegionState> rits = status.getRegionsInTransition();
List<RegionState> rits = status.getRegionStatesInTransition();
errors.print("Number of regions in transition: " + rits.size());
if (details) {
for (RegionState state: rits) {

View File

@ -23,10 +23,9 @@ import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@ -120,7 +119,7 @@ public class HBaseFsckRepair {
try {
boolean inTransition = false;
for (RegionState rs : admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
.getRegionsInTransition()) {
.getRegionStatesInTransition()) {
if (RegionInfo.COMPARATOR.compare(rs.getRegion(), region) == 0) {
inTransition = true;
break;

View File

@ -41,10 +41,9 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.cli.CommandLine;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.util;
import java.io.IOException;
import java.math.BigInteger;
import java.util.Arrays;
import java.util.Collection;
import java.util.EnumSet;
@ -42,10 +41,8 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -53,6 +50,8 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -434,7 +433,7 @@ public class RegionSplitter {
* Alternative getCurrentNrHRS which is no longer available.
* @param connection
* @return Rough count of regionservers out on cluster.
* @throws IOException
* @throws IOException if a remote or network exception occurs
*/
private static int getRegionServerCount(final Connection connection) throws IOException {
try (Admin admin = connection.getAdmin()) {
@ -785,7 +784,7 @@ public class RegionSplitter {
* @param conf
* @param tableName
* @return A Pair where first item is table dir and second is the split file.
* @throws IOException
* @throws IOException if a remote or network exception occurs
*/
private static Pair<Path, Path> getTableDirAndSplitFile(final Configuration conf,
final TableName tableName)

View File

@ -60,7 +60,7 @@ import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Admin;

View File

@ -0,0 +1,233 @@
/**
* 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.io.IOException;
import java.util.EnumSet;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.AsyncAdmin;
import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestClientClusterMetrics {
private static HBaseTestingUtility UTIL;
private static Admin ADMIN;
private final static int SLAVES = 5;
private final static int MASTERS = 3;
private static MiniHBaseCluster CLUSTER;
private static HRegionServer DEAD;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
Configuration conf = HBaseConfiguration.create();
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, MyObserver.class.getName());
UTIL = new HBaseTestingUtility(conf);
UTIL.startMiniCluster(MASTERS, SLAVES);
CLUSTER = UTIL.getHBaseCluster();
CLUSTER.waitForActiveAndReadyMaster();
ADMIN = UTIL.getAdmin();
// Kill one region server
List<RegionServerThread> rsts = CLUSTER.getLiveRegionServerThreads();
RegionServerThread rst = rsts.get(rsts.size() - 1);
DEAD = rst.getRegionServer();
DEAD.stop("Test dead servers metrics");
while (rst.isAlive()) {
Thread.sleep(500);
}
}
@Test
public void testDefaults() throws Exception {
ClusterMetrics origin = ADMIN.getClusterStatus();
ClusterMetrics defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
Assert.assertEquals(origin.getAverageLoad(), defaults.getAverageLoad(), 0);
Assert.assertEquals(origin.getBackupMasterNames().size(),
defaults.getBackupMasterNames().size());
Assert.assertEquals(origin.getDeadServerNames().size(), defaults.getDeadServerNames().size());
Assert.assertEquals(origin.getRegionCount(), defaults.getRegionCount());
Assert.assertEquals(origin.getLiveServerMetrics().size(),
defaults.getLiveServerMetrics().size());
Assert.assertEquals(origin.getMasterInfoPort(), defaults.getMasterInfoPort());
}
@Test
public void testAsyncClient() throws Exception {
try (AsyncConnection asyncConnect = ConnectionFactory.createAsyncConnection(
UTIL.getConfiguration()).get()) {
AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
CompletableFuture<ClusterStatus> originFuture =
asyncAdmin.getClusterStatus();
CompletableFuture<ClusterStatus> defaultsFuture =
asyncAdmin.getClusterStatus(EnumSet.allOf(Option.class));
ClusterMetrics origin = originFuture.get();
ClusterMetrics defaults = defaultsFuture.get();
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
Assert.assertEquals(origin.getAverageLoad(), defaults.getAverageLoad(), 0);
Assert.assertEquals(origin.getBackupMasterNames().size(),
defaults.getBackupMasterNames().size());
Assert.assertEquals(origin.getDeadServerNames().size(), defaults.getDeadServerNames().size());
Assert.assertEquals(origin.getRegionCount(), defaults.getRegionCount());
Assert.assertEquals(origin.getLiveServerMetrics().size(),
defaults.getLiveServerMetrics().size());
Assert.assertEquals(origin.getMasterInfoPort(), defaults.getMasterInfoPort());
}
}
@Test
public void testLiveAndDeadServersStatus() throws Exception {
// Count the number of live regionservers
List<RegionServerThread> regionserverThreads = CLUSTER.getLiveRegionServerThreads();
int numRs = 0;
int len = regionserverThreads.size();
for (int i = 0; i < len; i++) {
if (regionserverThreads.get(i).isAlive()) {
numRs++;
}
}
// Depending on the (random) order of unit execution we may run this unit before the
// minicluster is fully up and recovered from the RS shutdown done during test init.
Waiter.waitFor(CLUSTER.getConfiguration(), 10 * 1000, 100, new Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
ClusterMetrics metrics = ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
Assert.assertNotNull(metrics);
return metrics.getRegionCount() > 0;
}
});
// Retrieve live servers and dead servers info.
EnumSet<Option> options = EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS);
ClusterMetrics metrics = ADMIN.getClusterStatus(options);
Assert.assertNotNull(metrics);
// exclude a dead region server
Assert.assertEquals(SLAVES -1, numRs);
// live servers = nums of regionservers
// By default, HMaster don't carry any regions so it won't report its load.
// Hence, it won't be in the server list.
Assert.assertEquals(numRs, metrics.getLiveServerMetrics().size());
Assert.assertTrue(metrics.getRegionCount() > 0);
Assert.assertNotNull(metrics.getDeadServerNames());
Assert.assertEquals(1, metrics.getDeadServerNames().size());
ServerName deadServerName = metrics.getDeadServerNames().iterator().next();
Assert.assertEquals(DEAD.getServerName(), deadServerName);
}
@Test
public void testMasterAndBackupMastersStatus() throws Exception {
// get all the master threads
List<MasterThread> masterThreads = CLUSTER.getMasterThreads();
int numActive = 0;
int activeIndex = 0;
ServerName activeName = null;
HMaster active = null;
for (int i = 0; i < masterThreads.size(); i++) {
if (masterThreads.get(i).getMaster().isActiveMaster()) {
numActive++;
activeIndex = i;
active = masterThreads.get(activeIndex).getMaster();
activeName = active.getServerName();
}
}
Assert.assertNotNull(active);
Assert.assertEquals(1, numActive);
Assert.assertEquals(MASTERS, masterThreads.size());
// Retrieve master and backup masters infos only.
EnumSet<Option> options = EnumSet.of(Option.MASTER, Option.BACKUP_MASTERS);
ClusterMetrics metrics = ADMIN.getClusterStatus(options);
Assert.assertTrue(metrics.getMasterName().equals(activeName));
Assert.assertEquals(MASTERS - 1, metrics.getBackupMasterNames().size());
}
@Test
public void testOtherStatusInfos() throws Exception {
EnumSet<Option> options =
EnumSet.of(Option.MASTER_COPROCESSORS, Option.HBASE_VERSION,
Option.CLUSTER_ID, Option.BALANCER_ON);
ClusterMetrics metrics = ADMIN.getClusterStatus(options);
Assert.assertEquals(1, metrics.getMasterCoprocessorNames().size());
Assert.assertNotNull(metrics.getHBaseVersion());
Assert.assertNotNull(metrics.getClusterId());
Assert.assertTrue(metrics.getAverageLoad() == 0.0);
Assert.assertNotNull(metrics.getBalancerOn());
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
if (ADMIN != null) {
ADMIN.close();
}
UTIL.shutdownMiniCluster();
}
@Test
public void testObserver() throws IOException {
int preCount = MyObserver.PRE_COUNT.get();
int postCount = MyObserver.POST_COUNT.get();
Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
.anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
}
public static class MyObserver implements MasterCoprocessor, MasterObserver {
private static final AtomicInteger PRE_COUNT = new AtomicInteger(0);
private static final AtomicInteger POST_COUNT = new AtomicInteger(0);
@Override public Optional<MasterObserver> getMasterObserver() {
return Optional.of(this);
}
@Override public void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
PRE_COUNT.incrementAndGet();
}
@Override public void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
ClusterStatus metrics) throws IOException {
POST_COUNT.incrementAndGet();
}
}
}

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.EnumSet;
@ -25,14 +25,12 @@ import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.AsyncAdmin;
import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@ -40,7 +38,6 @@ import org.apache.hadoop.hbase.coprocessor.MasterObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@ -85,6 +82,8 @@ public class TestClientClusterStatus {
public void testDefaults() throws Exception {
ClusterStatus origin = ADMIN.getClusterStatus();
ClusterStatus defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
checkPbObjectNotNull(origin);
checkPbObjectNotNull(defaults);
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
@ -101,29 +100,30 @@ public class TestClientClusterStatus {
ClusterStatus status0 = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
ClusterStatus status1 = ADMIN.getClusterStatus(EnumSet.noneOf(Option.class));
Assert.assertEquals(status0, status1);
checkPbObjectNotNull(status0);
checkPbObjectNotNull(status1);
}
@Test
public void testAsyncClient() throws Exception {
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(UTIL.getConfiguration());
AsyncConnectionImpl asyncConnect = new AsyncConnectionImpl(UTIL.getConfiguration(), registry,
registry.getClusterId().get(), User.getCurrent());
AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
CompletableFuture<ClusterStatus> originFuture =
try (AsyncConnection asyncConnect = ConnectionFactory.createAsyncConnection(
UTIL.getConfiguration()).get()) {
AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
CompletableFuture<ClusterStatus> originFuture =
asyncAdmin.getClusterStatus();
CompletableFuture<ClusterStatus> defaultsFuture =
CompletableFuture<ClusterStatus> defaultsFuture =
asyncAdmin.getClusterStatus(EnumSet.allOf(Option.class));
ClusterStatus origin = originFuture.get();
ClusterStatus defaults = defaultsFuture.get();
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
if (asyncConnect != null) {
asyncConnect.close();
ClusterStatus origin = originFuture.get();
ClusterStatus defaults = defaultsFuture.get();
checkPbObjectNotNull(origin);
checkPbObjectNotNull(defaults);
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
}
}
@ -151,6 +151,7 @@ public class TestClientClusterStatus {
// Retrieve live servers and dead servers info.
EnumSet<Option> options = EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS);
ClusterStatus status = ADMIN.getClusterStatus(options);
checkPbObjectNotNull(status);
Assert.assertNotNull(status);
Assert.assertNotNull(status.getServers());
// exclude a dead region server
@ -221,6 +222,21 @@ public class TestClientClusterStatus {
Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
}
/**
* HBASE-19496 do the refactor for ServerLoad and RegionLoad so the inner pb object is useless
* now. However, they are Public classes, and consequently we must make sure the all pb objects
* have initialized.
*/
private static void checkPbObjectNotNull(ClusterStatus status) {
for (ServerName name : status.getLiveServerMetrics().keySet()) {
ServerLoad load = status.getLoad(name);
Assert.assertNotNull(load.obtainServerLoadPB());
for (RegionLoad rl : load.getRegionsLoad().values()) {
Assert.assertNotNull(rl.regionLoadPB);
}
}
}
public static class MyObserver implements MasterCoprocessor, MasterObserver {
private static final AtomicInteger PRE_COUNT = new AtomicInteger(0);
private static final AtomicInteger POST_COUNT = new AtomicInteger(0);

View File

@ -19,11 +19,16 @@
*/
package org.apache.hadoop.hbase;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import java.io.IOException;
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -34,11 +39,8 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@Category({MiscTests.class, MediumTests.class})
public class TestRegionLoad {
@ -124,6 +126,10 @@ public class TestRegionLoad {
private void checkRegionsAndRegionLoads(Collection<HRegionInfo> regions,
Collection<RegionLoad> regionLoads) {
for (RegionLoad load : regionLoads) {
assertNotNull(load.regionLoadPB);
}
assertEquals("No of regions and regionloads doesn't match",
regions.size(), regionLoads.size());

View File

@ -0,0 +1,130 @@
/**
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@Category({ MiscTests.class, MediumTests.class })
public class TestRegionMetrics {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static Admin admin;
private static final TableName TABLE_1 = TableName.valueOf("table_1");
private static final TableName TABLE_2 = TableName.valueOf("table_2");
private static final TableName TABLE_3 = TableName.valueOf("table_3");
private static final TableName[] tables = new TableName[] { TABLE_1, TABLE_2, TABLE_3 };
@BeforeClass
public static void beforeClass() throws Exception {
UTIL.startMiniCluster(4);
admin = UTIL.getAdmin();
admin.balancerSwitch(false, true);
byte[] FAMILY = Bytes.toBytes("f");
for (TableName tableName : tables) {
Table table = UTIL.createMultiRegionTable(tableName, FAMILY, 16);
UTIL.waitTableAvailable(tableName);
UTIL.loadTable(table, FAMILY);
}
}
@AfterClass
public static void afterClass() throws Exception {
for (TableName table : tables) {
UTIL.deleteTableIfAny(table);
}
UTIL.shutdownMiniCluster();
}
@Test
public void testRegionMetrics() throws Exception {
// Check if regions match with the RegionMetrics from the server
for (ServerName serverName : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
List<RegionInfo> regions = admin.getRegions(serverName);
Collection<RegionMetrics> regionMetricsList =
admin.getRegionLoads(serverName).stream().collect(Collectors.toList());
checkRegionsAndRegionMetrics(regions, regionMetricsList);
}
// Check if regionMetrics matches the table's regions and nothing is missed
for (TableName table : new TableName[] { TABLE_1, TABLE_2, TABLE_3 }) {
List<RegionInfo> tableRegions = admin.getRegions(table);
List<RegionMetrics> regionMetrics = new ArrayList<>();
for (ServerName serverName : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
regionMetrics.addAll(admin.getRegionLoads(serverName, table));
}
checkRegionsAndRegionMetrics(tableRegions, regionMetrics);
}
// Check RegionMetrics matches the RegionMetrics from ClusterStatus
ClusterMetrics clusterStatus = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
for (Map.Entry<ServerName, ServerMetrics> entry : clusterStatus.getLiveServerMetrics()
.entrySet()) {
ServerName serverName = entry.getKey();
ServerMetrics serverMetrics = entry.getValue();
List<RegionMetrics> regionMetrics =
admin.getRegionLoads(serverName).stream().collect(Collectors.toList());
assertEquals(serverMetrics.getRegionMetrics().size(), regionMetrics.size());
}
}
private void checkRegionsAndRegionMetrics(Collection<RegionInfo> regions,
Collection<RegionMetrics> regionMetrics) {
assertEquals("No of regions and regionMetrics doesn't match", regions.size(),
regionMetrics.size());
Map<byte[], RegionMetrics> regionMetricsMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
for (RegionMetrics r : regionMetrics) {
regionMetricsMap.put(r.getRegionName(), r);
}
for (RegionInfo info : regions) {
assertTrue("Region not in RegionMetricsMap region:"
+ info.getRegionNameAsString() + " regionMap: "
+ regionMetricsMap, regionMetricsMap.containsKey(info.getRegionName()));
}
}
}

View File

@ -20,42 +20,44 @@
package org.apache.hadoop.hbase;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@Category({MiscTests.class, SmallTests.class})
@Category({ MiscTests.class, SmallTests.class })
public class TestServerLoad {
@Test
public void testRegionLoadAggregation() {
ServerLoad sl = new ServerLoad(createServerLoadProto());
ServerLoad sl = new ServerLoad(ServerName.valueOf("localhost,1,1"), createServerLoadProto());
assertEquals(13, sl.getStores());
assertEquals(114, sl.getStorefiles());
assertEquals(129, sl.getStoreUncompressedSizeMB());
assertEquals(504, sl.getRootIndexSizeKB());
assertEquals(820, sl.getStorefileSizeMB());
assertEquals(82, sl.getStorefileIndexSizeKB());
assertEquals(((long)Integer.MAX_VALUE)*2, sl.getReadRequestsCount());
assertEquals(((long) Integer.MAX_VALUE) * 2, sl.getReadRequestsCount());
assertEquals(300, sl.getFilteredReadRequestsCount());
}
@Test
public void testToString() {
ServerLoad sl = new ServerLoad(createServerLoadProto());
ServerLoad sl = new ServerLoad(ServerName.valueOf("localhost,1,1"), createServerLoadProto());
String slToString = sl.toString();
assertNotNull(sl.obtainServerLoadPB());
assertTrue(slToString.contains("numberOfStores=13"));
assertTrue(slToString.contains("numberOfStorefiles=114"));
assertTrue(slToString.contains("storefileUncompressedSizeMB=129"));
assertTrue(slToString.contains("storefileSizeMB=820"));
assertTrue(slToString.contains("storefileSizeMB=820"));
assertTrue(slToString.contains("rootIndexSizeKB=504"));
assertTrue(slToString.contains("coprocessors=[]"));
assertTrue(slToString.contains("filteredReadRequestsCount=300"));
@ -63,36 +65,35 @@ public class TestServerLoad {
@Test
public void testRegionLoadWrapAroundAggregation() {
ServerLoad sl = new ServerLoad(createServerLoadProto());
long totalCount = ((long)Integer.MAX_VALUE)*2;
assertEquals(totalCount, sl.getReadRequestsCount());
assertEquals(totalCount, sl.getWriteRequestsCount());
ServerLoad sl = new ServerLoad(ServerName.valueOf("localhost,1,1"), createServerLoadProto());
assertNotNull(sl.obtainServerLoadPB());
long totalCount = ((long) Integer.MAX_VALUE) * 2;
assertEquals(totalCount, sl.getReadRequestsCount());
assertEquals(totalCount, sl.getWriteRequestsCount());
}
private ClusterStatusProtos.ServerLoad createServerLoadProto() {
HBaseProtos.RegionSpecifier rSpecOne =
HBaseProtos.RegionSpecifier.newBuilder()
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
.setValue(ByteString.copyFromUtf8("ASDFGQWERT")).build();
HBaseProtos.RegionSpecifier rSpecTwo =
HBaseProtos.RegionSpecifier.newBuilder()
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
.setValue(ByteString.copyFromUtf8("QWERTYUIOP")).build();
HBaseProtos.RegionSpecifier rSpecOne = HBaseProtos.RegionSpecifier.newBuilder()
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
.setValue(ByteString.copyFromUtf8("ASDFGQWERT")).build();
HBaseProtos.RegionSpecifier rSpecTwo = HBaseProtos.RegionSpecifier.newBuilder()
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
.setValue(ByteString.copyFromUtf8("QWERTYUIOP")).build();
ClusterStatusProtos.RegionLoad rlOne =
ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10)
.setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520)
.setFilteredReadRequestsCount(100)
.setStorefileIndexSizeKB(42).setRootIndexSizeKB(201).setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10)
.setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520)
.setFilteredReadRequestsCount(100).setStorefileIndexSizeKB(42).setRootIndexSizeKB(201)
.setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
ClusterStatusProtos.RegionLoad rlTwo =
ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecTwo).setStores(3)
.setStorefiles(13).setStoreUncompressedSizeMB(23).setStorefileSizeMB(300)
.setFilteredReadRequestsCount(200)
.setStorefileIndexSizeKB(40).setRootIndexSizeKB(303).setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecTwo).setStores(3)
.setStorefiles(13).setStoreUncompressedSizeMB(23).setStorefileSizeMB(300)
.setFilteredReadRequestsCount(200).setStorefileIndexSizeKB(40).setRootIndexSizeKB(303)
.setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
ClusterStatusProtos.ServerLoad sl =
ClusterStatusProtos.ServerLoad.newBuilder().addRegionLoads(rlOne).
addRegionLoads(rlTwo).build();
ClusterStatusProtos.ServerLoad.newBuilder().addRegionLoads(rlOne).
addRegionLoads(rlTwo).build();
return sl;
}

View File

@ -0,0 +1,114 @@
/**
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@Category({ MiscTests.class, SmallTests.class })
public class TestServerMetrics {
@Test
public void testRegionLoadAggregation() {
ServerMetrics metrics = ServerMetricsBuilder.toServerMetrics(
ServerName.valueOf("localhost,1,1"), createServerLoadProto());
assertEquals(13,
metrics.getRegionMetrics().values().stream().mapToInt(v -> v.getStoreCount()).sum());
assertEquals(114,
metrics.getRegionMetrics().values().stream().mapToInt(v -> v.getStoreFileCount()).sum());
assertEquals(129, metrics.getRegionMetrics().values().stream()
.mapToDouble(v -> v.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE)).sum(), 0);
assertEquals(504, metrics.getRegionMetrics().values().stream()
.mapToDouble(v -> v.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE)).sum(), 0);
assertEquals(820, metrics.getRegionMetrics().values().stream()
.mapToDouble(v -> v.getStoreFileSize().get(Size.Unit.MEGABYTE)).sum(), 0);
assertEquals(82, metrics.getRegionMetrics().values().stream()
.mapToDouble(v -> v.getStoreFileIndexSize().get(Size.Unit.KILOBYTE)).sum(), 0);
assertEquals(((long) Integer.MAX_VALUE) * 2,
metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getReadRequestCount()).sum());
assertEquals(300,
metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getFilteredReadRequestCount())
.sum());
}
@Test
public void testToString() {
ServerMetrics metrics = ServerMetricsBuilder.toServerMetrics(
ServerName.valueOf("localhost,1,1"), createServerLoadProto());
String slToString = metrics.toString();
assertTrue(slToString.contains("numberOfStores=13"));
assertTrue(slToString.contains("numberOfStorefiles=114"));
assertTrue(slToString.contains("storefileUncompressedSizeMB=129"));
assertTrue(slToString.contains("storefileSizeMB=820"));
assertTrue(slToString.contains("rootIndexSizeKB=504"));
assertTrue(slToString.contains("coprocessors=[]"));
assertTrue(slToString.contains("filteredReadRequestsCount=300"));
}
@Test
public void testRegionLoadWrapAroundAggregation() {
ServerMetrics metrics = ServerMetricsBuilder.toServerMetrics(
ServerName.valueOf("localhost,1,1"), createServerLoadProto());
long totalCount = ((long) Integer.MAX_VALUE) * 2;
assertEquals(totalCount,
metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getReadRequestCount()).sum());
assertEquals(totalCount,
metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getWriteRequestCount())
.sum());
}
private ClusterStatusProtos.ServerLoad createServerLoadProto() {
HBaseProtos.RegionSpecifier rSpecOne = HBaseProtos.RegionSpecifier.newBuilder()
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
.setValue(ByteString.copyFromUtf8("ASDFGQWERT")).build();
HBaseProtos.RegionSpecifier rSpecTwo = HBaseProtos.RegionSpecifier.newBuilder()
.setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
.setValue(ByteString.copyFromUtf8("QWERTYUIOP")).build();
ClusterStatusProtos.RegionLoad rlOne =
ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10)
.setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520)
.setFilteredReadRequestsCount(100).setStorefileIndexSizeKB(42).setRootIndexSizeKB(201)
.setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE)
.build();
ClusterStatusProtos.RegionLoad rlTwo =
ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecTwo).setStores(3)
.setStorefiles(13).setStoreUncompressedSizeMB(23).setStorefileSizeMB(300)
.setFilteredReadRequestsCount(200).setStorefileIndexSizeKB(40).setRootIndexSizeKB(303)
.setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE)
.build();
ClusterStatusProtos.ServerLoad sl =
ClusterStatusProtos.ServerLoad.newBuilder().addRegionLoads(rlOne).
addRegionLoads(rlTwo).build();
return sl;
}
}

View File

@ -0,0 +1,84 @@
/**
* 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 static org.junit.Assert.assertEquals;
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({MiscTests.class, SmallTests.class})
public class TestSize {
@Test
public void testConversion() {
Size kbSize = new Size(1024D, Size.Unit.MEGABYTE);
assertEquals(1D, kbSize.get(Size.Unit.GIGABYTE), 0);
assertEquals(1024D, kbSize.get(), 0);
assertEquals(1024D * 1024D, kbSize.get(Size.Unit.KILOBYTE), 0);
assertEquals(1024D * 1024D * 1024D, kbSize.get(Size.Unit.BYTE), 0);
}
@Test
public void testCompare() {
Size size00 = new Size(100D, Size.Unit.GIGABYTE);
Size size01 = new Size(100D, Size.Unit.MEGABYTE);
Size size02 = new Size(100D, Size.Unit.BYTE);
Set<Size> sizes = new TreeSet<>();
sizes.add(size00);
sizes.add(size01);
sizes.add(size02);
int count = 0;
for (Size s : sizes) {
switch (count++) {
case 0:
assertEquals(size02, s);
break;
case 1:
assertEquals(size01, s);
break;
default:
assertEquals(size00, s);
break;
}
}
assertEquals(3, count);
}
@Test
public void testEqual() {
assertEquals(new Size(1024D, Size.Unit.TERABYTE),
new Size(1D, Size.Unit.PETABYTE));
assertEquals(new Size(1024D, Size.Unit.GIGABYTE),
new Size(1D, Size.Unit.TERABYTE));
assertEquals(new Size(1024D, Size.Unit.MEGABYTE),
new Size(1D, Size.Unit.GIGABYTE));
assertEquals(new Size(1024D, Size.Unit.KILOBYTE),
new Size(1D, Size.Unit.MEGABYTE));
assertEquals(new Size(1024D, Size.Unit.BYTE),
new Size(1D, Size.Unit.KILOBYTE));
}
}

View File

@ -25,20 +25,14 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Random;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@ -58,7 +52,6 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -68,13 +61,13 @@ import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;

View File

@ -21,6 +21,8 @@ import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.io.IOException;
import java.nio.file.FileSystems;
import java.nio.file.Files;
@ -31,10 +33,9 @@ import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.RegionLoad;
@ -53,9 +54,6 @@ import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
@RunWith(Parameterized.class)
@Category({ ClientTests.class, MediumTests.class })
public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {

View File

@ -25,9 +25,8 @@ import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.Test;

View File

@ -50,7 +50,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;

View File

@ -33,7 +33,7 @@ import java.util.concurrent.ExecutorService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;

View File

@ -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.client;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.io.ByteBufferPool;
import org.apache.hadoop.hbase.ipc.NettyRpcServer;
import org.apache.hadoop.hbase.ipc.RpcServerFactory;
import org.apache.hadoop.hbase.ipc.SimpleRpcServer;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
/**
* HBASE-19496 noticed that the RegionLoad/ServerLoad may be corrupted if rpc server
* reuses the bytebuffer backed, so this test call the Admin#getLastMajorCompactionTimestamp() to
* invoke HMaster to iterate all stored server/region loads.
*/
@RunWith(Parameterized.class)
@Category({ MediumTests.class, ClientTests.class })
public class TestServerLoadDurability {
private static final byte[] FAMILY = Bytes.toBytes("testFamily");
@Parameterized.Parameter
public Configuration conf;
@Parameterized.Parameters
public static final Collection<Object[]> parameters() {
List<Object[]> configurations = new ArrayList<>();
configurations.add(new Object[] { createConfigurationForSimpleRpcServer() });
configurations.add(new Object[] { createConfigurationForNettyRpcServer() });
return configurations;
}
private static Configuration createConfigurationForSimpleRpcServer() {
Configuration conf = HBaseConfiguration.create();
conf.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY,
SimpleRpcServer.class.getName());
conf.setInt(ByteBufferPool.BUFFER_SIZE_KEY, 20);
return conf;
}
private static Configuration createConfigurationForNettyRpcServer() {
Configuration conf = HBaseConfiguration.create();
conf.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY,
NettyRpcServer.class.getName());
return conf;
}
protected HBaseTestingUtility utility;
protected Connection conn;
protected Admin admin;
@Rule
public TestName testName = new TestName();
protected TableName tableName;
@Before
public void setUp() throws Exception {
utility = new HBaseTestingUtility(conf);
utility.startMiniCluster(2);
conn = ConnectionFactory.createConnection(utility.getConfiguration());
admin = conn.getAdmin();
String methodName = testName.getMethodName();
tableName = TableName.valueOf(methodName.substring(0, methodName.length() - 3));
}
@After
public void tearDown() throws Exception {
utility.shutdownMiniCluster();
}
@Test
public void testCompactionTimestamps() throws Exception {
createTableWithDefaultConf(tableName);
try (Table table = conn.getTable(tableName)) {
long ts = admin.getLastMajorCompactionTimestamp(tableName);
}
}
private void createTableWithDefaultConf(TableName tableName) throws IOException {
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
admin.createTable(builder.build());
}
}

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerMetricsBuilder;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
@ -341,8 +342,8 @@ public class TestAssignmentListener {
// is started (just as we see when we failover to the Backup HMaster).
// One of these will already be a draining server.
HashMap<ServerName, ServerLoad> onlineServers = new HashMap<>();
onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
onlineServers.put(SERVERNAME_C, ServerLoad.EMPTY_SERVERLOAD);
onlineServers.put(SERVERNAME_A, new ServerLoad(ServerMetricsBuilder.of(SERVERNAME_A)));
onlineServers.put(SERVERNAME_C, new ServerLoad(ServerMetricsBuilder.of(SERVERNAME_C)));
// Create draining znodes for the draining servers, which would have been
// performed when the previous HMaster was running.

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaMockingUtil;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerMetricsBuilder;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@ -230,7 +231,7 @@ public class TestMasterNoCluster {
RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();;
ServerName sn = ServerName.parseVersionedServerName(sns[i].getVersionedBytes());
request.setServer(ProtobufUtil.toServerName(sn));
request.setLoad(ServerLoad.EMPTY_SERVERLOAD.obtainServerLoadPB());
request.setLoad(ServerMetricsBuilder.toServerLoad(ServerMetricsBuilder.of(sn)));
master.getMasterRpcServices().regionServerReport(null, request.build());
}
// Master should now come up.
@ -272,7 +273,8 @@ public class TestMasterNoCluster {
KeeperException, CoordinatedStateException {
super.initializeZKBasedSystemTrackers();
// Record a newer server in server manager at first
getServerManager().recordNewServerWithLock(newServer, ServerLoad.EMPTY_SERVERLOAD);
getServerManager().recordNewServerWithLock(newServer,
new ServerLoad(ServerMetricsBuilder.of(newServer)));
List<ServerName> onlineServers = new ArrayList<>();
onlineServers.add(deadServer);

View File

@ -31,9 +31,8 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerMetricsBuilder;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
@ -168,8 +169,8 @@ public class MockMasterServices extends MockNoopMasterServices {
startProcedureExecutor(remoteDispatcher);
this.assignmentManager.start();
for (int i = 0; i < numServes; ++i) {
serverManager.regionServerReport(
ServerName.valueOf("localhost", 100 + i, 1), ServerLoad.EMPTY_SERVERLOAD);
ServerName sn = ServerName.valueOf("localhost", 100 + i, 1);
serverManager.regionServerReport(sn, new ServerLoad(ServerMetricsBuilder.of(sn)));
}
this.procedureExecutor.getEnvironment().setEventReady(initialized, true);
}
@ -195,7 +196,7 @@ public class MockMasterServices extends MockNoopMasterServices {
return;
}
ServerName sn = ServerName.valueOf(serverName.getAddress().toString(), startCode);
serverManager.regionServerReport(sn, ServerLoad.EMPTY_SERVERLOAD);
serverManager.regionServerReport(sn, new ServerLoad(ServerMetricsBuilder.of(sn)));
}
@Override

View File

@ -27,10 +27,9 @@ import java.util.Arrays;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@ -55,15 +54,15 @@ import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;

View File

@ -30,9 +30,8 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@ -63,6 +62,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;

View File

@ -18,8 +18,19 @@
*/
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
@ -57,18 +68,6 @@ import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
@Category(MediumTests.class)
public class TestRegionServerReadRequestMetrics {
private static final Logger LOG =

View File

@ -30,13 +30,12 @@ import java.util.List;
import java.util.Optional;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
@ -189,7 +188,7 @@ public class TestMasterReplication {
@Override
public boolean evaluate() throws Exception {
ClusterStatus clusterStatus = utilities[0].getAdmin()
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
.getClusterStatus(EnumSet.of(ClusterMetrics.Option.LIVE_SERVERS));
ServerLoad serverLoad = clusterStatus.getLoad(rsName);
List<ReplicationLoadSource> replicationLoadSourceList =
serverLoad.getReplicationLoadSourceList();

View File

@ -22,11 +22,10 @@ import static org.junit.Assert.assertTrue;
import java.util.EnumSet;
import java.util.List;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.testclassification.MediumTests;

View File

@ -33,13 +33,12 @@ import java.util.UUID;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@ -77,6 +76,7 @@ import org.apache.zookeeper.KeeperException;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;

View File

@ -23,15 +23,14 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;