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