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:
Umesh Agashe 2018-08-07 11:46:42 -07:00 committed by Michael Stack
parent 6dd5383033
commit 3813f0ac3d
11 changed files with 370 additions and 2 deletions

View File

@ -326,4 +326,36 @@ public interface ClusterConnection extends Connection {
* @throws IOException if a remote or network exception occurs
*/
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;
}

View File

@ -422,6 +422,28 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
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
public MetricsConnection getConnectionMetrics() {
return this.metrics;

View File

@ -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);
}
}
}

View File

@ -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;
}

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
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.exceptions.DeserializationException;
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.SetNormalizerRunningRequest;
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.TruncateTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
@ -1444,6 +1446,16 @@ public final class RequestConverter {
.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
*

View File

@ -48,4 +48,9 @@ public final class HBaseInterfaceAudience {
* for class name, and arguments.
*/
public static final String TOOLS = "Tools";
/**
* Denotes classes used by hbck tool for fixing inconsistent state of HBase.
*/
public static final String HBCK = "HBCK";
}

View File

@ -485,6 +485,11 @@ message GetTableStateResponse {
required TableState table_state = 1;
}
message SetTableStateInMetaRequest {
required TableName table_name = 1;
required TableState table_state = 2;
}
message GetClusterStatusRequest {
repeated Option options = 1;
}
@ -991,3 +996,9 @@ service MasterService {
returns(ClearDeadServersResponse);
}
service HbckService {
/** Update state of the table in meta only*/
rpc SetTableStateInMeta(SetTableStateInMetaRequest)
returns(GetTableStateResponse);
}

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
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.RegionInfo;
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.GetTableStateRequest;
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.IsBalancerEnabledResponse;
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.SetSplitOrMergeEnabledRequest;
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.ShutdownResponse;
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")
public class MasterRpcServices extends RSRpcServices
implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
LockService.BlockingInterface {
LockService.BlockingInterface, HbckService.BlockingInterface {
private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
private final HMaster master;
@ -443,6 +446,8 @@ public class MasterRpcServices extends RSRpcServices
RegionServerStatusService.BlockingInterface.class));
bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
LockService.BlockingInterface.class));
bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this),
HbckService.BlockingInterface.class));
bssi.addAll(super.getServices());
return bssi;
}
@ -2312,4 +2317,28 @@ public class MasterRpcServices extends RSRpcServices
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);
}
}
}

View File

@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Admin;
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.ColumnFamilyDescriptorBuilder;
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.Get;
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.ImmutableHTableDescriptor;
import org.apache.hadoop.hbase.client.Put;
@ -2945,7 +2947,12 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
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.

View File

@ -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());
}
}

View File

@ -66,6 +66,7 @@ public class TestMasterMetrics {
long reportStartTime, long reportEndTime) {
// do nothing
}
}
@BeforeClass