HBASE-24505 Reimplement Hbck.setRegionStateInMeta (#1852)

Signed-off-by: Wellington Ramos Chevreuil <wchevreuil@apache.org>
This commit is contained in:
Duo Zhang 2020-06-05 08:48:39 +08:00 committed by GitHub
parent 2eb993a0f0
commit b2ec4c1ea0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 107 additions and 98 deletions

View File

@ -18,16 +18,21 @@
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.ServerName;
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.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
@ -37,18 +42,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BypassProc
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService.BlockingInterface;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RegionSpecifierAndState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ScheduleServerCrashProcedureResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignsResponse;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Use {@link Connection#getHbck()} to obtain an instance of {@link Hbck} instead of
* constructing an HBaseHbck directly.
@ -112,18 +111,20 @@ public class HBaseHbck implements Hbck {
}
@Override
public List<RegionState> setRegionStateInMeta(List<RegionState> states) throws IOException {
public Map<String, RegionState.State> setRegionStateInMeta(
Map<String, RegionState.State> nameOrEncodedName2State) throws IOException {
try {
if(LOG.isDebugEnabled()) {
states.forEach(s ->
LOG.debug("region={}, state={}", s.getRegion().getRegionName(), s.getState())
);
if (LOG.isDebugEnabled()) {
nameOrEncodedName2State.forEach((k, v) -> LOG.debug("region={}, state={}", k, v));
}
MasterProtos.SetRegionStateInMetaResponse response =
hbck.setRegionStateInMeta(rpcControllerFactory.newController(),
RequestConverter.buildSetRegionStateInMetaRequest(nameOrEncodedName2State));
Map<String, RegionState.State> result = new HashMap<>();
for (RegionSpecifierAndState nameAndState : response.getStatesList()) {
result.put(nameAndState.getRegionSpecifier().getValue().toStringUtf8(),
RegionState.State.convert(nameAndState.getState()));
}
MasterProtos.GetRegionStateInMetaResponse response = hbck.setRegionStateInMeta(
rpcControllerFactory.newController(),
RequestConverter.buildSetRegionStateInMetaRequest(states));
final List<RegionState> result = new ArrayList<>();
response.getStatesList().forEach(s -> result.add(RegionState.convert(s)));
return result;
} catch (ServiceException se) {
throw new IOException(se);

View File

@ -22,17 +22,14 @@ import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
/**
* Hbck fixup tool APIs. Obtain an instance from {@link Connection#getHbck()} and call
@ -56,12 +53,13 @@ public interface Hbck extends Abortable, Closeable {
TableState setTableStateInMeta(TableState state) throws IOException;
/**
* Update region state in Meta only. No procedures are submitted to manipulate the given region
* or any other region from same table.
* @param states list of all region states to be updated in meta
* Update region state in Meta only. No procedures are submitted to manipulate the given region or
* any other region from same table.
* @param nameOrEncodedName2State list of all region states to be updated in meta
* @return previous state of the region in Meta
*/
List<RegionState> setRegionStateInMeta(List<RegionState> states) throws IOException;
Map<String, RegionState.State>
setRegionStateInMeta(Map<String, RegionState.State> nameOrEncodedName2State) throws IOException;
/**
* Like {@link Admin#assign(byte[])} but 'raw' in that it can do more than one Region at a time

View File

@ -2345,14 +2345,6 @@ public final class ProtobufUtil {
.setQualifier(UnsafeByteOperations.unsafeWrap(tableName.getQualifier())).build();
}
public static HBaseProtos.RegionInfo toProtoRegionInfo(
org.apache.hadoop.hbase.client.RegionInfo regionInfo) {
return HBaseProtos.RegionInfo.newBuilder()
.setRegionId(regionInfo.getRegionId())
.setRegionEncodedName(regionInfo.getEncodedName())
.setTableName(toProtoTableName(regionInfo.getTable())).build();
}
public static List<TableName> toTableNameList(List<HBaseProtos.TableName> tableNamesList) {
if (tableNamesList == null) {
return new ArrayList<>();
@ -3315,7 +3307,6 @@ public final class ProtobufUtil {
builder.setOffline(info.isOffline());
builder.setSplit(info.isSplit());
builder.setReplicaId(info.getReplicaId());
builder.setRegionEncodedName(info.getEncodedName());
return builder.build();
}
@ -3355,9 +3346,6 @@ public final class ProtobufUtil {
if (proto.hasOffline()) {
rib.setOffline(proto.getOffline());
}
if (proto.hasRegionEncodedName()) {
rib.setEncodedName(proto.getRegionEncodedName());
}
return rib.build();
}

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.LogQueryFilter;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
@ -50,7 +51,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
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.LogQueryFilter;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
@ -135,6 +135,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegion
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RegionSpecifierAndState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
@ -1251,13 +1252,23 @@ public final class RequestConverter {
/**
* Creates a protocol buffer SetRegionStateInMetaRequest
* @param states list of regions states to update in Meta
* @param nameOrEncodedName2State list of regions states to update in Meta
* @return a SetRegionStateInMetaRequest
*/
public static SetRegionStateInMetaRequest buildSetRegionStateInMetaRequest(
final List<RegionState> states) {
final SetRegionStateInMetaRequest.Builder builder = SetRegionStateInMetaRequest.newBuilder();
states.forEach(s -> builder.addStates(s.convert()));
public static SetRegionStateInMetaRequest
buildSetRegionStateInMetaRequest(Map<String, RegionState.State> nameOrEncodedName2State) {
SetRegionStateInMetaRequest.Builder builder = SetRegionStateInMetaRequest.newBuilder();
nameOrEncodedName2State.forEach((name, state) -> {
byte[] bytes = Bytes.toBytes(name);
RegionSpecifier spec;
if (RegionInfo.isEncodedRegionName(bytes)) {
spec = buildRegionSpecifier(RegionSpecifierType.ENCODED_REGION_NAME, bytes);
} else {
spec = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, bytes);
}
builder.addStates(RegionSpecifierAndState.newBuilder().setRegionSpecifier(spec)
.setState(state.convert()).build());
});
return builder.build();
}

View File

@ -69,7 +69,7 @@ message ColumnFamilySchema {
}
/**
* Protocol buffer version of HRegionInfo.
* Protocol buffer version of RegionInfo.
*/
message RegionInfo {
required uint64 region_id = 1;
@ -79,7 +79,6 @@ message RegionInfo {
optional bool offline = 5;
optional bool split = 6;
optional int32 replica_id = 7 [default = 0];
optional string region_encoded_name = 8;
}
/**

View File

@ -518,11 +518,6 @@ message GetTableStateResponse {
required TableState table_state = 1;
}
message GetRegionStateInMetaResponse {
repeated RegionState states = 1;
}
message GetClusterStatusRequest {
repeated Option options = 1;
}
@ -1131,8 +1126,17 @@ message SetTableStateInMetaRequest {
required TableState table_state = 2;
}
message RegionSpecifierAndState {
required RegionSpecifier region_specifier = 1;
required RegionState.State state = 2;
}
message SetRegionStateInMetaRequest {
repeated RegionState states = 2;
repeated RegionSpecifierAndState states = 1;
}
message SetRegionStateInMetaResponse {
repeated RegionSpecifierAndState states = 1;
}
/** Like Admin's AssignRegionRequest except it can
@ -1199,7 +1203,7 @@ service HbckService {
/** Update state of the region in meta only*/
rpc SetRegionStateInMeta(SetRegionStateInMetaRequest)
returns(GetRegionStateInMetaResponse);
returns(SetRegionStateInMetaResponse);
/**
* Assign regions.

View File

@ -151,6 +151,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.Reg
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
@ -212,7 +213,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedu
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetRegionStateInMetaResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
@ -272,6 +272,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineReg
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RegionSpecifierAndState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
@ -291,6 +292,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormali
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetRegionStateInMetaRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetRegionStateInMetaResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSnapshotCleanupRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSnapshotCleanupResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
@ -2500,31 +2502,39 @@ public class MasterRpcServices extends RSRpcServices implements
* @return previous states of the regions
*/
@Override
public GetRegionStateInMetaResponse setRegionStateInMeta(RpcController controller,
public SetRegionStateInMetaResponse setRegionStateInMeta(RpcController controller,
SetRegionStateInMetaRequest request) throws ServiceException {
final GetRegionStateInMetaResponse.Builder builder = GetRegionStateInMetaResponse.newBuilder();
for(ClusterStatusProtos.RegionState s : request.getStatesList()) {
try {
RegionInfo info = this.master.getAssignmentManager().
loadRegionFromMeta(s.getRegionInfo().getRegionEncodedName());
SetRegionStateInMetaResponse.Builder builder = SetRegionStateInMetaResponse.newBuilder();
try {
for (RegionSpecifierAndState s : request.getStatesList()) {
RegionSpecifier spec = s.getRegionSpecifier();
String encodedName;
if (spec.getType() == RegionSpecifierType.ENCODED_REGION_NAME) {
encodedName = spec.getValue().toStringUtf8();
} else {
// TODO: actually, a full region name can save a lot on meta scan, improve later.
encodedName = RegionInfo.encodeRegionName(spec.getValue().toByteArray());
}
RegionInfo info = this.master.getAssignmentManager().loadRegionFromMeta(encodedName);
LOG.trace("region info loaded from meta table: {}", info);
RegionState prevState = this.master.getAssignmentManager().getRegionStates().
getRegionState(info);
RegionState newState = RegionState.convert(s);
RegionState prevState =
this.master.getAssignmentManager().getRegionStates().getRegionState(info);
RegionState.State newState = RegionState.State.convert(s.getState());
LOG.info("{} set region={} state from {} to {}", master.getClientIdAuditPrefix(), info,
prevState.getState(), newState.getState());
prevState.getState(), newState);
Put metaPut = MetaTableAccessor.makePutFromRegionInfo(info, System.currentTimeMillis());
metaPut.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
Bytes.toBytes(newState.getState().name()));
Bytes.toBytes(newState.name()));
List<Put> putList = new ArrayList<>();
putList.add(metaPut);
MetaTableAccessor.putsToMetaTable(this.master.getConnection(), putList);
//Loads from meta again to refresh AM cache with the new region state
this.master.getAssignmentManager().loadRegionFromMeta(info.getEncodedName());
builder.addStates(prevState.convert());
} catch (Exception e) {
throw new ServiceException(e);
// Loads from meta again to refresh AM cache with the new region state
this.master.getAssignmentManager().loadRegionFromMeta(encodedName);
builder.addStates(RegionSpecifierAndState.newBuilder().setRegionSpecifier(spec)
.setState(prevState.getState().convert()));
}
} catch (Exception e) {
throw new ServiceException(e);
}
return builder.build();
}

View File

@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
@ -190,31 +189,30 @@ public class TestHbck {
@Test
public void testSetRegionStateInMeta() throws Exception {
Hbck hbck = getHbck();
try(Admin admin = TEST_UTIL.getAdmin()){
final List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
final List<RegionState> prevStates = new ArrayList<>();
final List<RegionState> newStates = new ArrayList<>();
final Map<String, Pair<RegionState, RegionState>> regionsMap = new HashMap<>();
regions.forEach(r -> {
RegionState prevState = am.getRegionStates().getRegionState(r);
prevStates.add(prevState);
RegionState newState = RegionState.createForTesting(r, RegionState.State.CLOSED);
newStates.add(newState);
regionsMap.put(r.getEncodedName(), new Pair<>(prevState, newState));
});
final List<RegionState> result = hbck.setRegionStateInMeta(newStates);
result.forEach(r -> {
RegionState prevState = regionsMap.get(r.getRegion().getEncodedName()).getFirst();
assertEquals(prevState.getState(), r.getState());
});
regions.forEach(r -> {
RegionState cachedState = am.getRegionStates().getRegionState(r.getEncodedName());
RegionState newState = regionsMap.get(r.getEncodedName()).getSecond();
assertEquals(newState.getState(), cachedState.getState());
});
hbck.setRegionStateInMeta(prevStates);
}
Admin admin = TEST_UTIL.getAdmin();
final List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
Map<String, RegionState.State> prevStates = new HashMap<>();
Map<String, RegionState.State> newStates = new HashMap<>();
final Map<String, Pair<RegionState.State, RegionState.State>> regionsMap = new HashMap<>();
regions.forEach(r -> {
RegionState prevState = am.getRegionStates().getRegionState(r);
prevStates.put(r.getEncodedName(), prevState.getState());
newStates.put(r.getEncodedName(), RegionState.State.CLOSED);
regionsMap.put(r.getEncodedName(),
new Pair<>(prevState.getState(), RegionState.State.CLOSED));
});
final Map<String, RegionState.State> result = hbck.setRegionStateInMeta(newStates);
result.forEach((k, v) -> {
RegionState.State prevState = regionsMap.get(k).getFirst();
assertEquals(prevState, v);
});
regions.forEach(r -> {
RegionState cachedState = am.getRegionStates().getRegionState(r.getEncodedName());
RegionState.State newState = regionsMap.get(r.getEncodedName()).getSecond();
assertEquals(newState, cachedState.getState());
});
hbck.setRegionStateInMeta(prevStates);
}
@Test