HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all public classes

This commit is contained in:
Chia-Ping Tsai 2018-01-04 12:40:09 +08:00
parent 72631a08c1
commit 654edc5fa5
83 changed files with 907 additions and 938 deletions

View File

@ -29,12 +29,20 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
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 List<RegionMetrics> toRegionMetrics(
AdminProtos.GetRegionLoadResponse regionLoadResponse) {
return regionLoadResponse.getRegionLoadsList().stream()
.map(RegionMetricsBuilder::toRegionMetrics).collect(Collectors.toList());
}
public static RegionMetrics toRegionMetrics(ClusterStatusProtos.RegionLoad regionLoadPB) {
return RegionMetricsBuilder
.newBuilder(regionLoadPB.getRegionSpecifier().getValue().toByteArray())

View File

@ -25,19 +25,19 @@ import java.util.EnumSet;
import java.util.List;
import java.util.Map;
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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
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;
@ -1452,22 +1451,45 @@ public interface Admin extends Abortable, Closeable {
* </pre>
* @return cluster status
* @throws IOException if a remote or network exception occurs
* @deprecated since 2.0 version and will be removed in 3.0 version.
* use {@link #getClusterMetrics()}
*/
ClusterStatus getClusterStatus() throws IOException;
@Deprecated
default ClusterStatus getClusterStatus() throws IOException {
return new ClusterStatus(getClusterMetrics());
}
/**
* Get whole cluster metrics, containing status about:
* <pre>
* hbase version
* cluster id
* primary/backup master(s)
* master's coprocessors
* live/dead regionservers
* balancer
* regions in transition
* </pre>
* @return cluster metrics
* @throws IOException if a remote or network exception occurs
*/
default ClusterMetrics getClusterMetrics() throws IOException {
return getClusterMetrics(EnumSet.allOf(ClusterMetrics.Option.class));
}
/**
* Get cluster status with a set of {@link Option} to get desired status.
* @return cluster status
* @throws IOException if a remote or network exception occurs
*/
ClusterStatus getClusterStatus(EnumSet<Option> options) throws IOException;
ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException;
/**
* @return current master server name
* @throws IOException if a remote or network exception occurs
*/
default ServerName getMaster() throws IOException {
return getClusterStatus(EnumSet.of(Option.MASTER)).getMaster();
return getClusterMetrics(EnumSet.of(Option.MASTER)).getMasterName();
}
/**
@ -1475,7 +1497,7 @@ public interface Admin extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs
*/
default Collection<ServerName> getBackupMasters() throws IOException {
return getClusterStatus(EnumSet.of(Option.BACKUP_MASTERS)).getBackupMasters();
return getClusterMetrics(EnumSet.of(Option.BACKUP_MASTERS)).getBackupMasterNames();
}
/**
@ -1483,64 +1505,30 @@ public interface Admin extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs
*/
default Collection<ServerName> getRegionServers() throws IOException {
return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet();
}
/**
* Get {@link RegionLoad} of all regions hosted on a regionserver.
* Get {@link RegionMetrics} of all regions hosted on a regionserver.
*
* @param serverName region server from which regionload is required.
* @return region load map of all regions hosted on a region server
* @param serverName region server from which {@link RegionMetrics} is required.
* @return a {@link RegionMetrics} list of all regions hosted on a region server
* @throws IOException if a remote or network exception occurs
* @deprecated since 2.0 version and will be removed in 3.0 version.
* use {@link #getRegionLoads(ServerName)}
*/
@Deprecated
default Map<byte[], RegionLoad> getRegionLoad(ServerName serverName) throws IOException {
return getRegionLoad(serverName, null);
default List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
return getRegionMetrics(serverName, null);
}
/**
* Get {@link RegionLoad} of all regions hosted on a regionserver.
* Get {@link RegionMetrics} of all regions hosted on a regionserver for a table.
*
* @param serverName region server from which regionload is required.
* @return a region load list of all regions hosted on a region server
* @param serverName region server from which {@link RegionMetrics} is required.
* @param tableName get {@link RegionMetrics} of regions belonging to the table
* @return region metrics map of all regions of a table hosted on a region server
* @throws IOException if a remote or network exception occurs
*/
default List<RegionLoad> getRegionLoads(ServerName serverName) throws IOException {
return getRegionLoads(serverName, null);
}
/**
* Get {@link RegionLoad} of all regions hosted on a regionserver for a table.
*
* @param serverName region server from which regionload is required.
* @param tableName get region load of regions belonging to the table
* @return region load map of all regions of a table hosted on a region server
* @throws IOException if a remote or network exception occurs
* @deprecated since 2.0 version and will be removed in 3.0 version.
* use {@link #getRegionLoads(ServerName, TableName)}
*/
@Deprecated
default Map<byte[], RegionLoad> getRegionLoad(ServerName serverName, TableName tableName)
throws IOException {
List<RegionLoad> regionLoads = getRegionLoads(serverName, tableName);
Map<byte[], RegionLoad> resultMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (RegionLoad regionLoad : regionLoads) {
resultMap.put(regionLoad.getName(), regionLoad);
}
return resultMap;
}
/**
* Get {@link RegionLoad} of all regions hosted on a regionserver for a table.
*
* @param serverName region server from which regionload is required.
* @param tableName get region load of regions belonging to the table
* @return region load map of all regions of a table hosted on a region server
* @throws IOException if a remote or network exception occurs
*/
List<RegionLoad> getRegionLoads(ServerName serverName, TableName tableName) throws IOException;
List<RegionMetrics> getRegionMetrics(ServerName serverName,
TableName tableName) throws IOException;
/**
* @return Configuration used by the instance.
@ -1772,11 +1760,26 @@ public interface Admin extends Abortable, Closeable {
void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException;
/**
* Helper that delegates to getClusterStatus().getMasterCoprocessors().
* Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
* @return an array of master coprocessors
* @see org.apache.hadoop.hbase.ClusterStatus#getMasterCoprocessors()
* @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
* @deprecated since 2.0 version and will be removed in 3.0 version.
* use {@link #getMasterCoprocessorNames()}
*/
String[] getMasterCoprocessors() throws IOException;
@Deprecated
default String[] getMasterCoprocessors() throws IOException {
return getMasterCoprocessorNames().stream().toArray(size -> new String[size]);
}
/**
* Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
* @return an array of master coprocessors
* @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
*/
default List<String> getMasterCoprocessorNames() throws IOException {
return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS))
.getMasterCoprocessorNames();
}
/**
* Get the current compaction state of a table. It could be in a major compaction, a minor
@ -2371,7 +2374,7 @@ public interface Admin extends Abortable, Closeable {
* @throws IOException
*/
default int getMasterInfoPort() throws IOException {
return getClusterStatus(EnumSet.of(Option.MASTER_INFO_PORT)).getMasterInfoPort();
return getClusterMetrics(EnumSet.of(Option.MASTER_INFO_PORT)).getMasterInfoPort();
}
/**
@ -2617,7 +2620,7 @@ public interface Admin extends Abortable, Closeable {
* @return List of dead region servers.
*/
default List<ServerName> listDeadServers() throws IOException {
return getClusterStatus(EnumSet.of(Option.DEAD_SERVERS)).getDeadServerNames();
return getClusterMetrics(EnumSet.of(Option.DEAD_SERVERS)).getDeadServerNames();
}
/**

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;
import java.util.List;
@ -28,10 +27,10 @@ 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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.replication.TableCFs;
@ -886,40 +885,42 @@ public interface AsyncAdmin {
/**
* @return cluster status wrapped by {@link CompletableFuture}
*/
CompletableFuture<ClusterStatus> getClusterStatus();
CompletableFuture<ClusterMetrics> getClusterMetrics();
/**
* @return cluster status wrapped by {@link CompletableFuture}
*/
CompletableFuture<ClusterStatus> getClusterStatus(EnumSet<Option> options);
CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options);
/**
* @return current master server name wrapped by {@link CompletableFuture}
*/
default CompletableFuture<ServerName> getMaster() {
return getClusterStatus(EnumSet.of(Option.MASTER)).thenApply(ClusterStatus::getMaster);
return getClusterMetrics(EnumSet.of(Option.MASTER)).thenApply(ClusterMetrics::getMasterName);
}
/**
* @return current backup master list wrapped by {@link CompletableFuture}
*/
default CompletableFuture<Collection<ServerName>> getBackupMasters() {
return getClusterStatus(EnumSet.of(Option.BACKUP_MASTERS)).thenApply(ClusterStatus::getBackupMasters);
return getClusterMetrics(EnumSet.of(Option.BACKUP_MASTERS))
.thenApply(ClusterMetrics::getBackupMasterNames);
}
/**
* @return current live region servers list wrapped by {@link CompletableFuture}
*/
default CompletableFuture<Collection<ServerName>> getRegionServers() {
return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).thenApply(ClusterStatus::getServers);
return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.thenApply(cm -> cm.getLiveServerMetrics().keySet());
}
/**
* @return a list of master coprocessors wrapped by {@link CompletableFuture}
*/
default CompletableFuture<List<String>> getMasterCoprocessors() {
return getClusterStatus(EnumSet.of(Option.MASTER_COPROCESSORS))
.thenApply(ClusterStatus::getMasterCoprocessors).thenApply(Arrays::asList);
default CompletableFuture<List<String>> getMasterCoprocessorNames() {
return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS))
.thenApply(ClusterMetrics::getMasterCoprocessorNames);
}
/**
@ -927,8 +928,8 @@ public interface AsyncAdmin {
* @return master info port
*/
default CompletableFuture<Integer> getMasterInfoPort() {
return getClusterStatus(EnumSet.of(Option.MASTER_INFO_PORT)).thenApply(
ClusterStatus::getMasterInfoPort);
return getClusterMetrics(EnumSet.of(Option.MASTER_INFO_PORT)).thenApply(
ClusterMetrics::getMasterInfoPort);
}
/**
@ -978,19 +979,20 @@ public interface AsyncAdmin {
CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues);
/**
* Get a list of {@link RegionLoad} of all regions hosted on a region seerver.
* Get a list of {@link RegionMetrics} of all regions hosted on a region seerver.
* @param serverName
* @return a list of {@link RegionLoad} wrapped by {@link CompletableFuture}
* @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
*/
CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName);
CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName);
/**
* Get a list of {@link RegionLoad} of all regions hosted on a region seerver for a table.
* Get a list of {@link RegionMetrics} of all regions hosted on a region seerver for a table.
* @param serverName
* @param tableName
* @return a list of {@link RegionLoad} wrapped by {@link CompletableFuture}
* @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
*/
CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName, TableName tableName);
CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
TableName tableName);
/**
* Check whether master is in maintenance mode
@ -1199,8 +1201,8 @@ public interface AsyncAdmin {
* List all the dead region servers.
*/
default CompletableFuture<List<ServerName>> listDeadServers() {
return this.getClusterStatus(EnumSet.of(Option.DEAD_SERVERS))
.thenApply(ClusterStatus::getDeadServerNames);
return this.getClusterMetrics(EnumSet.of(Option.DEAD_SERVERS))
.thenApply(ClusterMetrics::getDeadServerNames);
}
/**

View File

@ -27,10 +27,10 @@ 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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.replication.TableCFs;
@ -566,13 +566,13 @@ class AsyncHBaseAdmin implements AsyncAdmin {
}
@Override
public CompletableFuture<ClusterStatus> getClusterStatus() {
return getClusterStatus(EnumSet.allOf(Option.class));
public CompletableFuture<ClusterMetrics> getClusterMetrics() {
return getClusterMetrics(EnumSet.allOf(Option.class));
}
@Override
public CompletableFuture<ClusterStatus> getClusterStatus(EnumSet<Option> options) {
return wrap(rawAdmin.getClusterStatus(options));
public CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options) {
return wrap(rawAdmin.getClusterMetrics(options));
}
@Override
@ -616,14 +616,14 @@ class AsyncHBaseAdmin implements AsyncAdmin {
}
@Override
public CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName) {
return wrap(rawAdmin.getRegionLoads(serverName));
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName) {
return wrap(rawAdmin.getRegionMetrics(serverName));
}
@Override
public CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName,
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
TableName tableName) {
return wrap(rawAdmin.getRegionLoads(serverName, tableName));
return wrap(rawAdmin.getRegionMetrics(serverName, tableName));
}
@Override

View File

@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.IOException;
import java.lang.reflect.Constructor;
@ -30,9 +28,9 @@ import java.net.NetworkInterface;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
@ -42,6 +40,7 @@ import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.io.netty.bootstrap.Bootstrap;
import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufInputStream;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
@ -52,7 +51,7 @@ import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramChannel;
import org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramPacket;
import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioDatagramChannel;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
/**
@ -133,7 +132,7 @@ class ClusterStatusListener implements Closeable {
*
* @param ncs the cluster status
*/
public void receive(ClusterStatus ncs) {
public void receive(ClusterMetrics ncs) {
if (ncs.getDeadServerNames() != null) {
for (ServerName sn : ncs.getDeadServerNames()) {
if (!isDeadServer(sn)) {
@ -264,7 +263,7 @@ class ClusterStatusListener implements Closeable {
ByteBufInputStream bis = new ByteBufInputStream(dp.content());
try {
ClusterStatusProtos.ClusterStatus csp = ClusterStatusProtos.ClusterStatus.parseFrom(bis);
ClusterStatus ncs = ProtobufUtil.toClusterStatus(csp);
ClusterMetrics ncs = ClusterMetricsBuilder.toClusterMetrics(csp);
receive(ncs);
} finally {
bis.close();

View File

@ -46,8 +46,9 @@ 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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
@ -59,8 +60,9 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.RegionMetricsBuilder;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@ -2072,29 +2074,31 @@ public class HBaseAdmin implements Admin {
}
@Override
public ClusterStatus getClusterStatus() throws IOException {
return getClusterStatus(EnumSet.allOf(Option.class));
}
@Override
public ClusterStatus getClusterStatus(EnumSet<Option> options) throws IOException {
return executeCallable(new MasterCallable<ClusterStatus>(getConnection(),
public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
return executeCallable(new MasterCallable<ClusterMetrics>(getConnection(),
this.rpcControllerFactory) {
@Override
protected ClusterStatus rpcCall() throws Exception {
protected ClusterMetrics rpcCall() throws Exception {
GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest(options);
return ProtobufUtil.toClusterStatus(
return ClusterMetricsBuilder.toClusterMetrics(
master.getClusterStatus(getRpcController(), req).getClusterStatus());
}
});
}
@Override
public List<RegionLoad> getRegionLoads(ServerName serverName, TableName tableName)
public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
throws IOException {
AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
HBaseRpcController controller = rpcControllerFactory.newController();
return ProtobufUtil.getRegionLoad(controller, admin, tableName);
AdminProtos.GetRegionLoadRequest request =
RequestConverter.buildGetRegionLoadRequest(tableName);
try {
return admin.getRegionLoad(controller, request).getRegionLoadsList().stream()
.map(RegionMetricsBuilder::toRegionMetrics).collect(Collectors.toList());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
@Override
@ -2428,16 +2432,6 @@ public class HBaseAdmin implements Admin {
rollWALWriterImpl(serverName);
}
@Override
public String[] getMasterCoprocessors() {
try {
return getClusterStatus(EnumSet.of(Option.MASTER_COPROCESSORS)).getMasterCoprocessors();
} catch (IOException e) {
LOG.error("Could not getClusterStatus()",e);
return null;
}
}
@Override
public CompactionState getCompactionState(final TableName tableName)
throws IOException {
@ -3148,15 +3142,15 @@ public class HBaseAdmin implements Admin {
@Override
public void updateConfiguration() throws IOException {
ClusterStatus status = getClusterStatus(
ClusterMetrics status = getClusterMetrics(
EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS));
for (ServerName server : status.getServers()) {
for (ServerName server : status.getLiveServerMetrics().keySet()) {
updateConfiguration(server);
}
updateConfiguration(status.getMaster());
updateConfiguration(status.getMasterName());
for (ServerName server : status.getBackupMasters()) {
for (ServerName server : status.getBackupMasterNames()) {
updateConfiguration(server);
}
}

View File

@ -42,15 +42,17 @@ 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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.RegionMetricsBuilder;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@ -2601,20 +2603,20 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
@Override
public CompletableFuture<ClusterStatus> getClusterStatus() {
return getClusterStatus(EnumSet.allOf(Option.class));
public CompletableFuture<ClusterMetrics> getClusterMetrics() {
return getClusterMetrics(EnumSet.allOf(Option.class));
}
@Override
public CompletableFuture<ClusterStatus>getClusterStatus(EnumSet<Option> options) {
public CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options) {
return this
.<ClusterStatus> newMasterCaller()
.<ClusterMetrics> newMasterCaller()
.action(
(controller, stub) -> this
.<GetClusterStatusRequest, GetClusterStatusResponse, ClusterStatus> call(controller,
.<GetClusterStatusRequest, GetClusterStatusResponse, ClusterMetrics> call(controller,
stub, RequestConverter.buildGetClusterStatusRequest(options),
(s, c, req, done) -> s.getClusterStatus(c, req, done),
resp -> ProtobufUtil.toClusterStatus(resp.getClusterStatus()))).call();
resp -> ClusterMetricsBuilder.toClusterMetrics(resp.getClusterStatus()))).call();
}
@Override
@ -2665,17 +2667,16 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<Void> updateConfiguration() {
CompletableFuture<Void> future = new CompletableFuture<Void>();
getClusterStatus(
EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS))
.whenComplete(
(status, err) -> {
getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS))
.whenComplete((status, err) -> {
if (err != null) {
future.completeExceptionally(err);
} else {
List<CompletableFuture<Void>> futures = new ArrayList<>();
status.getServers().forEach((server) -> futures.add(updateConfiguration(server)));
futures.add(updateConfiguration(status.getMaster()));
status.getBackupMasters().forEach(master -> futures.add(updateConfiguration(master)));
status.getLiveServerMetrics().keySet()
.forEach(server -> futures.add(updateConfiguration(server)));
futures.add(updateConfiguration(status.getMasterName()));
status.getBackupMasterNames().forEach(master -> futures.add(updateConfiguration(master)));
CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()]))
.whenComplete((result, err2) -> {
if (err2 != null) {
@ -2725,25 +2726,25 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
@Override
public CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName) {
return getRegionLoad(GetRegionLoadRequest.newBuilder().build(), serverName);
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName) {
return getRegionMetrics(GetRegionLoadRequest.newBuilder().build(), serverName);
}
@Override
public CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName,
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
TableName tableName) {
Preconditions.checkNotNull(tableName,
"tableName is null. If you don't specify a tableName, use getRegionLoads() instead");
return getRegionLoad(RequestConverter.buildGetRegionLoadRequest(tableName), serverName);
return getRegionMetrics(RequestConverter.buildGetRegionLoadRequest(tableName), serverName);
}
private CompletableFuture<List<RegionLoad>> getRegionLoad(GetRegionLoadRequest request,
private CompletableFuture<List<RegionMetrics>> getRegionMetrics(GetRegionLoadRequest request,
ServerName serverName) {
return this.<List<RegionLoad>> newAdminCaller()
return this.<List<RegionMetrics>> newAdminCaller()
.action((controller, stub) -> this
.<GetRegionLoadRequest, GetRegionLoadResponse, List<RegionLoad>> adminCall(controller,
stub, request, (s, c, req, done) -> s.getRegionLoad(controller, req, done),
ProtobufUtil::getRegionLoadInfo))
.<GetRegionLoadRequest, GetRegionLoadResponse, List<RegionMetrics>>
adminCall(controller, stub, request, (s, c, req, done) ->
s.getRegionLoad(controller, req, done), RegionMetricsBuilder::toRegionMetrics))
.serverName(serverName).call();
}

View File

@ -49,8 +49,6 @@ import org.apache.hadoop.hbase.Cell.Type;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.ExtendedCellBuilder;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
@ -126,7 +124,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRe
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
@ -1764,20 +1761,6 @@ public final class ProtobufUtil {
}
}
public static List<org.apache.hadoop.hbase.RegionLoad> getRegionLoad(
final RpcController controller, final AdminService.BlockingInterface admin,
final TableName tableName) throws IOException {
GetRegionLoadRequest request =
RequestConverter.buildGetRegionLoadRequest(tableName);
GetRegionLoadResponse response;
try {
response = admin.getRegionLoad(controller, request);
} catch (ServiceException se) {
throw getRemoteException(se);
}
return getRegionLoadInfo(response);
}
public static List<org.apache.hadoop.hbase.RegionLoad> getRegionLoadInfo(
GetRegionLoadResponse regionLoadResponse) {
List<org.apache.hadoop.hbase.RegionLoad> regionLoadList =
@ -2969,16 +2952,6 @@ public final class ProtobufUtil {
snapshotDesc.getCreationTime(), snapshotDesc.getVersion());
}
/**
* Convert a protobuf ClusterStatus to a ClusterStatus
*
* @param proto the protobuf ClusterStatus
* @return the converted ClusterStatus
*/
public static ClusterStatus toClusterStatus(ClusterStatusProtos.ClusterStatus proto) {
return new ClusterStatus(ClusterMetricsBuilder.toClusterMetrics(proto));
}
public static RegionLoadStats createRegionLoadStats(ClientProtos.RegionLoadStats stats) {
return new RegionLoadStats(stats.getMemStoreLoad(), stats.getHeapOccupancy(),
stats.getCompactionPressure());

View File

@ -75,7 +75,8 @@ public class TestInterfaceAlign {
}
private <T> List<String> getMethodNames(Class<T> c) {
return Arrays.asList(c.getDeclaredMethods()).stream().filter(m -> !isDeprecated(m))
// DON'T use the getDeclaredMethods as we want to check the Public APIs only.
return Arrays.asList(c.getMethods()).stream().filter(m -> !isDeprecated(m))
.map(Method::getName).distinct().collect(Collectors.toList());
}

View File

@ -25,22 +25,22 @@ import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterManager.ServiceType;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
/**
* Manages the interactions with an already deployed distributed cluster (as opposed to
@ -65,7 +65,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
this.clusterManager = clusterManager;
this.connection = ConnectionFactory.createConnection(conf);
this.admin = this.connection.getAdmin();
this.initialClusterStatus = getClusterStatus();
this.initialClusterStatus = getClusterMetrics();
}
public void setClusterManager(ClusterManager clusterManager) {
@ -81,12 +81,12 @@ public class DistributedHBaseCluster extends HBaseCluster {
* @throws IOException
*/
@Override
public ClusterStatus getClusterStatus() throws IOException {
return admin.getClusterStatus();
public ClusterMetrics getClusterMetrics() throws IOException {
return admin.getClusterMetrics();
}
@Override
public ClusterStatus getInitialClusterStatus() throws IOException {
public ClusterMetrics getInitialClusterMetrics() throws IOException {
return initialClusterStatus;
}
@ -315,8 +315,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
}
@Override
public boolean restoreClusterStatus(ClusterStatus initial) throws IOException {
ClusterStatus current = getClusterStatus();
public boolean restoreClusterMetrics(ClusterMetrics initial) throws IOException {
ClusterMetrics current = getClusterMetrics();
LOG.info("Restoring cluster - started");
@ -330,15 +330,15 @@ public class DistributedHBaseCluster extends HBaseCluster {
return success;
}
protected boolean restoreMasters(ClusterStatus initial, ClusterStatus current) {
protected boolean restoreMasters(ClusterMetrics initial, ClusterMetrics current) {
List<IOException> deferred = new ArrayList<>();
//check whether current master has changed
final ServerName initMaster = initial.getMaster();
if (!ServerName.isSameAddress(initMaster, current.getMaster())) {
final ServerName initMaster = initial.getMasterName();
if (!ServerName.isSameAddress(initMaster, current.getMasterName())) {
LOG.info("Restoring cluster - Initial active master : "
+ initMaster.getHostAndPort()
+ " has changed to : "
+ current.getMaster().getHostAndPort());
+ current.getMasterName().getHostAndPort());
// If initial master is stopped, start it, before restoring the state.
// It will come up as a backup master, if there is already an active master.
try {
@ -353,14 +353,14 @@ public class DistributedHBaseCluster extends HBaseCluster {
// 1. Kill the current backups
// 2. Stop current master
// 3. Start backup masters
for (ServerName currentBackup : current.getBackupMasters()) {
for (ServerName currentBackup : current.getBackupMasterNames()) {
if (!ServerName.isSameAddress(currentBackup, initMaster)) {
LOG.info("Restoring cluster - stopping backup master: " + currentBackup);
stopMaster(currentBackup);
}
}
LOG.info("Restoring cluster - stopping active master: " + current.getMaster());
stopMaster(current.getMaster());
LOG.info("Restoring cluster - stopping active master: " + current.getMasterName());
stopMaster(current.getMasterName());
waitForActiveAndReadyMaster(); // wait so that active master takes over
} catch (IOException ex) {
// if we fail to start the initial active master, we do not want to continue stopping
@ -369,7 +369,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
}
//start backup masters
for (ServerName backup : initial.getBackupMasters()) {
for (ServerName backup : initial.getBackupMasterNames()) {
try {
//these are not started in backup mode, but we should already have an active master
if (!clusterManager.isRunning(ServiceType.HBASE_MASTER,
@ -387,13 +387,13 @@ public class DistributedHBaseCluster extends HBaseCluster {
//current master has not changed, match up backup masters
Set<ServerName> toStart = new TreeSet<>(new ServerNameIgnoreStartCodeComparator());
Set<ServerName> toKill = new TreeSet<>(new ServerNameIgnoreStartCodeComparator());
toStart.addAll(initial.getBackupMasters());
toKill.addAll(current.getBackupMasters());
toStart.addAll(initial.getBackupMasterNames());
toKill.addAll(current.getBackupMasterNames());
for (ServerName server : current.getBackupMasters()) {
for (ServerName server : current.getBackupMasterNames()) {
toStart.remove(server);
}
for (ServerName server: initial.getBackupMasters()) {
for (ServerName server: initial.getBackupMasterNames()) {
toKill.remove(server);
}
@ -442,18 +442,18 @@ public class DistributedHBaseCluster extends HBaseCluster {
}
}
protected boolean restoreRegionServers(ClusterStatus initial, ClusterStatus current) {
protected boolean restoreRegionServers(ClusterMetrics initial, ClusterMetrics current) {
Set<ServerName> toStart = new TreeSet<>(new ServerNameIgnoreStartCodeComparator());
Set<ServerName> toKill = new TreeSet<>(new ServerNameIgnoreStartCodeComparator());
toStart.addAll(initial.getServers());
toKill.addAll(current.getServers());
toStart.addAll(initial.getLiveServerMetrics().keySet());
toKill.addAll(current.getLiveServerMetrics().keySet());
ServerName master = initial.getMaster();
ServerName master = initial.getMasterName();
for (ServerName server : current.getServers()) {
for (ServerName server : current.getLiveServerMetrics().keySet()) {
toStart.remove(server);
}
for (ServerName server: initial.getServers()) {
for (ServerName server: initial.getLiveServerMetrics().keySet()) {
toKill.remove(server);
}

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -160,7 +159,8 @@ public class IntegrationTestIngest extends IntegrationTestBase {
int recordSize, int writeThreads, int readThreads) throws Exception {
LOG.info("Running ingest");
LOG.info("Cluster size:" + util.getHBaseClusterInterface().getClusterStatus().getServersSize());
LOG.info("Cluster size:" + util.getHBaseClusterInterface()
.getClusterMetrics().getLiveServerMetrics().size());
long start = System.currentTimeMillis();
String runtimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());
@ -248,7 +248,7 @@ public class IntegrationTestIngest extends IntegrationTestBase {
/** Estimates a data size based on the cluster size */
protected long getNumKeys(long keysPerServer)
throws IOException {
int numRegionServers = cluster.getClusterStatus().getServersSize();
int numRegionServers = cluster.getClusterMetrics().getLiveServerMetrics().size();
return keysPerServer * numRegionServers;
}

View File

@ -23,7 +23,6 @@ import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
@ -31,17 +30,16 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
import org.apache.hadoop.hbase.util.MultiThreadedWriter;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -195,7 +193,8 @@ public class IntegrationTestLazyCfLoading {
hcd.setDataBlockEncoding(blockEncoding);
htd.addFamily(hcd);
}
int serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
int serverCount = util.getHBaseClusterInterface().getClusterMetrics()
.getLiveServerMetrics().size();
byte[][] splits = new RegionSplitter.HexStringSplit().split(serverCount * REGIONS_PER_SERVER);
util.getAdmin().createTable(htd, splits);
LOG.info("Created table");
@ -222,7 +221,8 @@ public class IntegrationTestLazyCfLoading {
Configuration conf = util.getConfiguration();
String timeoutKey = String.format(TIMEOUT_KEY, this.getClass().getSimpleName());
long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES);
long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
long serverCount = util.getHBaseClusterInterface().getClusterMetrics()
.getLiveServerMetrics().size();
long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER;
Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(TABLE_NAME);

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.ConstantDelayQueue;
@ -54,8 +53,9 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
* The job will run for <b>at least<b> given runtime (default 10min) by running a concurrent
* writer and reader workload followed by a concurrent updater and reader workload for
* num_keys_per_server.
*<p>
* <p>
* Example usage:
* </p>
* <pre>
* hbase org.apache.hadoop.hbase.IntegrationTestRegionReplicaReplication
* -DIntegrationTestRegionReplicaReplication.num_keys_per_server=10000
@ -154,7 +154,8 @@ public class IntegrationTestRegionReplicaReplication extends IntegrationTestInge
int recordSize, int writeThreads, int readThreads) throws Exception {
LOG.info("Running ingest");
LOG.info("Cluster size:" + util.getHBaseClusterInterface().getClusterStatus().getServersSize());
LOG.info("Cluster size:" + util.getHBaseClusterInterface()
.getClusterMetrics().getLiveServerMetrics().size());
// sleep for some time so that the cache for disabled tables does not interfere.
Threads.sleep(

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.ReflectionUtils;
@ -61,9 +60,9 @@ public class IntegrationTestingUtility extends HBaseTestingUtility {
/** Config for pluggable hbase cluster manager */
private static final String HBASE_CLUSTER_MANAGER_CLASS = "hbase.it.clustermanager.class";
private static final Class<? extends ClusterManager> DEFAULT_HBASE_CLUSTER_MANAGER_CLASS =
private static final Class<? extends ClusterManager> DEFAULT_HBASE_CLUSTER_MANAGER_CLASS =
HBaseClusterManager.class;
/**
* Initializes the state of the cluster. It starts a new in-process mini cluster, OR
* if we are given an already deployed distributed cluster it initializes the state.
@ -85,7 +84,7 @@ public class IntegrationTestingUtility extends HBaseTestingUtility {
*/
public void checkNodeCount(int numSlaves) throws Exception {
HBaseCluster cluster = getHBaseClusterInterface();
if (cluster.getClusterStatus().getServers().size() < numSlaves) {
if (cluster.getClusterMetrics().getLiveServerMetrics().size() < numSlaves) {
throw new Exception("Cluster does not have enough nodes:" + numSlaves);
}
}

View File

@ -21,10 +21,8 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.Locale;
import java.util.Set;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.lang3.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.regionserver.HStore;
@ -33,12 +31,13 @@ import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
import org.apache.hadoop.hbase.regionserver.StripeStoreEngine;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
import org.apache.hadoop.hbase.util.MultiThreadedAction;
import org.apache.hadoop.hbase.util.MultiThreadedReader;
import org.apache.hadoop.hbase.util.MultiThreadedWriter;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
import org.apache.yetus.audience.InterfaceAudience;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -196,7 +195,8 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool {
}
private void runOneTest(String description, Configuration conf) throws Exception {
int numServers = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
int numServers = util.getHBaseClusterInterface()
.getClusterMetrics().getLiveServerMetrics().size();
long startKey = preloadKeys * numServers;
long endKey = startKey + writeKeys * numServers;
status(String.format("%s test starting on %d servers; preloading 0 to %d and writing to %d",
@ -298,7 +298,7 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool {
htd.setConfiguration(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, "1048576");
}
byte[][] splits = new RegionSplitter.HexStringSplit().split(
util.getHBaseClusterInterface().getClusterStatus().getServersSize());
util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size());
util.getAdmin().createTable(htd, splits);
}

View File

@ -23,17 +23,17 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseCluster;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
@ -79,7 +79,7 @@ public class Action {
protected ActionContext context;
protected HBaseCluster cluster;
protected ClusterStatus initialStatus;
protected ClusterMetrics initialStatus;
protected ServerName[] initialServers;
protected long killMasterTimeout;
@ -94,8 +94,8 @@ public class Action {
public void init(ActionContext context) throws IOException {
this.context = context;
cluster = context.getHBaseCluster();
initialStatus = cluster.getInitialClusterStatus();
Collection<ServerName> regionServers = initialStatus.getServers();
initialStatus = cluster.getInitialClusterMetrics();
Collection<ServerName> regionServers = initialStatus.getLiveServerMetrics().keySet();
initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
killMasterTimeout = cluster.getConf().getLong(KILL_MASTER_TIMEOUT_KEY,
@ -118,13 +118,13 @@ public class Action {
/** Returns current region servers - active master */
protected ServerName[] getCurrentServers() throws IOException {
ClusterStatus clusterStatus = cluster.getClusterStatus();
Collection<ServerName> regionServers = clusterStatus.getServers();
ClusterMetrics clusterStatus = cluster.getClusterMetrics();
Collection<ServerName> regionServers = clusterStatus.getLiveServerMetrics().keySet();
int count = regionServers == null ? 0 : regionServers.size();
if (count <= 0) {
return new ServerName [] {};
}
ServerName master = clusterStatus.getMaster();
ServerName master = clusterStatus.getMasterName();
if (master == null || !regionServers.contains(master)) {
return regionServers.toArray(new ServerName[count]);
}
@ -156,7 +156,7 @@ public class Action {
cluster.killRegionServer(server);
cluster.waitForRegionServerToStop(server, killRsTimeout);
LOG.info("Killed region server:" + server + ". Reported num of rs:"
+ cluster.getClusterStatus().getServersSize());
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void startRs(ServerName server) throws IOException {
@ -164,7 +164,7 @@ public class Action {
cluster.startRegionServer(server.getHostname(), server.getPort());
cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
LOG.info("Started region server:" + server + ". Reported num of rs:"
+ cluster.getClusterStatus().getServersSize());
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void killZKNode(ServerName server) throws IOException {
@ -172,7 +172,7 @@ public class Action {
cluster.killZkNode(server);
cluster.waitForZkNodeToStop(server, killZkNodeTimeout);
LOG.info("Killed zookeeper node:" + server + ". Reported num of rs:"
+ cluster.getClusterStatus().getServersSize());
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void startZKNode(ServerName server) throws IOException {
@ -187,7 +187,7 @@ public class Action {
cluster.killDataNode(server);
cluster.waitForDataNodeToStop(server, killDataNodeTimeout);
LOG.info("Killed datanode:" + server + ". Reported num of rs:"
+ cluster.getClusterStatus().getServersSize());
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void startDataNode(ServerName server) throws IOException {
@ -197,16 +197,18 @@ public class Action {
LOG.info("Started datanode:" + server);
}
protected void unbalanceRegions(ClusterStatus clusterStatus,
protected void unbalanceRegions(ClusterMetrics clusterStatus,
List<ServerName> fromServers, List<ServerName> toServers,
double fractionOfRegions) throws Exception {
List<byte[]> victimRegions = new LinkedList<>();
for (ServerName server : fromServers) {
ServerLoad serverLoad = clusterStatus.getLoad(server);
for (Map.Entry<ServerName, ServerMetrics> entry
: clusterStatus.getLiveServerMetrics().entrySet()) {
ServerName sn = entry.getKey();
ServerMetrics serverLoad = entry.getValue();
// Ugh.
List<byte[]> regions = new LinkedList<>(serverLoad.getRegionsLoad().keySet());
List<byte[]> regions = new LinkedList<>(serverLoad.getRegionMetrics().keySet());
int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
LOG.debug("Removing " + victimRegionCount + " regions from " + server.getServerName());
LOG.debug("Removing " + victimRegionCount + " regions from " + sn);
for (int i = 0; i < victimRegionCount; ++i) {
int victimIx = RandomUtils.nextInt(0, regions.size());
String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
@ -61,7 +60,7 @@ public class BatchRestartRsAction extends RestartActionBaseAction {
}
LOG.info("Killed " + killedServers.size() + " region servers. Reported num of rs:"
+ cluster.getClusterStatus().getServersSize());
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
sleep(sleepTime);
@ -76,6 +75,6 @@ public class BatchRestartRsAction extends RestartActionBaseAction {
PolicyBasedChaosMonkey.TIMEOUT);
}
LOG.info("Started " + killedServers.size() +" region servers. Reported num of rs:"
+ cluster.getClusterStatus().getServersSize());
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
}

View File

@ -33,6 +33,6 @@ public class DumpClusterStatusAction extends Action {
@Override
public void perform() throws Exception {
LOG.debug("Performing action: Dump cluster status");
LOG.info("Cluster status\n" + cluster.getClusterStatus());
LOG.info("Cluster status\n" + cluster.getClusterMetrics());
}
}

View File

@ -57,7 +57,7 @@ public class MoveRegionsOfTableAction extends Action {
Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
Collection<ServerName> serversList =
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet();
ServerName[] servers = serversList.toArray(new ServerName[serversList.size()]);
LOG.info("Performing action: Move regions of table " + tableName);

View File

@ -31,7 +31,7 @@ public class RestartActiveMasterAction extends RestartActionBaseAction {
public void perform() throws Exception {
LOG.info("Performing action: Restart active master");
ServerName master = cluster.getClusterStatus().getMaster();
ServerName master = cluster.getClusterMetrics().getMasterName();
restartMaster(master, sleepTime);
}
}

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hbase.chaos.actions;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ServerName;
/**
@ -36,8 +36,8 @@ public class RestartRsHoldingMetaAction extends RestartActionBaseAction {
LOG.warn("No server is holding hbase:meta right now.");
return;
}
ClusterStatus clusterStatus = cluster.getClusterStatus();
if (server.equals(clusterStatus.getMaster())) {
ClusterMetrics clusterStatus = cluster.getClusterMetrics();
if (server.equals(clusterStatus.getMasterName())) {
// Master holds the meta, so restart the master.
restartMaster(server, sleepTime);
} else {

View File

@ -23,9 +23,8 @@ import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.junit.Assert;
@ -54,8 +53,8 @@ public class UnbalanceKillAndRebalanceAction extends Action {
@Override
public void perform() throws Exception {
ClusterStatus status = this.cluster.getClusterStatus();
List<ServerName> victimServers = new LinkedList<>(status.getServers());
ClusterMetrics status = this.cluster.getClusterMetrics();
List<ServerName> victimServers = new LinkedList<>(status.getLiveServerMetrics().keySet());
Set<ServerName> killedServers = new HashSet<>();
int liveCount = (int)Math.ceil(FRC_SERVERS_THAT_HOARD_AND_LIVE * victimServers.size());

View File

@ -21,9 +21,8 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ServerName;
/**
@ -47,8 +46,8 @@ public class UnbalanceRegionsAction extends Action {
@Override
public void perform() throws Exception {
LOG.info("Unbalancing regions");
ClusterStatus status = this.cluster.getClusterStatus();
List<ServerName> victimServers = new LinkedList<>(status.getServers());
ClusterMetrics status = this.cluster.getClusterMetrics();
List<ServerName> victimServers = new LinkedList<>(status.getLiveServerMetrics().keySet());
int targetServerCount = (int)Math.ceil(fractionOfServers * victimServers.size());
List<ServerName> targetServers = new ArrayList<>(targetServerCount);
for (int i = 0; i < targetServerCount; ++i) {

View File

@ -31,7 +31,6 @@ import java.util.Optional;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.conf.Configuration;
@ -757,8 +756,8 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
if (util.isDistributedCluster()) {
util.getConfiguration().setIfUnset(NUM_MAPS_KEY,
Integer.toString(util.getAdmin()
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getServersSize() * 10)
.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().size() * 10)
);
util.getConfiguration().setIfUnset(NUM_IMPORT_ROUNDS_KEY, "5");
} else {

View File

@ -28,7 +28,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
import org.apache.hadoop.hbase.ClusterStatus;
@ -618,7 +617,8 @@ public class IntegrationTestMTTR {
@Override
public Boolean call() throws Exception {
int colsPerKey = 10;
int numServers = util.getHBaseClusterInterface().getInitialClusterStatus().getServersSize();
int numServers = util.getHBaseClusterInterface().getInitialClusterMetrics()
.getLiveServerMetrics().size();
int numKeys = numServers * 5000;
int writeThreads = 10;

View File

@ -36,7 +36,6 @@ import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter;
@ -82,11 +81,11 @@ import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl;
import org.apache.hadoop.hbase.mapreduce.WALPlayer;
import org.apache.hadoop.hbase.regionserver.FlushAllLargeStoresPolicy;
import org.apache.hadoop.hbase.regionserver.FlushPolicyFactory;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.NullWritable;
@ -710,8 +709,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
if (conf.getBoolean(HBaseTestingUtility.PRESPLIT_TEST_TABLE_KEY,
HBaseTestingUtility.PRESPLIT_TEST_TABLE)) {
int numberOfServers =
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getServers().size();
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().size();
if (numberOfServers == 0) {
throw new IllegalStateException("No live regionservers");
}

View File

@ -25,7 +25,6 @@ import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.IntegrationTestIngest;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
import org.apache.hadoop.hbase.client.Admin;
@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.LoadTestTool;
import org.apache.hadoop.hbase.util.MultiThreadedReader;
import org.apache.hadoop.hbase.util.Threads;
@ -142,7 +141,7 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr
protected void runIngestTest(long defaultRunTime, long keysPerServerPerIter, int colsPerKey,
int recordSize, int writeThreads, int readThreads) throws Exception {
LOG.info("Cluster size:"+
util.getHBaseClusterInterface().getClusterStatus().getServersSize());
util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size());
long start = System.currentTimeMillis();
String runtimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());

View File

@ -23,11 +23,11 @@ import java.util.Collections;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Size;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.util.Bytes;
@ -79,12 +79,13 @@ public class RegionSizeCalculator {
Set<ServerName> tableServers = getRegionServersOfTable(regionLocator);
for (ServerName tableServerName : tableServers) {
Map<byte[], RegionLoad> regionLoads =
admin.getRegionLoad(tableServerName, regionLocator.getName());
for (RegionLoad regionLoad : regionLoads.values()) {
for (RegionMetrics regionLoad : admin.getRegionMetrics(
tableServerName,regionLocator.getName())) {
byte[] regionId = regionLoad.getRegionName();
long regionSizeBytes
= ((long) regionLoad.getStoreFileSize().get(Size.Unit.MEGABYTE)) * MEGABYTE;
byte[] regionId = regionLoad.getName();
long regionSizeBytes = regionLoad.getStorefileSizeMB() * MEGABYTE;
sizeMap.put(regionId, regionSizeBytes);
if (LOG.isDebugEnabled()) {

View File

@ -17,30 +17,27 @@
*/
package org.apache.hadoop.hbase.mapreduce;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import static org.apache.hadoop.hbase.HConstants.DEFAULT_REGIONSERVER_PORT;
import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import static org.apache.hadoop.hbase.HConstants.DEFAULT_REGIONSERVER_PORT;
import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.when;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Size;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionLocator;
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.mockito.Mockito;
@Category({MiscTests.class, SmallTests.class})
public class TestRegionSizeCalculator {
@ -134,14 +131,15 @@ public class TestRegionSizeCalculator {
/**
* Creates mock returning RegionLoad info about given servers.
*/
private Admin mockAdmin(RegionLoad... regionLoadArray) throws Exception {
private Admin mockAdmin(RegionMetrics... regionLoadArray) throws Exception {
Admin mockAdmin = Mockito.mock(Admin.class);
Map<byte[], RegionLoad> regionLoads = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (RegionLoad regionLoad : regionLoadArray) {
regionLoads.put(regionLoad.getName(), regionLoad);
List<RegionMetrics> regionLoads = new ArrayList<>();
for (RegionMetrics regionLoad : regionLoadArray) {
regionLoads.add(regionLoad);
}
when(mockAdmin.getConfiguration()).thenReturn(configuration);
when(mockAdmin.getRegionLoad(sn, TableName.valueOf("sizeTestTable"))).thenReturn(regionLoads);
when(mockAdmin.getRegionMetrics(sn, TableName.valueOf("sizeTestTable")))
.thenReturn(regionLoads);
return mockAdmin;
}
@ -150,11 +148,11 @@ public class TestRegionSizeCalculator {
*
* @param fileSizeMb number of megabytes occupied by region in file store in megabytes
* */
private RegionLoad mockRegion(String regionName, int fileSizeMb) {
RegionLoad region = Mockito.mock(RegionLoad.class);
when(region.getName()).thenReturn(regionName.getBytes());
private RegionMetrics mockRegion(String regionName, int fileSizeMb) {
RegionMetrics region = Mockito.mock(RegionMetrics.class);
when(region.getRegionName()).thenReturn(regionName.getBytes());
when(region.getNameAsString()).thenReturn(regionName);
when(region.getStorefileSizeMB()).thenReturn(fileSizeMb);
when(region.getStoreFileSize()).thenReturn(new Size(fileSizeMb, Size.Unit.MEGABYTE));
return region;
}
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.rest;
import java.io.IOException;
import java.util.EnumSet;
import java.util.Map;
import javax.ws.rs.GET;
import javax.ws.rs.Produces;
import javax.ws.rs.core.CacheControl;
@ -28,11 +29,12 @@ 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;
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.RegionMetrics;
import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Size;
import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@ -67,29 +69,35 @@ public class StorageClusterStatusResource extends ResourceBase {
}
servlet.getMetrics().incrementRequests(1);
try {
ClusterStatus status = servlet.getAdmin().getClusterStatus(
ClusterMetrics status = servlet.getAdmin().getClusterMetrics(
EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS));
StorageClusterStatusModel model = new StorageClusterStatusModel();
model.setRegions(status.getRegionsCount());
model.setRegions(status.getRegionCount());
model.setRequests(status.getRequestCount());
model.setAverageLoad(status.getAverageLoad());
for (ServerName info: status.getServers()) {
ServerLoad load = status.getLoad(info);
for (Map.Entry<ServerName, ServerMetrics> entry: status.getLiveServerMetrics().entrySet()) {
ServerName sn = entry.getKey();
ServerMetrics load = entry.getValue();
StorageClusterStatusModel.Node node =
model.addLiveNode(
info.getHostname() + ":" +
Integer.toString(info.getPort()),
info.getStartcode(), load.getUsedHeapMB(),
load.getMaxHeapMB());
node.setRequests(load.getNumberOfRequests());
for (RegionLoad region: load.getRegionsLoad().values()) {
node.addRegion(region.getName(), region.getStores(),
region.getStorefiles(), region.getStorefileSizeMB(),
region.getMemStoreSizeMB(), region.getStorefileIndexSizeKB(),
region.getReadRequestsCount(), region.getWriteRequestsCount(),
region.getRootIndexSizeKB(), region.getTotalStaticIndexSizeKB(),
region.getTotalStaticBloomSizeKB(), region.getTotalCompactingKVs(),
region.getCurrentCompactedKVs());
sn.getHostname() + ":" +
Integer.toString(sn.getPort()),
sn.getStartcode(), (int) load.getUsedHeapSize().get(Size.Unit.MEGABYTE),
(int) load.getMaxHeapSize().get(Size.Unit.MEGABYTE));
node.setRequests(load.getRequestCount());
for (RegionMetrics region: load.getRegionMetrics().values()) {
node.addRegion(region.getRegionName(), region.getStoreCount(),
region.getStoreFileCount(),
(int) region.getStoreFileSize().get(Size.Unit.MEGABYTE),
(int) region.getMemStoreSize().get(Size.Unit.MEGABYTE),
(long) region.getStoreFileIndexSize().get(Size.Unit.KILOBYTE),
region.getReadRequestCount(),
region.getWriteRequestCount(),
(int) region.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE),
(int) region.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE),
(int) region.getBloomFilterSize().get(Size.Unit.KILOBYTE),
region.getCompactingCellCount(),
region.getCompactedCellCount());
}
}
for (ServerName name: status.getDeadServerNames()) {

View File

@ -64,7 +64,7 @@ public class StorageClusterVersionResource extends ResourceBase {
try {
StorageClusterVersionModel model = new StorageClusterVersionModel();
model.setVersion(
servlet.getAdmin().getClusterStatus(EnumSet.of(Option.HBASE_VERSION))
servlet.getAdmin().getClusterMetrics(EnumSet.of(Option.HBASE_VERSION))
.getHBaseVersion());
ResponseBuilder response = Response.ok(model);
response.cacheControl(cacheControl);

View File

@ -29,9 +29,8 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
@ -73,7 +72,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
private static final Logger LOG = LoggerFactory.getLogger(RSGroupBasedLoadBalancer.class);
private Configuration config;
private ClusterStatus clusterStatus;
private ClusterMetrics clusterStatus;
private MasterServices masterServices;
private volatile RSGroupInfoManager rsGroupInfoManager;
private LoadBalancer internalBalancer;
@ -95,8 +94,8 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
public void setClusterStatus(ClusterStatus st) {
this.clusterStatus = st;
public void setClusterMetrics(ClusterMetrics sm) {
this.clusterStatus = sm;
}
@Override
@ -386,7 +385,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
StochasticLoadBalancer.class, LoadBalancer.class);
internalBalancer = ReflectionUtils.newInstance(balancerKlass, config);
internalBalancer.setMasterServices(masterServices);
internalBalancer.setClusterStatus(clusterStatus);
internalBalancer.setClusterMetrics(clusterStatus);
internalBalancer.setConf(config);
internalBalancer.initialize();
}

View File

@ -32,14 +32,15 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseCluster;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
@ -163,15 +164,16 @@ public abstract class TestRSGroupsBase {
public Map<TableName, Map<ServerName, List<String>>> getTableServerRegionMap()
throws IOException {
Map<TableName, Map<ServerName, List<String>>> map = Maps.newTreeMap();
ClusterStatus status = TEST_UTIL.getHBaseClusterInterface().getClusterStatus();
for(ServerName serverName : status.getServers()) {
for(RegionLoad rl : status.getLoad(serverName).getRegionsLoad().values()) {
ClusterMetrics status = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics();
for (Map.Entry<ServerName, ServerMetrics> entry : status.getLiveServerMetrics().entrySet()) {
ServerName serverName = entry.getKey();
for(RegionMetrics rl : entry.getValue().getRegionMetrics().values()) {
TableName tableName = null;
try {
tableName = RegionInfo.getTable(rl.getName());
tableName = RegionInfo.getTable(rl.getRegionName());
} catch (IllegalArgumentException e) {
LOG.warn("Failed parse a table name from regionname=" +
Bytes.toStringBinary(rl.getName()));
Bytes.toStringBinary(rl.getRegionName()));
continue;
}
if(!map.containsKey(tableName)) {
@ -267,11 +269,11 @@ public abstract class TestRSGroupsBase {
// return the real number of region servers, excluding the master embedded region server in 2.0+
public int getNumServers() throws IOException {
ClusterStatus status =
admin.getClusterStatus(EnumSet.of(Option.MASTER, Option.LIVE_SERVERS));
ServerName master = status.getMaster();
ClusterMetrics status =
admin.getClusterMetrics(EnumSet.of(Option.MASTER, Option.LIVE_SERVERS));
ServerName master = status.getMasterName();
int count = 0;
for (ServerName sn : status.getServers()) {
for (ServerName sn : status.getLiveServerMetrics().keySet()) {
if (!sn.equals(master)) {
count++;
}
@ -494,8 +496,8 @@ public abstract class TestRSGroupsBase {
}
//get server which is not a member of new group
ServerName targetServer = null;
for (ServerName server : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getServers()) {
for (ServerName server : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
if (!newGroup.containsServer(server.getAddress())) {
targetServer = server;
break;
@ -524,7 +526,7 @@ public abstract class TestRSGroupsBase {
return
getTableRegionMap().get(tableName) != null &&
getTableRegionMap().get(tableName).size() == 6 &&
admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
admin.getClusterMetrics(EnumSet.of(Option.REGIONS_IN_TRANSITION))
.getRegionStatesInTransition().size() < 1;
}
});
@ -603,13 +605,13 @@ public abstract class TestRSGroupsBase {
AdminProtos.StopServerRequest.newBuilder().setReason("Die").build());
} catch(Exception e) {
}
assertFalse(cluster.getClusterStatus().getServers().contains(targetServer));
assertFalse(cluster.getClusterMetrics().getLiveServerMetrics().containsKey(targetServer));
//wait for created table to be assigned
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return cluster.getClusterStatus().getRegionStatesInTransition().isEmpty();
return cluster.getClusterMetrics().getRegionStatesInTransition().isEmpty();
}
});
Set<Address> newServers = Sets.newHashSet();
@ -626,7 +628,7 @@ public abstract class TestRSGroupsBase {
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return cluster.getClusterStatus().getRegionStatesInTransition().isEmpty();
return cluster.getClusterMetrics().getRegionStatesInTransition().isEmpty();
}
});
@ -766,7 +768,8 @@ public abstract class TestRSGroupsBase {
//get server which is not a member of new group
ServerName targetServer = null;
for(ServerName server : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
for(ServerName server : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
if(!newGroup.containsServer(server.getAddress()) &&
!rsGroupAdmin.getRSGroupInfo("master").containsServer(server.getAddress())) {
targetServer = server;
@ -829,7 +832,7 @@ public abstract class TestRSGroupsBase {
return getTableRegionMap().get(tableName) != null &&
getTableRegionMap().get(tableName).size() == 5 &&
getTableServerRegionMap().get(tableName).size() == 1 &&
admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
admin.getClusterMetrics(EnumSet.of(Option.REGIONS_IN_TRANSITION))
.getRegionStatesInTransition().size() < 1;
}
});
@ -890,11 +893,11 @@ public abstract class TestRSGroupsBase {
@Override
public boolean evaluate() throws Exception {
return !master.getServerManager().areDeadServersInProgress()
&& cluster.getClusterStatus().getDeadServerNames().size() > 0;
&& cluster.getClusterMetrics().getDeadServerNames().size() > 0;
}
});
assertFalse(cluster.getClusterStatus().getServers().contains(targetServer));
assertTrue(cluster.getClusterStatus().getDeadServerNames().contains(targetServer));
assertFalse(cluster.getClusterMetrics().getLiveServerMetrics().containsKey(targetServer));
assertTrue(cluster.getClusterMetrics().getDeadServerNames().contains(targetServer));
assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
//clear dead servers list
@ -941,7 +944,7 @@ public abstract class TestRSGroupsBase {
@Override
public boolean evaluate() throws Exception {
return !master.getServerManager().areDeadServersInProgress()
&& cluster.getClusterStatus().getDeadServerNames().size() > 0;
&& cluster.getClusterMetrics().getDeadServerNames().size() > 0;
}
});
@ -956,15 +959,15 @@ public abstract class TestRSGroupsBase {
}
assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics().getMasterName();
TEST_UTIL.getHBaseClusterInterface().stopMaster(sn);
TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(sn, 60000);
TEST_UTIL.getHBaseClusterInterface().startMaster(sn.getHostname(), 0);
TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster(60000);
assertEquals(3, cluster.getClusterStatus().getServersSize());
assertFalse(cluster.getClusterStatus().getServers().contains(targetServer));
assertFalse(cluster.getClusterStatus().getDeadServerNames().contains(targetServer));
assertEquals(3, cluster.getClusterMetrics().getLiveServerMetrics().size());
assertFalse(cluster.getClusterMetrics().getLiveServerMetrics().containsKey(targetServer));
assertFalse(cluster.getClusterMetrics().getDeadServerNames().contains(targetServer));
assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
rsGroupAdmin.removeServers(Sets.newHashSet(targetServer.getAddress()));

View File

@ -54,8 +54,8 @@ MasterAddressTracker masterAddressTracker = master.getMasterAddressTracker();
<th>Start Time</th>
</tr>
<%java>
Collection<ServerName> backup_masters = master.getClusterStatusWithoutCoprocessor(
EnumSet.of(ClusterMetrics.Option.BACKUP_MASTERS)).getBackupMasters();
Collection<ServerName> backup_masters = master.getClusterMetricsWithoutCoprocessor(
EnumSet.of(ClusterMetrics.Option.BACKUP_MASTERS)).getBackupMasterNames();
ServerName [] backupServerNames = backup_masters.toArray(new ServerName[backup_masters.size()]);
Arrays.sort(backupServerNames);
for (ServerName serverName : backupServerNames) {

View File

@ -21,8 +21,7 @@ package org.apache.hadoop.hbase.coprocessor;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@ -1263,14 +1262,14 @@ public interface MasterObserver {
/**
* Called before get cluster status.
*/
default void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
default void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {}
/**
* Called after get cluster status.
*/
default void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
ClusterStatus status) throws IOException {}
default void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
ClusterMetrics status) throws IOException {}
/**
* Called before clear dead region servers.

View File

@ -35,8 +35,8 @@ 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.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ScheduledChore;
@ -65,7 +65,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.socket.InternetProtocolFamil
import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioDatagramChannel;
import org.apache.hbase.thirdparty.io.netty.handler.codec.MessageToMessageEncoder;
import org.apache.hbase.thirdparty.io.netty.util.internal.StringUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
/**
@ -159,12 +158,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.
publisher.publish(new ClusterStatus(ClusterMetricsBuilder.newBuilder()
.setHBaseVersion(VersionInfo.getVersion())
.setClusterId(master.getMasterFileSystem().getClusterId().toString())
.setMasterName(master.getServerName())
.setDeadServerNames(sns)
.build()));
publisher.publish(ClusterMetricsBuilder.newBuilder()
.setHBaseVersion(VersionInfo.getVersion())
.setClusterId(master.getMasterFileSystem().getClusterId().toString())
.setMasterName(master.getServerName())
.setDeadServerNames(sns)
.build());
}
protected void cleanup() {
@ -229,7 +228,7 @@ public class ClusterStatusPublisher extends ScheduledChore {
void connect(Configuration conf) throws IOException;
void publish(ClusterStatus cs);
void publish(ClusterMetrics cs);
@Override
void close();
@ -289,7 +288,7 @@ public class ClusterStatusPublisher extends ScheduledChore {
b.group(group)
.channelFactory(new HBaseDatagramChannelFactory<Channel>(NioDatagramChannel.class, family))
.option(ChannelOption.SO_REUSEADDR, true)
.handler(new ClusterStatusEncoder(isa));
.handler(new ClusterMetricsEncoder(isa));
try {
channel = (DatagramChannel) b.bind(bindAddress, 0).sync().channel();
@ -328,24 +327,24 @@ public class ClusterStatusPublisher extends ScheduledChore {
}
}
private static final class ClusterStatusEncoder extends MessageToMessageEncoder<ClusterStatus> {
private static final class ClusterMetricsEncoder
extends MessageToMessageEncoder<ClusterMetrics> {
final private InetSocketAddress isa;
private ClusterStatusEncoder(InetSocketAddress isa) {
private ClusterMetricsEncoder(InetSocketAddress isa) {
this.isa = isa;
}
@Override
protected void encode(ChannelHandlerContext channelHandlerContext,
ClusterStatus clusterStatus, List<Object> objects) {
ClusterStatusProtos.ClusterStatus csp
= ClusterMetricsBuilder.toClusterStatus(clusterStatus);
objects.add(new DatagramPacket(Unpooled.wrappedBuffer(csp.toByteArray()), isa));
ClusterMetrics clusterStatus, List<Object> objects) {
objects.add(new DatagramPacket(Unpooled.wrappedBuffer(
ClusterMetricsBuilder.toClusterStatus(clusterStatus).toByteArray()), isa));
}
}
@Override
public void publish(ClusterStatus cs) {
public void publish(ClusterMetrics cs) {
channel.writeAndFlush(cs).syncUninterruptibly();
}

View File

@ -56,9 +56,9 @@ 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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
@ -834,7 +834,7 @@ public class HMaster extends HRegionServer implements MasterServices {
//initialize load balancer
this.balancer.setMasterServices(this);
this.balancer.setClusterStatus(getClusterStatusWithoutCoprocessor());
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
this.balancer.initialize();
// Check if master is shutting down because of some issue
@ -878,7 +878,7 @@ public class HMaster extends HRegionServer implements MasterServices {
this.assignmentManager.joinCluster();
// set cluster status again after user regions are assigned
this.balancer.setClusterStatus(getClusterStatusWithoutCoprocessor());
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
// Start balancer and meta catalog janitor after meta and regions have been assigned.
status.setStatus("Starting balancer and catalog janitor");
@ -1404,7 +1404,7 @@ public class HMaster extends HRegionServer implements MasterServices {
List<RegionPlan> plans = new ArrayList<>();
//Give the balancer the current cluster state.
this.balancer.setClusterStatus(getClusterStatusWithoutCoprocessor());
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
this.balancer.setClusterLoad(assignmentsByTable);
for (Map<ServerName, List<RegionInfo>> serverMap : assignmentsByTable.values()) {
@ -2404,11 +2404,11 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
public ClusterStatus getClusterStatusWithoutCoprocessor() throws InterruptedIOException {
return getClusterStatusWithoutCoprocessor(EnumSet.allOf(Option.class));
public ClusterMetrics getClusterMetricsWithoutCoprocessor() throws InterruptedIOException {
return getClusterMetricsWithoutCoprocessor(EnumSet.allOf(Option.class));
}
public ClusterStatus getClusterStatusWithoutCoprocessor(EnumSet<Option> options)
public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet<Option> options)
throws InterruptedIOException {
ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
// given that hbase1 can't submit the request with Option,
@ -2464,23 +2464,23 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
}
return new ClusterStatus(builder.build());
return builder.build();
}
/**
* @return cluster status
*/
public ClusterStatus getClusterStatus() throws IOException {
return getClusterStatus(EnumSet.allOf(Option.class));
public ClusterMetrics getClusterMetrics() throws IOException {
return getClusterMetrics(EnumSet.allOf(Option.class));
}
public ClusterStatus getClusterStatus(EnumSet<Option> options) throws IOException {
public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
if (cpHost != null) {
cpHost.preGetClusterStatus();
cpHost.preGetClusterMetrics();
}
ClusterStatus status = getClusterStatusWithoutCoprocessor(options);
ClusterMetrics status = getClusterMetricsWithoutCoprocessor(options);
if (cpHost != null) {
cpHost.postGetClusterStatus(status);
cpHost.postGetClusterMetrics(status);
}
return status;
}
@ -3173,14 +3173,14 @@ public class HMaster extends HRegionServer implements MasterServices {
@Override
public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getLastMajorCompactionTsForTable(table);
return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLastMajorCompactionTimestamp(table);
}
@Override
public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getLastMajorCompactionTsForRegion(regionName);
return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLastMajorCompactionTimestamp(regionName);
}
/**

View File

@ -18,12 +18,12 @@
*/
package org.apache.hadoop.hbase.master;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
@ -32,8 +32,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
import org.apache.yetus.audience.InterfaceAudience;
import edu.umd.cs.findbugs.annotations.Nullable;
/**
* Makes decisions about the placement and movement of Regions across
* RegionServers.
@ -71,7 +69,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* Set the current cluster status. This allows a LoadBalancer to map host name to a server
* @param st
*/
void setClusterStatus(ClusterStatus st);
void setClusterMetrics(ClusterMetrics st);
/**
* Pass RegionStates and allow balancer to set the current cluster load.

View File

@ -18,14 +18,12 @@
package org.apache.hadoop.hbase.master;
import com.google.protobuf.Service;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import com.google.protobuf.Service;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
@ -1563,20 +1561,20 @@ public class MasterCoprocessorHost
});
}
public void preGetClusterStatus() throws IOException {
public void preGetClusterMetrics() throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.preGetClusterStatus(this);
observer.preGetClusterMetrics(this);
}
});
}
public void postGetClusterStatus(ClusterStatus status) throws IOException {
public void postGetClusterMetrics(ClusterMetrics status) throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.postGetClusterStatus(this, status);
observer.postGetClusterMetrics(this, status);
}
});
}

View File

@ -30,7 +30,6 @@ import java.util.Map;
import java.util.Map.Entry;
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;
@ -905,7 +904,7 @@ public class MasterRpcServices extends RSRpcServices
try {
master.checkInitialized();
response.setClusterStatus(ClusterMetricsBuilder.toClusterStatus(
master.getClusterStatus(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
master.getClusterMetrics(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
} catch (IOException e) {
throw new ServiceException(e);
}

View File

@ -191,265 +191,268 @@ public class RegionPlacementMaintainer {
* @throws IOException
*/
private void genAssignmentPlan(TableName tableName,
SnapshotOfRegionAssignmentFromMeta assignmentSnapshot,
Map<String, Map<String, Float>> regionLocalityMap, FavoredNodesPlan plan,
boolean munkresForSecondaryAndTertiary) throws IOException {
// Get the all the regions for the current table
List<RegionInfo> regions =
assignmentSnapshot.getTableToRegionMap().get(tableName);
int numRegions = regions.size();
SnapshotOfRegionAssignmentFromMeta assignmentSnapshot,
Map<String, Map<String, Float>> regionLocalityMap, FavoredNodesPlan plan,
boolean munkresForSecondaryAndTertiary) throws IOException {
// Get the all the regions for the current table
List<RegionInfo> regions =
assignmentSnapshot.getTableToRegionMap().get(tableName);
int numRegions = regions.size();
// Get the current assignment map
Map<RegionInfo, ServerName> currentAssignmentMap =
assignmentSnapshot.getRegionToRegionServerMap();
// Get the current assignment map
Map<RegionInfo, ServerName> currentAssignmentMap =
assignmentSnapshot.getRegionToRegionServerMap();
// Get the all the region servers
List<ServerName> servers = new ArrayList<>();
try (Admin admin = this.connection.getAdmin()) {
servers.addAll(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
}
// Get the all the region servers
List<ServerName> servers = new ArrayList<>();
try (Admin admin = this.connection.getAdmin()) {
servers.addAll(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet());
}
LOG.info("Start to generate assignment plan for " + numRegions +
" regions from table " + tableName + " with " +
servers.size() + " region servers");
LOG.info("Start to generate assignment plan for " + numRegions +
" regions from table " + tableName + " with " +
servers.size() + " region servers");
int slotsPerServer = (int) Math.ceil((float) numRegions /
servers.size());
int regionSlots = slotsPerServer * servers.size();
int slotsPerServer = (int) Math.ceil((float) numRegions /
servers.size());
int regionSlots = slotsPerServer * servers.size();
// Compute the primary, secondary and tertiary costs for each region/server
// pair. These costs are based only on node locality and rack locality, and
// will be modified later.
float[][] primaryCost = new float[numRegions][regionSlots];
float[][] secondaryCost = new float[numRegions][regionSlots];
float[][] tertiaryCost = new float[numRegions][regionSlots];
// Compute the primary, secondary and tertiary costs for each region/server
// pair. These costs are based only on node locality and rack locality, and
// will be modified later.
float[][] primaryCost = new float[numRegions][regionSlots];
float[][] secondaryCost = new float[numRegions][regionSlots];
float[][] tertiaryCost = new float[numRegions][regionSlots];
if (this.enforceLocality && regionLocalityMap != null) {
// Transform the locality mapping into a 2D array, assuming that any
// unspecified locality value is 0.
float[][] localityPerServer = new float[numRegions][regionSlots];
for (int i = 0; i < numRegions; i++) {
Map<String, Float> serverLocalityMap =
regionLocalityMap.get(regions.get(i).getEncodedName());
if (serverLocalityMap == null) {
if (this.enforceLocality && regionLocalityMap != null) {
// Transform the locality mapping into a 2D array, assuming that any
// unspecified locality value is 0.
float[][] localityPerServer = new float[numRegions][regionSlots];
for (int i = 0; i < numRegions; i++) {
Map<String, Float> serverLocalityMap =
regionLocalityMap.get(regions.get(i).getEncodedName());
if (serverLocalityMap == null) {
continue;
}
for (int j = 0; j < servers.size(); j++) {
String serverName = servers.get(j).getHostname();
if (serverName == null) {
continue;
}
for (int j = 0; j < servers.size(); j++) {
String serverName = servers.get(j).getHostname();
if (serverName == null) {
continue;
}
Float locality = serverLocalityMap.get(serverName);
if (locality == null) {
continue;
}
for (int k = 0; k < slotsPerServer; k++) {
// If we can't find the locality of a region to a server, which occurs
// because locality is only reported for servers which have some
// blocks of a region local, then the locality for that pair is 0.
localityPerServer[i][j * slotsPerServer + k] = locality.floatValue();
}
Float locality = serverLocalityMap.get(serverName);
if (locality == null) {
continue;
}
}
// Compute the total rack locality for each region in each rack. The total
// rack locality is the sum of the localities of a region on all servers in
// a rack.
Map<String, Map<RegionInfo, Float>> rackRegionLocality = new HashMap<>();
for (int i = 0; i < numRegions; i++) {
RegionInfo region = regions.get(i);
for (int j = 0; j < regionSlots; j += slotsPerServer) {
String rack = rackManager.getRack(servers.get(j / slotsPerServer));
Map<RegionInfo, Float> rackLocality = rackRegionLocality.get(rack);
if (rackLocality == null) {
rackLocality = new HashMap<>();
rackRegionLocality.put(rack, rackLocality);
}
Float localityObj = rackLocality.get(region);
float locality = localityObj == null ? 0 : localityObj.floatValue();
locality += localityPerServer[i][j];
rackLocality.put(region, locality);
}
}
for (int i = 0; i < numRegions; i++) {
for (int j = 0; j < regionSlots; j++) {
String rack = rackManager.getRack(servers.get(j / slotsPerServer));
Float totalRackLocalityObj =
rackRegionLocality.get(rack).get(regions.get(i));
float totalRackLocality = totalRackLocalityObj == null ?
0 : totalRackLocalityObj.floatValue();
// Primary cost aims to favor servers with high node locality and low
// rack locality, so that secondaries and tertiaries can be chosen for
// nodes with high rack locality. This might give primaries with
// slightly less locality at first compared to a cost which only
// considers the node locality, but should be better in the long run.
primaryCost[i][j] = 1 - (2 * localityPerServer[i][j] -
totalRackLocality);
// Secondary cost aims to favor servers with high node locality and high
// rack locality since the tertiary will be chosen from the same rack as
// the secondary. This could be negative, but that is okay.
secondaryCost[i][j] = 2 - (localityPerServer[i][j] + totalRackLocality);
// Tertiary cost is only concerned with the node locality. It will later
// be restricted to only hosts on the same rack as the secondary.
tertiaryCost[i][j] = 1 - localityPerServer[i][j];
for (int k = 0; k < slotsPerServer; k++) {
// If we can't find the locality of a region to a server, which occurs
// because locality is only reported for servers which have some
// blocks of a region local, then the locality for that pair is 0.
localityPerServer[i][j * slotsPerServer + k] = locality.floatValue();
}
}
}
if (this.enforceMinAssignmentMove && currentAssignmentMap != null) {
// We want to minimize the number of regions which move as the result of a
// new assignment. Therefore, slightly penalize any placement which is for
// a host that is not currently serving the region.
for (int i = 0; i < numRegions; i++) {
for (int j = 0; j < servers.size(); j++) {
ServerName currentAddress = currentAssignmentMap.get(regions.get(i));
if (currentAddress != null &&
!currentAddress.equals(servers.get(j))) {
for (int k = 0; k < slotsPerServer; k++) {
primaryCost[i][j * slotsPerServer + k] += NOT_CURRENT_HOST_PENALTY;
}
}
}
}
}
// Artificially increase cost of last slot of each server to evenly
// distribute the slop, otherwise there will be a few servers with too few
// regions and many servers with the max number of regions.
// Compute the total rack locality for each region in each rack. The total
// rack locality is the sum of the localities of a region on all servers in
// a rack.
Map<String, Map<RegionInfo, Float>> rackRegionLocality = new HashMap<>();
for (int i = 0; i < numRegions; i++) {
RegionInfo region = regions.get(i);
for (int j = 0; j < regionSlots; j += slotsPerServer) {
primaryCost[i][j] += LAST_SLOT_COST_PENALTY;
secondaryCost[i][j] += LAST_SLOT_COST_PENALTY;
tertiaryCost[i][j] += LAST_SLOT_COST_PENALTY;
String rack = rackManager.getRack(servers.get(j / slotsPerServer));
Map<RegionInfo, Float> rackLocality = rackRegionLocality.get(rack);
if (rackLocality == null) {
rackLocality = new HashMap<>();
rackRegionLocality.put(rack, rackLocality);
}
Float localityObj = rackLocality.get(region);
float locality = localityObj == null ? 0 : localityObj.floatValue();
locality += localityPerServer[i][j];
rackLocality.put(region, locality);
}
}
RandomizedMatrix randomizedMatrix = new RandomizedMatrix(numRegions,
regionSlots);
primaryCost = randomizedMatrix.transform(primaryCost);
int[] primaryAssignment = new MunkresAssignment(primaryCost).solve();
primaryAssignment = randomizedMatrix.invertIndices(primaryAssignment);
// Modify the secondary and tertiary costs for each region/server pair to
// prevent a region from being assigned to the same rack for both primary
// and either one of secondary or tertiary.
for (int i = 0; i < numRegions; i++) {
int slot = primaryAssignment[i];
for (int j = 0; j < regionSlots; j++) {
String rack = rackManager.getRack(servers.get(j / slotsPerServer));
Float totalRackLocalityObj =
rackRegionLocality.get(rack).get(regions.get(i));
float totalRackLocality = totalRackLocalityObj == null ?
0 : totalRackLocalityObj.floatValue();
// Primary cost aims to favor servers with high node locality and low
// rack locality, so that secondaries and tertiaries can be chosen for
// nodes with high rack locality. This might give primaries with
// slightly less locality at first compared to a cost which only
// considers the node locality, but should be better in the long run.
primaryCost[i][j] = 1 - (2 * localityPerServer[i][j] -
totalRackLocality);
// Secondary cost aims to favor servers with high node locality and high
// rack locality since the tertiary will be chosen from the same rack as
// the secondary. This could be negative, but that is okay.
secondaryCost[i][j] = 2 - (localityPerServer[i][j] + totalRackLocality);
// Tertiary cost is only concerned with the node locality. It will later
// be restricted to only hosts on the same rack as the secondary.
tertiaryCost[i][j] = 1 - localityPerServer[i][j];
}
}
}
if (this.enforceMinAssignmentMove && currentAssignmentMap != null) {
// We want to minimize the number of regions which move as the result of a
// new assignment. Therefore, slightly penalize any placement which is for
// a host that is not currently serving the region.
for (int i = 0; i < numRegions; i++) {
for (int j = 0; j < servers.size(); j++) {
ServerName currentAddress = currentAssignmentMap.get(regions.get(i));
if (currentAddress != null &&
!currentAddress.equals(servers.get(j))) {
for (int k = 0; k < slotsPerServer; k++) {
primaryCost[i][j * slotsPerServer + k] += NOT_CURRENT_HOST_PENALTY;
}
}
}
}
}
// Artificially increase cost of last slot of each server to evenly
// distribute the slop, otherwise there will be a few servers with too few
// regions and many servers with the max number of regions.
for (int i = 0; i < numRegions; i++) {
for (int j = 0; j < regionSlots; j += slotsPerServer) {
primaryCost[i][j] += LAST_SLOT_COST_PENALTY;
secondaryCost[i][j] += LAST_SLOT_COST_PENALTY;
tertiaryCost[i][j] += LAST_SLOT_COST_PENALTY;
}
}
RandomizedMatrix randomizedMatrix = new RandomizedMatrix(numRegions,
regionSlots);
primaryCost = randomizedMatrix.transform(primaryCost);
int[] primaryAssignment = new MunkresAssignment(primaryCost).solve();
primaryAssignment = randomizedMatrix.invertIndices(primaryAssignment);
// Modify the secondary and tertiary costs for each region/server pair to
// prevent a region from being assigned to the same rack for both primary
// and either one of secondary or tertiary.
for (int i = 0; i < numRegions; i++) {
int slot = primaryAssignment[i];
String rack = rackManager.getRack(servers.get(slot / slotsPerServer));
for (int k = 0; k < servers.size(); k++) {
if (!rackManager.getRack(servers.get(k)).equals(rack)) {
continue;
}
if (k == slot / slotsPerServer) {
// Same node, do not place secondary or tertiary here ever.
for (int m = 0; m < slotsPerServer; m++) {
secondaryCost[i][k * slotsPerServer + m] = MAX_COST;
tertiaryCost[i][k * slotsPerServer + m] = MAX_COST;
}
} else {
// Same rack, do not place secondary or tertiary here if possible.
for (int m = 0; m < slotsPerServer; m++) {
secondaryCost[i][k * slotsPerServer + m] = AVOID_COST;
tertiaryCost[i][k * slotsPerServer + m] = AVOID_COST;
}
}
}
}
if (munkresForSecondaryAndTertiary) {
randomizedMatrix = new RandomizedMatrix(numRegions, regionSlots);
secondaryCost = randomizedMatrix.transform(secondaryCost);
int[] secondaryAssignment = new MunkresAssignment(secondaryCost).solve();
secondaryAssignment = randomizedMatrix.invertIndices(secondaryAssignment);
// Modify the tertiary costs for each region/server pair to ensure that a
// region is assigned to a tertiary server on the same rack as its secondary
// server, but not the same server in that rack.
for (int i = 0; i < numRegions; i++) {
int slot = secondaryAssignment[i];
String rack = rackManager.getRack(servers.get(slot / slotsPerServer));
for (int k = 0; k < servers.size(); k++) {
if (!rackManager.getRack(servers.get(k)).equals(rack)) {
continue;
}
if (k == slot / slotsPerServer) {
// Same node, do not place secondary or tertiary here ever.
// Same node, do not place tertiary here ever.
for (int m = 0; m < slotsPerServer; m++) {
secondaryCost[i][k * slotsPerServer + m] = MAX_COST;
tertiaryCost[i][k * slotsPerServer + m] = MAX_COST;
}
} else {
// Same rack, do not place secondary or tertiary here if possible.
if (rackManager.getRack(servers.get(k)).equals(rack)) {
continue;
}
// Different rack, do not place tertiary here if possible.
for (int m = 0; m < slotsPerServer; m++) {
secondaryCost[i][k * slotsPerServer + m] = AVOID_COST;
tertiaryCost[i][k * slotsPerServer + m] = AVOID_COST;
}
}
}
}
if (munkresForSecondaryAndTertiary) {
randomizedMatrix = new RandomizedMatrix(numRegions, regionSlots);
secondaryCost = randomizedMatrix.transform(secondaryCost);
int[] secondaryAssignment = new MunkresAssignment(secondaryCost).solve();
secondaryAssignment = randomizedMatrix.invertIndices(secondaryAssignment);
// Modify the tertiary costs for each region/server pair to ensure that a
// region is assigned to a tertiary server on the same rack as its secondary
// server, but not the same server in that rack.
for (int i = 0; i < numRegions; i++) {
int slot = secondaryAssignment[i];
String rack = rackManager.getRack(servers.get(slot / slotsPerServer));
for (int k = 0; k < servers.size(); k++) {
if (k == slot / slotsPerServer) {
// Same node, do not place tertiary here ever.
for (int m = 0; m < slotsPerServer; m++) {
tertiaryCost[i][k * slotsPerServer + m] = MAX_COST;
}
} else {
if (rackManager.getRack(servers.get(k)).equals(rack)) {
continue;
}
// Different rack, do not place tertiary here if possible.
for (int m = 0; m < slotsPerServer; m++) {
tertiaryCost[i][k * slotsPerServer + m] = AVOID_COST;
}
}
}
}
randomizedMatrix = new RandomizedMatrix(numRegions, regionSlots);
tertiaryCost = randomizedMatrix.transform(tertiaryCost);
int[] tertiaryAssignment = new MunkresAssignment(tertiaryCost).solve();
tertiaryAssignment = randomizedMatrix.invertIndices(tertiaryAssignment);
randomizedMatrix = new RandomizedMatrix(numRegions, regionSlots);
tertiaryCost = randomizedMatrix.transform(tertiaryCost);
int[] tertiaryAssignment = new MunkresAssignment(tertiaryCost).solve();
tertiaryAssignment = randomizedMatrix.invertIndices(tertiaryAssignment);
for (int i = 0; i < numRegions; i++) {
List<ServerName> favoredServers
= new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
ServerName s = servers.get(primaryAssignment[i] / slotsPerServer);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
for (int i = 0; i < numRegions; i++) {
List<ServerName> favoredServers = new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
ServerName s = servers.get(primaryAssignment[i] / slotsPerServer);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
s = servers.get(secondaryAssignment[i] / slotsPerServer);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
s = servers.get(secondaryAssignment[i] / slotsPerServer);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
s = servers.get(tertiaryAssignment[i] / slotsPerServer);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
// Update the assignment plan
plan.updateFavoredNodesMap(regions.get(i), favoredServers);
}
LOG.info("Generated the assignment plan for " + numRegions +
" regions from table " + tableName + " with " +
servers.size() + " region servers");
LOG.info("Assignment plan for secondary and tertiary generated " +
"using MunkresAssignment");
} else {
Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
for (int i = 0; i < numRegions; i++) {
primaryRSMap.put(regions.get(i), servers.get(primaryAssignment[i] / slotsPerServer));
}
FavoredNodeAssignmentHelper favoredNodeHelper =
new FavoredNodeAssignmentHelper(servers, conf);
favoredNodeHelper.initialize();
Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
favoredNodeHelper.placeSecondaryAndTertiaryWithRestrictions(primaryRSMap);
for (int i = 0; i < numRegions; i++) {
List<ServerName> favoredServers = new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
RegionInfo currentRegion = regions.get(i);
ServerName s = primaryRSMap.get(currentRegion);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
ServerName[] secondaryAndTertiary =
secondaryAndTertiaryMap.get(currentRegion);
s = secondaryAndTertiary[0];
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
s = secondaryAndTertiary[1];
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
// Update the assignment plan
plan.updateFavoredNodesMap(regions.get(i), favoredServers);
}
LOG.info("Generated the assignment plan for " + numRegions +
" regions from table " + tableName + " with " +
servers.size() + " region servers");
LOG.info("Assignment plan for secondary and tertiary generated " +
"using placeSecondaryAndTertiaryWithRestrictions method");
s = servers.get(tertiaryAssignment[i] / slotsPerServer);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
// Update the assignment plan
plan.updateFavoredNodesMap(regions.get(i), favoredServers);
}
LOG.info("Generated the assignment plan for " + numRegions +
" regions from table " + tableName + " with " +
servers.size() + " region servers");
LOG.info("Assignment plan for secondary and tertiary generated " +
"using MunkresAssignment");
} else {
Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
for (int i = 0; i < numRegions; i++) {
primaryRSMap.put(regions.get(i), servers.get(primaryAssignment[i] / slotsPerServer));
}
FavoredNodeAssignmentHelper favoredNodeHelper =
new FavoredNodeAssignmentHelper(servers, conf);
favoredNodeHelper.initialize();
Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
favoredNodeHelper.placeSecondaryAndTertiaryWithRestrictions(primaryRSMap);
for (int i = 0; i < numRegions; i++) {
List<ServerName> favoredServers
= new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
RegionInfo currentRegion = regions.get(i);
ServerName s = primaryRSMap.get(currentRegion);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
ServerName[] secondaryAndTertiary =
secondaryAndTertiaryMap.get(currentRegion);
s = secondaryAndTertiary[0];
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
s = secondaryAndTertiary[1];
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
// Update the assignment plan
plan.updateFavoredNodesMap(regions.get(i), favoredServers);
}
LOG.info("Generated the assignment plan for " + numRegions +
" regions from table " + tableName + " with " +
servers.size() + " region servers");
LOG.info("Assignment plan for secondary and tertiary generated " +
"using placeSecondaryAndTertiaryWithRestrictions method");
}
}
public FavoredNodesPlan getNewAssignmentPlan() throws IOException {
// Get the current region assignment snapshot by scanning from the META

View File

@ -18,13 +18,16 @@
package org.apache.hadoop.hbase.master.balancer;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.Size;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
/**
* Wrapper class for the few fields required by the {@link StochasticLoadBalancer}
* from the full {@link RegionLoad}.
* from the full {@link RegionMetrics}.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
class BalancerRegionLoad {
private final long readRequestsCount;
@ -32,11 +35,11 @@ class BalancerRegionLoad {
private final int memStoreSizeMB;
private final int storefileSizeMB;
BalancerRegionLoad(RegionLoad regionLoad) {
readRequestsCount = regionLoad.getReadRequestsCount();
writeRequestsCount = regionLoad.getWriteRequestsCount();
memStoreSizeMB = regionLoad.getMemStoreSizeMB();
storefileSizeMB = regionLoad.getStorefileSizeMB();
BalancerRegionLoad(RegionMetrics regionMetrics) {
readRequestsCount = regionMetrics.getReadRequestCount();
writeRequestsCount = regionMetrics.getWriteRequestCount();
memStoreSizeMB = (int) regionMetrics.getMemStoreSize().get(Size.Unit.MEGABYTE);
storefileSizeMB = (int) regionMetrics.getStoreFileSize().get(Size.Unit.MEGABYTE);
}
public long getReadRequestsCount() {

View File

@ -36,7 +36,7 @@ import java.util.stream.Collectors;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
@ -1006,7 +1006,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
private static final Random RANDOM = new Random(System.currentTimeMillis());
private static final Logger LOG = LoggerFactory.getLogger(BaseLoadBalancer.class);
protected MetricsBalancer metricsBalancer = null;
protected ClusterStatus clusterStatus = null;
protected ClusterMetrics clusterStatus = null;
protected ServerName masterServerName;
protected MasterServices services;
protected boolean tablesOnMaster;
@ -1128,10 +1128,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
@Override
public synchronized void setClusterStatus(ClusterStatus st) {
public synchronized void setClusterMetrics(ClusterMetrics st) {
this.clusterStatus = st;
if (useRegionFinder) {
regionFinder.setClusterStatus(st);
regionFinder.setClusterMetrics(st);
}
}

View File

@ -46,7 +46,7 @@ public class ClusterStatusChore extends ScheduledChore {
@Override
protected void chore() {
try {
balancer.setClusterStatus(master.getClusterStatusWithoutCoprocessor());
balancer.setClusterMetrics(master.getClusterMetricsWithoutCoprocessor());
} catch (InterruptedIOException e) {
LOG.warn("Ignoring interruption", e);
}

View File

@ -27,9 +27,8 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@ -64,7 +63,7 @@ class RegionLocationFinder {
private static final long CACHE_TIME = 240 * 60 * 1000;
private static final HDFSBlocksDistribution EMPTY_BLOCK_DISTRIBUTION = new HDFSBlocksDistribution();
private Configuration conf;
private volatile ClusterStatus status;
private volatile ClusterMetrics status;
private MasterServices services;
private final ListeningExecutorService executor;
// Do not scheduleFullRefresh at master startup
@ -105,7 +104,6 @@ class RegionLocationFinder {
/**
* Create a cache for region to list of servers
* @param time time to cache the locations
* @return A new Cache.
*/
private LoadingCache<RegionInfo, HDFSBlocksDistribution> createCache() {
@ -126,7 +124,7 @@ class RegionLocationFinder {
this.services = services;
}
public void setClusterStatus(ClusterStatus status) {
public void setClusterMetrics(ClusterMetrics status) {
long currentTime = EnvironmentEdgeManager.currentTime();
this.status = status;
if (currentTime > lastFullRefresh + (CACHE_TIME / 2)) {
@ -244,7 +242,7 @@ class RegionLocationFinder {
}
List<ServerName> topServerNames = new ArrayList<>();
Collection<ServerName> regionServers = status.getServers();
Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet();
// create a mapping from hostname to ServerName for fast lookup
HashMap<String, List<ServerName>> hostToServerName = new HashMap<>();

View File

@ -27,14 +27,13 @@ import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
@ -226,11 +225,11 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
}
@Override
public synchronized void setClusterStatus(ClusterStatus st) {
super.setClusterStatus(st);
public synchronized void setClusterMetrics(ClusterMetrics st) {
super.setClusterMetrics(st);
updateRegionLoad();
for(CostFromRegionLoadFunction cost : regionLoadFunctions) {
cost.setClusterStatus(st);
cost.setClusterMetrics(st);
}
// update metrics size
@ -527,23 +526,19 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
Map<String, Deque<BalancerRegionLoad>> oldLoads = loads;
loads = new HashMap<>();
for (ServerName sn : clusterStatus.getServers()) {
ServerLoad sl = clusterStatus.getLoad(sn);
if (sl == null) {
continue;
}
for (Entry<byte[], RegionLoad> entry : sl.getRegionsLoad().entrySet()) {
Deque<BalancerRegionLoad> rLoads = oldLoads.get(Bytes.toString(entry.getKey()));
clusterStatus.getLiveServerMetrics().forEach((ServerName sn, ServerMetrics sm) -> {
sm.getRegionMetrics().forEach((byte[] regionName, RegionMetrics rm) -> {
Deque<BalancerRegionLoad> rLoads = oldLoads.get(Bytes.toString(regionName));
if (rLoads == null) {
// There was nothing there
rLoads = new ArrayDeque<>();
} else if (rLoads.size() >= numRegionLoadsToRemember) {
rLoads.remove();
}
rLoads.add(new BalancerRegionLoad(entry.getValue()));
loads.put(Bytes.toString(entry.getKey()), rLoads);
}
}
rLoads.add(new BalancerRegionLoad(rm));
loads.put(Bytes.toString(regionName), rLoads);
});
});
for(CostFromRegionLoadFunction cost : regionLoadFunctions) {
cost.setLoads(loads);
@ -1371,14 +1366,14 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
*/
abstract static class CostFromRegionLoadFunction extends CostFunction {
private ClusterStatus clusterStatus = null;
private ClusterMetrics clusterStatus = null;
private Map<String, Deque<BalancerRegionLoad>> loads = null;
private double[] stats = null;
CostFromRegionLoadFunction(Configuration conf) {
super(conf);
}
void setClusterStatus(ClusterStatus status) {
void setClusterMetrics(ClusterMetrics status) {
this.clusterStatus = status;
}

View File

@ -22,7 +22,6 @@ import com.google.protobuf.Message;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import java.io.IOException;
import java.net.InetAddress;
import java.security.PrivilegedExceptionAction;
@ -38,7 +37,6 @@ import java.util.Optional;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ArrayBackedTag;
import org.apache.hadoop.hbase.Cell;

View File

@ -51,8 +51,8 @@ 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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
@ -1168,8 +1168,7 @@ public final class Canary implements Tool {
private void checkWriteTableDistribution() throws IOException {
if (!admin.tableExists(writeTableName)) {
int numberOfServers =
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()
.size();
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().size();
if (numberOfServers == 0) {
throw new IllegalStateException("No live regionservers");
}
@ -1180,10 +1179,10 @@ public final class Canary implements Tool {
admin.enableTable(writeTableName);
}
ClusterStatus status =
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER));
int numberOfServers = status.getServersSize();
if (status.getServers().contains(status.getMaster())) {
ClusterMetrics status =
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER));
int numberOfServers = status.getLiveServerMetrics().size();
if (status.getLiveServerMetrics().containsKey(status.getMasterName())) {
numberOfServers -= 1;
}
@ -1502,8 +1501,8 @@ public final class Canary implements Tool {
}
// get any live regionservers not serving any regions
for (ServerName rs : this.admin
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
for (ServerName rs : this.admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
String rsName = rs.getHostname();
if (!rsAndRMap.containsKey(rsName)) {
rsAndRMap.put(rsName, Collections.<RegionInfo> emptyList());

View File

@ -72,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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
@ -224,7 +224,7 @@ public class HBaseFsck extends Configured implements Closeable {
* Internal resources
**********************/
private static final Logger LOG = LoggerFactory.getLogger(HBaseFsck.class.getName());
private ClusterStatus status;
private ClusterMetrics status;
private ClusterConnection connection;
private Admin admin;
private Table meta;
@ -525,7 +525,7 @@ public class HBaseFsck extends Configured implements Closeable {
connection = (ClusterConnection)ConnectionFactory.createConnection(getConf());
admin = connection.getAdmin();
meta = connection.getTable(TableName.META_TABLE_NAME);
status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS,
status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS,
Option.DEAD_SERVERS, Option.MASTER, Option.BACKUP_MASTERS,
Option.REGIONS_IN_TRANSITION, Option.HBASE_VERSION));
}
@ -535,7 +535,7 @@ public class HBaseFsck extends Configured implements Closeable {
*/
private void loadDeployedRegions() throws IOException, InterruptedException {
// From the master, get a list of all known live region servers
Collection<ServerName> regionServers = status.getServers();
Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet();
errors.print("Number of live region servers: " + regionServers.size());
if (details) {
for (ServerName rsinfo: regionServers) {
@ -553,10 +553,10 @@ public class HBaseFsck extends Configured implements Closeable {
}
// Print the current master name and state
errors.print("Master: " + status.getMaster());
errors.print("Master: " + status.getMasterName());
// Print the list of all backup masters
Collection<ServerName> backupMasters = status.getBackupMasters();
Collection<ServerName> backupMasters = status.getBackupMasterNames();
errors.print("Number of backup masters: " + backupMasters.size());
if (details) {
for (ServerName name: backupMasters) {
@ -566,7 +566,7 @@ public class HBaseFsck extends Configured implements Closeable {
errors.print("Average load: " + status.getAverageLoad());
errors.print("Number of requests: " + status.getRequestCount());
errors.print("Number of regions: " + status.getRegionsCount());
errors.print("Number of regions: " + status.getRegionCount());
List<RegionState> rits = status.getRegionStatesInTransition();
errors.print("Number of regions in transition: " + rits.size());
@ -2451,7 +2451,8 @@ public class HBaseFsck extends Configured implements Closeable {
LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers(), numReplicas);
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet(), numReplicas);
tryAssignmentRepair(hbi, "Trying to reassign region...");
}
@ -2478,7 +2479,8 @@ public class HBaseFsck extends Configured implements Closeable {
LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers(), numReplicas);
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet(), numReplicas);
tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
}

View File

@ -118,7 +118,7 @@ public class HBaseFsckRepair {
while (EnvironmentEdgeManager.currentTime() < expiration) {
try {
boolean inTransition = false;
for (RegionState rs : admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
for (RegionState rs : admin.getClusterMetrics(EnumSet.of(Option.REGIONS_IN_TRANSITION))
.getRegionStatesInTransition()) {
if (RegionInfo.COMPARATOR.compare(rs.getRegion(), region) == 0) {
inTransition = true;

View File

@ -747,7 +747,7 @@ public class RegionMover extends AbstractHBaseTool {
* @throws IOException
*/
private void stripMaster(ArrayList<String> regionServers, Admin admin) throws IOException {
ServerName master = admin.getClusterStatus(EnumSet.of(Option.MASTER)).getMaster();
ServerName master = admin.getClusterMetrics(EnumSet.of(Option.MASTER)).getMasterName();
String masterHostname = master.getHostname();
int masterPort = master.getPort();
try {
@ -825,7 +825,7 @@ public class RegionMover extends AbstractHBaseTool {
*/
private ArrayList<String> getServers(Admin admin) throws IOException {
ArrayList<ServerName> serverInfo = new ArrayList<>(
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet());
ArrayList<String> regionServers = new ArrayList<>(serverInfo.size());
for (ServerName server : serverInfo) {
regionServers.add(server.getServerName().toLowerCase());

View File

@ -41,8 +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.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -437,8 +437,8 @@ public class RegionSplitter {
*/
private static int getRegionServerCount(final Connection connection) throws IOException {
try (Admin admin = connection.getAdmin()) {
ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
Collection<ServerName> servers = status.getServers();
ClusterMetrics status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
Collection<ServerName> servers = status.getLiveServerMetrics().keySet();
return servers == null || servers.isEmpty()? 0: servers.size();
}
}

View File

@ -19,16 +19,16 @@ package org.apache.hadoop.hbase;
import java.io.Closeable;
import java.io.IOException;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.util.Threads;
/**
* This class defines methods that can help with managing HBase clusters
@ -61,7 +61,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
protected Configuration conf;
/** the status of the cluster before we begin */
protected ClusterStatus initialClusterStatus;
protected ClusterMetrics initialClusterStatus;
/**
* Construct an HBaseCluster
@ -82,16 +82,16 @@ public abstract class HBaseCluster implements Closeable, Configurable {
}
/**
* Returns a ClusterStatus for this HBase cluster.
* @see #getInitialClusterStatus()
* Returns a ClusterMetrics for this HBase cluster.
* @see #getInitialClusterMetrics()
*/
public abstract ClusterStatus getClusterStatus() throws IOException;
public abstract ClusterMetrics getClusterMetrics() throws IOException;
/**
* Returns a ClusterStatus for this HBase cluster as observed at the
* starting of the HBaseCluster
*/
public ClusterStatus getInitialClusterStatus() throws IOException {
public ClusterMetrics getInitialClusterMetrics() throws IOException {
return initialClusterStatus;
}
@ -153,7 +153,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
throws IOException {
long start = System.currentTimeMillis();
while ((System.currentTimeMillis() - start) < timeout) {
for (ServerName server : getClusterStatus().getServers()) {
for (ServerName server : getClusterMetrics().getLiveServerMetrics().keySet()) {
if (server.getHostname().equals(hostname) && server.getPort() == port) {
return;
}
@ -317,7 +317,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
* @return whether restoration is complete
*/
public boolean restoreInitialStatus() throws IOException {
return restoreClusterStatus(getInitialClusterStatus());
return restoreClusterMetrics(getInitialClusterMetrics());
}
/**
@ -327,7 +327,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
* permissions, etc. restoration might be partial.
* @return whether restoration is complete
*/
public boolean restoreClusterStatus(ClusterStatus desiredStatus) throws IOException {
public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOException {
return true;
}

View File

@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
@ -52,7 +51,6 @@ import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.logging.impl.Jdk14Logger;
@ -151,6 +149,7 @@ import org.apache.zookeeper.ZooKeeper.States;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.impl.Log4jLoggerAdapter;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
/**
@ -3735,7 +3734,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
// The number of splits is set as:
// region servers * regions per region server).
int numberOfServers =
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics()
.size();
if (numberOfServers == 0) {
throw new IllegalStateException("No live regionservers");

View File

@ -23,19 +23,10 @@ import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
@ -45,6 +36,14 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
/**
* This class creates a single process HBase cluster.
@ -629,10 +628,20 @@ public class MiniHBaseCluster extends HBaseCluster {
public void close() throws IOException {
}
@Override
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getClusterMetrics()} instead.
*/
@Deprecated
public ClusterStatus getClusterStatus() throws IOException {
HMaster master = getMaster();
return master == null ? null : master.getClusterStatus();
return master == null ? null : new ClusterStatus(master.getClusterMetrics());
}
@Override
public ClusterMetrics getClusterMetrics() throws IOException {
HMaster master = getMaster();
return master == null ? null : master.getClusterMetrics();
}
/**

View File

@ -23,7 +23,6 @@ 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;
@ -77,8 +76,8 @@ public class TestClientClusterMetrics {
@Test
public void testDefaults() throws Exception {
ClusterMetrics origin = ADMIN.getClusterStatus();
ClusterMetrics defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
ClusterMetrics origin = ADMIN.getClusterMetrics();
ClusterMetrics defaults = ADMIN.getClusterMetrics(EnumSet.allOf(Option.class));
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
Assert.assertEquals(origin.getAverageLoad(), defaults.getAverageLoad(), 0);
@ -96,10 +95,10 @@ public class TestClientClusterMetrics {
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));
CompletableFuture<ClusterMetrics> originFuture =
asyncAdmin.getClusterMetrics();
CompletableFuture<ClusterMetrics> defaultsFuture =
asyncAdmin.getClusterMetrics(EnumSet.allOf(Option.class));
ClusterMetrics origin = originFuture.get();
ClusterMetrics defaults = defaultsFuture.get();
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
@ -133,14 +132,14 @@ public class TestClientClusterMetrics {
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));
ClusterMetrics metrics = ADMIN.getClusterMetrics(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);
ClusterMetrics metrics = ADMIN.getClusterMetrics(options);
Assert.assertNotNull(metrics);
// exclude a dead region server
Assert.assertEquals(SLAVES -1, numRs);
@ -176,7 +175,7 @@ public class TestClientClusterMetrics {
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);
ClusterMetrics metrics = ADMIN.getClusterMetrics(options);
Assert.assertTrue(metrics.getMasterName().equals(activeName));
Assert.assertEquals(MASTERS - 1, metrics.getBackupMasterNames().size());
}
@ -186,7 +185,7 @@ public class TestClientClusterMetrics {
EnumSet<Option> options =
EnumSet.of(Option.MASTER_COPROCESSORS, Option.HBASE_VERSION,
Option.CLUSTER_ID, Option.BALANCER_ON);
ClusterMetrics metrics = ADMIN.getClusterStatus(options);
ClusterMetrics metrics = ADMIN.getClusterMetrics(options);
Assert.assertEquals(1, metrics.getMasterCoprocessorNames().size());
Assert.assertNotNull(metrics.getHBaseVersion());
Assert.assertNotNull(metrics.getClusterId());
@ -206,7 +205,7 @@ public class TestClientClusterMetrics {
public void testObserver() throws IOException {
int preCount = MyObserver.PRE_COUNT.get();
int postCount = MyObserver.POST_COUNT.get();
Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
Assert.assertTrue(ADMIN.getClusterMetrics().getMasterCoprocessorNames().stream()
.anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
@ -220,13 +219,13 @@ public class TestClientClusterMetrics {
return Optional.of(this);
}
@Override public void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
@Override public void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
PRE_COUNT.incrementAndGet();
}
@Override public void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
ClusterStatus metrics) throws IOException {
@Override public void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
ClusterMetrics metrics) throws IOException {
POST_COUNT.incrementAndGet();
}
}

View File

@ -21,16 +21,12 @@ 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;
@ -81,7 +77,8 @@ public class TestClientClusterStatus {
@Test
public void testDefaults() throws Exception {
ClusterStatus origin = ADMIN.getClusterStatus();
ClusterStatus defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
ClusterStatus defaults
= new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)));
checkPbObjectNotNull(origin);
checkPbObjectNotNull(defaults);
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
@ -97,36 +94,15 @@ public class TestClientClusterStatus {
@Test
public void testNone() throws Exception {
ClusterStatus status0 = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
ClusterStatus status1 = ADMIN.getClusterStatus(EnumSet.noneOf(Option.class));
ClusterStatus status0
= new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)));
ClusterStatus status1
= new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class)));
Assert.assertEquals(status0, status1);
checkPbObjectNotNull(status0);
checkPbObjectNotNull(status1);
}
@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));
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());
}
}
@Test
public void testLiveAndDeadServersStatus() throws Exception {
// Count the number of live regionservers
@ -143,14 +119,15 @@ public class TestClientClusterStatus {
Waiter.waitFor(CLUSTER.getConfiguration(), 10 * 1000, 100, new Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
ClusterStatus status = ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
ClusterStatus status
= new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)));
Assert.assertNotNull(status);
return status.getRegionsCount() > 0;
}
});
// Retrieve live servers and dead servers info.
EnumSet<Option> options = EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS);
ClusterStatus status = ADMIN.getClusterStatus(options);
ClusterStatus status = new ClusterStatus(ADMIN.getClusterMetrics(options));
checkPbObjectNotNull(status);
Assert.assertNotNull(status);
Assert.assertNotNull(status.getServers());
@ -188,7 +165,7 @@ public class TestClientClusterStatus {
Assert.assertEquals(MASTERS, masterThreads.size());
// Retrieve master and backup masters infos only.
EnumSet<Option> options = EnumSet.of(Option.MASTER, Option.BACKUP_MASTERS);
ClusterStatus status = ADMIN.getClusterStatus(options);
ClusterStatus status = new ClusterStatus(ADMIN.getClusterMetrics(options));
Assert.assertTrue(status.getMaster().equals(activeName));
Assert.assertEquals(MASTERS - 1, status.getBackupMastersSize());
}
@ -198,7 +175,7 @@ public class TestClientClusterStatus {
EnumSet<Option> options =
EnumSet.of(Option.MASTER_COPROCESSORS, Option.HBASE_VERSION,
Option.CLUSTER_ID, Option.BALANCER_ON);
ClusterStatus status = ADMIN.getClusterStatus(options);
ClusterStatus status = new ClusterStatus(ADMIN.getClusterMetrics(options));
Assert.assertTrue(status.getMasterCoprocessors().length == 1);
Assert.assertNotNull(status.getHBaseVersion());
Assert.assertNotNull(status.getClusterId());
@ -245,13 +222,13 @@ public class TestClientClusterStatus {
return Optional.of(this);
}
@Override public void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
@Override public void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
PRE_COUNT.incrementAndGet();
}
@Override public void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
ClusterStatus status) throws IOException {
@Override public void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
ClusterMetrics status) throws IOException {
POST_COUNT.incrementAndGet();
}
}

View File

@ -28,6 +28,8 @@ import java.util.Collection;
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.ClusterMetrics.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table;
@ -83,9 +85,10 @@ public class TestRegionLoad {
// Check if regions match with the regionLoad from the server
for (ServerName serverName : admin
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()) {
List<HRegionInfo> regions = admin.getOnlineRegions(serverName);
Collection<RegionLoad> regionLoads = admin.getRegionLoad(serverName).values();
Collection<RegionLoad> regionLoads = admin.getRegionMetrics(serverName)
.stream().map(r -> new RegionLoad(r)).collect(Collectors.toList());
checkRegionsAndRegionLoads(regions, regionLoads);
}
@ -95,17 +98,23 @@ public class TestRegionLoad {
List<RegionLoad> regionLoads = Lists.newArrayList();
for (ServerName serverName : admin
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
regionLoads.addAll(admin.getRegionLoad(serverName, table).values());
.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()) {
regionLoads.addAll(admin.getRegionMetrics(serverName, table)
.stream().map(r -> new RegionLoad(r)).collect(Collectors.toList()));
}
checkRegionsAndRegionLoads(tableRegions, regionLoads);
}
// Check RegionLoad matches the regionLoad from ClusterStatus
ClusterStatus clusterStatus = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
ClusterStatus clusterStatus
= new ClusterStatus(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)));
for (ServerName serverName : clusterStatus.getServers()) {
ServerLoad serverLoad = clusterStatus.getLoad(serverName);
Map<byte[], RegionLoad> regionLoads = admin.getRegionLoad(serverName);
Map<byte[], RegionLoad> regionLoads = admin.getRegionMetrics(serverName).stream()
.collect(Collectors.toMap(e -> e.getRegionName(), e -> new RegionLoad(e),
(v1, v2) -> {
throw new RuntimeException("impossible!!");
}, () -> new TreeMap<>(Bytes.BYTES_COMPARATOR)));
compareRegionLoads(serverLoad.getRegionsLoad(), regionLoads);
}
}

View File

@ -27,7 +27,6 @@ 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;
@ -79,11 +78,11 @@ public class TestRegionMetrics {
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))
for (ServerName serverName : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
List<RegionInfo> regions = admin.getRegions(serverName);
Collection<RegionMetrics> regionMetricsList =
admin.getRegionLoads(serverName).stream().collect(Collectors.toList());
admin.getRegionMetrics(serverName);
checkRegionsAndRegionMetrics(regions, regionMetricsList);
}
@ -92,21 +91,20 @@ public class TestRegionMetrics {
List<RegionInfo> tableRegions = admin.getRegions(table);
List<RegionMetrics> regionMetrics = new ArrayList<>();
for (ServerName serverName : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
for (ServerName serverName : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
regionMetrics.addAll(admin.getRegionLoads(serverName, table));
regionMetrics.addAll(admin.getRegionMetrics(serverName, table));
}
checkRegionsAndRegionMetrics(tableRegions, regionMetrics);
}
// Check RegionMetrics matches the RegionMetrics from ClusterStatus
ClusterMetrics clusterStatus = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
ClusterMetrics clusterStatus = admin.getClusterMetrics(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());
List<RegionMetrics> regionMetrics = admin.getRegionMetrics(serverName);
assertEquals(serverMetrics.getRegionMetrics().size(), regionMetrics.size());
}
}

View File

@ -20,13 +20,6 @@ package org.apache.hadoop.hbase;
import static org.junit.Assert.assertTrue;
import javax.management.MBeanAttributeInfo;
import javax.management.MBeanInfo;
import javax.management.MBeanServerConnection;
import javax.management.ObjectInstance;
import javax.management.ObjectName;
import javax.management.remote.JMXConnector;
import javax.management.remote.JMXConnectorFactory;
import java.io.IOException;
import java.util.HashSet;
import java.util.Hashtable;
@ -35,7 +28,13 @@ import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import javax.management.MBeanAttributeInfo;
import javax.management.MBeanInfo;
import javax.management.MBeanServerConnection;
import javax.management.ObjectInstance;
import javax.management.ObjectName;
import javax.management.remote.JMXConnector;
import javax.management.remote.JMXConnectorFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@ -156,7 +155,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
conf.setBoolean(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, true);
loadBalancer.setConf(conf);
// NOTE the size is normally set in setClusterStatus, for test purpose, we set it manually
// NOTE the size is normally set in setClusterMetrics, for test purpose, we set it manually
// Tables: hbase:namespace, table1, table2
// Functions: costFunctions, overall
String[] functionNames = loadBalancer.getCostFunctionNames();

View File

@ -698,7 +698,8 @@ public class TestAdmin2 {
TEST_UTIL.createMultiRegionTable(tableName, "f".getBytes(), 6);
ArrayList<ServerName> clusterRegionServers =
new ArrayList<>(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
new ArrayList<>(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet());
assertEquals(clusterRegionServers.size(), 3);

View File

@ -32,12 +32,12 @@ import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -244,38 +244,43 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
Collection<ServerName> servers = admin.getRegionServers().get();
for (ServerName serverName : servers) {
List<RegionInfo> regions = admin.getRegions(serverName).get();
checkRegionsAndRegionLoads(regions, admin.getRegionLoads(serverName).get());
checkRegionsAndRegionLoads(regions, admin.getRegionMetrics(serverName).get());
}
// Check if regionLoad matches the table's regions and nothing is missed
for (TableName table : tables) {
List<RegionInfo> tableRegions = admin.getRegions(table).get();
List<RegionLoad> regionLoads = Lists.newArrayList();
List<RegionMetrics> regionLoads = Lists.newArrayList();
for (ServerName serverName : servers) {
regionLoads.addAll(admin.getRegionLoads(serverName, table).get());
regionLoads.addAll(admin.getRegionMetrics(serverName, table).get());
}
checkRegionsAndRegionLoads(tableRegions, regionLoads);
}
// Check RegionLoad matches the regionLoad from ClusterStatus
ClusterStatus clusterStatus = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).get();
for (ServerName serverName : clusterStatus.getServers()) {
ServerLoad serverLoad = clusterStatus.getLoad(serverName);
compareRegionLoads(serverLoad.getRegionsLoad().values(), admin.getRegionLoads(serverName)
.get());
ClusterMetrics clusterStatus = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).get();
for (Map.Entry<ServerName, ServerMetrics> entry :
clusterStatus.getLiveServerMetrics().entrySet()) {
ServerName sn = entry.getKey();
ServerMetrics sm = entry.getValue();
compareRegionLoads(sm.getRegionMetrics().values(), admin.getRegionMetrics(sn).get());
}
for (ServerName serverName : clusterStatus.getLiveServerMetrics().keySet()) {
ServerMetrics serverLoad = clusterStatus.getLiveServerMetrics().get(serverName);
}
}
private void compareRegionLoads(Collection<RegionLoad> regionLoadCluster,
Collection<RegionLoad> regionLoads) {
private void compareRegionLoads(Collection<RegionMetrics> regionLoadCluster,
Collection<RegionMetrics> regionLoads) {
assertEquals("No of regionLoads from clusterStatus and regionloads from RS doesn't match",
regionLoadCluster.size(), regionLoads.size());
for (RegionLoad loadCluster : regionLoadCluster) {
for (RegionMetrics loadCluster : regionLoadCluster) {
boolean matched = false;
for (RegionLoad load : regionLoads) {
if (Bytes.equals(loadCluster.getName(), load.getName())) {
for (RegionMetrics load : regionLoads) {
if (Bytes.equals(loadCluster.getRegionName(), load.getRegionName())) {
matched = true;
continue;
}
@ -285,13 +290,13 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
}
private void checkRegionsAndRegionLoads(Collection<RegionInfo> regions,
Collection<RegionLoad> regionLoads) {
Collection<RegionMetrics> regionLoads) {
assertEquals("No of regions and regionloads doesn't match", regions.size(), regionLoads.size());
Map<byte[], RegionLoad> regionLoadMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
for (RegionLoad regionLoad : regionLoads) {
regionLoadMap.put(regionLoad.getName(), regionLoad);
Map<byte[], RegionMetrics> regionLoadMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
for (RegionMetrics regionLoad : regionLoads) {
regionLoadMap.put(regionLoad.getRegionName(), regionLoad);
}
for (RegionInfo info : regions) {
assertTrue("Region not in regionLoadMap region:" + info.getRegionNameAsString()

View File

@ -46,7 +46,8 @@ public class TestAsyncDecommissionAdminApi extends TestAsyncAdminBase {
TEST_UTIL.createMultiRegionTable(tableName, FAMILY, 4);
ArrayList<ServerName> clusterRegionServers =
new ArrayList<>(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).get().getServers());
new ArrayList<>(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).get()
.getLiveServerMetrics().keySet());
assertEquals(clusterRegionServers.size(), 2);

View File

@ -44,7 +44,6 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
@ -58,10 +57,10 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeepDeletedCells;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@ -4261,8 +4260,8 @@ public class TestFromClientSide {
boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
try (Admin admin = conn.getAdmin()) {
assertTrue(admin.tableExists(tableName));
assertTrue(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getServersSize() == SLAVES + (tablesOnMaster ? 1 : 0));
assertTrue(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().size() == SLAVES + (tablesOnMaster ? 1 : 0));
}
}

View File

@ -23,13 +23,13 @@ import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
import java.util.concurrent.ExecutorService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
@ -68,8 +68,6 @@ import org.junit.rules.TestRule;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import edu.umd.cs.findbugs.annotations.Nullable;
/**
* Tests the scenarios where replicas are enabled for the meta table
*/
@ -207,7 +205,7 @@ public class TestMetaWithReplicas {
Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
30000) * 3);
}
master = util.getHBaseClusterInterface().getClusterStatus().getMaster();
master = util.getHBaseClusterInterface().getClusterMetrics().getMasterName();
// kill the master so that regionserver recovery is not triggered at all
// for the meta server
util.getHBaseClusterInterface().stopMaster(master);
@ -276,7 +274,7 @@ public class TestMetaWithReplicas {
private void stopMasterAndValidateReplicaCount(final int originalReplicaCount,
final int newReplicaCount)
throws Exception {
ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics().getMasterName();
TEST_UTIL.getHBaseClusterInterface().stopMaster(sn);
TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(sn, 60000);
List<String> metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
@ -409,7 +407,7 @@ public class TestMetaWithReplicas {
byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
ServerName currentServer = ProtobufUtil.toServerName(data);
Collection<ServerName> liveServers = TEST_UTIL.getAdmin()
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet();
ServerName moveToServer = null;
for (ServerName s : liveServers) {
if (!currentServer.equals(s)) {

View File

@ -336,7 +336,7 @@ public class TestMultiParallel {
public boolean evaluate() throws Exception {
// Master is also a regionserver, so the count is liveRScount
return UTIL.getMiniHBaseCluster().getMaster()
.getClusterStatus().getServersSize() == liveRScount;
.getClusterMetrics().getLiveServerMetrics().size() == liveRScount;
}
});
UTIL.waitFor(15 * 1000, UTIL.predicateNoRegionsInTransition());

View File

@ -29,7 +29,6 @@ import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -70,6 +69,7 @@ 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;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
@ -1523,7 +1523,7 @@ public class TestMasterObserver {
assertNotNull("Found a non-null entry", firstGoodPair);
LOG.info("Found " + firstGoodPair.toString());
// Try to force a move
Collection<ServerName> servers = master.getClusterStatus().getServers();
Collection<ServerName> servers = master.getClusterMetrics().getLiveServerMetrics().keySet();
String destName = null;
String serverNameForFirstRegion = firstGoodPair.getServerName().toString();
LOG.info("serverNameForFirstRegion=" + serverNameForFirstRegion);

View File

@ -23,9 +23,8 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.util.List;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
@ -97,10 +96,9 @@ public class TestMasterFailover {
// Check that ClusterStatus reports the correct active and backup masters
assertNotNull(active);
ClusterStatus status = active.getClusterStatus();
assertTrue(status.getMaster().equals(activeName));
assertEquals(2, status.getBackupMastersSize());
assertEquals(2, status.getBackupMasters().size());
ClusterMetrics status = active.getClusterMetrics();
assertTrue(status.getMasterName().equals(activeName));
assertEquals(2, status.getBackupMasterNames().size());
// attempt to stop one of the inactive masters
int backupIndex = (activeIndex == 0 ? 1 : activeIndex - 1);
@ -119,17 +117,17 @@ public class TestMasterFailover {
}
assertEquals(1, numActive);
assertEquals(2, masterThreads.size());
int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
int rsCount = masterThreads.get(activeIndex).getMaster().getClusterMetrics()
.getLiveServerMetrics().size();
LOG.info("Active master " + active.getServerName() + " managing " + rsCount +
" regions servers");
assertEquals(3, rsCount);
// Check that ClusterStatus reports the correct active and backup masters
assertNotNull(active);
status = active.getClusterStatus();
assertTrue(status.getMaster().equals(activeName));
assertEquals(1, status.getBackupMastersSize());
assertEquals(1, status.getBackupMasters().size());
status = active.getClusterMetrics();
assertTrue(status.getMasterName().equals(activeName));
assertEquals(1, status.getBackupMasterNames().size());
// kill the active master
LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
@ -146,13 +144,12 @@ public class TestMasterFailover {
// and he should be active
active = masterThreads.get(0).getMaster();
assertNotNull(active);
status = active.getClusterStatus();
ServerName mastername = status.getMaster();
status = active.getClusterMetrics();
ServerName mastername = status.getMasterName();
assertTrue(mastername.equals(active.getServerName()));
assertTrue(active.isActiveMaster());
assertEquals(0, status.getBackupMastersSize());
assertEquals(0, status.getBackupMasters().size());
int rss = status.getServersSize();
assertEquals(0, status.getBackupMasterNames().size());
int rss = status.getLiveServerMetrics().size();
LOG.info("Active master " + mastername.getServerName() + " managing " +
rss + " region servers");
assertEquals(3, rss);

View File

@ -18,7 +18,12 @@
*/
package org.apache.hadoop.hbase.master;
import org.apache.hadoop.hbase.ClusterStatus;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@ -28,12 +33,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
import java.util.List;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@Category({MasterTests.class, LargeTests.class})
public class TestMasterFailoverBalancerPersistence {
@ -57,14 +56,14 @@ public class TestMasterFailoverBalancerPersistence {
assertTrue(cluster.waitForActiveAndReadyMaster());
HMaster active = cluster.getMaster();
// check that the balancer is on by default for the active master
ClusterStatus clusterStatus = active.getClusterStatus();
assertTrue(clusterStatus.isBalancerOn());
ClusterMetrics clusterStatus = active.getClusterMetrics();
assertTrue(clusterStatus.getBalancerOn());
active = killActiveAndWaitForNewActive(cluster);
// ensure the load balancer is still running on new master
clusterStatus = active.getClusterStatus();
assertTrue(clusterStatus.isBalancerOn());
clusterStatus = active.getClusterMetrics();
assertTrue(clusterStatus.getBalancerOn());
// turn off the load balancer
active.balanceSwitch(false);
@ -73,8 +72,8 @@ public class TestMasterFailoverBalancerPersistence {
active = killActiveAndWaitForNewActive(cluster);
// ensure the load balancer is not running on the new master
clusterStatus = active.getClusterStatus();
assertFalse(clusterStatus.isBalancerOn());
clusterStatus = active.getClusterMetrics();
assertFalse(clusterStatus.getBalancerOn());
// Stop the cluster
TEST_UTIL.shutdownMiniCluster();

View File

@ -84,8 +84,8 @@ public class TestMasterOperationsForRegionReplicas {
TEST_UTIL.startMiniCluster(numSlaves);
CONNECTION = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
ADMIN = CONNECTION.getAdmin();
while(ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getServers().size() < numSlaves) {
while(ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().size() < numSlaves) {
Thread.sleep(100);
}
}
@ -163,7 +163,7 @@ public class TestMasterOperationsForRegionReplicas {
ADMIN.getConnection());
// Now kill the master, restart it and see if the assignments are kept
ServerName master = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
ServerName master = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics().getMasterName();
TEST_UTIL.getHBaseClusterInterface().stopMaster(master);
TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(master, 30000);
TEST_UTIL.getHBaseClusterInterface().startMaster(master.getHostname(), master.getPort());

View File

@ -24,9 +24,8 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LocalHBaseCluster;
@ -84,9 +83,8 @@ public class TestMasterShutdown {
}
assertNotNull(active);
// make sure the other two are backup masters
ClusterStatus status = active.getClusterStatus();
assertEquals(2, status.getBackupMastersSize());
assertEquals(2, status.getBackupMasters().size());
ClusterMetrics status = active.getClusterMetrics();
assertEquals(2, status.getBackupMasterNames().size());
// tell the active master to shutdown the cluster
active.shutdown();

View File

@ -28,8 +28,8 @@ import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@ -169,14 +169,14 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
TEST_UTIL.getHBaseCluster().startRegionServerAndWait(60000);
Map<ServerName, List<RegionInfo>> serverAssignments = Maps.newHashMap();
ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
for (ServerName sn : status.getServers()) {
ClusterMetrics status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
for (ServerName sn : status.getLiveServerMetrics().keySet()) {
if (!ServerName.isSameAddress(sn, masterServerName)) {
serverAssignments.put(sn, getTableRegionsFromServer(tableName, sn));
}
}
RegionLocationFinder regionFinder = new RegionLocationFinder();
regionFinder.setClusterStatus(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)));
regionFinder.setClusterMetrics(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)));
regionFinder.setConf(conf);
regionFinder.setServices(TEST_UTIL.getMiniHBaseCluster().getMaster());
Cluster cluster = new Cluster(serverAssignments, null, regionFinder, new RackManager(conf));

View File

@ -30,6 +30,7 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -154,7 +155,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
regions.addAll(admin.getTableRegions(TableName.META_TABLE_NAME));
regions.addAll(admin.getTableRegions(TableName.NAMESPACE_TABLE_NAME));
List<ServerName> servers = Lists.newArrayList(
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet());
Map<ServerName, List<RegionInfo>> map = balancer.roundRobinAssignment(regions, servers);
for (List<RegionInfo> regionInfos : map.values()) {
regions.removeAll(regionInfos);
@ -181,11 +182,11 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
}
Map<ServerName, List<Integer>> replicaLoadMap = fnm.getReplicaLoad(
Lists.newArrayList(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getServers()));
Lists.newArrayList(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()));
assertTrue("Not all replica load collected.",
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getServers().size() == replicaLoadMap.size());
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().size() == replicaLoadMap.size());
for (Entry<ServerName, List<Integer>> entry : replicaLoadMap.entrySet()) {
assertTrue(entry.getValue().size() == FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
assertTrue(entry.getValue().get(0) >= 0);
@ -196,10 +197,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.disableTable(TableName.valueOf(tableName));
admin.deleteTable(TableName.valueOf(tableName));
replicaLoadMap = fnm.getReplicaLoad(Lists.newArrayList(
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()));
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()));
assertTrue("replica load found " + replicaLoadMap.size() + " instead of 0.",
replicaLoadMap.size() == admin
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()
.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics()
.size());
}
@ -220,7 +221,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
LoadBalancer balancer = master.getLoadBalancer();
ServerName destination = balancer.randomAssignment(hri, Lists.newArrayList(admin
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()));
.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics()
.keySet().stream().collect(Collectors.toList())));
assertNotNull(destination);
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
assertNotNull(favoredNodes);
@ -286,7 +288,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
assertNotNull(currentFN);
List<ServerName> serversForNewFN = Lists.newArrayList();
for (ServerName sn : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
for (ServerName sn : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
serversForNewFN.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), NON_STARTCODE));
}
for (ServerName sn : currentFN) {
@ -386,7 +389,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
// Regenerate FN and assign, everything else should be fine
List<ServerName> serversForNewFN = Lists.newArrayList();
for (ServerName sn : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
for (ServerName sn : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
serversForNewFN.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), NON_STARTCODE));
}
@ -480,7 +484,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
// Regenerate FN and assign, everything else should be fine
List<ServerName> serversForNewFN = Lists.newArrayList();
for (ServerName sn : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
for (ServerName sn : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet()) {
serversForNewFN.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), NON_STARTCODE));
}

View File

@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@ -69,7 +68,7 @@ public class TestRegionLocationFinder {
finder.setConf(TEST_UTIL.getConfiguration());
finder.setServices(cluster.getMaster());
finder.setClusterStatus(cluster.getMaster().getClusterStatus());
finder.setClusterMetrics(cluster.getMaster().getClusterMetrics());
}
@AfterClass

View File

@ -25,7 +25,6 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@ -33,13 +32,13 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Queue;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Size;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.MockNoopMasterServices;
@ -126,20 +125,25 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
ServerName sn = ServerName.valueOf("test:8080", 100);
int numClusterStatusToAdd = 20000;
for (int i = 0; i < numClusterStatusToAdd; i++) {
ServerLoad sl = mock(ServerLoad.class);
ServerMetrics sl = mock(ServerMetrics.class);
RegionLoad rl = mock(RegionLoad.class);
when(rl.getStorefileSizeMB()).thenReturn(i);
RegionMetrics rl = mock(RegionMetrics.class);
when(rl.getReadRequestCount()).thenReturn(0L);
when(rl.getWriteRequestCount()).thenReturn(0L);
when(rl.getMemStoreSize()).thenReturn(Size.ZERO);
when(rl.getStoreFileSize()).thenReturn(new Size(i, Size.Unit.MEGABYTE));
Map<byte[], RegionLoad> regionLoadMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
Map<byte[], RegionMetrics> regionLoadMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
regionLoadMap.put(Bytes.toBytes(REGION_KEY), rl);
when(sl.getRegionsLoad()).thenReturn(regionLoadMap);
when(sl.getRegionMetrics()).thenReturn(regionLoadMap);
ClusterStatus clusterStatus = mock(ClusterStatus.class);
when(clusterStatus.getServers()).thenReturn(Arrays.asList(sn));
when(clusterStatus.getLoad(sn)).thenReturn(sl);
ClusterMetrics clusterStatus = mock(ClusterMetrics.class);
Map<ServerName, ServerMetrics> serverMetricsMap = new TreeMap<>();
serverMetricsMap.put(sn, sl);
when(clusterStatus.getLiveServerMetrics()).thenReturn(serverMetricsMap);
// when(clusterStatus.getLoad(sn)).thenReturn(sl);
loadBalancer.setClusterStatus(clusterStatus);
loadBalancer.setClusterMetrics(clusterStatus);
}
assertTrue(loadBalancer.loads.get(REGION_KEY) != null);
assertTrue(loadBalancer.loads.get(REGION_KEY).size() == 15);

View File

@ -106,7 +106,8 @@ public class TestRegionServerReadRequestMetrics {
TEST_UTIL.getConfiguration().setBoolean(LoadBalancer.SYSTEM_TABLES_ON_MASTER, true);
TEST_UTIL.startMiniCluster();
admin = TEST_UTIL.getAdmin();
serverNames = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
serverNames = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet();
table = createTable();
putData();
List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
@ -166,7 +167,8 @@ public class TestRegionServerReadRequestMetrics {
boolean metricsUpdated = false;
for (int i = 0; i < MAX_TRY; i++) {
for (ServerName serverName : serverNames) {
serverLoad = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getLoad(serverName);
serverLoad = new ServerLoad(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().get(serverName));
Map<byte[], RegionLoad> regionsLoad = serverLoad.getRegionsLoad();
RegionLoad regionLoad = regionsLoad.get(regionInfo.getRegionName());
@ -442,8 +444,8 @@ public class TestRegionServerReadRequestMetrics {
private void testReadRequests(byte[] regionName, int expectedReadRequests) throws Exception {
for (ServerName serverName : serverNames) {
ServerLoad serverLoad =
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getLoad(serverName);
ServerLoad serverLoad = new ServerLoad(admin.getClusterMetrics(
EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().get(serverName));
Map<byte[], RegionLoad> regionsLoad = serverLoad.getRegionsLoad();
RegionLoad regionLoad = regionsLoad.get(regionName);
if (regionLoad != null) {

View File

@ -33,7 +33,6 @@ import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -904,13 +903,14 @@ public class TestSplitTransactionOnCluster {
private void waitUntilRegionServerDead() throws InterruptedException, IOException {
// Wait until the master processes the RS shutdown
for (int i=0; (cluster.getMaster().getClusterStatus().getServers().size() > NB_SERVERS
for (int i=0; (cluster.getMaster().getClusterMetrics()
.getLiveServerMetrics().size() > NB_SERVERS
|| cluster.getLiveRegionServerThreads().size() > NB_SERVERS) && i<100; i++) {
LOG.info("Waiting on server to go down");
Thread.sleep(100);
}
assertFalse("Waited too long for RS to die",
cluster.getMaster().getClusterStatus(). getServers().size() > NB_SERVERS
cluster.getMaster().getClusterMetrics(). getLiveServerMetrics().size() > NB_SERVERS
|| cluster.getLiveRegionServerThreads().size() > NB_SERVERS);
}

View File

@ -35,13 +35,12 @@ 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.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
@ -187,9 +186,9 @@ public class TestMasterReplication {
Waiter.waitFor(baseConfiguration, 10000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
ClusterStatus clusterStatus = utilities[0].getAdmin()
.getClusterStatus(EnumSet.of(ClusterMetrics.Option.LIVE_SERVERS));
ServerLoad serverLoad = clusterStatus.getLoad(rsName);
ClusterMetrics clusterStatus = utilities[0].getAdmin()
.getClusterMetrics(EnumSet.of(ClusterMetrics.Option.LIVE_SERVERS));
ServerMetrics serverLoad = clusterStatus.getLiveServerMetrics().get(rsName);
List<ReplicationLoadSource> replicationLoadSourceList =
serverLoad.getReplicationLoadSourceList();
return replicationLoadSourceList.isEmpty();

View File

@ -67,7 +67,8 @@ public class TestReplicationStatus extends TestReplicationBase {
htable1.put(p);
}
ClusterStatus status = hbaseAdmin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
ClusterStatus status = new ClusterStatus(hbaseAdmin.getClusterMetrics(
EnumSet.of(Option.LIVE_SERVERS)));
for (JVMClusterUtil.RegionServerThread thread : utility1.getHBaseCluster()
.getRegionServerThreads()) {
@ -90,7 +91,7 @@ public class TestReplicationStatus extends TestReplicationBase {
// Stop rs1, then the queue of rs1 will be transfered to rs0
utility1.getHBaseCluster().getRegionServer(1).stop("Stop RegionServer");
Thread.sleep(10000);
status = hbaseAdmin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
status = new ClusterStatus(hbaseAdmin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)));
ServerName server = utility1.getHBaseCluster().getRegionServer(0).getServerName();
ServerLoad sl = status.getLoad(server);
List<ReplicationLoadSource> rLoadSourceList = sl.getReplicationLoadSourceList();

View File

@ -120,8 +120,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.Permission.Action;
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
@ -137,6 +135,9 @@ import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
/**
* Performs authorization checks for common operations, according to different
* levels of authorized users.
@ -3129,7 +3130,7 @@ public class TestAccessController extends SecureTestUtil {
AccessTestAction action = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preGetClusterStatus(ObserverContextImpl.createAndPrepare(CP_ENV));
ACCESS_CONTROLLER.preGetClusterMetrics(ObserverContextImpl.createAndPrepare(CP_ENV));
return null;
}
};

View File

@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.google.protobuf.ServiceException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
@ -29,7 +30,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
@ -72,14 +71,13 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import com.google.protobuf.ServiceException;
/**
* Utilities class for snapshots
*/
@ -229,9 +227,6 @@ public final class SnapshotTestingUtils {
}
});
}
for (byte[] b : snapshotFamilies) {
LOG.info("[CHIA] " + Bytes.toStringBinary(b));
}
// Verify that there are store files in the specified families
if (nonEmptyTestFamilies != null) {
for (final byte[] familyName: nonEmptyTestFamilies) {

View File

@ -37,14 +37,13 @@ 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;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
@ -54,7 +53,6 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
@ -319,8 +317,8 @@ public class BaseTestHBaseFsck {
* Get region info from local cluster.
*/
Map<ServerName, List<String>> getDeployedHRIs(final Admin admin) throws IOException {
ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
Collection<ServerName> regionServers = status.getServers();
ClusterMetrics status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet();
Map<ServerName, List<String>> mm = new HashMap<>();
for (ServerName hsi : regionServers) {
AdminProtos.AdminService.BlockingInterface server = connection.getAdmin(hsi);

View File

@ -153,8 +153,8 @@ public class TestMiniClusterLoadSequential {
", isMultiPut=" + isMultiPut);
numKeys = numKeys();
Admin admin = TEST_UTIL.getAdmin();
while (admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
.getServers().size() < NUM_RS) {
while (admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().size() < NUM_RS) {
LOG.info("Sleeping until " + NUM_RS + " RSs are online");
Threads.sleepWithoutInterrupt(1000);
}