001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019package org.apache.hadoop.hbase.master; 020 021import static org.apache.hadoop.hbase.master.MasterWalManager.META_FILTER; 022 023import java.io.IOException; 024import java.net.BindException; 025import java.net.InetAddress; 026import java.net.InetSocketAddress; 027import java.util.ArrayList; 028import java.util.HashMap; 029import java.util.HashSet; 030import java.util.List; 031import java.util.Map; 032import java.util.Map.Entry; 033import java.util.Set; 034import java.util.function.BiFunction; 035import java.util.stream.Collectors; 036import org.apache.hadoop.conf.Configuration; 037import org.apache.hadoop.fs.Path; 038import org.apache.hadoop.hbase.ClusterMetricsBuilder; 039import org.apache.hadoop.hbase.DoNotRetryIOException; 040import org.apache.hadoop.hbase.HConstants; 041import org.apache.hadoop.hbase.MetaTableAccessor; 042import org.apache.hadoop.hbase.NamespaceDescriptor; 043import org.apache.hadoop.hbase.Server; 044import org.apache.hadoop.hbase.ServerMetrics; 045import org.apache.hadoop.hbase.ServerMetricsBuilder; 046import org.apache.hadoop.hbase.ServerName; 047import org.apache.hadoop.hbase.TableName; 048import org.apache.hadoop.hbase.UnknownRegionException; 049import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 050import org.apache.hadoop.hbase.client.MasterSwitchType; 051import org.apache.hadoop.hbase.client.RegionInfo; 052import org.apache.hadoop.hbase.client.RegionInfoBuilder; 053import org.apache.hadoop.hbase.client.TableDescriptor; 054import org.apache.hadoop.hbase.client.TableState; 055import org.apache.hadoop.hbase.client.VersionInfoUtil; 056import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; 057import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; 058import org.apache.hadoop.hbase.errorhandling.ForeignException; 059import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; 060import org.apache.hadoop.hbase.io.hfile.HFile; 061import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; 062import org.apache.hadoop.hbase.ipc.PriorityFunction; 063import org.apache.hadoop.hbase.ipc.QosPriority; 064import org.apache.hadoop.hbase.ipc.RpcServer; 065import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; 066import org.apache.hadoop.hbase.ipc.RpcServerFactory; 067import org.apache.hadoop.hbase.ipc.RpcServerInterface; 068import org.apache.hadoop.hbase.ipc.ServerRpcController; 069import org.apache.hadoop.hbase.master.assignment.RegionStates; 070import org.apache.hadoop.hbase.master.locking.LockProcedure; 071import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; 072import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil; 073import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable; 074import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure; 075import org.apache.hadoop.hbase.mob.MobUtils; 076import org.apache.hadoop.hbase.procedure.MasterProcedureManager; 077import org.apache.hadoop.hbase.procedure2.LockType; 078import org.apache.hadoop.hbase.procedure2.LockedResource; 079import org.apache.hadoop.hbase.procedure2.Procedure; 080import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; 081import org.apache.hadoop.hbase.procedure2.ProcedureUtil; 082import org.apache.hadoop.hbase.procedure2.RemoteProcedureException; 083import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService; 084import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService; 085import org.apache.hadoop.hbase.quotas.MasterQuotaManager; 086import org.apache.hadoop.hbase.quotas.QuotaObserverChore; 087import org.apache.hadoop.hbase.quotas.QuotaUtil; 088import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; 089import org.apache.hadoop.hbase.regionserver.RSRpcServices; 090import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory; 091import org.apache.hadoop.hbase.replication.ReplicationException; 092import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; 093import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; 094import org.apache.hadoop.hbase.security.User; 095import org.apache.hadoop.hbase.security.access.AccessChecker; 096import org.apache.hadoop.hbase.security.access.AccessController; 097import org.apache.hadoop.hbase.security.access.Permission; 098import org.apache.hadoop.hbase.security.visibility.VisibilityController; 099import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; 100import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; 101import org.apache.hadoop.hbase.util.Bytes; 102import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 103import org.apache.hadoop.hbase.util.ForeignExceptionUtil; 104import org.apache.hadoop.hbase.util.Pair; 105import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; 106import org.apache.yetus.audience.InterfaceAudience; 107import org.apache.zookeeper.KeeperException; 108import org.slf4j.Logger; 109import org.slf4j.LoggerFactory; 110 111import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; 112import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; 113import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; 114 115import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 116import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; 117import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; 118import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse; 119import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest; 120import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; 121import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; 122import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; 123import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; 124import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; 125import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair; 126import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; 127import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; 128import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo; 129import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest; 130import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse; 131import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest; 132import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse; 133import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService; 134import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; 135import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest; 136import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse; 137import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest; 138import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse; 139import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest; 140import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse; 141import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; 142import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse; 143import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest; 144import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse; 145import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest; 146import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse; 147import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; 148import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse; 149import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersRequest; 150import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersResponse; 151import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest; 152import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse; 153import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest; 154import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse; 155import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest; 156import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse; 157import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest; 158import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse; 159import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest; 160import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse; 161import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest; 162import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse; 163import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest; 164import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse; 165import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest; 166import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse; 167import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest; 168import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse; 169import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest; 170import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse; 171import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest; 172import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse; 173import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest; 174import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse; 175import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest; 176import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse; 177import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest; 178import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse; 179import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest; 180import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse; 181import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; 182import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse; 183import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; 184import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse; 185import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest; 186import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse; 187import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService; 188import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest; 189import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse; 190import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; 191import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse; 192import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest; 193import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse; 194import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest; 195import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse; 196import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest; 197import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse; 198import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest; 199import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse; 200import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest; 201import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse; 202import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; 203import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; 204import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest; 205import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse; 206import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest; 207import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse; 208import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest; 209import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse; 210import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest; 211import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse; 212import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest; 213import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse; 214import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest; 215import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest; 216import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse; 217import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService; 218import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; 219import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse; 220import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; 221import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse; 222import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; 223import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse; 224import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; 225import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse; 226import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; 227import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse; 228import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; 229import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse; 230import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest; 231import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse; 232import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest; 233import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse; 234import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest; 235import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse; 236import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest; 237import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse; 238import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest; 239import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse; 240import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest; 241import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse; 242import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest; 243import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse; 244import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest; 245import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse; 246import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest; 247import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse; 248import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest; 249import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse; 250import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest; 251import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse; 252import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest; 253import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest; 254import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse; 255import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest; 256import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; 257import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest; 258import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse; 259import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest; 260import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse; 261import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest; 262import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse; 263import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest; 264import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse; 265import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest; 266import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse; 267import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot; 268import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot; 269import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest; 270import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse; 271import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes; 272import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; 273import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse; 274import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; 275import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse; 276import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; 277import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse; 278import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService; 279import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse; 280import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest; 281import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse; 282import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult; 283import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest; 284import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse; 285import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; 286import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse; 287import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; 288import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; 289import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest; 290import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse; 291import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest; 292import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse; 293import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest; 294import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse; 295import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest; 296import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse; 297import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest; 298import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse; 299import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest; 300import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; 301import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState; 302import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; 303import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; 304import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; 305 306/** 307 * Implements the master RPC services. 308 */ 309@InterfaceAudience.Private 310@SuppressWarnings("deprecation") 311public class MasterRpcServices extends RSRpcServices 312 implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface, 313 LockService.BlockingInterface, HbckService.BlockingInterface { 314 private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName()); 315 316 private final HMaster master; 317 318 /** 319 * @return Subset of configuration to pass initializing regionservers: e.g. 320 * the filesystem to use and root directory to use. 321 */ 322 private RegionServerStartupResponse.Builder createConfigurationSubset() { 323 RegionServerStartupResponse.Builder resp = addConfig( 324 RegionServerStartupResponse.newBuilder(), HConstants.HBASE_DIR); 325 resp = addConfig(resp, "fs.defaultFS"); 326 return addConfig(resp, "hbase.master.info.port"); 327 } 328 329 private RegionServerStartupResponse.Builder addConfig( 330 final RegionServerStartupResponse.Builder resp, final String key) { 331 NameStringPair.Builder entry = NameStringPair.newBuilder() 332 .setName(key) 333 .setValue(master.getConfiguration().get(key)); 334 resp.addMapEntries(entry.build()); 335 return resp; 336 } 337 338 public MasterRpcServices(HMaster m) throws IOException { 339 super(m); 340 master = m; 341 } 342 343 @Override 344 protected Class<?> getRpcSchedulerFactoryClass() { 345 Configuration conf = getConfiguration(); 346 if (conf != null) { 347 return conf.getClass(MASTER_RPC_SCHEDULER_FACTORY_CLASS, super.getRpcSchedulerFactoryClass()); 348 } else { 349 return super.getRpcSchedulerFactoryClass(); 350 } 351 } 352 353 @Override 354 protected RpcServerInterface createRpcServer(Server server, Configuration conf, 355 RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name) 356 throws IOException { 357 // RpcServer at HM by default enable ByteBufferPool iff HM having user table region in it 358 boolean reservoirEnabled = conf.getBoolean(RESERVOIR_ENABLED_KEY, 359 LoadBalancer.isMasterCanHostUserRegions(conf)); 360 try { 361 return RpcServerFactory.createRpcServer(server, name, getServices(), 362 bindAddress, // use final bindAddress for this server. 363 conf, rpcSchedulerFactory.create(conf, this, server), reservoirEnabled); 364 } catch (BindException be) { 365 throw new IOException(be.getMessage() + ". To switch ports use the '" 366 + HConstants.MASTER_PORT + "' configuration property.", 367 be.getCause() != null ? be.getCause() : be); 368 } 369 } 370 371 @Override 372 protected PriorityFunction createPriority() { 373 return new MasterAnnotationReadingPriorityFunction(this); 374 } 375 376 /** 377 * Checks for the following pre-checks in order: 378 * <ol> 379 * <li>Master is initialized</li> 380 * <li>Rpc caller has admin permissions</li> 381 * </ol> 382 * @param requestName name of rpc request. Used in reporting failures to provide context. 383 * @throws ServiceException If any of the above listed pre-check fails. 384 */ 385 private void rpcPreCheck(String requestName) throws ServiceException { 386 try { 387 master.checkInitialized(); 388 requirePermission(requestName, Permission.Action.ADMIN); 389 } catch (IOException ioe) { 390 throw new ServiceException(ioe); 391 } 392 } 393 394 enum BalanceSwitchMode { 395 SYNC, 396 ASYNC 397 } 398 399 /** 400 * Assigns balancer switch according to BalanceSwitchMode 401 * @param b new balancer switch 402 * @param mode BalanceSwitchMode 403 * @return old balancer switch 404 */ 405 boolean switchBalancer(final boolean b, BalanceSwitchMode mode) throws IOException { 406 boolean oldValue = master.loadBalancerTracker.isBalancerOn(); 407 boolean newValue = b; 408 try { 409 if (master.cpHost != null) { 410 master.cpHost.preBalanceSwitch(newValue); 411 } 412 try { 413 if (mode == BalanceSwitchMode.SYNC) { 414 synchronized (master.getLoadBalancer()) { 415 master.loadBalancerTracker.setBalancerOn(newValue); 416 } 417 } else { 418 master.loadBalancerTracker.setBalancerOn(newValue); 419 } 420 } catch (KeeperException ke) { 421 throw new IOException(ke); 422 } 423 LOG.info(master.getClientIdAuditPrefix() + " set balanceSwitch=" + newValue); 424 if (master.cpHost != null) { 425 master.cpHost.postBalanceSwitch(oldValue, newValue); 426 } 427 master.getLoadBalancer().updateBalancerStatus(newValue); 428 } catch (IOException ioe) { 429 LOG.warn("Error flipping balance switch", ioe); 430 } 431 return oldValue; 432 } 433 434 boolean synchronousBalanceSwitch(final boolean b) throws IOException { 435 return switchBalancer(b, BalanceSwitchMode.SYNC); 436 } 437 438 /** 439 * @return list of blocking services and their security info classes that this server supports 440 */ 441 @Override 442 protected List<BlockingServiceAndInterface> getServices() { 443 List<BlockingServiceAndInterface> bssi = new ArrayList<>(5); 444 bssi.add(new BlockingServiceAndInterface( 445 MasterService.newReflectiveBlockingService(this), 446 MasterService.BlockingInterface.class)); 447 bssi.add(new BlockingServiceAndInterface( 448 RegionServerStatusService.newReflectiveBlockingService(this), 449 RegionServerStatusService.BlockingInterface.class)); 450 bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this), 451 LockService.BlockingInterface.class)); 452 bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this), 453 HbckService.BlockingInterface.class)); 454 bssi.addAll(super.getServices()); 455 return bssi; 456 } 457 458 @Override 459 @QosPriority(priority = HConstants.ADMIN_QOS) 460 public GetLastFlushedSequenceIdResponse getLastFlushedSequenceId(RpcController controller, 461 GetLastFlushedSequenceIdRequest request) throws ServiceException { 462 try { 463 master.checkServiceStarted(); 464 } catch (IOException ioe) { 465 throw new ServiceException(ioe); 466 } 467 byte[] encodedRegionName = request.getRegionName().toByteArray(); 468 RegionStoreSequenceIds ids = master.getServerManager() 469 .getLastFlushedSequenceId(encodedRegionName); 470 return ResponseConverter.buildGetLastFlushedSequenceIdResponse(ids); 471 } 472 473 @Override 474 public RegionServerReportResponse regionServerReport(RpcController controller, 475 RegionServerReportRequest request) throws ServiceException { 476 try { 477 master.checkServiceStarted(); 478 int versionNumber = 0; 479 String version = "0.0.0"; 480 VersionInfo versionInfo = VersionInfoUtil.getCurrentClientVersionInfo(); 481 if (versionInfo != null) { 482 version = versionInfo.getVersion(); 483 versionNumber = VersionInfoUtil.getVersionNumber(versionInfo); 484 } 485 ClusterStatusProtos.ServerLoad sl = request.getLoad(); 486 ServerName serverName = ProtobufUtil.toServerName(request.getServer()); 487 ServerMetrics oldLoad = master.getServerManager().getLoad(serverName); 488 ServerMetrics newLoad = 489 ServerMetricsBuilder.toServerMetrics(serverName, versionNumber, version, sl); 490 master.getServerManager().regionServerReport(serverName, newLoad); 491 master.getAssignmentManager().reportOnlineRegions(serverName, 492 newLoad.getRegionMetrics().keySet()); 493 if (sl != null && master.metricsMaster != null) { 494 // Up our metrics. 495 master.metricsMaster.incrementRequests( 496 sl.getTotalNumberOfRequests() - (oldLoad != null ? oldLoad.getRequestCount() : 0)); 497 } 498 } catch (IOException ioe) { 499 throw new ServiceException(ioe); 500 } 501 return RegionServerReportResponse.newBuilder().build(); 502 } 503 504 @Override 505 public RegionServerStartupResponse regionServerStartup(RpcController controller, 506 RegionServerStartupRequest request) throws ServiceException { 507 // Register with server manager 508 try { 509 master.checkServiceStarted(); 510 int versionNumber = 0; 511 String version = "0.0.0"; 512 VersionInfo versionInfo = VersionInfoUtil.getCurrentClientVersionInfo(); 513 if (versionInfo != null) { 514 version = versionInfo.getVersion(); 515 versionNumber = VersionInfoUtil.getVersionNumber(versionInfo); 516 } 517 InetAddress ia = master.getRemoteInetAddress(request.getPort(), request.getServerStartCode()); 518 // if regionserver passed hostname to use, 519 // then use it instead of doing a reverse DNS lookup 520 ServerName rs = 521 master.getServerManager().regionServerStartup(request, versionNumber, version, ia); 522 523 // Send back some config info 524 RegionServerStartupResponse.Builder resp = createConfigurationSubset(); 525 NameStringPair.Builder entry = NameStringPair.newBuilder() 526 .setName(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER).setValue(rs.getHostname()); 527 resp.addMapEntries(entry.build()); 528 529 return resp.build(); 530 } catch (IOException ioe) { 531 throw new ServiceException(ioe); 532 } 533 } 534 535 @Override 536 public ReportRSFatalErrorResponse reportRSFatalError( 537 RpcController controller, ReportRSFatalErrorRequest request) throws ServiceException { 538 String errorText = request.getErrorMessage(); 539 ServerName sn = ProtobufUtil.toServerName(request.getServer()); 540 String msg = sn + " reported a fatal error:\n" + errorText; 541 LOG.warn(msg); 542 master.rsFatals.add(msg); 543 return ReportRSFatalErrorResponse.newBuilder().build(); 544 } 545 546 @Override 547 public AddColumnResponse addColumn(RpcController controller, 548 AddColumnRequest req) throws ServiceException { 549 try { 550 long procId = master.addColumn( 551 ProtobufUtil.toTableName(req.getTableName()), 552 ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()), 553 req.getNonceGroup(), 554 req.getNonce()); 555 if (procId == -1) { 556 // This mean operation was not performed in server, so do not set any procId 557 return AddColumnResponse.newBuilder().build(); 558 } else { 559 return AddColumnResponse.newBuilder().setProcId(procId).build(); 560 } 561 } catch (IOException ioe) { 562 throw new ServiceException(ioe); 563 } 564 } 565 566 @Override 567 public AssignRegionResponse assignRegion(RpcController controller, 568 AssignRegionRequest req) throws ServiceException { 569 try { 570 master.checkInitialized(); 571 572 final RegionSpecifierType type = req.getRegion().getType(); 573 if (type != RegionSpecifierType.REGION_NAME) { 574 LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME 575 + " actual: " + type); 576 } 577 578 final byte[] regionName = req.getRegion().getValue().toByteArray(); 579 final RegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName); 580 if (regionInfo == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName)); 581 582 final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build(); 583 if (master.cpHost != null) { 584 master.cpHost.preAssign(regionInfo); 585 } 586 LOG.info(master.getClientIdAuditPrefix() + " assign " + regionInfo.getRegionNameAsString()); 587 master.getAssignmentManager().assign(regionInfo); 588 if (master.cpHost != null) { 589 master.cpHost.postAssign(regionInfo); 590 } 591 return arr; 592 } catch (IOException ioe) { 593 throw new ServiceException(ioe); 594 } 595 } 596 597 598 @Override 599 public BalanceResponse balance(RpcController controller, 600 BalanceRequest request) throws ServiceException { 601 try { 602 return BalanceResponse.newBuilder().setBalancerRan(master.balance( 603 request.hasForce()? request.getForce(): false)).build(); 604 } catch (IOException ex) { 605 throw new ServiceException(ex); 606 } 607 } 608 609 @Override 610 public CreateNamespaceResponse createNamespace(RpcController controller, 611 CreateNamespaceRequest request) throws ServiceException { 612 try { 613 long procId = master.createNamespace( 614 ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()), 615 request.getNonceGroup(), 616 request.getNonce()); 617 return CreateNamespaceResponse.newBuilder().setProcId(procId).build(); 618 } catch (IOException e) { 619 throw new ServiceException(e); 620 } 621 } 622 623 @Override 624 public CreateTableResponse createTable(RpcController controller, CreateTableRequest req) 625 throws ServiceException { 626 TableDescriptor tableDescriptor = ProtobufUtil.toTableDescriptor(req.getTableSchema()); 627 byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req); 628 try { 629 long procId = 630 master.createTable(tableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce()); 631 LOG.info(master.getClientIdAuditPrefix() + " procedure request for creating table: " + 632 req.getTableSchema().getTableName() + " procId is: " + procId); 633 return CreateTableResponse.newBuilder().setProcId(procId).build(); 634 } catch (IOException ioe) { 635 throw new ServiceException(ioe); 636 } 637 } 638 639 @Override 640 public DeleteColumnResponse deleteColumn(RpcController controller, 641 DeleteColumnRequest req) throws ServiceException { 642 try { 643 long procId = master.deleteColumn( 644 ProtobufUtil.toTableName(req.getTableName()), 645 req.getColumnName().toByteArray(), 646 req.getNonceGroup(), 647 req.getNonce()); 648 if (procId == -1) { 649 // This mean operation was not performed in server, so do not set any procId 650 return DeleteColumnResponse.newBuilder().build(); 651 } else { 652 return DeleteColumnResponse.newBuilder().setProcId(procId).build(); 653 } 654 } catch (IOException ioe) { 655 throw new ServiceException(ioe); 656 } 657 } 658 659 @Override 660 public DeleteNamespaceResponse deleteNamespace(RpcController controller, 661 DeleteNamespaceRequest request) throws ServiceException { 662 try { 663 long procId = master.deleteNamespace( 664 request.getNamespaceName(), 665 request.getNonceGroup(), 666 request.getNonce()); 667 return DeleteNamespaceResponse.newBuilder().setProcId(procId).build(); 668 } catch (IOException e) { 669 throw new ServiceException(e); 670 } 671 } 672 673 /** 674 * Execute Delete Snapshot operation. 675 * @return DeleteSnapshotResponse (a protobuf wrapped void) if the snapshot existed and was 676 * deleted properly. 677 * @throws ServiceException wrapping SnapshotDoesNotExistException if specified snapshot did not 678 * exist. 679 */ 680 @Override 681 public DeleteSnapshotResponse deleteSnapshot(RpcController controller, 682 DeleteSnapshotRequest request) throws ServiceException { 683 try { 684 master.checkInitialized(); 685 master.snapshotManager.checkSnapshotSupport(); 686 687 LOG.info(master.getClientIdAuditPrefix() + " delete " + request.getSnapshot()); 688 master.snapshotManager.deleteSnapshot(request.getSnapshot()); 689 return DeleteSnapshotResponse.newBuilder().build(); 690 } catch (IOException e) { 691 throw new ServiceException(e); 692 } 693 } 694 695 @Override 696 public DeleteTableResponse deleteTable(RpcController controller, 697 DeleteTableRequest request) throws ServiceException { 698 try { 699 long procId = master.deleteTable(ProtobufUtil.toTableName( 700 request.getTableName()), request.getNonceGroup(), request.getNonce()); 701 return DeleteTableResponse.newBuilder().setProcId(procId).build(); 702 } catch (IOException ioe) { 703 throw new ServiceException(ioe); 704 } 705 } 706 707 @Override 708 public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request) 709 throws ServiceException { 710 try { 711 long procId = master.truncateTable( 712 ProtobufUtil.toTableName(request.getTableName()), 713 request.getPreserveSplits(), 714 request.getNonceGroup(), 715 request.getNonce()); 716 return TruncateTableResponse.newBuilder().setProcId(procId).build(); 717 } catch (IOException ioe) { 718 throw new ServiceException(ioe); 719 } 720 } 721 722 @Override 723 public DisableTableResponse disableTable(RpcController controller, 724 DisableTableRequest request) throws ServiceException { 725 try { 726 long procId = master.disableTable( 727 ProtobufUtil.toTableName(request.getTableName()), 728 request.getNonceGroup(), 729 request.getNonce()); 730 return DisableTableResponse.newBuilder().setProcId(procId).build(); 731 } catch (IOException ioe) { 732 throw new ServiceException(ioe); 733 } 734 } 735 736 @Override 737 public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c, 738 EnableCatalogJanitorRequest req) throws ServiceException { 739 rpcPreCheck("enableCatalogJanitor"); 740 return EnableCatalogJanitorResponse.newBuilder().setPrevValue( 741 master.catalogJanitorChore.setEnabled(req.getEnable())).build(); 742 } 743 744 @Override 745 public SetCleanerChoreRunningResponse setCleanerChoreRunning( 746 RpcController c, SetCleanerChoreRunningRequest req) throws ServiceException { 747 rpcPreCheck("setCleanerChoreRunning"); 748 749 boolean prevValue = 750 master.getLogCleaner().getEnabled() && master.getHFileCleaner().getEnabled(); 751 master.getLogCleaner().setEnabled(req.getOn()); 752 master.getHFileCleaner().setEnabled(req.getOn()); 753 return SetCleanerChoreRunningResponse.newBuilder().setPrevValue(prevValue).build(); 754 } 755 756 @Override 757 public EnableTableResponse enableTable(RpcController controller, 758 EnableTableRequest request) throws ServiceException { 759 try { 760 long procId = master.enableTable( 761 ProtobufUtil.toTableName(request.getTableName()), 762 request.getNonceGroup(), 763 request.getNonce()); 764 return EnableTableResponse.newBuilder().setProcId(procId).build(); 765 } catch (IOException ioe) { 766 throw new ServiceException(ioe); 767 } 768 } 769 770 @Override 771 public MergeTableRegionsResponse mergeTableRegions( 772 RpcController c, MergeTableRegionsRequest request) throws ServiceException { 773 try { 774 master.checkInitialized(); 775 } catch (IOException ioe) { 776 throw new ServiceException(ioe); 777 } 778 779 RegionStates regionStates = master.getAssignmentManager().getRegionStates(); 780 781 assert(request.getRegionCount() == 2); 782 RegionInfo[] regionsToMerge = new RegionInfo[request.getRegionCount()]; 783 for (int i = 0; i < request.getRegionCount(); i++) { 784 final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray(); 785 if (request.getRegion(i).getType() != RegionSpecifierType.ENCODED_REGION_NAME) { 786 LOG.warn("MergeRegions specifier type: expected: " 787 + RegionSpecifierType.ENCODED_REGION_NAME + " actual: region " + i + " =" 788 + request.getRegion(i).getType()); 789 } 790 RegionState regionState = regionStates.getRegionState(Bytes.toString(encodedNameOfRegion)); 791 if (regionState == null) { 792 throw new ServiceException( 793 new UnknownRegionException(Bytes.toStringBinary(encodedNameOfRegion))); 794 } 795 regionsToMerge[i] = regionState.getRegion(); 796 } 797 798 try { 799 long procId = master.mergeRegions( 800 regionsToMerge, 801 request.getForcible(), 802 request.getNonceGroup(), 803 request.getNonce()); 804 return MergeTableRegionsResponse.newBuilder().setProcId(procId).build(); 805 } catch (IOException ioe) { 806 throw new ServiceException(ioe); 807 } 808 } 809 810 @Override 811 public SplitTableRegionResponse splitRegion(final RpcController controller, 812 final SplitTableRegionRequest request) throws ServiceException { 813 try { 814 long procId = master.splitRegion( 815 ProtobufUtil.toRegionInfo(request.getRegionInfo()), 816 request.hasSplitRow() ? request.getSplitRow().toByteArray() : null, 817 request.getNonceGroup(), 818 request.getNonce()); 819 return SplitTableRegionResponse.newBuilder().setProcId(procId).build(); 820 } catch (IOException ie) { 821 throw new ServiceException(ie); 822 } 823 } 824 825 @Override 826 public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller, 827 final ClientProtos.CoprocessorServiceRequest request) throws ServiceException { 828 rpcPreCheck("execMasterService"); 829 try { 830 ServerRpcController execController = new ServerRpcController(); 831 ClientProtos.CoprocessorServiceCall call = request.getCall(); 832 String serviceName = call.getServiceName(); 833 String methodName = call.getMethodName(); 834 if (!master.coprocessorServiceHandlers.containsKey(serviceName)) { 835 throw new UnknownProtocolException(null, 836 "No registered Master Coprocessor Endpoint found for " + serviceName + 837 ". Has it been enabled?"); 838 } 839 840 com.google.protobuf.Service service = master.coprocessorServiceHandlers.get(serviceName); 841 com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType(); 842 com.google.protobuf.Descriptors.MethodDescriptor methodDesc = 843 CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc); 844 845 com.google.protobuf.Message execRequest = 846 CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest()); 847 final com.google.protobuf.Message.Builder responseBuilder = 848 service.getResponsePrototype(methodDesc).newBuilderForType(); 849 service.callMethod(methodDesc, execController, execRequest, 850 (message) -> { 851 if (message != null) { 852 responseBuilder.mergeFrom(message); 853 } 854 }); 855 com.google.protobuf.Message execResult = responseBuilder.build(); 856 if (execController.getFailedOn() != null) { 857 throw execController.getFailedOn(); 858 } 859 return CoprocessorRpcUtils.getResponse(execResult, HConstants.EMPTY_BYTE_ARRAY); 860 } catch (IOException ie) { 861 throw new ServiceException(ie); 862 } 863 } 864 865 /** 866 * Triggers an asynchronous attempt to run a distributed procedure. 867 * {@inheritDoc} 868 */ 869 @Override 870 public ExecProcedureResponse execProcedure(RpcController controller, 871 ExecProcedureRequest request) throws ServiceException { 872 try { 873 master.checkInitialized(); 874 ProcedureDescription desc = request.getProcedure(); 875 MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager( 876 desc.getSignature()); 877 if (mpm == null) { 878 throw new ServiceException(new DoNotRetryIOException("The procedure is not registered: " 879 + desc.getSignature())); 880 } 881 LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature()); 882 mpm.checkPermissions(desc, accessChecker, RpcServer.getRequestUser().orElse(null)); 883 mpm.execProcedure(desc); 884 // send back the max amount of time the client should wait for the procedure 885 // to complete 886 long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME; 887 return ExecProcedureResponse.newBuilder().setExpectedTimeout( 888 waitTime).build(); 889 } catch (ForeignException e) { 890 throw new ServiceException(e.getCause()); 891 } catch (IOException e) { 892 throw new ServiceException(e); 893 } 894 } 895 896 /** 897 * Triggers a synchronous attempt to run a distributed procedure and sets 898 * return data in response. 899 * {@inheritDoc} 900 */ 901 @Override 902 public ExecProcedureResponse execProcedureWithRet(RpcController controller, 903 ExecProcedureRequest request) throws ServiceException { 904 rpcPreCheck("execProcedureWithRet"); 905 try { 906 ProcedureDescription desc = request.getProcedure(); 907 MasterProcedureManager mpm = 908 master.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature()); 909 if (mpm == null) { 910 throw new ServiceException("The procedure is not registered: " + desc.getSignature()); 911 } 912 LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature()); 913 byte[] data = mpm.execProcedureWithRet(desc); 914 ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder(); 915 // set return data if available 916 if (data != null) { 917 builder.setReturnData(UnsafeByteOperations.unsafeWrap(data)); 918 } 919 return builder.build(); 920 } catch (IOException e) { 921 throw new ServiceException(e); 922 } 923 } 924 925 @Override 926 public GetClusterStatusResponse getClusterStatus(RpcController controller, 927 GetClusterStatusRequest req) throws ServiceException { 928 GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder(); 929 try { 930 // We used to check if Master was up at this point but let this call proceed even if 931 // Master is initializing... else we shut out stuff like hbck2 tool from making progress 932 // since it queries this method to figure cluster version. hbck2 wants to be able to work 933 // against Master even if it is 'initializing' so it can do fixup. 934 response.setClusterStatus(ClusterMetricsBuilder.toClusterStatus( 935 master.getClusterMetrics(ClusterMetricsBuilder.toOptions(req.getOptionsList())))); 936 } catch (IOException e) { 937 throw new ServiceException(e); 938 } 939 return response.build(); 940 } 941 942 /** 943 * List the currently available/stored snapshots. Any in-progress snapshots are ignored 944 */ 945 @Override 946 public GetCompletedSnapshotsResponse getCompletedSnapshots(RpcController controller, 947 GetCompletedSnapshotsRequest request) throws ServiceException { 948 try { 949 master.checkInitialized(); 950 GetCompletedSnapshotsResponse.Builder builder = GetCompletedSnapshotsResponse.newBuilder(); 951 List<SnapshotDescription> snapshots = master.snapshotManager.getCompletedSnapshots(); 952 953 // convert to protobuf 954 for (SnapshotDescription snapshot : snapshots) { 955 builder.addSnapshots(snapshot); 956 } 957 return builder.build(); 958 } catch (IOException e) { 959 throw new ServiceException(e); 960 } 961 } 962 963 @Override 964 public GetNamespaceDescriptorResponse getNamespaceDescriptor( 965 RpcController controller, GetNamespaceDescriptorRequest request) 966 throws ServiceException { 967 try { 968 return GetNamespaceDescriptorResponse.newBuilder() 969 .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor( 970 master.getNamespace(request.getNamespaceName()))) 971 .build(); 972 } catch (IOException e) { 973 throw new ServiceException(e); 974 } 975 } 976 977 /** 978 * Get the number of regions of the table that have been updated by the alter. 979 * 980 * @return Pair indicating the number of regions updated Pair.getFirst is the 981 * regions that are yet to be updated Pair.getSecond is the total number 982 * of regions of the table 983 * @throws ServiceException 984 */ 985 @Override 986 public GetSchemaAlterStatusResponse getSchemaAlterStatus( 987 RpcController controller, GetSchemaAlterStatusRequest req) throws ServiceException { 988 // TODO: currently, we query using the table name on the client side. this 989 // may overlap with other table operations or the table operation may 990 // have completed before querying this API. We need to refactor to a 991 // transaction system in the future to avoid these ambiguities. 992 TableName tableName = ProtobufUtil.toTableName(req.getTableName()); 993 994 try { 995 master.checkInitialized(); 996 Pair<Integer,Integer> pair = master.getAssignmentManager().getReopenStatus(tableName); 997 GetSchemaAlterStatusResponse.Builder ret = GetSchemaAlterStatusResponse.newBuilder(); 998 ret.setYetToUpdateRegions(pair.getFirst()); 999 ret.setTotalRegions(pair.getSecond()); 1000 return ret.build(); 1001 } catch (IOException ioe) { 1002 throw new ServiceException(ioe); 1003 } 1004 } 1005 1006 /** 1007 * Get list of TableDescriptors for requested tables. 1008 * @param c Unused (set to null). 1009 * @param req GetTableDescriptorsRequest that contains: 1010 * - tableNames: requested tables, or if empty, all are requested 1011 * @return GetTableDescriptorsResponse 1012 * @throws ServiceException 1013 */ 1014 @Override 1015 public GetTableDescriptorsResponse getTableDescriptors(RpcController c, 1016 GetTableDescriptorsRequest req) throws ServiceException { 1017 try { 1018 master.checkInitialized(); 1019 1020 final String regex = req.hasRegex() ? req.getRegex() : null; 1021 final String namespace = req.hasNamespace() ? req.getNamespace() : null; 1022 List<TableName> tableNameList = null; 1023 if (req.getTableNamesCount() > 0) { 1024 tableNameList = new ArrayList<TableName>(req.getTableNamesCount()); 1025 for (HBaseProtos.TableName tableNamePB: req.getTableNamesList()) { 1026 tableNameList.add(ProtobufUtil.toTableName(tableNamePB)); 1027 } 1028 } 1029 1030 List<TableDescriptor> descriptors = master.listTableDescriptors(namespace, regex, 1031 tableNameList, req.getIncludeSysTables()); 1032 1033 GetTableDescriptorsResponse.Builder builder = GetTableDescriptorsResponse.newBuilder(); 1034 if (descriptors != null && descriptors.size() > 0) { 1035 // Add the table descriptors to the response 1036 for (TableDescriptor htd: descriptors) { 1037 builder.addTableSchema(ProtobufUtil.toTableSchema(htd)); 1038 } 1039 } 1040 return builder.build(); 1041 } catch (IOException ioe) { 1042 throw new ServiceException(ioe); 1043 } 1044 } 1045 1046 /** 1047 * Get list of userspace table names 1048 * @param controller Unused (set to null). 1049 * @param req GetTableNamesRequest 1050 * @return GetTableNamesResponse 1051 * @throws ServiceException 1052 */ 1053 @Override 1054 public GetTableNamesResponse getTableNames(RpcController controller, 1055 GetTableNamesRequest req) throws ServiceException { 1056 try { 1057 master.checkServiceStarted(); 1058 1059 final String regex = req.hasRegex() ? req.getRegex() : null; 1060 final String namespace = req.hasNamespace() ? req.getNamespace() : null; 1061 List<TableName> tableNames = master.listTableNames(namespace, regex, 1062 req.getIncludeSysTables()); 1063 1064 GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder(); 1065 if (tableNames != null && tableNames.size() > 0) { 1066 // Add the table names to the response 1067 for (TableName table: tableNames) { 1068 builder.addTableNames(ProtobufUtil.toProtoTableName(table)); 1069 } 1070 } 1071 return builder.build(); 1072 } catch (IOException e) { 1073 throw new ServiceException(e); 1074 } 1075 } 1076 1077 @Override 1078 public GetTableStateResponse getTableState(RpcController controller, 1079 GetTableStateRequest request) throws ServiceException { 1080 try { 1081 master.checkServiceStarted(); 1082 TableName tableName = ProtobufUtil.toTableName(request.getTableName()); 1083 TableState ts = master.getTableStateManager().getTableState(tableName); 1084 GetTableStateResponse.Builder builder = GetTableStateResponse.newBuilder(); 1085 builder.setTableState(ts.convert()); 1086 return builder.build(); 1087 } catch (IOException e) { 1088 throw new ServiceException(e); 1089 } 1090 } 1091 1092 @Override 1093 public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c, 1094 IsCatalogJanitorEnabledRequest req) throws ServiceException { 1095 return IsCatalogJanitorEnabledResponse.newBuilder().setValue( 1096 master.isCatalogJanitorEnabled()).build(); 1097 } 1098 1099 @Override 1100 public IsCleanerChoreEnabledResponse isCleanerChoreEnabled(RpcController c, 1101 IsCleanerChoreEnabledRequest req) 1102 throws ServiceException { 1103 return IsCleanerChoreEnabledResponse.newBuilder().setValue(master.isCleanerChoreEnabled()) 1104 .build(); 1105 } 1106 1107 @Override 1108 public IsMasterRunningResponse isMasterRunning(RpcController c, 1109 IsMasterRunningRequest req) throws ServiceException { 1110 try { 1111 master.checkServiceStarted(); 1112 return IsMasterRunningResponse.newBuilder().setIsMasterRunning( 1113 !master.isStopped()).build(); 1114 } catch (IOException e) { 1115 throw new ServiceException(e); 1116 } 1117 } 1118 1119 /** 1120 * Checks if the specified procedure is done. 1121 * @return true if the procedure is done, false if the procedure is in the process of completing 1122 * @throws ServiceException if invalid procedure or failed procedure with progress failure reason. 1123 */ 1124 @Override 1125 public IsProcedureDoneResponse isProcedureDone(RpcController controller, 1126 IsProcedureDoneRequest request) throws ServiceException { 1127 try { 1128 master.checkInitialized(); 1129 ProcedureDescription desc = request.getProcedure(); 1130 MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager( 1131 desc.getSignature()); 1132 if (mpm == null) { 1133 throw new ServiceException("The procedure is not registered: " 1134 + desc.getSignature()); 1135 } 1136 LOG.debug("Checking to see if procedure from request:" 1137 + desc.getSignature() + " is done"); 1138 1139 IsProcedureDoneResponse.Builder builder = 1140 IsProcedureDoneResponse.newBuilder(); 1141 boolean done = mpm.isProcedureDone(desc); 1142 builder.setDone(done); 1143 return builder.build(); 1144 } catch (ForeignException e) { 1145 throw new ServiceException(e.getCause()); 1146 } catch (IOException e) { 1147 throw new ServiceException(e); 1148 } 1149 } 1150 1151 /** 1152 * Checks if the specified snapshot is done. 1153 * @return true if the snapshot is in file system ready to use, 1154 * false if the snapshot is in the process of completing 1155 * @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or 1156 * a wrapped HBaseSnapshotException with progress failure reason. 1157 */ 1158 @Override 1159 public IsSnapshotDoneResponse isSnapshotDone(RpcController controller, 1160 IsSnapshotDoneRequest request) throws ServiceException { 1161 LOG.debug("Checking to see if snapshot from request:" + 1162 ClientSnapshotDescriptionUtils.toString(request.getSnapshot()) + " is done"); 1163 try { 1164 master.checkInitialized(); 1165 IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder(); 1166 boolean done = master.snapshotManager.isSnapshotDone(request.getSnapshot()); 1167 builder.setDone(done); 1168 return builder.build(); 1169 } catch (ForeignException e) { 1170 throw new ServiceException(e.getCause()); 1171 } catch (IOException e) { 1172 throw new ServiceException(e); 1173 } 1174 } 1175 1176 @Override 1177 public GetProcedureResultResponse getProcedureResult(RpcController controller, 1178 GetProcedureResultRequest request) throws ServiceException { 1179 LOG.debug("Checking to see if procedure is done pid=" + request.getProcId()); 1180 try { 1181 master.checkInitialized(); 1182 GetProcedureResultResponse.Builder builder = GetProcedureResultResponse.newBuilder(); 1183 long procId = request.getProcId(); 1184 ProcedureExecutor<?> executor = master.getMasterProcedureExecutor(); 1185 Procedure<?> result = executor.getResultOrProcedure(procId); 1186 if (result != null) { 1187 builder.setSubmittedTime(result.getSubmittedTime()); 1188 builder.setLastUpdate(result.getLastUpdate()); 1189 if (executor.isFinished(procId)) { 1190 builder.setState(GetProcedureResultResponse.State.FINISHED); 1191 if (result.isFailed()) { 1192 IOException exception = 1193 MasterProcedureUtil.unwrapRemoteIOException(result); 1194 builder.setException(ForeignExceptionUtil.toProtoForeignException(exception)); 1195 } 1196 byte[] resultData = result.getResult(); 1197 if (resultData != null) { 1198 builder.setResult(UnsafeByteOperations.unsafeWrap(resultData)); 1199 } 1200 master.getMasterProcedureExecutor().removeResult(request.getProcId()); 1201 } else { 1202 builder.setState(GetProcedureResultResponse.State.RUNNING); 1203 } 1204 } else { 1205 builder.setState(GetProcedureResultResponse.State.NOT_FOUND); 1206 } 1207 return builder.build(); 1208 } catch (IOException e) { 1209 throw new ServiceException(e); 1210 } 1211 } 1212 1213 @Override 1214 public AbortProcedureResponse abortProcedure( 1215 RpcController rpcController, AbortProcedureRequest request) throws ServiceException { 1216 try { 1217 AbortProcedureResponse.Builder response = AbortProcedureResponse.newBuilder(); 1218 boolean abortResult = 1219 master.abortProcedure(request.getProcId(), request.getMayInterruptIfRunning()); 1220 response.setIsProcedureAborted(abortResult); 1221 return response.build(); 1222 } catch (IOException e) { 1223 throw new ServiceException(e); 1224 } 1225 } 1226 1227 @Override 1228 public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController c, 1229 ListNamespaceDescriptorsRequest request) throws ServiceException { 1230 try { 1231 ListNamespaceDescriptorsResponse.Builder response = 1232 ListNamespaceDescriptorsResponse.newBuilder(); 1233 for(NamespaceDescriptor ns: master.getNamespaces()) { 1234 response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns)); 1235 } 1236 return response.build(); 1237 } catch (IOException e) { 1238 throw new ServiceException(e); 1239 } 1240 } 1241 1242 @Override 1243 public GetProceduresResponse getProcedures( 1244 RpcController rpcController, 1245 GetProceduresRequest request) throws ServiceException { 1246 try { 1247 final GetProceduresResponse.Builder response = GetProceduresResponse.newBuilder(); 1248 for (Procedure<?> p: master.getProcedures()) { 1249 response.addProcedure(ProcedureUtil.convertToProtoProcedure(p)); 1250 } 1251 return response.build(); 1252 } catch (IOException e) { 1253 throw new ServiceException(e); 1254 } 1255 } 1256 1257 @Override 1258 public GetLocksResponse getLocks( 1259 RpcController controller, 1260 GetLocksRequest request) throws ServiceException { 1261 try { 1262 final GetLocksResponse.Builder builder = GetLocksResponse.newBuilder(); 1263 1264 for (LockedResource lockedResource: master.getLocks()) { 1265 builder.addLock(ProcedureUtil.convertToProtoLockedResource(lockedResource)); 1266 } 1267 1268 return builder.build(); 1269 } catch (IOException e) { 1270 throw new ServiceException(e); 1271 } 1272 } 1273 1274 @Override 1275 public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(RpcController c, 1276 ListTableDescriptorsByNamespaceRequest request) throws ServiceException { 1277 try { 1278 ListTableDescriptorsByNamespaceResponse.Builder b = 1279 ListTableDescriptorsByNamespaceResponse.newBuilder(); 1280 for (TableDescriptor htd : master 1281 .listTableDescriptorsByNamespace(request.getNamespaceName())) { 1282 b.addTableSchema(ProtobufUtil.toTableSchema(htd)); 1283 } 1284 return b.build(); 1285 } catch (IOException e) { 1286 throw new ServiceException(e); 1287 } 1288 } 1289 1290 @Override 1291 public ListTableNamesByNamespaceResponse listTableNamesByNamespace(RpcController c, 1292 ListTableNamesByNamespaceRequest request) throws ServiceException { 1293 try { 1294 ListTableNamesByNamespaceResponse.Builder b = 1295 ListTableNamesByNamespaceResponse.newBuilder(); 1296 for (TableName tableName: master.listTableNamesByNamespace(request.getNamespaceName())) { 1297 b.addTableName(ProtobufUtil.toProtoTableName(tableName)); 1298 } 1299 return b.build(); 1300 } catch (IOException e) { 1301 throw new ServiceException(e); 1302 } 1303 } 1304 1305 @Override 1306 public ModifyColumnResponse modifyColumn(RpcController controller, 1307 ModifyColumnRequest req) throws ServiceException { 1308 try { 1309 long procId = master.modifyColumn( 1310 ProtobufUtil.toTableName(req.getTableName()), 1311 ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()), 1312 req.getNonceGroup(), 1313 req.getNonce()); 1314 if (procId == -1) { 1315 // This mean operation was not performed in server, so do not set any procId 1316 return ModifyColumnResponse.newBuilder().build(); 1317 } else { 1318 return ModifyColumnResponse.newBuilder().setProcId(procId).build(); 1319 } 1320 } catch (IOException ioe) { 1321 throw new ServiceException(ioe); 1322 } 1323 } 1324 1325 @Override 1326 public ModifyNamespaceResponse modifyNamespace(RpcController controller, 1327 ModifyNamespaceRequest request) throws ServiceException { 1328 try { 1329 long procId = master.modifyNamespace( 1330 ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()), 1331 request.getNonceGroup(), 1332 request.getNonce()); 1333 return ModifyNamespaceResponse.newBuilder().setProcId(procId).build(); 1334 } catch (IOException e) { 1335 throw new ServiceException(e); 1336 } 1337 } 1338 1339 @Override 1340 public ModifyTableResponse modifyTable(RpcController controller, 1341 ModifyTableRequest req) throws ServiceException { 1342 try { 1343 long procId = master.modifyTable( 1344 ProtobufUtil.toTableName(req.getTableName()), 1345 ProtobufUtil.toTableDescriptor(req.getTableSchema()), 1346 req.getNonceGroup(), 1347 req.getNonce()); 1348 return ModifyTableResponse.newBuilder().setProcId(procId).build(); 1349 } catch (IOException ioe) { 1350 throw new ServiceException(ioe); 1351 } 1352 } 1353 1354 @Override 1355 public MoveRegionResponse moveRegion(RpcController controller, 1356 MoveRegionRequest req) throws ServiceException { 1357 final byte [] encodedRegionName = req.getRegion().getValue().toByteArray(); 1358 RegionSpecifierType type = req.getRegion().getType(); 1359 final byte [] destServerName = (req.hasDestServerName())? 1360 Bytes.toBytes(ProtobufUtil.toServerName(req.getDestServerName()).getServerName()):null; 1361 MoveRegionResponse mrr = MoveRegionResponse.newBuilder().build(); 1362 1363 if (type != RegionSpecifierType.ENCODED_REGION_NAME) { 1364 LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME 1365 + " actual: " + type); 1366 } 1367 1368 try { 1369 master.checkInitialized(); 1370 master.move(encodedRegionName, destServerName); 1371 } catch (IOException ioe) { 1372 throw new ServiceException(ioe); 1373 } 1374 return mrr; 1375 } 1376 1377 /** 1378 * Offline specified region from master's in-memory state. It will not attempt to 1379 * reassign the region as in unassign. 1380 * 1381 * This is a special method that should be used by experts or hbck. 1382 * 1383 */ 1384 @Override 1385 public OfflineRegionResponse offlineRegion(RpcController controller, 1386 OfflineRegionRequest request) throws ServiceException { 1387 try { 1388 master.checkInitialized(); 1389 1390 final RegionSpecifierType type = request.getRegion().getType(); 1391 if (type != RegionSpecifierType.REGION_NAME) { 1392 LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME 1393 + " actual: " + type); 1394 } 1395 1396 final byte[] regionName = request.getRegion().getValue().toByteArray(); 1397 final RegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName); 1398 if (hri == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName)); 1399 1400 if (master.cpHost != null) { 1401 master.cpHost.preRegionOffline(hri); 1402 } 1403 LOG.info(master.getClientIdAuditPrefix() + " offline " + hri.getRegionNameAsString()); 1404 master.getAssignmentManager().offlineRegion(hri); 1405 if (master.cpHost != null) { 1406 master.cpHost.postRegionOffline(hri); 1407 } 1408 } catch (IOException ioe) { 1409 throw new ServiceException(ioe); 1410 } 1411 return OfflineRegionResponse.newBuilder().build(); 1412 } 1413 1414 /** 1415 * Execute Restore/Clone snapshot operation. 1416 * 1417 * <p>If the specified table exists a "Restore" is executed, replacing the table 1418 * schema and directory data with the content of the snapshot. 1419 * The table must be disabled, or a UnsupportedOperationException will be thrown. 1420 * 1421 * <p>If the table doesn't exist a "Clone" is executed, a new table is created 1422 * using the schema at the time of the snapshot, and the content of the snapshot. 1423 * 1424 * <p>The restore/clone operation does not require copying HFiles. Since HFiles 1425 * are immutable the table can point to and use the same files as the original one. 1426 */ 1427 @Override 1428 public RestoreSnapshotResponse restoreSnapshot(RpcController controller, 1429 RestoreSnapshotRequest request) throws ServiceException { 1430 try { 1431 long procId = master.restoreSnapshot(request.getSnapshot(), request.getNonceGroup(), 1432 request.getNonce(), request.getRestoreACL()); 1433 return RestoreSnapshotResponse.newBuilder().setProcId(procId).build(); 1434 } catch (ForeignException e) { 1435 throw new ServiceException(e.getCause()); 1436 } catch (IOException e) { 1437 throw new ServiceException(e); 1438 } 1439 } 1440 1441 @Override 1442 public RunCatalogScanResponse runCatalogScan(RpcController c, 1443 RunCatalogScanRequest req) throws ServiceException { 1444 rpcPreCheck("runCatalogScan"); 1445 try { 1446 return ResponseConverter.buildRunCatalogScanResponse(master.catalogJanitorChore.scan()); 1447 } catch (IOException ioe) { 1448 throw new ServiceException(ioe); 1449 } 1450 } 1451 1452 @Override 1453 public RunCleanerChoreResponse runCleanerChore(RpcController c, RunCleanerChoreRequest req) 1454 throws ServiceException { 1455 rpcPreCheck("runCleanerChore"); 1456 boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner(); 1457 return ResponseConverter.buildRunCleanerChoreResponse(result); 1458 } 1459 1460 @Override 1461 public SetBalancerRunningResponse setBalancerRunning(RpcController c, 1462 SetBalancerRunningRequest req) throws ServiceException { 1463 try { 1464 master.checkInitialized(); 1465 boolean prevValue = (req.getSynchronous())? 1466 synchronousBalanceSwitch(req.getOn()) : master.balanceSwitch(req.getOn()); 1467 return SetBalancerRunningResponse.newBuilder().setPrevBalanceValue(prevValue).build(); 1468 } catch (IOException ioe) { 1469 throw new ServiceException(ioe); 1470 } 1471 } 1472 1473 @Override 1474 public ShutdownResponse shutdown(RpcController controller, 1475 ShutdownRequest request) throws ServiceException { 1476 LOG.info(master.getClientIdAuditPrefix() + " shutdown"); 1477 try { 1478 master.shutdown(); 1479 } catch (IOException e) { 1480 LOG.error("Exception occurred in HMaster.shutdown()", e); 1481 throw new ServiceException(e); 1482 } 1483 return ShutdownResponse.newBuilder().build(); 1484 } 1485 1486 /** 1487 * Triggers an asynchronous attempt to take a snapshot. 1488 * {@inheritDoc} 1489 */ 1490 @Override 1491 public SnapshotResponse snapshot(RpcController controller, 1492 SnapshotRequest request) throws ServiceException { 1493 try { 1494 master.checkInitialized(); 1495 master.snapshotManager.checkSnapshotSupport(); 1496 1497 LOG.info(master.getClientIdAuditPrefix() + " snapshot request for:" + 1498 ClientSnapshotDescriptionUtils.toString(request.getSnapshot())); 1499 // get the snapshot information 1500 SnapshotDescription snapshot = SnapshotDescriptionUtils.validate( 1501 request.getSnapshot(), master.getConfiguration()); 1502 master.snapshotManager.takeSnapshot(snapshot); 1503 1504 // send back the max amount of time the client should wait for the snapshot to complete 1505 long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(master.getConfiguration(), 1506 snapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME); 1507 return SnapshotResponse.newBuilder().setExpectedTimeout(waitTime).build(); 1508 } catch (ForeignException e) { 1509 throw new ServiceException(e.getCause()); 1510 } catch (IOException e) { 1511 throw new ServiceException(e); 1512 } 1513 } 1514 1515 @Override 1516 public StopMasterResponse stopMaster(RpcController controller, 1517 StopMasterRequest request) throws ServiceException { 1518 LOG.info(master.getClientIdAuditPrefix() + " stop"); 1519 try { 1520 master.stopMaster(); 1521 } catch (IOException e) { 1522 LOG.error("Exception occurred while stopping master", e); 1523 throw new ServiceException(e); 1524 } 1525 return StopMasterResponse.newBuilder().build(); 1526 } 1527 1528 @Override 1529 public IsInMaintenanceModeResponse isMasterInMaintenanceMode( 1530 final RpcController controller, 1531 final IsInMaintenanceModeRequest request) throws ServiceException { 1532 IsInMaintenanceModeResponse.Builder response = IsInMaintenanceModeResponse.newBuilder(); 1533 response.setInMaintenanceMode(master.isInMaintenanceMode()); 1534 return response.build(); 1535 } 1536 1537 @Override 1538 public UnassignRegionResponse unassignRegion(RpcController controller, 1539 UnassignRegionRequest req) throws ServiceException { 1540 try { 1541 final byte [] regionName = req.getRegion().getValue().toByteArray(); 1542 RegionSpecifierType type = req.getRegion().getType(); 1543 final boolean force = req.getForce(); 1544 UnassignRegionResponse urr = UnassignRegionResponse.newBuilder().build(); 1545 1546 master.checkInitialized(); 1547 if (type != RegionSpecifierType.REGION_NAME) { 1548 LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME 1549 + " actual: " + type); 1550 } 1551 Pair<RegionInfo, ServerName> pair = 1552 MetaTableAccessor.getRegion(master.getConnection(), regionName); 1553 if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(),regionName)) { 1554 pair = new Pair<>(RegionInfoBuilder.FIRST_META_REGIONINFO, 1555 master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper())); 1556 } 1557 if (pair == null) { 1558 throw new UnknownRegionException(Bytes.toString(regionName)); 1559 } 1560 1561 RegionInfo hri = pair.getFirst(); 1562 if (master.cpHost != null) { 1563 master.cpHost.preUnassign(hri, force); 1564 } 1565 LOG.debug(master.getClientIdAuditPrefix() + " unassign " + hri.getRegionNameAsString() 1566 + " in current location if it is online and reassign.force=" + force); 1567 master.getAssignmentManager().unassign(hri); 1568 if (master.cpHost != null) { 1569 master.cpHost.postUnassign(hri, force); 1570 } 1571 1572 return urr; 1573 } catch (IOException ioe) { 1574 throw new ServiceException(ioe); 1575 } 1576 } 1577 1578 @Override 1579 public ReportRegionStateTransitionResponse reportRegionStateTransition(RpcController c, 1580 ReportRegionStateTransitionRequest req) throws ServiceException { 1581 try { 1582 master.checkServiceStarted(); 1583 return master.getAssignmentManager().reportRegionStateTransition(req); 1584 } catch (IOException ioe) { 1585 throw new ServiceException(ioe); 1586 } 1587 } 1588 1589 @Override 1590 public SetQuotaResponse setQuota(RpcController c, SetQuotaRequest req) 1591 throws ServiceException { 1592 try { 1593 master.checkInitialized(); 1594 return master.getMasterQuotaManager().setQuota(req); 1595 } catch (Exception e) { 1596 throw new ServiceException(e); 1597 } 1598 } 1599 1600 @Override 1601 public MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(RpcController controller, 1602 MajorCompactionTimestampRequest request) throws ServiceException { 1603 MajorCompactionTimestampResponse.Builder response = 1604 MajorCompactionTimestampResponse.newBuilder(); 1605 try { 1606 master.checkInitialized(); 1607 response.setCompactionTimestamp(master.getLastMajorCompactionTimestamp(ProtobufUtil 1608 .toTableName(request.getTableName()))); 1609 } catch (IOException e) { 1610 throw new ServiceException(e); 1611 } 1612 return response.build(); 1613 } 1614 1615 @Override 1616 public MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion( 1617 RpcController controller, MajorCompactionTimestampForRegionRequest request) 1618 throws ServiceException { 1619 MajorCompactionTimestampResponse.Builder response = 1620 MajorCompactionTimestampResponse.newBuilder(); 1621 try { 1622 master.checkInitialized(); 1623 response.setCompactionTimestamp(master.getLastMajorCompactionTimestampForRegion(request 1624 .getRegion().getValue().toByteArray())); 1625 } catch (IOException e) { 1626 throw new ServiceException(e); 1627 } 1628 return response.build(); 1629 } 1630 1631 /** 1632 * Compact a region on the master. 1633 * 1634 * @param controller the RPC controller 1635 * @param request the request 1636 * @throws ServiceException 1637 */ 1638 @Override 1639 @QosPriority(priority=HConstants.ADMIN_QOS) 1640 public CompactRegionResponse compactRegion(final RpcController controller, 1641 final CompactRegionRequest request) throws ServiceException { 1642 try { 1643 master.checkInitialized(); 1644 byte[] regionName = request.getRegion().getValue().toByteArray(); 1645 TableName tableName = RegionInfo.getTable(regionName); 1646 // if the region is a mob region, do the mob file compaction. 1647 if (MobUtils.isMobRegionName(tableName, regionName)) { 1648 checkHFileFormatVersionForMob(); 1649 return compactMob(request, tableName); 1650 } else { 1651 return super.compactRegion(controller, request); 1652 } 1653 } catch (IOException ie) { 1654 throw new ServiceException(ie); 1655 } 1656 } 1657 1658 /** 1659 * check configured hfile format version before to do compaction 1660 * @throws IOException throw IOException 1661 */ 1662 private void checkHFileFormatVersionForMob() throws IOException { 1663 if (HFile.getFormatVersion(master.getConfiguration()) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) { 1664 LOG.error("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS 1665 + " is required for MOB compaction. Compaction will not run."); 1666 throw new IOException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS 1667 + " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY 1668 + " accordingly."); 1669 } 1670 } 1671 1672 @Override 1673 @QosPriority(priority=HConstants.ADMIN_QOS) 1674 public GetRegionInfoResponse getRegionInfo(final RpcController controller, 1675 final GetRegionInfoRequest request) throws ServiceException { 1676 byte[] regionName = request.getRegion().getValue().toByteArray(); 1677 TableName tableName = RegionInfo.getTable(regionName); 1678 if (MobUtils.isMobRegionName(tableName, regionName)) { 1679 // a dummy region info contains the compaction state. 1680 RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName); 1681 GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder(); 1682 builder.setRegionInfo(ProtobufUtil.toRegionInfo(mobRegionInfo)); 1683 if (request.hasCompactionState() && request.getCompactionState()) { 1684 builder.setCompactionState(master.getMobCompactionState(tableName)); 1685 } 1686 return builder.build(); 1687 } else { 1688 return super.getRegionInfo(controller, request); 1689 } 1690 } 1691 1692 /** 1693 * Compacts the mob files in the current table. 1694 * @param request the request. 1695 * @param tableName the current table name. 1696 * @return The response of the mob file compaction. 1697 * @throws IOException 1698 */ 1699 private CompactRegionResponse compactMob(final CompactRegionRequest request, 1700 TableName tableName) throws IOException { 1701 if (!master.getTableStateManager().isTableState(tableName, TableState.State.ENABLED)) { 1702 throw new DoNotRetryIOException("Table " + tableName + " is not enabled"); 1703 } 1704 boolean allFiles = false; 1705 List<ColumnFamilyDescriptor> compactedColumns = new ArrayList<>(); 1706 ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies(); 1707 byte[] family = null; 1708 if (request.hasFamily()) { 1709 family = request.getFamily().toByteArray(); 1710 for (ColumnFamilyDescriptor hcd : hcds) { 1711 if (Bytes.equals(family, hcd.getName())) { 1712 if (!hcd.isMobEnabled()) { 1713 LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family"); 1714 throw new DoNotRetryIOException("Column family " + hcd.getNameAsString() 1715 + " is not a mob column family"); 1716 } 1717 compactedColumns.add(hcd); 1718 } 1719 } 1720 } else { 1721 for (ColumnFamilyDescriptor hcd : hcds) { 1722 if (hcd.isMobEnabled()) { 1723 compactedColumns.add(hcd); 1724 } 1725 } 1726 } 1727 if (compactedColumns.isEmpty()) { 1728 LOG.error("No mob column families are assigned in the mob compaction"); 1729 throw new DoNotRetryIOException( 1730 "No mob column families are assigned in the mob compaction"); 1731 } 1732 if (request.hasMajor() && request.getMajor()) { 1733 allFiles = true; 1734 } 1735 String familyLogMsg = (family != null) ? Bytes.toString(family) : ""; 1736 if (LOG.isTraceEnabled()) { 1737 LOG.trace("User-triggered mob compaction requested for table: " 1738 + tableName.getNameAsString() + " for column family: " + familyLogMsg); 1739 } 1740 master.requestMobCompaction(tableName, compactedColumns, allFiles); 1741 return CompactRegionResponse.newBuilder().build(); 1742 } 1743 1744 @Override 1745 public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller, 1746 IsBalancerEnabledRequest request) throws ServiceException { 1747 IsBalancerEnabledResponse.Builder response = IsBalancerEnabledResponse.newBuilder(); 1748 response.setEnabled(master.isBalancerOn()); 1749 return response.build(); 1750 } 1751 1752 @Override 1753 public SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(RpcController controller, 1754 SetSplitOrMergeEnabledRequest request) throws ServiceException { 1755 SetSplitOrMergeEnabledResponse.Builder response = SetSplitOrMergeEnabledResponse.newBuilder(); 1756 try { 1757 master.checkInitialized(); 1758 boolean newValue = request.getEnabled(); 1759 for (MasterProtos.MasterSwitchType masterSwitchType: request.getSwitchTypesList()) { 1760 MasterSwitchType switchType = convert(masterSwitchType); 1761 boolean oldValue = master.isSplitOrMergeEnabled(switchType); 1762 response.addPrevValue(oldValue); 1763 if (master.cpHost != null) { 1764 master.cpHost.preSetSplitOrMergeEnabled(newValue, switchType); 1765 } 1766 master.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue, switchType); 1767 if (master.cpHost != null) { 1768 master.cpHost.postSetSplitOrMergeEnabled(newValue, switchType); 1769 } 1770 } 1771 } catch (IOException e) { 1772 throw new ServiceException(e); 1773 } catch (KeeperException e) { 1774 throw new ServiceException(e); 1775 } 1776 return response.build(); 1777 } 1778 1779 @Override 1780 public IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(RpcController controller, 1781 IsSplitOrMergeEnabledRequest request) throws ServiceException { 1782 IsSplitOrMergeEnabledResponse.Builder response = IsSplitOrMergeEnabledResponse.newBuilder(); 1783 response.setEnabled(master.isSplitOrMergeEnabled(convert(request.getSwitchType()))); 1784 return response.build(); 1785 } 1786 1787 @Override 1788 public NormalizeResponse normalize(RpcController controller, 1789 NormalizeRequest request) throws ServiceException { 1790 rpcPreCheck("normalize"); 1791 try { 1792 return NormalizeResponse.newBuilder().setNormalizerRan(master.normalizeRegions()).build(); 1793 } catch (IOException ex) { 1794 throw new ServiceException(ex); 1795 } 1796 } 1797 1798 @Override 1799 public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller, 1800 SetNormalizerRunningRequest request) throws ServiceException { 1801 rpcPreCheck("setNormalizerRunning"); 1802 1803 // Sets normalizer on/off flag in ZK. 1804 boolean prevValue = master.getRegionNormalizerTracker().isNormalizerOn(); 1805 boolean newValue = request.getOn(); 1806 try { 1807 master.getRegionNormalizerTracker().setNormalizerOn(newValue); 1808 } catch (KeeperException ke) { 1809 LOG.warn("Error flipping normalizer switch", ke); 1810 } 1811 LOG.info("{} set normalizerSwitch={}", master.getClientIdAuditPrefix(), newValue); 1812 return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build(); 1813 } 1814 1815 @Override 1816 public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller, 1817 IsNormalizerEnabledRequest request) throws ServiceException { 1818 IsNormalizerEnabledResponse.Builder response = IsNormalizerEnabledResponse.newBuilder(); 1819 response.setEnabled(master.isNormalizerOn()); 1820 return response.build(); 1821 } 1822 1823 /** 1824 * Returns the security capabilities in effect on the cluster 1825 */ 1826 @Override 1827 public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller, 1828 SecurityCapabilitiesRequest request) throws ServiceException { 1829 SecurityCapabilitiesResponse.Builder response = SecurityCapabilitiesResponse.newBuilder(); 1830 try { 1831 master.checkInitialized(); 1832 Set<SecurityCapabilitiesResponse.Capability> capabilities = new HashSet<>(); 1833 // Authentication 1834 if (User.isHBaseSecurityEnabled(master.getConfiguration())) { 1835 capabilities.add(SecurityCapabilitiesResponse.Capability.SECURE_AUTHENTICATION); 1836 } else { 1837 capabilities.add(SecurityCapabilitiesResponse.Capability.SIMPLE_AUTHENTICATION); 1838 } 1839 // A coprocessor that implements AccessControlService can provide AUTHORIZATION and 1840 // CELL_AUTHORIZATION 1841 if (master.cpHost != null && hasAccessControlServiceCoprocessor(master.cpHost)) { 1842 if (AccessChecker.isAuthorizationSupported(master.getConfiguration())) { 1843 capabilities.add(SecurityCapabilitiesResponse.Capability.AUTHORIZATION); 1844 } 1845 if (AccessController.isCellAuthorizationSupported(master.getConfiguration())) { 1846 capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_AUTHORIZATION); 1847 } 1848 } 1849 // A coprocessor that implements VisibilityLabelsService can provide CELL_VISIBILITY. 1850 if (master.cpHost != null && hasVisibilityLabelsServiceCoprocessor(master.cpHost)) { 1851 if (VisibilityController.isCellAuthorizationSupported(master.getConfiguration())) { 1852 capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_VISIBILITY); 1853 } 1854 } 1855 response.addAllCapabilities(capabilities); 1856 } catch (IOException e) { 1857 throw new ServiceException(e); 1858 } 1859 return response.build(); 1860 } 1861 1862 /** 1863 * Determines if there is a MasterCoprocessor deployed which implements 1864 * {@link org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface}. 1865 */ 1866 boolean hasAccessControlServiceCoprocessor(MasterCoprocessorHost cpHost) { 1867 return checkCoprocessorWithService( 1868 cpHost.findCoprocessors(MasterCoprocessor.class), AccessControlService.Interface.class); 1869 } 1870 1871 /** 1872 * Determines if there is a MasterCoprocessor deployed which implements 1873 * {@link org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface}. 1874 */ 1875 boolean hasVisibilityLabelsServiceCoprocessor(MasterCoprocessorHost cpHost) { 1876 return checkCoprocessorWithService( 1877 cpHost.findCoprocessors(MasterCoprocessor.class), 1878 VisibilityLabelsService.Interface.class); 1879 } 1880 1881 /** 1882 * Determines if there is a coprocessor implementation in the provided argument which extends 1883 * or implements the provided {@code service}. 1884 */ 1885 boolean checkCoprocessorWithService( 1886 List<MasterCoprocessor> coprocessorsToCheck, Class<?> service) { 1887 if (coprocessorsToCheck == null || coprocessorsToCheck.isEmpty()) { 1888 return false; 1889 } 1890 for (MasterCoprocessor cp : coprocessorsToCheck) { 1891 if (service.isAssignableFrom(cp.getClass())) { 1892 return true; 1893 } 1894 } 1895 return false; 1896 } 1897 1898 private MasterSwitchType convert(MasterProtos.MasterSwitchType switchType) { 1899 switch (switchType) { 1900 case SPLIT: 1901 return MasterSwitchType.SPLIT; 1902 case MERGE: 1903 return MasterSwitchType.MERGE; 1904 default: 1905 break; 1906 } 1907 return null; 1908 } 1909 1910 @Override 1911 public AddReplicationPeerResponse addReplicationPeer(RpcController controller, 1912 AddReplicationPeerRequest request) throws ServiceException { 1913 try { 1914 long procId = master.addReplicationPeer(request.getPeerId(), 1915 ReplicationPeerConfigUtil.convert(request.getPeerConfig()), 1916 request.getPeerState().getState().equals(ReplicationState.State.ENABLED)); 1917 return AddReplicationPeerResponse.newBuilder().setProcId(procId).build(); 1918 } catch (ReplicationException | IOException e) { 1919 throw new ServiceException(e); 1920 } 1921 } 1922 1923 @Override 1924 public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller, 1925 RemoveReplicationPeerRequest request) throws ServiceException { 1926 try { 1927 long procId = master.removeReplicationPeer(request.getPeerId()); 1928 return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build(); 1929 } catch (ReplicationException | IOException e) { 1930 throw new ServiceException(e); 1931 } 1932 } 1933 1934 @Override 1935 public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller, 1936 EnableReplicationPeerRequest request) throws ServiceException { 1937 try { 1938 long procId = master.enableReplicationPeer(request.getPeerId()); 1939 return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build(); 1940 } catch (ReplicationException | IOException e) { 1941 throw new ServiceException(e); 1942 } 1943 } 1944 1945 @Override 1946 public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller, 1947 DisableReplicationPeerRequest request) throws ServiceException { 1948 try { 1949 long procId = master.disableReplicationPeer(request.getPeerId()); 1950 return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build(); 1951 } catch (ReplicationException | IOException e) { 1952 throw new ServiceException(e); 1953 } 1954 } 1955 1956 @Override 1957 public GetReplicationPeerConfigResponse getReplicationPeerConfig(RpcController controller, 1958 GetReplicationPeerConfigRequest request) throws ServiceException { 1959 GetReplicationPeerConfigResponse.Builder response = GetReplicationPeerConfigResponse 1960 .newBuilder(); 1961 try { 1962 String peerId = request.getPeerId(); 1963 ReplicationPeerConfig peerConfig = master.getReplicationPeerConfig(peerId); 1964 response.setPeerId(peerId); 1965 response.setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)); 1966 } catch (ReplicationException | IOException e) { 1967 throw new ServiceException(e); 1968 } 1969 return response.build(); 1970 } 1971 1972 @Override 1973 public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller, 1974 UpdateReplicationPeerConfigRequest request) throws ServiceException { 1975 try { 1976 long procId = master.updateReplicationPeerConfig(request.getPeerId(), 1977 ReplicationPeerConfigUtil.convert(request.getPeerConfig())); 1978 return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build(); 1979 } catch (ReplicationException | IOException e) { 1980 throw new ServiceException(e); 1981 } 1982 } 1983 1984 @Override 1985 public ListReplicationPeersResponse listReplicationPeers(RpcController controller, 1986 ListReplicationPeersRequest request) throws ServiceException { 1987 ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder(); 1988 try { 1989 List<ReplicationPeerDescription> peers = master 1990 .listReplicationPeers(request.hasRegex() ? request.getRegex() : null); 1991 for (ReplicationPeerDescription peer : peers) { 1992 response.addPeerDesc(ReplicationPeerConfigUtil.toProtoReplicationPeerDescription(peer)); 1993 } 1994 } catch (ReplicationException | IOException e) { 1995 throw new ServiceException(e); 1996 } 1997 return response.build(); 1998 } 1999 2000 @Override 2001 public ListDecommissionedRegionServersResponse listDecommissionedRegionServers( 2002 RpcController controller, ListDecommissionedRegionServersRequest request) 2003 throws ServiceException { 2004 ListDecommissionedRegionServersResponse.Builder response = 2005 ListDecommissionedRegionServersResponse.newBuilder(); 2006 try { 2007 master.checkInitialized(); 2008 if (master.cpHost != null) { 2009 master.cpHost.preListDecommissionedRegionServers(); 2010 } 2011 List<ServerName> servers = master.listDecommissionedRegionServers(); 2012 response.addAllServerName((servers.stream().map(server -> ProtobufUtil.toServerName(server))) 2013 .collect(Collectors.toList())); 2014 if (master.cpHost != null) { 2015 master.cpHost.postListDecommissionedRegionServers(); 2016 } 2017 } catch (IOException io) { 2018 throw new ServiceException(io); 2019 } 2020 2021 return response.build(); 2022 } 2023 2024 @Override 2025 public DecommissionRegionServersResponse decommissionRegionServers(RpcController controller, 2026 DecommissionRegionServersRequest request) throws ServiceException { 2027 try { 2028 master.checkInitialized(); 2029 List<ServerName> servers = request.getServerNameList().stream() 2030 .map(pbServer -> ProtobufUtil.toServerName(pbServer)).collect(Collectors.toList()); 2031 boolean offload = request.getOffload(); 2032 if (master.cpHost != null) { 2033 master.cpHost.preDecommissionRegionServers(servers, offload); 2034 } 2035 master.decommissionRegionServers(servers, offload); 2036 if (master.cpHost != null) { 2037 master.cpHost.postDecommissionRegionServers(servers, offload); 2038 } 2039 } catch (IOException io) { 2040 throw new ServiceException(io); 2041 } 2042 2043 return DecommissionRegionServersResponse.newBuilder().build(); 2044 } 2045 2046 @Override 2047 public RecommissionRegionServerResponse recommissionRegionServer(RpcController controller, 2048 RecommissionRegionServerRequest request) throws ServiceException { 2049 try { 2050 master.checkInitialized(); 2051 ServerName server = ProtobufUtil.toServerName(request.getServerName()); 2052 List<byte[]> encodedRegionNames = request.getRegionList().stream() 2053 .map(regionSpecifier -> regionSpecifier.getValue().toByteArray()) 2054 .collect(Collectors.toList()); 2055 if (master.cpHost != null) { 2056 master.cpHost.preRecommissionRegionServer(server, encodedRegionNames); 2057 } 2058 master.recommissionRegionServer(server, encodedRegionNames); 2059 if (master.cpHost != null) { 2060 master.cpHost.postRecommissionRegionServer(server, encodedRegionNames); 2061 } 2062 } catch (IOException io) { 2063 throw new ServiceException(io); 2064 } 2065 2066 return RecommissionRegionServerResponse.newBuilder().build(); 2067 } 2068 2069 @Override 2070 public LockResponse requestLock(RpcController controller, final LockRequest request) 2071 throws ServiceException { 2072 try { 2073 if (request.getDescription().isEmpty()) { 2074 throw new IllegalArgumentException("Empty description"); 2075 } 2076 NonceProcedureRunnable npr; 2077 LockType type = LockType.valueOf(request.getLockType().name()); 2078 if (request.getRegionInfoCount() > 0) { 2079 final RegionInfo[] regionInfos = new RegionInfo[request.getRegionInfoCount()]; 2080 for (int i = 0; i < request.getRegionInfoCount(); ++i) { 2081 regionInfos[i] = ProtobufUtil.toRegionInfo(request.getRegionInfo(i)); 2082 } 2083 npr = new NonceProcedureRunnable(master, request.getNonceGroup(), request.getNonce()) { 2084 @Override 2085 protected void run() throws IOException { 2086 setProcId(master.getLockManager().remoteLocks().requestRegionsLock(regionInfos, 2087 request.getDescription(), getNonceKey())); 2088 } 2089 2090 @Override 2091 protected String getDescription() { 2092 return "RequestLock"; 2093 } 2094 }; 2095 } else if (request.hasTableName()) { 2096 final TableName tableName = ProtobufUtil.toTableName(request.getTableName()); 2097 npr = new NonceProcedureRunnable(master, request.getNonceGroup(), request.getNonce()) { 2098 @Override 2099 protected void run() throws IOException { 2100 setProcId(master.getLockManager().remoteLocks().requestTableLock(tableName, type, 2101 request.getDescription(), getNonceKey())); 2102 } 2103 2104 @Override 2105 protected String getDescription() { 2106 return "RequestLock"; 2107 } 2108 }; 2109 } else if (request.hasNamespace()) { 2110 npr = new NonceProcedureRunnable(master, request.getNonceGroup(), request.getNonce()) { 2111 @Override 2112 protected void run() throws IOException { 2113 setProcId(master.getLockManager().remoteLocks().requestNamespaceLock( 2114 request.getNamespace(), type, request.getDescription(), getNonceKey())); 2115 } 2116 2117 @Override 2118 protected String getDescription() { 2119 return "RequestLock"; 2120 } 2121 }; 2122 } else { 2123 throw new IllegalArgumentException("one of table/namespace/region should be specified"); 2124 } 2125 long procId = MasterProcedureUtil.submitProcedure(npr); 2126 return LockResponse.newBuilder().setProcId(procId).build(); 2127 } catch (IllegalArgumentException e) { 2128 LOG.warn("Exception when queuing lock", e); 2129 throw new ServiceException(new DoNotRetryIOException(e)); 2130 } catch (IOException e) { 2131 LOG.warn("Exception when queuing lock", e); 2132 throw new ServiceException(e); 2133 } 2134 } 2135 2136 /** 2137 * @return LOCKED, if procedure is found and it has the lock; else UNLOCKED. 2138 * @throws ServiceException if given proc id is found but it is not a LockProcedure. 2139 */ 2140 @Override 2141 public LockHeartbeatResponse lockHeartbeat(RpcController controller, LockHeartbeatRequest request) 2142 throws ServiceException { 2143 try { 2144 if (master.getLockManager().remoteLocks().lockHeartbeat(request.getProcId(), 2145 request.getKeepAlive())) { 2146 return LockHeartbeatResponse.newBuilder().setTimeoutMs( 2147 master.getConfiguration().getInt(LockProcedure.REMOTE_LOCKS_TIMEOUT_MS_CONF, 2148 LockProcedure.DEFAULT_REMOTE_LOCKS_TIMEOUT_MS)) 2149 .setLockStatus(LockHeartbeatResponse.LockStatus.LOCKED).build(); 2150 } else { 2151 return LockHeartbeatResponse.newBuilder() 2152 .setLockStatus(LockHeartbeatResponse.LockStatus.UNLOCKED).build(); 2153 } 2154 } catch (IOException e) { 2155 throw new ServiceException(e); 2156 } 2157 } 2158 2159 @Override 2160 public RegionSpaceUseReportResponse reportRegionSpaceUse(RpcController controller, 2161 RegionSpaceUseReportRequest request) throws ServiceException { 2162 try { 2163 master.checkInitialized(); 2164 if (!QuotaUtil.isQuotaEnabled(master.getConfiguration())) { 2165 return RegionSpaceUseReportResponse.newBuilder().build(); 2166 } 2167 MasterQuotaManager quotaManager = this.master.getMasterQuotaManager(); 2168 final long now = EnvironmentEdgeManager.currentTime(); 2169 for (RegionSpaceUse report : request.getSpaceUseList()) { 2170 quotaManager.addRegionSize(ProtobufUtil.toRegionInfo( 2171 report.getRegionInfo()), report.getRegionSize(), now); 2172 } 2173 return RegionSpaceUseReportResponse.newBuilder().build(); 2174 } catch (Exception e) { 2175 throw new ServiceException(e); 2176 } 2177 } 2178 2179 @Override 2180 public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes( 2181 RpcController controller, GetSpaceQuotaRegionSizesRequest request) throws ServiceException { 2182 try { 2183 master.checkInitialized(); 2184 MasterQuotaManager quotaManager = this.master.getMasterQuotaManager(); 2185 GetSpaceQuotaRegionSizesResponse.Builder builder = 2186 GetSpaceQuotaRegionSizesResponse.newBuilder(); 2187 if (quotaManager != null) { 2188 Map<RegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes(); 2189 Map<TableName,Long> regionSizesByTable = new HashMap<>(); 2190 // Translate hregioninfo+long -> tablename+long 2191 for (Entry<RegionInfo,Long> entry : regionSizes.entrySet()) { 2192 final TableName tableName = entry.getKey().getTable(); 2193 Long prevSize = regionSizesByTable.get(tableName); 2194 if (prevSize == null) { 2195 prevSize = 0L; 2196 } 2197 regionSizesByTable.put(tableName, prevSize + entry.getValue()); 2198 } 2199 // Serialize them into the protobuf 2200 for (Entry<TableName,Long> tableSize : regionSizesByTable.entrySet()) { 2201 builder.addSizes(RegionSizes.newBuilder() 2202 .setTableName(ProtobufUtil.toProtoTableName(tableSize.getKey())) 2203 .setSize(tableSize.getValue()).build()); 2204 } 2205 return builder.build(); 2206 } 2207 return builder.build(); 2208 } catch (Exception e) { 2209 throw new ServiceException(e); 2210 } 2211 } 2212 2213 @Override 2214 public GetQuotaStatesResponse getQuotaStates( 2215 RpcController controller, GetQuotaStatesRequest request) throws ServiceException { 2216 try { 2217 master.checkInitialized(); 2218 QuotaObserverChore quotaChore = this.master.getQuotaObserverChore(); 2219 GetQuotaStatesResponse.Builder builder = GetQuotaStatesResponse.newBuilder(); 2220 if (quotaChore != null) { 2221 // The "current" view of all tables with quotas 2222 Map<TableName, SpaceQuotaSnapshot> tableSnapshots = quotaChore.getTableQuotaSnapshots(); 2223 for (Entry<TableName, SpaceQuotaSnapshot> entry : tableSnapshots.entrySet()) { 2224 builder.addTableSnapshots( 2225 TableQuotaSnapshot.newBuilder() 2226 .setTableName(ProtobufUtil.toProtoTableName(entry.getKey())) 2227 .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build()); 2228 } 2229 // The "current" view of all namespaces with quotas 2230 Map<String, SpaceQuotaSnapshot> nsSnapshots = quotaChore.getNamespaceQuotaSnapshots(); 2231 for (Entry<String, SpaceQuotaSnapshot> entry : nsSnapshots.entrySet()) { 2232 builder.addNsSnapshots( 2233 NamespaceQuotaSnapshot.newBuilder() 2234 .setNamespace(entry.getKey()) 2235 .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build()); 2236 } 2237 return builder.build(); 2238 } 2239 return builder.build(); 2240 } catch (Exception e) { 2241 throw new ServiceException(e); 2242 } 2243 } 2244 2245 @Override 2246 public ClearDeadServersResponse clearDeadServers(RpcController controller, 2247 ClearDeadServersRequest request) throws ServiceException { 2248 LOG.debug(master.getClientIdAuditPrefix() + " clear dead region servers."); 2249 ClearDeadServersResponse.Builder response = ClearDeadServersResponse.newBuilder(); 2250 try { 2251 master.checkInitialized(); 2252 if (master.cpHost != null) { 2253 master.cpHost.preClearDeadServers(); 2254 } 2255 2256 if (master.getServerManager().areDeadServersInProgress()) { 2257 LOG.debug("Some dead server is still under processing, won't clear the dead server list"); 2258 response.addAllServerName(request.getServerNameList()); 2259 } else { 2260 for (HBaseProtos.ServerName pbServer : request.getServerNameList()) { 2261 if (!master.getServerManager().getDeadServers() 2262 .removeDeadServer(ProtobufUtil.toServerName(pbServer))) { 2263 response.addServerName(pbServer); 2264 } 2265 } 2266 } 2267 2268 if (master.cpHost != null) { 2269 master.cpHost.postClearDeadServers( 2270 ProtobufUtil.toServerNameList(request.getServerNameList()), 2271 ProtobufUtil.toServerNameList(response.getServerNameList())); 2272 } 2273 } catch (IOException io) { 2274 throw new ServiceException(io); 2275 } 2276 return response.build(); 2277 } 2278 2279 @Override 2280 public ReportProcedureDoneResponse reportProcedureDone(RpcController controller, 2281 ReportProcedureDoneRequest request) throws ServiceException { 2282 request.getResultList().forEach(result -> { 2283 if (result.getStatus() == RemoteProcedureResult.Status.SUCCESS) { 2284 master.remoteProcedureCompleted(result.getProcId()); 2285 } else { 2286 master.remoteProcedureFailed(result.getProcId(), 2287 RemoteProcedureException.fromProto(result.getError())); 2288 } 2289 }); 2290 return ReportProcedureDoneResponse.getDefaultInstance(); 2291 } 2292 2293 // HBCK Services 2294 2295 /** 2296 * Update state of the table in meta only. This is required by hbck in some situations to cleanup 2297 * stuck assign/ unassign regions procedures for the table. 2298 * 2299 * @return previous state of the table 2300 */ 2301 @Override 2302 public GetTableStateResponse setTableStateInMeta(RpcController controller, 2303 SetTableStateInMetaRequest request) throws ServiceException { 2304 TableName tn = ProtobufUtil.toTableName(request.getTableName()); 2305 try { 2306 HBaseProtos.TableState prevState = 2307 this.master.getTableStateManager().getTableState(tn).convert(); 2308 this.master.getTableStateManager().setTableState(tn, 2309 TableState.convert(tn, request.getTableState()).getState()); 2310 return GetTableStateResponse.newBuilder().setTableState(prevState).build(); 2311 } catch (Exception e) { 2312 throw new ServiceException(e); 2313 } 2314 } 2315 2316 /** 2317 * Get RegionInfo from Master using content of RegionSpecifier as key. 2318 * @return RegionInfo found by decoding <code>rs</code> or null if none found 2319 */ 2320 private RegionInfo getRegionInfo(HBaseProtos.RegionSpecifier rs) throws UnknownRegionException { 2321 RegionInfo ri = null; 2322 switch(rs.getType()) { 2323 case REGION_NAME: 2324 final byte[] regionName = rs.getValue().toByteArray(); 2325 ri = this.master.getAssignmentManager().getRegionInfo(regionName); 2326 break; 2327 case ENCODED_REGION_NAME: 2328 String encodedRegionName = Bytes.toString(rs.getValue().toByteArray()); 2329 RegionState regionState = this.master.getAssignmentManager().getRegionStates(). 2330 getRegionState(encodedRegionName); 2331 ri = regionState == null? null: regionState.getRegion(); 2332 break; 2333 default: 2334 break; 2335 } 2336 return ri; 2337 } 2338 2339 /** 2340 * Submit the Procedure that gets created by <code>f</code> 2341 * @return pid of the submitted Procedure. 2342 */ 2343 private long submitProcedure(HBaseProtos.RegionSpecifier rs, boolean override, 2344 BiFunction<RegionInfo, Boolean, Procedure> f) 2345 throws UnknownRegionException { 2346 RegionInfo ri = getRegionInfo(rs); 2347 long pid = Procedure.NO_PROC_ID; 2348 if (ri == null) { 2349 LOG.warn("No RegionInfo found to match {}", rs); 2350 } else { 2351 pid = this.master.getMasterProcedureExecutor().submitProcedure(f.apply(ri, override)); 2352 } 2353 return pid; 2354 } 2355 2356 /** 2357 * A 'raw' version of assign that does bulk and skirts Master state checks (assigns can be made 2358 * during Master startup). For use by Hbck2. 2359 */ 2360 @Override 2361 public MasterProtos.AssignsResponse assigns(RpcController controller, 2362 MasterProtos.AssignsRequest request) 2363 throws ServiceException { 2364 LOG.info(master.getClientIdAuditPrefix() + " assigns"); 2365 if (this.master.getMasterProcedureExecutor() == null) { 2366 throw new ServiceException("Master's ProcedureExecutor not initialized; retry later"); 2367 } 2368 MasterProtos.AssignsResponse.Builder responseBuilder = 2369 MasterProtos.AssignsResponse.newBuilder(); 2370 try { 2371 boolean override = request.getOverride(); 2372 for (HBaseProtos.RegionSpecifier rs: request.getRegionList()) { 2373 long pid = submitProcedure(rs, override, 2374 (r, b) -> this.master.getAssignmentManager().createAssignProcedure(r, b)); 2375 responseBuilder.addPid(pid); 2376 } 2377 return responseBuilder.build(); 2378 } catch (IOException ioe) { 2379 throw new ServiceException(ioe); 2380 } 2381 } 2382 2383 /** 2384 * A 'raw' version of unassign that does bulk and skirts Master state checks (unassigns can be 2385 * made during Master startup). For use by Hbck2. 2386 */ 2387 @Override 2388 public MasterProtos.UnassignsResponse unassigns(RpcController controller, 2389 MasterProtos.UnassignsRequest request) 2390 throws ServiceException { 2391 LOG.info(master.getClientIdAuditPrefix() + " unassigns"); 2392 if (this.master.getMasterProcedureExecutor() == null) { 2393 throw new ServiceException("Master's ProcedureExecutor not initialized; retry later"); 2394 } 2395 MasterProtos.UnassignsResponse.Builder responseBuilder = 2396 MasterProtos.UnassignsResponse.newBuilder(); 2397 try { 2398 boolean override = request.getOverride(); 2399 for (HBaseProtos.RegionSpecifier rs: request.getRegionList()) { 2400 long pid = submitProcedure(rs, override, 2401 (r, b) -> this.master.getAssignmentManager().createUnassignProcedure(r, b)); 2402 responseBuilder.addPid(pid); 2403 } 2404 return responseBuilder.build(); 2405 } catch (IOException ioe) { 2406 throw new ServiceException(ioe); 2407 } 2408 } 2409 2410 /** 2411 * Bypass specified procedure to completion. Procedure is marked completed but no actual work 2412 * is done from the current state/ step onwards. Parents of the procedure are also marked for 2413 * bypass. 2414 * 2415 * NOTE: this is a dangerous operation and may be used to unstuck buggy procedures. This may 2416 * leave system in inconherent state. This may need to be followed by some cleanup steps/ 2417 * actions by operator. 2418 * 2419 * @return BypassProcedureToCompletionResponse indicating success or failure 2420 */ 2421 @Override 2422 public MasterProtos.BypassProcedureResponse bypassProcedure(RpcController controller, 2423 MasterProtos.BypassProcedureRequest request) throws ServiceException { 2424 try { 2425 List<Boolean> ret = 2426 master.getMasterProcedureExecutor().bypassProcedure(request.getProcIdList(), 2427 request.getWaitTime(), request.getOverride(), request.getRecursive()); 2428 return MasterProtos.BypassProcedureResponse.newBuilder().addAllBypassed(ret).build(); 2429 } catch (IOException e) { 2430 throw new ServiceException(e); 2431 } 2432 } 2433 2434 @Override 2435 public MasterProtos.ScheduleServerCrashProcedureResponse scheduleServerCrashProcedure( 2436 RpcController controller, MasterProtos.ScheduleServerCrashProcedureRequest request) 2437 throws ServiceException { 2438 List<HBaseProtos.ServerName> serverNames = request.getServerNameList(); 2439 List<Long> pids = new ArrayList<>(); 2440 try { 2441 for (HBaseProtos.ServerName serverName : serverNames) { 2442 ServerName server = ProtobufUtil.toServerName(serverName); 2443 if (shouldSubmitSCP(server)) { 2444 ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor(); 2445 pids.add(procExec.submitProcedure(new ServerCrashProcedure(procExec.getEnvironment(), 2446 server, true, containMetaWals(server)))); 2447 } else { 2448 pids.add(-1L); 2449 } 2450 } 2451 return MasterProtos.ScheduleServerCrashProcedureResponse.newBuilder().addAllPid(pids).build(); 2452 } catch (IOException e) { 2453 throw new ServiceException(e); 2454 } 2455 } 2456 2457 private boolean containMetaWals(ServerName serverName) throws IOException { 2458 Path logDir = new Path(master.getWALRootDir(), 2459 AbstractFSWALProvider.getWALDirectoryName(serverName.toString())); 2460 Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT); 2461 Path checkDir = master.getFileSystem().exists(splitDir) ? splitDir : logDir; 2462 return master.getFileSystem().listStatus(checkDir, META_FILTER).length > 0; 2463 } 2464 2465 private boolean shouldSubmitSCP(ServerName serverName) { 2466 // check if there is already a SCP of this server running 2467 List<Procedure<MasterProcedureEnv>> procedures = 2468 master.getMasterProcedureExecutor().getProcedures(); 2469 for (Procedure<MasterProcedureEnv> procedure : procedures) { 2470 if (procedure instanceof ServerCrashProcedure) { 2471 if (serverName.compareTo(((ServerCrashProcedure) procedure).getServerName()) == 0 2472 && !procedure.isFinished()) { 2473 LOG.info("there is already a SCP of this server {} running, pid {}", serverName, 2474 procedure.getProcId()); 2475 return false; 2476 } 2477 } 2478 } 2479 return true; 2480 } 2481}