HBASE-20941 Created and implemented HbckService in master
Added API setTableStateInMeta() to update table state only in Meta. This will be used by hbck2 tool.
This commit is contained in:
parent
6dd5383033
commit
3813f0ac3d
|
@ -326,4 +326,36 @@ public interface ClusterConnection extends Connection {
|
||||||
* @throws IOException if a remote or network exception occurs
|
* @throws IOException if a remote or network exception occurs
|
||||||
*/
|
*/
|
||||||
int getCurrentNrHRS() throws IOException;
|
int getCurrentNrHRS() throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve an Hbck implementation to fix an HBase cluster.
|
||||||
|
* The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by
|
||||||
|
* each thread. This is a lightweight operation. Pooling or caching of the returned Hbck instance
|
||||||
|
* is not recommended.
|
||||||
|
* <br>
|
||||||
|
* The caller is responsible for calling {@link Hbck#close()} on the returned Hbck instance.
|
||||||
|
*<br>
|
||||||
|
* This will be used mostly by hbck tool.
|
||||||
|
*
|
||||||
|
* @return an Hbck instance for active master. Active master is fetched from the zookeeper.
|
||||||
|
*/
|
||||||
|
Hbck getHbck() throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieve an Hbck implementation to fix an HBase cluster.
|
||||||
|
* The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by
|
||||||
|
* each thread. This is a lightweight operation. Pooling or caching of the returned Hbck instance
|
||||||
|
* is not recommended.
|
||||||
|
* <br>
|
||||||
|
* The caller is responsible for calling {@link Hbck#close()} on the returned Hbck instance.
|
||||||
|
*<br>
|
||||||
|
* This will be used mostly by hbck tool. This may only be used to by pass getting
|
||||||
|
* registered master from ZK. In situations where ZK is not available or active master is not
|
||||||
|
* registered with ZK and user can get master address by other means, master can be explicitly
|
||||||
|
* specified.
|
||||||
|
*
|
||||||
|
* @param masterServer explicit {@link ServerName} for master server
|
||||||
|
* @return an Hbck instance for a specified master server
|
||||||
|
*/
|
||||||
|
Hbck getHbck(ServerName masterServer) throws IOException;
|
||||||
}
|
}
|
||||||
|
|
|
@ -422,6 +422,28 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
return new HBaseAdmin(this);
|
return new HBaseAdmin(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Hbck getHbck() throws IOException {
|
||||||
|
return getHbck(get(registry.getMasterAddress()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Hbck getHbck(ServerName masterServer) throws IOException {
|
||||||
|
checkClosed();
|
||||||
|
if (isDeadServer(masterServer)) {
|
||||||
|
throw new RegionServerStoppedException(masterServer + " is dead.");
|
||||||
|
}
|
||||||
|
String key = getStubKey(MasterProtos.HbckService.BlockingInterface.class.getName(),
|
||||||
|
masterServer, this.hostnamesCanChange);
|
||||||
|
|
||||||
|
return new HBaseHbck(this,
|
||||||
|
(MasterProtos.HbckService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
|
||||||
|
BlockingRpcChannel channel =
|
||||||
|
this.rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout);
|
||||||
|
return MasterProtos.HbckService.newBlockingStub(channel);
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public MetricsConnection getConnectionMetrics() {
|
public MetricsConnection getConnectionMetrics() {
|
||||||
return this.metrics;
|
return this.metrics;
|
||||||
|
|
|
@ -0,0 +1,95 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService.BlockingInterface;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Use {@link ClusterConnection#getHbck()} to obtain an instance of {@link Hbck} instead of
|
||||||
|
* constructing
|
||||||
|
* an HBaseHbck directly. This will be mostly used by hbck tool.
|
||||||
|
*
|
||||||
|
* <p>Connection should be an <i>unmanaged</i> connection obtained via
|
||||||
|
* {@link ConnectionFactory#createConnection(Configuration)}.</p>
|
||||||
|
*
|
||||||
|
* <p>An instance of this class is lightweight and not-thread safe. A new instance should be created
|
||||||
|
* by each thread. Pooling or caching of the instance is not recommended.</p>
|
||||||
|
*
|
||||||
|
* @see ConnectionFactory
|
||||||
|
* @see ClusterConnection
|
||||||
|
* @see Hbck
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class HBaseHbck implements Hbck {
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(HBaseHbck.class);
|
||||||
|
|
||||||
|
private boolean aborted;
|
||||||
|
private final BlockingInterface hbck;
|
||||||
|
|
||||||
|
private RpcControllerFactory rpcControllerFactory;
|
||||||
|
|
||||||
|
HBaseHbck(ClusterConnection connection, BlockingInterface hbck) throws IOException {
|
||||||
|
this.hbck = hbck;
|
||||||
|
this.rpcControllerFactory = connection.getRpcControllerFactory();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
// currently does nothing
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void abort(String why, Throwable e) {
|
||||||
|
this.aborted = true;
|
||||||
|
// Currently does nothing but throw the passed message and exception
|
||||||
|
throw new RuntimeException(why, e);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isAborted() {
|
||||||
|
return this.aborted;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* NOTE: This is a dangerous action, as existing running procedures for the table or regions
|
||||||
|
* which belong to the table may get confused.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public TableState setTableStateInMeta(TableState state) throws IOException {
|
||||||
|
try {
|
||||||
|
GetTableStateResponse response = hbck.setTableStateInMeta(
|
||||||
|
rpcControllerFactory.newController(),
|
||||||
|
RequestConverter.buildSetTableStateInMetaRequest(state));
|
||||||
|
return TableState.convert(state.getTableName(), response.getTableState());
|
||||||
|
} catch (ServiceException se) {
|
||||||
|
LOG.debug("ServiceException while updating table state in meta. table={}, state={}",
|
||||||
|
state.getTableName(), state.getState());
|
||||||
|
throw new IOException(se);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,50 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Hbck APIs for HBase. Obtain an instance from {@link ClusterConnection#getHbck()} and call
|
||||||
|
* {@link #close()} when done.
|
||||||
|
* <p>Hbck client APIs will be mostly used by hbck tool which in turn can be used by operators to
|
||||||
|
* fix HBase and bringging it to consistent state.</p>
|
||||||
|
*
|
||||||
|
* @see ConnectionFactory
|
||||||
|
* @see ClusterConnection
|
||||||
|
* @since 2.2.0
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.HBCK)
|
||||||
|
public interface Hbck extends Abortable, Closeable {
|
||||||
|
/**
|
||||||
|
* Update table state in Meta only. No procedures are submitted to open/ assign or close/
|
||||||
|
* unassign regions of the table. This is useful only when some procedures/ actions are stuck
|
||||||
|
* beause of inconsistency between region and table states.
|
||||||
|
*
|
||||||
|
* NOTE: This is a dangerous action, as existing running procedures for the table or regions
|
||||||
|
* which belong to the table may get confused.
|
||||||
|
*
|
||||||
|
* @param state table state
|
||||||
|
* @return previous state of the table in Meta
|
||||||
|
*/
|
||||||
|
TableState setTableStateInMeta(TableState state) throws IOException;
|
||||||
|
}
|
|
@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Row;
|
||||||
import org.apache.hadoop.hbase.client.RowMutations;
|
import org.apache.hadoop.hbase.client.RowMutations;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
|
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
|
||||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||||
|
@ -134,6 +135,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalance
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
|
||||||
|
@ -1444,6 +1446,16 @@ public final class RequestConverter {
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a protocol buffer SetTableStateInMetaRequest
|
||||||
|
* @param state table state to update in Meta
|
||||||
|
* @return a SetTableStateInMetaRequest
|
||||||
|
*/
|
||||||
|
public static SetTableStateInMetaRequest buildSetTableStateInMetaRequest(final TableState state) {
|
||||||
|
return SetTableStateInMetaRequest.newBuilder().setTableState(state.convert())
|
||||||
|
.setTableName(ProtobufUtil.toProtoTableName(state.getTableName())).build();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a protocol buffer GetTableDescriptorsRequest for a single table
|
* Creates a protocol buffer GetTableDescriptorsRequest for a single table
|
||||||
*
|
*
|
||||||
|
|
|
@ -48,4 +48,9 @@ public final class HBaseInterfaceAudience {
|
||||||
* for class name, and arguments.
|
* for class name, and arguments.
|
||||||
*/
|
*/
|
||||||
public static final String TOOLS = "Tools";
|
public static final String TOOLS = "Tools";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Denotes classes used by hbck tool for fixing inconsistent state of HBase.
|
||||||
|
*/
|
||||||
|
public static final String HBCK = "HBCK";
|
||||||
}
|
}
|
||||||
|
|
|
@ -485,6 +485,11 @@ message GetTableStateResponse {
|
||||||
required TableState table_state = 1;
|
required TableState table_state = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message SetTableStateInMetaRequest {
|
||||||
|
required TableName table_name = 1;
|
||||||
|
required TableState table_state = 2;
|
||||||
|
}
|
||||||
|
|
||||||
message GetClusterStatusRequest {
|
message GetClusterStatusRequest {
|
||||||
repeated Option options = 1;
|
repeated Option options = 1;
|
||||||
}
|
}
|
||||||
|
@ -991,3 +996,9 @@ service MasterService {
|
||||||
returns(ClearDeadServersResponse);
|
returns(ClearDeadServersResponse);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
service HbckService {
|
||||||
|
/** Update state of the table in meta only*/
|
||||||
|
rpc SetTableStateInMeta(SetTableStateInMetaRequest)
|
||||||
|
returns(GetTableStateResponse);
|
||||||
|
}
|
||||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||||
|
@ -177,6 +178,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNa
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
|
||||||
|
@ -241,6 +243,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRe
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
|
||||||
|
@ -305,7 +308,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.Snapshot
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public class MasterRpcServices extends RSRpcServices
|
public class MasterRpcServices extends RSRpcServices
|
||||||
implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
|
implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
|
||||||
LockService.BlockingInterface {
|
LockService.BlockingInterface, HbckService.BlockingInterface {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
|
private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
|
||||||
|
|
||||||
private final HMaster master;
|
private final HMaster master;
|
||||||
|
@ -443,6 +446,8 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
RegionServerStatusService.BlockingInterface.class));
|
RegionServerStatusService.BlockingInterface.class));
|
||||||
bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
|
bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
|
||||||
LockService.BlockingInterface.class));
|
LockService.BlockingInterface.class));
|
||||||
|
bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this),
|
||||||
|
HbckService.BlockingInterface.class));
|
||||||
bssi.addAll(super.getServices());
|
bssi.addAll(super.getServices());
|
||||||
return bssi;
|
return bssi;
|
||||||
}
|
}
|
||||||
|
@ -2312,4 +2317,28 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
throw new ServiceException(e);
|
throw new ServiceException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// HBCK Services
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update state of the table in meta only. This is required by hbck in some situations to cleanup
|
||||||
|
* stuck assign/ unassign regions procedures for the table.
|
||||||
|
*
|
||||||
|
* @return previous state of the table
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public GetTableStateResponse setTableStateInMeta(RpcController controller,
|
||||||
|
SetTableStateInMetaRequest request) throws ServiceException {
|
||||||
|
Connection conn = master.getConnection();
|
||||||
|
TableName tn = ProtobufUtil.toTableName(request.getTableName());
|
||||||
|
|
||||||
|
try {
|
||||||
|
HBaseProtos.TableState prevState = MetaTableAccessor.getTableState(conn, tn).convert();
|
||||||
|
MetaTableAccessor.updateTableState(conn, tn,
|
||||||
|
TableState.convert(tn, request.getTableState()).getState());
|
||||||
|
return GetTableStateResponse.newBuilder().setTableState(prevState).build();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new ServiceException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
|
||||||
import org.apache.hadoop.hbase.Waiter.Predicate;
|
import org.apache.hadoop.hbase.Waiter.Predicate;
|
||||||
import org.apache.hadoop.hbase.client.Admin;
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||||
|
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
@ -73,6 +74,7 @@ import org.apache.hadoop.hbase.client.Delete;
|
||||||
import org.apache.hadoop.hbase.client.Durability;
|
import org.apache.hadoop.hbase.client.Durability;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||||
|
import org.apache.hadoop.hbase.client.Hbck;
|
||||||
import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
|
import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
|
import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
@ -2945,7 +2947,12 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
|
||||||
|
|
||||||
private HBaseAdmin hbaseAdmin = null;
|
private HBaseAdmin hbaseAdmin = null;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns an {@link Hbck} instance. Needs be closed when done.
|
||||||
|
*/
|
||||||
|
public Hbck getHbck() throws IOException {
|
||||||
|
return ((ClusterConnection) getConnection()).getHbck();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Unassign the named region.
|
* Unassign the named region.
|
||||||
|
|
|
@ -0,0 +1,104 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
|
import static junit.framework.TestCase.assertTrue;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.rules.TestName;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class to test HBaseHbck.
|
||||||
|
* Spins up the minicluster once at test start and then takes it down afterward.
|
||||||
|
* Add any testing of HBaseHbck functionality here.
|
||||||
|
*/
|
||||||
|
@Category({LargeTests.class, ClientTests.class})
|
||||||
|
public class TestHbck {
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestHbck.class);
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(TestHbck.class);
|
||||||
|
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||||
|
private Admin admin;
|
||||||
|
private Hbck hbck;
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public TestName name = new TestName();
|
||||||
|
|
||||||
|
private static final TableName tableName = TableName.valueOf(TestHbck.class.getSimpleName());
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void setUpBeforeClass() throws Exception {
|
||||||
|
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
|
||||||
|
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
|
||||||
|
TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
|
||||||
|
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
|
||||||
|
TEST_UTIL.startMiniCluster(3);
|
||||||
|
|
||||||
|
TEST_UTIL.createTable(tableName, "family1");
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void tearDownAfterClass() throws Exception {
|
||||||
|
TEST_UTIL.shutdownMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
this.admin = TEST_UTIL.getAdmin();
|
||||||
|
this.hbck = TEST_UTIL.getHbck();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
for (HTableDescriptor htd : this.admin.listTables()) {
|
||||||
|
TEST_UTIL.deleteTable(htd.getTableName());
|
||||||
|
}
|
||||||
|
this.hbck.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSetTableStateInMeta() throws IOException {
|
||||||
|
// set table state to DISABLED
|
||||||
|
hbck.setTableStateInMeta(new TableState(tableName, TableState.State.DISABLED));
|
||||||
|
// Method {@link Hbck#setTableStateInMeta()} returns previous state, which in this case
|
||||||
|
// will be DISABLED
|
||||||
|
TableState prevState =
|
||||||
|
hbck.setTableStateInMeta(new TableState(tableName, TableState.State.ENABLED));
|
||||||
|
assertTrue("Incorrect previous state! expeced=DISABLED, found=" + prevState.getState(),
|
||||||
|
prevState.isDisabled());
|
||||||
|
}
|
||||||
|
}
|
|
@ -66,6 +66,7 @@ public class TestMasterMetrics {
|
||||||
long reportStartTime, long reportEndTime) {
|
long reportStartTime, long reportEndTime) {
|
||||||
// do nothing
|
// do nothing
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
|
|
Loading…
Reference in New Issue