HBASE-18131 Add an hbase shell command to clear deadserver list in ServerManager

Signed-off-by: tedyu <yuzhihong@gmail.com>
This commit is contained in:
Guangxu Cheng 2017-09-11 15:56:05 +08:00 committed by tedyu
parent 90265c9f8c
commit c75bb7424e
20 changed files with 447 additions and 0 deletions

View File

@ -2265,4 +2265,18 @@ public interface Admin extends Abortable, Closeable {
*/
void clearCompactionQueues(final ServerName sn, final Set<String> queues)
throws IOException, InterruptedException;
/**
* List dead region servers.
* @return List of dead region servers.
*/
List<ServerName> listDeadServers() throws IOException;
/**
* Clear dead region servers from master.
* @param servers list of dead region servers.
* @throws IOException if a remote or network exception occurs
* @return List of servers that are not cleared
*/
List<ServerName> clearDeadServers(final List<ServerName> servers) throws IOException;
}

View File

@ -1116,4 +1116,17 @@ public interface AsyncAdmin {
*/
<S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, ServerName serverName);
/**
* List all the dead region servers.
* @return - returns a list of dead region servers wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<List<ServerName>> listDeadServers();
/**
* Clear dead region servers from master.
* @param servers list of dead region servers.
* @return - returns a list of servers that not cleared wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<List<ServerName>> clearDeadServers(final List<ServerName> servers);
}

View File

@ -639,4 +639,14 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
CoprocessorCallable<S, R> callable, ServerName serverName) {
return wrap(rawAdmin.coprocessorService(stubMaker, callable, serverName));
}
@Override
public CompletableFuture<List<ServerName>> listDeadServers() {
return wrap(rawAdmin.listDeadServers());
}
@Override
public CompletableFuture<List<ServerName>> clearDeadServers(List<ServerName> servers) {
return wrap(rawAdmin.clearDeadServers(servers));
}
}

View File

@ -1775,6 +1775,18 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
RpcController controller, GetQuotaStatesRequest request) throws ServiceException {
return stub.getQuotaStates(controller, request);
}
@Override
public MasterProtos.ListDeadServersResponse listDeadServers(RpcController controller,
MasterProtos.ListDeadServersRequest request) throws ServiceException {
return stub.listDeadServers(controller, request);
}
@Override
public MasterProtos.ClearDeadServersResponse clearDeadServers(RpcController controller,
MasterProtos.ClearDeadServersRequest request) throws ServiceException {
return stub.clearDeadServers(controller, request);
}
};
}

View File

@ -113,6 +113,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProce
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
@ -150,6 +151,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedur
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
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.MasterProtos.ListDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
@ -4333,4 +4335,33 @@ public class HBaseAdmin implements Admin {
};
ProtobufUtil.call(callable);
}
@Override
public List<ServerName> listDeadServers() throws IOException {
return executeCallable(new MasterCallable<List<ServerName>>(getConnection(),
getRpcControllerFactory()) {
@Override
public List<ServerName> rpcCall() throws ServiceException {
ListDeadServersRequest req = ListDeadServersRequest.newBuilder().build();
return ProtobufUtil.toServerNameList(
master.listDeadServers(getRpcController(), req).getServerNameList());
}
});
}
@Override
public List<ServerName> clearDeadServers(final List<ServerName> servers) throws IOException {
if (servers == null || servers.size() == 0) {
throw new IllegalArgumentException("servers cannot be null or empty");
}
return executeCallable(new MasterCallable<List<ServerName>>(getConnection(),
getRpcControllerFactory()) {
@Override
protected List<ServerName> rpcCall() throws Exception {
ClearDeadServersRequest req = RequestConverter.buildClearDeadServersRequest(servers);
return ProtobufUtil.toServerNameList(
master.clearDeadServers(getRpcController(), req).getServerNameList());
}
});
}
}

View File

@ -113,6 +113,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegi
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
@ -169,6 +171,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshot
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDeadServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
@ -2900,6 +2904,28 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return future;
}
@Override
public CompletableFuture<List<ServerName>> listDeadServers() {
return this.<List<ServerName>> newMasterCaller()
.action((controller, stub) -> this
.<ListDeadServersRequest, ListDeadServersResponse, List<ServerName>> call(
controller, stub, ListDeadServersRequest.newBuilder().build(),
(s, c, req, done) -> s.listDeadServers(c, req, done),
(resp) -> ProtobufUtil.toServerNameList(resp.getServerNameList())))
.call();
}
@Override
public CompletableFuture<List<ServerName>> clearDeadServers(List<ServerName> servers) {
return this.<List<ServerName>> newMasterCaller()
.action((controller, stub) -> this
.<ClearDeadServersRequest, ClearDeadServersResponse, List<ServerName>> call(
controller, stub, RequestConverter.buildClearDeadServersRequest(servers),
(s, c, req, done) -> s.clearDeadServers(c, req, done),
(resp) -> ProtobufUtil.toServerNameList(resp.getServerNameList())))
.call();
}
private <T> ServerRequestCallerBuilder<T> newServerCaller() {
return this.connection.callerFactory.<T> serverRequest()
.rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)

View File

@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegi
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
@ -90,6 +92,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshot
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDeadServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
@ -625,6 +629,18 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
return stub.getQuotaStates(controller, request);
}
@Override
public ClearDeadServersResponse clearDeadServers(RpcController controller,
ClearDeadServersRequest request) throws ServiceException {
return stub.clearDeadServers(controller, request);
}
@Override
public ListDeadServersResponse listDeadServers(RpcController controller,
ListDeadServersRequest request) throws ServiceException {
return stub.listDeadServers(controller, request);
}
@Override
public SplitTableRegionResponse splitRegion(RpcController controller, SplitTableRegionRequest request)
throws ServiceException {

View File

@ -409,6 +409,17 @@ public final class ProtobufUtil {
return ServerName.valueOf(hostName, port, startCode);
}
/**
* Convert a list of protocol buffer ServerName to a list of ServerName
* @param proto protocol buffer ServerNameList
* @return a list of ServerName
*/
public static List<ServerName> toServerNameList(
List<HBaseProtos.ServerName> proto) {
return proto.stream().map(ProtobufUtil::toServerName)
.collect(Collectors.toList());
}
/**
* Get a list of NamespaceDescriptor from ListNamespaceDescriptorsResponse protobuf
* @param proto the ListNamespaceDescriptorsResponse

View File

@ -88,6 +88,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
@ -1812,6 +1813,14 @@ public final class RequestConverter {
return builder.build();
}
public static ClearDeadServersRequest buildClearDeadServersRequest(List<ServerName> deadServers) {
ClearDeadServersRequest.Builder builder = ClearDeadServersRequest.newBuilder();
for(ServerName server: deadServers) {
builder.addServerName(ProtobufUtil.toServerName(server));
}
return builder.build();
}
private static final GetSpaceQuotaRegionSizesRequest GET_SPACE_QUOTA_REGION_SIZES_REQUEST =
GetSpaceQuotaRegionSizesRequest.newBuilder().build();

View File

@ -622,6 +622,21 @@ message RemoveDrainFromRegionServersRequest {
message RemoveDrainFromRegionServersResponse {
}
message ListDeadServersRequest {
}
message ListDeadServersResponse {
repeated ServerName server_name = 1;
}
message ClearDeadServersRequest {
repeated ServerName server_name = 1;
}
message ClearDeadServersResponse {
repeated ServerName server_name = 1;
}
service MasterService {
/** Used by the client to get the number of regions that have received the updated schema */
rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
@ -970,4 +985,12 @@ service MasterService {
/** Fetches the Master's view of quotas */
rpc GetQuotaStates(GetQuotaStatesRequest)
returns(GetQuotaStatesResponse);
/** clear dead servers from master*/
rpc ClearDeadServers(ClearDeadServersRequest)
returns(ClearDeadServersResponse);
/** Returns a list of Dead Servers. */
rpc ListDeadServers(ListDeadServersRequest)
returns(ListDeadServersResponse);
}

View File

@ -1917,4 +1917,28 @@ public interface MasterObserver extends Coprocessor {
*/
default void postLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx,
LockProcedure proc, boolean keepAlive) throws IOException {}
/**
* Called before list dead region servers.
*/
default void preListDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {}
/**
* Called after list dead region servers.
*/
default void postListDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {}
/**
* Called before clear dead region servers.
*/
default void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {}
/**
* Called after clear dead region servers.
*/
default void postClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {}
}

View File

@ -202,4 +202,17 @@ public class DeadServer {
return o1.getSecond().compareTo(o2.getSecond());
}
};
/**
* remove the specified dead server
* @param deadServerName the dead server name
* @return true if this server was removed
*/
public synchronized boolean removeDeadServer(final ServerName deadServerName) {
if (deadServers.remove(deadServerName) == null) {
return false;
}
return true;
}
}

View File

@ -1879,6 +1879,46 @@ public class MasterCoprocessorHost
});
}
public void preListDeadServers() throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preListDeadServers(ctx);
}
});
}
public void postListDeadServers() throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postListDeadServers(ctx);
}
});
}
public void preClearDeadServers() throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preClearDeadServers(ctx);
}
});
}
public void postClearDeadServers() throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postClearDeadServers(ctx);
}
});
}
private static ImmutableHTableDescriptor toImmutableHTableDescriptor(TableDescriptor desc) {
return new ImmutableHTableDescriptor(desc);
}

View File

@ -104,6 +104,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegi
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
@ -164,6 +166,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshot
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDeadServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDeadServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
@ -2121,4 +2125,63 @@ public class MasterRpcServices extends RSRpcServices
throw new ServiceException(e);
}
}
@Override
public ListDeadServersResponse listDeadServers(RpcController controller,
ListDeadServersRequest request) throws ServiceException {
LOG.debug(master.getClientIdAuditPrefix() + " list dead region servers.");
ListDeadServersResponse.Builder response = ListDeadServersResponse.newBuilder();
try {
master.checkInitialized();
if (master.cpHost != null) {
master.cpHost.preListDeadServers();
}
Set<ServerName> servers = master.getServerManager().getDeadServers().copyServerNames();
for (ServerName server : servers) {
response.addServerName(ProtobufUtil.toServerName(server));
}
if (master.cpHost != null) {
master.cpHost.postListDeadServers();
}
} catch (IOException io) {
throw new ServiceException(io);
}
return response.build();
}
@Override
public ClearDeadServersResponse clearDeadServers(RpcController controller,
ClearDeadServersRequest request) throws ServiceException {
LOG.debug(master.getClientIdAuditPrefix() + " clear dead region servers.");
ClearDeadServersResponse.Builder response = ClearDeadServersResponse.newBuilder();
try {
master.checkInitialized();
if (master.cpHost != null) {
master.cpHost.preClearDeadServers();
}
if (master.getServerManager().areDeadServersInProgress()) {
LOG.debug("Some dead server is still under processing, won't clear the dead server list");
response.addAllServerName(request.getServerNameList());
} else {
for (HBaseProtos.ServerName pbServer : request.getServerNameList()) {
if (!master.getServerManager().getDeadServers()
.removeDeadServer(ProtobufUtil.toServerName(pbServer))) {
response.addServerName(pbServer);
}
}
}
if (master.cpHost != null) {
master.cpHost.postClearDeadServers();
}
} catch (IOException io) {
throw new ServiceException(io);
}
return response.build();
}
}

View File

@ -1461,6 +1461,11 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
requirePermission(getActiveUser(ctx), "split", tableName, null, null, Action.ADMIN);
}
@Override
public void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
requirePermission(getActiveUser(ctx), "clearDeadServers", Action.ADMIN);
}
/* ---- RegionObserver implementation ---- */
@Override

View File

@ -151,5 +151,23 @@ public class TestDeadServer {
Assert.assertTrue(d.isEmpty());
}
@Test
public void testClearDeadServer(){
DeadServer d = new DeadServer();
d.add(hostname123);
d.add(hostname1234);
Assert.assertEquals(2, d.size());
d.removeDeadServer(hostname123);
Assert.assertEquals(1, d.size());
d.removeDeadServer(hostname1234);
Assert.assertTrue(d.isEmpty());
d.add(hostname1234);
Assert.assertFalse(d.removeDeadServer(hostname123_2));
Assert.assertEquals(1, d.size());
Assert.assertTrue(d.removeDeadServer(hostname1234));
Assert.assertTrue(d.isEmpty());
}
}

View File

@ -1249,5 +1249,27 @@ module Hbase
end
@admin.clearCompactionQueues(ServerName.valueOf(server_name), queues)
end
#----------------------------------------------------------------------------------------------
# clear dead region servers
def list_deadservers
@admin.listDeadServers.to_a
end
#----------------------------------------------------------------------------------------------
# clear dead region servers
def clear_deadservers(dead_servers)
# Flatten params array
dead_servers = dead_servers.flatten.compact
if dead_servers.empty?
servers = list_deadservers
else
servers = java.util.ArrayList.new
dead_servers.each do |s|
servers.add(ServerName.valueOf(s))
end
end
@admin.clearDeadServers(servers).to_a
end
end
end

View File

@ -358,6 +358,8 @@ Shell.load_command_group(
splitormerge_switch
splitormerge_enabled
clear_compaction_queues
list_deadservers
clear_deadservers
],
# TODO: remove older hlog_roll command
aliases: {

View File

@ -0,0 +1,52 @@
#
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
module Shell
module Commands
class ClearDeadservers < Command
def help
<<-EOF
Clear the dead region servers that are never used.
Examples:
Clear all dead region servers:
hbase> clear_deadservers
Clear the specified dead region servers:
hbase> clear_deadservers 'host187.example.com,60020,1289493121758'
or
hbase> clear_deadservers 'host187.example.com,60020,1289493121758',
'host188.example.com,60020,1289493121758'
EOF
end
def command(*dead_servers)
servers = admin.clear_deadservers(dead_servers)
if servers.size <= 0
formatter.row(['true'])
else
formatter.row(['Some dead server clear failed'])
formatter.row(['SERVERNAME'])
servers.each do |server|
formatter.row([server.toString])
end
formatter.footer(servers.size)
end
end
end
end
end

View File

@ -0,0 +1,43 @@
#
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
module Shell
module Commands
class ListDeadservers < Command
def help
<<-EOF
List all dead region servers in hbase
Examples:
hbase> list_deadservers
EOF
end
def command
formatter.header(['SERVERNAME'])
servers = admin.list_deadservers
servers.each do |server|
formatter.row([server.toString])
end
formatter.footer(servers.size)
end
end
end
end