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