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 java.io.IOException;
021import java.lang.reflect.InvocationTargetException;
022import java.lang.reflect.Method;
023import java.net.InetAddress;
024import java.util.ArrayList;
025import java.util.Collections;
026import java.util.HashMap;
027import java.util.HashSet;
028import java.util.List;
029import java.util.Map;
030import java.util.Map.Entry;
031import java.util.Optional;
032import java.util.Set;
033import java.util.concurrent.CompletableFuture;
034import java.util.concurrent.ExecutionException;
035import java.util.concurrent.ThreadLocalRandom;
036import java.util.stream.Collectors;
037import org.apache.hadoop.conf.Configuration;
038import org.apache.hadoop.hbase.ClusterMetricsBuilder;
039import org.apache.hadoop.hbase.DoNotRetryIOException;
040import org.apache.hadoop.hbase.HBaseRpcServicesBase;
041import org.apache.hadoop.hbase.HConstants;
042import org.apache.hadoop.hbase.MasterNotRunningException;
043import org.apache.hadoop.hbase.MetaTableAccessor;
044import org.apache.hadoop.hbase.NamespaceDescriptor;
045import org.apache.hadoop.hbase.ServerMetrics;
046import org.apache.hadoop.hbase.ServerMetricsBuilder;
047import org.apache.hadoop.hbase.ServerName;
048import org.apache.hadoop.hbase.TableName;
049import org.apache.hadoop.hbase.UnknownRegionException;
050import org.apache.hadoop.hbase.client.BalanceRequest;
051import org.apache.hadoop.hbase.client.BalanceResponse;
052import org.apache.hadoop.hbase.client.MasterSwitchType;
053import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
054import org.apache.hadoop.hbase.client.RegionInfo;
055import org.apache.hadoop.hbase.client.Table;
056import org.apache.hadoop.hbase.client.TableDescriptor;
057import org.apache.hadoop.hbase.client.TableState;
058import org.apache.hadoop.hbase.client.VersionInfoUtil;
059import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
060import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
061import org.apache.hadoop.hbase.errorhandling.ForeignException;
062import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
063import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
064import org.apache.hadoop.hbase.ipc.PriorityFunction;
065import org.apache.hadoop.hbase.ipc.QosPriority;
066import org.apache.hadoop.hbase.ipc.RpcServer;
067import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
068import org.apache.hadoop.hbase.ipc.ServerRpcController;
069import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
070import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
071import org.apache.hadoop.hbase.master.assignment.RegionStates;
072import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
073import org.apache.hadoop.hbase.master.hbck.HbckChore;
074import org.apache.hadoop.hbase.master.janitor.MetaFixer;
075import org.apache.hadoop.hbase.master.locking.LockProcedure;
076import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
077import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
078import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
079import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
080import org.apache.hadoop.hbase.master.replication.AbstractPeerNoLockProcedure;
081import org.apache.hadoop.hbase.mob.MobUtils;
082import org.apache.hadoop.hbase.namequeues.BalancerDecisionDetails;
083import org.apache.hadoop.hbase.namequeues.BalancerRejectionDetails;
084import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
085import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
086import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
087import org.apache.hadoop.hbase.net.Address;
088import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
089import org.apache.hadoop.hbase.procedure2.LockType;
090import org.apache.hadoop.hbase.procedure2.LockedResource;
091import org.apache.hadoop.hbase.procedure2.Procedure;
092import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
093import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
094import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
095import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
096import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
097import org.apache.hadoop.hbase.quotas.QuotaUtil;
098import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
099import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
100import org.apache.hadoop.hbase.replication.ReplicationException;
101import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
102import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
103import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
104import org.apache.hadoop.hbase.rsgroup.RSGroupUtil;
105import org.apache.hadoop.hbase.security.Superusers;
106import org.apache.hadoop.hbase.security.User;
107import org.apache.hadoop.hbase.security.access.AccessChecker;
108import org.apache.hadoop.hbase.security.access.AccessChecker.InputUser;
109import org.apache.hadoop.hbase.security.access.AccessController;
110import org.apache.hadoop.hbase.security.access.Permission;
111import org.apache.hadoop.hbase.security.access.Permission.Action;
112import org.apache.hadoop.hbase.security.access.PermissionStorage;
113import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
114import org.apache.hadoop.hbase.security.access.UserPermission;
115import org.apache.hadoop.hbase.security.visibility.VisibilityController;
116import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
117import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
118import org.apache.hadoop.hbase.util.Bytes;
119import org.apache.hadoop.hbase.util.DNS;
120import org.apache.hadoop.hbase.util.DNS.ServerType;
121import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
122import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
123import org.apache.hadoop.hbase.util.Pair;
124import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
125import org.apache.yetus.audience.InterfaceAudience;
126import org.slf4j.Logger;
127import org.slf4j.LoggerFactory;
128
129import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
130import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
131import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
132import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
133import org.apache.hbase.thirdparty.com.google.protobuf.Message;
134import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
135import org.apache.hbase.thirdparty.com.google.protobuf.Service;
136import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
137import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
138
139import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
140import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
141import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
142import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService;
143import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest;
144import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
145import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
146import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantResponse;
147import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsRequest;
148import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsResponse;
149import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type;
150import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest;
151import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeResponse;
152import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
153import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
154import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
155import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
156import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
157import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
158import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
159import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
160import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
161import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest;
162import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
163import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
164import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
165import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
166import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
167import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListRequest;
168import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListResponse;
169import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
170import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
171import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
172import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
173import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
174import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
175import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
176import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
177import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
178import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
179import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
180import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
181import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
182import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
183import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
184import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse;
185import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest;
186import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse;
187import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
188import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
189import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
190import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse;
191import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
192import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
193import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
194import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
195import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
196import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
197import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
198import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
199import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo;
200import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
201import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
202import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
203import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
204import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
205import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
206import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
207import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
208import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
209import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
210import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
211import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
212import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
213import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse;
214import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
215import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
216import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
217import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
218import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersRequest;
219import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersResponse;
220import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
221import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
222import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
223import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
224import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
225import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
226import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
227import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
228import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
229import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
230import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
231import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
232import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
233import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
234import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
235import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
236import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
237import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
238import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreRequest;
239import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreResponse;
240import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushTableRequest;
241import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushTableResponse;
242import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
243import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
244import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
245import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
246import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
247import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
248import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
249import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
250import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
251import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
252import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
253import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
254import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
255import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
256import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
257import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
258import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
259import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
260import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
261import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
262import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService;
263import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
264import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
265import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
266import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
267import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
268import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse;
269import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
270import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
271import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
272import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
273import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
274import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
275import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
276import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
277import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledRequest;
278import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledResponse;
279import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
280import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
281import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
282import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
283import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest;
284import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse;
285import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
286import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
287import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesRequest;
288import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesResponse;
289import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
290import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
291import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateRequest;
292import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateResponse;
293import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
294import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
295import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByStateRequest;
296import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByStateResponse;
297import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
298import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
299import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
300import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
301import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
302import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
303import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
304import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
305import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest;
306import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse;
307import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
308import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
309import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
310import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
311import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest;
312import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse;
313import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
314import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
315import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
316import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
317import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
318import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
319import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest;
320import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse;
321import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RegionSpecifierAndState;
322import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
323import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
324import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RollAllWALWritersRequest;
325import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RollAllWALWritersResponse;
326import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
327import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
328import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
329import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
330import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreRequest;
331import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreResponse;
332import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
333import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
334import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
335import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
336import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
337import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse;
338import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
339import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
340import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
341import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
342import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetRegionStateInMetaRequest;
343import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetRegionStateInMetaResponse;
344import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSnapshotCleanupRequest;
345import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSnapshotCleanupResponse;
346import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
347import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
348import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest;
349import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
350import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
351import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
352import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
353import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
354import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse;
355import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
356import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse;
357import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaRequest;
358import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaResponse;
359import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleRequest;
360import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleResponse;
361import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
362import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
363import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
364import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
365import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
366import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
367import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot;
368import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot;
369import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
370import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
371import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
372import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
373import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
374import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.AddRSGroupRequest;
375import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.AddRSGroupResponse;
376import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.BalanceRSGroupRequest;
377import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.BalanceRSGroupResponse;
378import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetConfiguredNamespacesAndTablesInRSGroupRequest;
379import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetConfiguredNamespacesAndTablesInRSGroupResponse;
380import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoOfServerRequest;
381import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoOfServerResponse;
382import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoOfTableRequest;
383import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoOfTableResponse;
384import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoRequest;
385import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoResponse;
386import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosRequest;
387import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosResponse;
388import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListTablesInRSGroupRequest;
389import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListTablesInRSGroupResponse;
390import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersRequest;
391import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersResponse;
392import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
393import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
394import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
395import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersResponse;
396import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RenameRSGroupRequest;
397import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RenameRSGroupResponse;
398import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.UpdateRSGroupConfigRequest;
399import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.UpdateRSGroupConfigResponse;
400import org.apache.hadoop.hbase.shaded.protobuf.generated.RecentLogs;
401import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos;
402import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationRequest;
403import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationResponse;
404import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
405import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
406import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLiveRegionServersRequest;
407import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLiveRegionServersResponse;
408import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
409import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse;
410import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
411import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
412import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
413import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
414import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
415import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse;
416import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult;
417import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
418import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse;
419import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
420import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
421import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
422import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
423import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService;
424import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
425import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
426import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
427import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
428import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest;
429import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
430import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
431import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
432import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerModificationProceduresRequest;
433import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerModificationProceduresResponse;
434import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerStateRequest;
435import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerStateResponse;
436import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.IsReplicationPeerModificationEnabledRequest;
437import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.IsReplicationPeerModificationEnabledResponse;
438import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
439import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
440import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
441import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
442import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerModificationSwitchRequest;
443import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerModificationSwitchResponse;
444import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState;
445import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
446import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
447import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
448import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
449import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
450import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
451
452/**
453 * Implements the master RPC services.
454 */
455@InterfaceAudience.Private
456public class MasterRpcServices extends HBaseRpcServicesBase<HMaster>
457  implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
458  LockService.BlockingInterface, HbckService.BlockingInterface {
459
460  private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
461  private static final Logger AUDITLOG =
462    LoggerFactory.getLogger("SecurityLogger." + MasterRpcServices.class.getName());
463
464  /** RPC scheduler to use for the master. */
465  public static final String MASTER_RPC_SCHEDULER_FACTORY_CLASS =
466    "hbase.master.rpc.scheduler.factory.class";
467
468  /**
469   * @return Subset of configuration to pass initializing regionservers: e.g. the filesystem to use
470   *         and root directory to use.
471   */
472  private RegionServerStartupResponse.Builder createConfigurationSubset() {
473    RegionServerStartupResponse.Builder resp =
474      addConfig(RegionServerStartupResponse.newBuilder(), HConstants.HBASE_DIR);
475    resp = addConfig(resp, "fs.defaultFS");
476    return addConfig(resp, "hbase.master.info.port");
477  }
478
479  private RegionServerStartupResponse.Builder
480    addConfig(final RegionServerStartupResponse.Builder resp, final String key) {
481    NameStringPair.Builder entry =
482      NameStringPair.newBuilder().setName(key).setValue(server.getConfiguration().get(key));
483    resp.addMapEntries(entry.build());
484    return resp;
485  }
486
487  public MasterRpcServices(HMaster m) throws IOException {
488    super(m, m.getProcessName());
489  }
490
491  @Override
492  protected boolean defaultReservoirEnabled() {
493    return false;
494  }
495
496  @Override
497  protected ServerType getDNSServerType() {
498    return DNS.ServerType.MASTER;
499  }
500
501  @Override
502  protected String getHostname(Configuration conf, String defaultHostname) {
503    return conf.get("hbase.master.ipc.address", defaultHostname);
504  }
505
506  @Override
507  protected String getPortConfigName() {
508    return HConstants.MASTER_PORT;
509  }
510
511  @Override
512  protected int getDefaultPort() {
513    return HConstants.DEFAULT_MASTER_PORT;
514  }
515
516  @Override
517  protected Class<?> getRpcSchedulerFactoryClass(Configuration conf) {
518    return conf.getClass(MASTER_RPC_SCHEDULER_FACTORY_CLASS, SimpleRpcSchedulerFactory.class);
519  }
520
521  @Override
522  protected PriorityFunction createPriority() {
523    return new MasterAnnotationReadingPriorityFunction(this);
524  }
525
526  /**
527   * Checks for the following pre-checks in order:
528   * <ol>
529   * <li>Master is initialized</li>
530   * <li>Rpc caller has admin permissions</li>
531   * </ol>
532   * @param requestName name of rpc request. Used in reporting failures to provide context.
533   * @throws ServiceException If any of the above listed pre-check fails.
534   */
535  private void rpcPreCheck(String requestName) throws ServiceException {
536    try {
537      server.checkInitialized();
538      requirePermission(requestName, Permission.Action.ADMIN);
539    } catch (IOException ioe) {
540      throw new ServiceException(ioe);
541    }
542  }
543
544  enum BalanceSwitchMode {
545    SYNC,
546    ASYNC
547  }
548
549  /**
550   * Assigns balancer switch according to BalanceSwitchMode
551   * @param b    new balancer switch
552   * @param mode BalanceSwitchMode
553   * @return old balancer switch
554   */
555  boolean switchBalancer(final boolean b, BalanceSwitchMode mode) throws IOException {
556    boolean oldValue = server.loadBalancerStateStore.get();
557    boolean newValue = b;
558    try {
559      if (server.cpHost != null) {
560        server.cpHost.preBalanceSwitch(newValue);
561      }
562      if (mode == BalanceSwitchMode.SYNC) {
563        synchronized (server.getLoadBalancer()) {
564          server.loadBalancerStateStore.set(newValue);
565        }
566      } else {
567        server.loadBalancerStateStore.set(newValue);
568      }
569      LOG.info(server.getClientIdAuditPrefix() + " set balanceSwitch=" + newValue);
570      if (server.cpHost != null) {
571        server.cpHost.postBalanceSwitch(oldValue, newValue);
572      }
573      server.getLoadBalancer().updateBalancerStatus(newValue);
574    } catch (IOException ioe) {
575      LOG.warn("Error flipping balance switch", ioe);
576    }
577    return oldValue;
578  }
579
580  boolean synchronousBalanceSwitch(final boolean b) throws IOException {
581    return switchBalancer(b, BalanceSwitchMode.SYNC);
582  }
583
584  /** Returns list of blocking services and their security info classes that this server supports */
585  @Override
586  protected List<BlockingServiceAndInterface> getServices() {
587    List<BlockingServiceAndInterface> bssi = new ArrayList<>(5);
588    bssi.add(new BlockingServiceAndInterface(MasterService.newReflectiveBlockingService(this),
589      MasterService.BlockingInterface.class));
590    bssi.add(
591      new BlockingServiceAndInterface(RegionServerStatusService.newReflectiveBlockingService(this),
592        RegionServerStatusService.BlockingInterface.class));
593    bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
594      LockService.BlockingInterface.class));
595    bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this),
596      HbckService.BlockingInterface.class));
597    bssi.add(new BlockingServiceAndInterface(ClientMetaService.newReflectiveBlockingService(this),
598      ClientMetaService.BlockingInterface.class));
599    bssi.add(new BlockingServiceAndInterface(AdminService.newReflectiveBlockingService(this),
600      AdminService.BlockingInterface.class));
601    return bssi;
602  }
603
604  void start(ZKWatcher zkWatcher) {
605    internalStart(zkWatcher);
606  }
607
608  void stop() {
609    internalStop();
610  }
611
612  @Override
613  // priority for all RegionServerStatusProtos rpc's are set HIGH_QOS in
614  // MasterAnnotationReadingPriorityFunction itself
615  public GetLastFlushedSequenceIdResponse getLastFlushedSequenceId(RpcController controller,
616    GetLastFlushedSequenceIdRequest request) throws ServiceException {
617    try {
618      server.checkServiceStarted();
619    } catch (IOException ioe) {
620      throw new ServiceException(ioe);
621    }
622    byte[] encodedRegionName = request.getRegionName().toByteArray();
623    RegionStoreSequenceIds ids =
624      server.getServerManager().getLastFlushedSequenceId(encodedRegionName);
625    return ResponseConverter.buildGetLastFlushedSequenceIdResponse(ids);
626  }
627
628  @Override
629  public RegionServerReportResponse regionServerReport(RpcController controller,
630    RegionServerReportRequest request) throws ServiceException {
631    try {
632      server.checkServiceStarted();
633      int versionNumber = 0;
634      String version = "0.0.0";
635      VersionInfo versionInfo = VersionInfoUtil.getCurrentClientVersionInfo();
636      if (versionInfo != null) {
637        version = versionInfo.getVersion();
638        versionNumber = VersionInfoUtil.getVersionNumber(versionInfo);
639      }
640      ClusterStatusProtos.ServerLoad sl = request.getLoad();
641      ServerName serverName = ProtobufUtil.toServerName(request.getServer());
642      ServerMetrics oldLoad = server.getServerManager().getLoad(serverName);
643      ServerMetrics newLoad =
644        ServerMetricsBuilder.toServerMetrics(serverName, versionNumber, version, sl);
645      server.getServerManager().regionServerReport(serverName, newLoad);
646      server.getAssignmentManager().reportOnlineRegions(serverName,
647        newLoad.getRegionMetrics().keySet());
648      if (sl != null && server.metricsMaster != null) {
649        // Up our metrics.
650        server.metricsMaster.incrementRequests(
651          sl.getTotalNumberOfRequests() - (oldLoad != null ? oldLoad.getRequestCount() : 0));
652        server.metricsMaster.incrementReadRequests(
653          sl.getReadRequestsCount() - (oldLoad != null ? oldLoad.getReadRequestsCount() : 0));
654        server.metricsMaster.incrementWriteRequests(
655          sl.getWriteRequestsCount() - (oldLoad != null ? oldLoad.getWriteRequestsCount() : 0));
656      }
657    } catch (IOException ioe) {
658      throw new ServiceException(ioe);
659    }
660    return RegionServerReportResponse.newBuilder().build();
661  }
662
663  @Override
664  public RegionServerStartupResponse regionServerStartup(RpcController controller,
665    RegionServerStartupRequest request) throws ServiceException {
666    // Register with server manager
667    try {
668      server.checkServiceStarted();
669      int versionNumber = 0;
670      String version = "0.0.0";
671      VersionInfo versionInfo = VersionInfoUtil.getCurrentClientVersionInfo();
672      if (versionInfo != null) {
673        version = versionInfo.getVersion();
674        versionNumber = VersionInfoUtil.getVersionNumber(versionInfo);
675      }
676      InetAddress ia = server.getRemoteInetAddress(request.getPort(), request.getServerStartCode());
677      // if regionserver passed hostname to use,
678      // then use it instead of doing a reverse DNS lookup
679      ServerName rs =
680        server.getServerManager().regionServerStartup(request, versionNumber, version, ia);
681
682      // Send back some config info
683      RegionServerStartupResponse.Builder resp = createConfigurationSubset();
684      NameStringPair.Builder entry = NameStringPair.newBuilder()
685        .setName(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER).setValue(rs.getHostname());
686      resp.addMapEntries(entry.build());
687
688      return resp.build();
689    } catch (IOException ioe) {
690      throw new ServiceException(ioe);
691    }
692  }
693
694  @Override
695  public ReportRSFatalErrorResponse reportRSFatalError(RpcController controller,
696    ReportRSFatalErrorRequest request) throws ServiceException {
697    String errorText = request.getErrorMessage();
698    ServerName sn = ProtobufUtil.toServerName(request.getServer());
699    String msg = sn + " reported a fatal error:\n" + errorText;
700    LOG.warn(msg);
701    server.rsFatals.add(msg);
702    return ReportRSFatalErrorResponse.newBuilder().build();
703  }
704
705  @Override
706  public AddColumnResponse addColumn(RpcController controller, AddColumnRequest req)
707    throws ServiceException {
708    try {
709      long procId = server.addColumn(ProtobufUtil.toTableName(req.getTableName()),
710        ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()), req.getNonceGroup(),
711        req.getNonce());
712      if (procId == -1) {
713        // This mean operation was not performed in server, so do not set any procId
714        return AddColumnResponse.newBuilder().build();
715      } else {
716        return AddColumnResponse.newBuilder().setProcId(procId).build();
717      }
718    } catch (IOException ioe) {
719      throw new ServiceException(ioe);
720    }
721  }
722
723  @Override
724  public AssignRegionResponse assignRegion(RpcController controller, AssignRegionRequest req)
725    throws ServiceException {
726    try {
727      server.checkInitialized();
728
729      final RegionSpecifierType type = req.getRegion().getType();
730      if (type != RegionSpecifierType.REGION_NAME) {
731        LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
732          + " actual: " + type);
733      }
734
735      final byte[] regionName = req.getRegion().getValue().toByteArray();
736      final RegionInfo regionInfo = server.getAssignmentManager().getRegionInfo(regionName);
737      if (regionInfo == null) {
738        throw new UnknownRegionException(Bytes.toStringBinary(regionName));
739      }
740
741      final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
742      if (server.cpHost != null) {
743        server.cpHost.preAssign(regionInfo);
744      }
745      LOG.info(server.getClientIdAuditPrefix() + " assign " + regionInfo.getRegionNameAsString());
746      server.getAssignmentManager().assign(regionInfo);
747      if (server.cpHost != null) {
748        server.cpHost.postAssign(regionInfo);
749      }
750      return arr;
751    } catch (IOException ioe) {
752      throw new ServiceException(ioe);
753    }
754  }
755
756  @Override
757  public MasterProtos.BalanceResponse balance(RpcController controller,
758    MasterProtos.BalanceRequest request) throws ServiceException {
759    try {
760      return ProtobufUtil.toBalanceResponse(server.balance(ProtobufUtil.toBalanceRequest(request)));
761    } catch (IOException ex) {
762      throw new ServiceException(ex);
763    }
764  }
765
766  @Override
767  public CreateNamespaceResponse createNamespace(RpcController controller,
768    CreateNamespaceRequest request) throws ServiceException {
769    try {
770      long procId =
771        server.createNamespace(ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
772          request.getNonceGroup(), request.getNonce());
773      return CreateNamespaceResponse.newBuilder().setProcId(procId).build();
774    } catch (IOException e) {
775      throw new ServiceException(e);
776    }
777  }
778
779  @Override
780  public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
781    throws ServiceException {
782    TableDescriptor tableDescriptor = ProtobufUtil.toTableDescriptor(req.getTableSchema());
783    byte[][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
784    try {
785      long procId =
786        server.createTable(tableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce());
787      LOG.info(server.getClientIdAuditPrefix() + " procedure request for creating table: "
788        + req.getTableSchema().getTableName() + " procId is: " + procId);
789      return CreateTableResponse.newBuilder().setProcId(procId).build();
790    } catch (IOException ioe) {
791      throw new ServiceException(ioe);
792    }
793  }
794
795  @Override
796  public DeleteColumnResponse deleteColumn(RpcController controller, DeleteColumnRequest req)
797    throws ServiceException {
798    try {
799      long procId = server.deleteColumn(ProtobufUtil.toTableName(req.getTableName()),
800        req.getColumnName().toByteArray(), req.getNonceGroup(), req.getNonce());
801      if (procId == -1) {
802        // This mean operation was not performed in server, so do not set any procId
803        return DeleteColumnResponse.newBuilder().build();
804      } else {
805        return DeleteColumnResponse.newBuilder().setProcId(procId).build();
806      }
807    } catch (IOException ioe) {
808      throw new ServiceException(ioe);
809    }
810  }
811
812  @Override
813  public DeleteNamespaceResponse deleteNamespace(RpcController controller,
814    DeleteNamespaceRequest request) throws ServiceException {
815    try {
816      long procId = server.deleteNamespace(request.getNamespaceName(), request.getNonceGroup(),
817        request.getNonce());
818      return DeleteNamespaceResponse.newBuilder().setProcId(procId).build();
819    } catch (IOException e) {
820      throw new ServiceException(e);
821    }
822  }
823
824  /**
825   * Execute Delete Snapshot operation.
826   * @return DeleteSnapshotResponse (a protobuf wrapped void) if the snapshot existed and was
827   *         deleted properly.
828   * @throws ServiceException wrapping SnapshotDoesNotExistException if specified snapshot did not
829   *                          exist.
830   */
831  @Override
832  public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
833    DeleteSnapshotRequest request) throws ServiceException {
834    try {
835      server.checkInitialized();
836      server.snapshotManager.checkSnapshotSupport();
837
838      LOG.info(server.getClientIdAuditPrefix() + " delete " + request.getSnapshot());
839      server.snapshotManager.deleteSnapshot(request.getSnapshot());
840      return DeleteSnapshotResponse.newBuilder().build();
841    } catch (IOException e) {
842      throw new ServiceException(e);
843    }
844  }
845
846  @Override
847  public DeleteTableResponse deleteTable(RpcController controller, DeleteTableRequest request)
848    throws ServiceException {
849    try {
850      long procId = server.deleteTable(ProtobufUtil.toTableName(request.getTableName()),
851        request.getNonceGroup(), request.getNonce());
852      return DeleteTableResponse.newBuilder().setProcId(procId).build();
853    } catch (IOException ioe) {
854      throw new ServiceException(ioe);
855    }
856  }
857
858  @Override
859  public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
860    throws ServiceException {
861    try {
862      long procId = server.truncateTable(ProtobufUtil.toTableName(request.getTableName()),
863        request.getPreserveSplits(), request.getNonceGroup(), request.getNonce());
864      return TruncateTableResponse.newBuilder().setProcId(procId).build();
865    } catch (IOException ioe) {
866      throw new ServiceException(ioe);
867    }
868  }
869
870  @Override
871  public DisableTableResponse disableTable(RpcController controller, DisableTableRequest request)
872    throws ServiceException {
873    try {
874      long procId = server.disableTable(ProtobufUtil.toTableName(request.getTableName()),
875        request.getNonceGroup(), request.getNonce());
876      return DisableTableResponse.newBuilder().setProcId(procId).build();
877    } catch (IOException ioe) {
878      throw new ServiceException(ioe);
879    }
880  }
881
882  @Override
883  public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
884    EnableCatalogJanitorRequest req) throws ServiceException {
885    rpcPreCheck("enableCatalogJanitor");
886    return EnableCatalogJanitorResponse.newBuilder()
887      .setPrevValue(server.catalogJanitorChore.setEnabled(req.getEnable())).build();
888  }
889
890  @Override
891  public SetCleanerChoreRunningResponse setCleanerChoreRunning(RpcController c,
892    SetCleanerChoreRunningRequest req) throws ServiceException {
893    rpcPreCheck("setCleanerChoreRunning");
894
895    boolean prevValue =
896      server.getLogCleaner().getEnabled() && server.getHFileCleaner().getEnabled();
897    server.getLogCleaner().setEnabled(req.getOn());
898    for (HFileCleaner hFileCleaner : server.getHFileCleaners()) {
899      hFileCleaner.setEnabled(req.getOn());
900    }
901    return SetCleanerChoreRunningResponse.newBuilder().setPrevValue(prevValue).build();
902  }
903
904  @Override
905  public EnableTableResponse enableTable(RpcController controller, EnableTableRequest request)
906    throws ServiceException {
907    try {
908      long procId = server.enableTable(ProtobufUtil.toTableName(request.getTableName()),
909        request.getNonceGroup(), request.getNonce());
910      return EnableTableResponse.newBuilder().setProcId(procId).build();
911    } catch (IOException ioe) {
912      throw new ServiceException(ioe);
913    }
914  }
915
916  @Override
917  public MergeTableRegionsResponse mergeTableRegions(RpcController c,
918    MergeTableRegionsRequest request) throws ServiceException {
919    try {
920      server.checkInitialized();
921    } catch (IOException ioe) {
922      throw new ServiceException(ioe);
923    }
924
925    RegionStates regionStates = server.getAssignmentManager().getRegionStates();
926
927    RegionInfo[] regionsToMerge = new RegionInfo[request.getRegionCount()];
928    for (int i = 0; i < request.getRegionCount(); i++) {
929      final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray();
930      if (request.getRegion(i).getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
931        LOG.warn("MergeRegions specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME
932          + " actual: region " + i + " =" + request.getRegion(i).getType());
933      }
934      RegionState regionState = regionStates.getRegionState(Bytes.toString(encodedNameOfRegion));
935      if (regionState == null) {
936        throw new ServiceException(
937          new UnknownRegionException(Bytes.toStringBinary(encodedNameOfRegion)));
938      }
939      regionsToMerge[i] = regionState.getRegion();
940    }
941
942    try {
943      long procId = server.mergeRegions(regionsToMerge, request.getForcible(),
944        request.getNonceGroup(), request.getNonce());
945      return MergeTableRegionsResponse.newBuilder().setProcId(procId).build();
946    } catch (IOException ioe) {
947      throw new ServiceException(ioe);
948    }
949  }
950
951  @Override
952  public SplitTableRegionResponse splitRegion(final RpcController controller,
953    final SplitTableRegionRequest request) throws ServiceException {
954    try {
955      long procId = server.splitRegion(ProtobufUtil.toRegionInfo(request.getRegionInfo()),
956        request.hasSplitRow() ? request.getSplitRow().toByteArray() : null, request.getNonceGroup(),
957        request.getNonce());
958      return SplitTableRegionResponse.newBuilder().setProcId(procId).build();
959    } catch (IOException ie) {
960      throw new ServiceException(ie);
961    }
962  }
963
964  @Override
965  public MasterProtos.TruncateRegionResponse truncateRegion(RpcController controller,
966    final MasterProtos.TruncateRegionRequest request) throws ServiceException {
967    try {
968      long procId = server.truncateRegion(ProtobufUtil.toRegionInfo(request.getRegionInfo()),
969        request.getNonceGroup(), request.getNonce());
970      return MasterProtos.TruncateRegionResponse.newBuilder().setProcId(procId).build();
971    } catch (IOException ie) {
972      throw new ServiceException(ie);
973    }
974  }
975
976  @Override
977  public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
978    final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {
979    rpcPreCheck("execMasterService");
980    try {
981      ServerRpcController execController = new ServerRpcController();
982      ClientProtos.CoprocessorServiceCall call = request.getCall();
983      String serviceName = call.getServiceName();
984      String methodName = call.getMethodName();
985      if (!server.coprocessorServiceHandlers.containsKey(serviceName)) {
986        throw new UnknownProtocolException(null,
987          "No registered Master Coprocessor Endpoint found for " + serviceName
988            + ". Has it been enabled?");
989      }
990
991      Service service = server.coprocessorServiceHandlers.get(serviceName);
992      ServiceDescriptor serviceDesc = service.getDescriptorForType();
993      MethodDescriptor methodDesc =
994        CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
995
996      Message execRequest = CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
997      final Message.Builder responseBuilder =
998        service.getResponsePrototype(methodDesc).newBuilderForType();
999      service.callMethod(methodDesc, execController, execRequest, (message) -> {
1000        if (message != null) {
1001          responseBuilder.mergeFrom(message);
1002        }
1003      });
1004      Message execResult = responseBuilder.build();
1005      if (execController.getFailedOn() != null) {
1006        throw execController.getFailedOn();
1007      }
1008
1009      String remoteAddress = RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("");
1010      User caller = RpcServer.getRequestUser().orElse(null);
1011      AUDITLOG.info("User {} (remote address: {}) master service request for {}.{}", caller,
1012        remoteAddress, serviceName, methodName);
1013
1014      return CoprocessorRpcUtils.getResponse(execResult, HConstants.EMPTY_BYTE_ARRAY);
1015    } catch (IOException ie) {
1016      throw new ServiceException(ie);
1017    }
1018  }
1019
1020  /**
1021   * Triggers an asynchronous attempt to run a distributed procedure. {@inheritDoc}
1022   */
1023  @Override
1024  public ExecProcedureResponse execProcedure(RpcController controller, ExecProcedureRequest request)
1025    throws ServiceException {
1026    try {
1027      server.checkInitialized();
1028      ProcedureDescription desc = request.getProcedure();
1029      MasterProcedureManager mpm =
1030        server.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature());
1031      if (mpm == null) {
1032        throw new ServiceException(
1033          new DoNotRetryIOException("The procedure is not registered: " + desc.getSignature()));
1034      }
1035      LOG.info(server.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
1036      mpm.checkPermissions(desc, getAccessChecker(), RpcServer.getRequestUser().orElse(null));
1037      mpm.execProcedure(desc);
1038      // send back the max amount of time the client should wait for the procedure
1039      // to complete
1040      long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME;
1041      return ExecProcedureResponse.newBuilder().setExpectedTimeout(waitTime).build();
1042    } catch (ForeignException e) {
1043      throw new ServiceException(e.getCause());
1044    } catch (IOException e) {
1045      throw new ServiceException(e);
1046    }
1047  }
1048
1049  /**
1050   * Triggers a synchronous attempt to run a distributed procedure and sets return data in response.
1051   * {@inheritDoc}
1052   */
1053  @Override
1054  public ExecProcedureResponse execProcedureWithRet(RpcController controller,
1055    ExecProcedureRequest request) throws ServiceException {
1056    rpcPreCheck("execProcedureWithRet");
1057    try {
1058      ProcedureDescription desc = request.getProcedure();
1059      MasterProcedureManager mpm =
1060        server.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature());
1061      if (mpm == null) {
1062        throw new ServiceException("The procedure is not registered: " + desc.getSignature());
1063      }
1064      LOG.info(server.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
1065      byte[] data = mpm.execProcedureWithRet(desc);
1066      ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder();
1067      // set return data if available
1068      if (data != null) {
1069        builder.setReturnData(UnsafeByteOperations.unsafeWrap(data));
1070      }
1071      return builder.build();
1072    } catch (IOException e) {
1073      throw new ServiceException(e);
1074    }
1075  }
1076
1077  @Override
1078  public GetClusterStatusResponse getClusterStatus(RpcController controller,
1079    GetClusterStatusRequest req) throws ServiceException {
1080    GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
1081    try {
1082      // We used to check if Master was up at this point but let this call proceed even if
1083      // Master is initializing... else we shut out stuff like hbck2 tool from making progress
1084      // since it queries this method to figure cluster version. hbck2 wants to be able to work
1085      // against Master even if it is 'initializing' so it can do fixup.
1086      response.setClusterStatus(ClusterMetricsBuilder.toClusterStatus(
1087        server.getClusterMetrics(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
1088    } catch (IOException e) {
1089      throw new ServiceException(e);
1090    }
1091    return response.build();
1092  }
1093
1094  /**
1095   * List the currently available/stored snapshots. Any in-progress snapshots are ignored
1096   */
1097  @Override
1098  public GetCompletedSnapshotsResponse getCompletedSnapshots(RpcController controller,
1099    GetCompletedSnapshotsRequest request) throws ServiceException {
1100    try {
1101      server.checkInitialized();
1102      GetCompletedSnapshotsResponse.Builder builder = GetCompletedSnapshotsResponse.newBuilder();
1103      List<SnapshotDescription> snapshots = server.snapshotManager.getCompletedSnapshots();
1104
1105      // convert to protobuf
1106      for (SnapshotDescription snapshot : snapshots) {
1107        builder.addSnapshots(snapshot);
1108      }
1109      return builder.build();
1110    } catch (IOException e) {
1111      throw new ServiceException(e);
1112    }
1113  }
1114
1115  @Override
1116  public ListNamespacesResponse listNamespaces(RpcController controller,
1117    ListNamespacesRequest request) throws ServiceException {
1118    try {
1119      return ListNamespacesResponse.newBuilder().addAllNamespaceName(server.listNamespaces())
1120        .build();
1121    } catch (IOException e) {
1122      throw new ServiceException(e);
1123    }
1124  }
1125
1126  @Override
1127  public GetNamespaceDescriptorResponse getNamespaceDescriptor(RpcController controller,
1128    GetNamespaceDescriptorRequest request) throws ServiceException {
1129    try {
1130      return GetNamespaceDescriptorResponse.newBuilder()
1131        .setNamespaceDescriptor(
1132          ProtobufUtil.toProtoNamespaceDescriptor(server.getNamespace(request.getNamespaceName())))
1133        .build();
1134    } catch (IOException e) {
1135      throw new ServiceException(e);
1136    }
1137  }
1138
1139  /**
1140   * Get the number of regions of the table that have been updated by the alter.
1141   * @return Pair indicating the number of regions updated Pair.getFirst is the regions that are yet
1142   *         to be updated Pair.getSecond is the total number of regions of the table
1143   */
1144  @Override
1145  public GetSchemaAlterStatusResponse getSchemaAlterStatus(RpcController controller,
1146    GetSchemaAlterStatusRequest req) throws ServiceException {
1147    // TODO: currently, we query using the table name on the client side. this
1148    // may overlap with other table operations or the table operation may
1149    // have completed before querying this API. We need to refactor to a
1150    // transaction system in the future to avoid these ambiguities.
1151    TableName tableName = ProtobufUtil.toTableName(req.getTableName());
1152
1153    try {
1154      server.checkInitialized();
1155      Pair<Integer, Integer> pair = server.getAssignmentManager().getReopenStatus(tableName);
1156      GetSchemaAlterStatusResponse.Builder ret = GetSchemaAlterStatusResponse.newBuilder();
1157      ret.setYetToUpdateRegions(pair.getFirst());
1158      ret.setTotalRegions(pair.getSecond());
1159      return ret.build();
1160    } catch (IOException ioe) {
1161      throw new ServiceException(ioe);
1162    }
1163  }
1164
1165  /**
1166   * Get list of TableDescriptors for requested tables.
1167   * @param c   Unused (set to null).
1168   * @param req GetTableDescriptorsRequest that contains: - tableNames: requested tables, or if
1169   *            empty, all are requested.
1170   */
1171  @Override
1172  public GetTableDescriptorsResponse getTableDescriptors(RpcController c,
1173    GetTableDescriptorsRequest req) throws ServiceException {
1174    try {
1175      server.checkInitialized();
1176
1177      final String regex = req.hasRegex() ? req.getRegex() : null;
1178      final String namespace = req.hasNamespace() ? req.getNamespace() : null;
1179      List<TableName> tableNameList = null;
1180      if (req.getTableNamesCount() > 0) {
1181        tableNameList = new ArrayList<TableName>(req.getTableNamesCount());
1182        for (HBaseProtos.TableName tableNamePB : req.getTableNamesList()) {
1183          tableNameList.add(ProtobufUtil.toTableName(tableNamePB));
1184        }
1185      }
1186
1187      List<TableDescriptor> descriptors =
1188        server.listTableDescriptors(namespace, regex, tableNameList, req.getIncludeSysTables());
1189
1190      GetTableDescriptorsResponse.Builder builder = GetTableDescriptorsResponse.newBuilder();
1191      if (descriptors != null && descriptors.size() > 0) {
1192        // Add the table descriptors to the response
1193        for (TableDescriptor htd : descriptors) {
1194          builder.addTableSchema(ProtobufUtil.toTableSchema(htd));
1195        }
1196      }
1197      return builder.build();
1198    } catch (IOException ioe) {
1199      throw new ServiceException(ioe);
1200    }
1201  }
1202
1203  @Override
1204  public ListTableDescriptorsByStateResponse listTableDescriptorsByState(RpcController controller,
1205    ListTableDescriptorsByStateRequest request) throws ServiceException {
1206    try {
1207      server.checkInitialized();
1208      List<TableDescriptor> descriptors = server.listTableDescriptors(null, null, null, false);
1209
1210      ListTableDescriptorsByStateResponse.Builder builder =
1211        ListTableDescriptorsByStateResponse.newBuilder();
1212      if (descriptors != null && descriptors.size() > 0) {
1213        // Add the table descriptors to the response
1214        TableState.State state =
1215          request.getIsEnabled() ? TableState.State.ENABLED : TableState.State.DISABLED;
1216        for (TableDescriptor htd : descriptors) {
1217          if (server.getTableStateManager().isTableState(htd.getTableName(), state)) {
1218            builder.addTableSchema(ProtobufUtil.toTableSchema(htd));
1219          }
1220        }
1221      }
1222      return builder.build();
1223    } catch (IOException ioe) {
1224      throw new ServiceException(ioe);
1225    }
1226  }
1227
1228  /**
1229   * Get list of userspace table names
1230   * @param controller Unused (set to null).
1231   * @param req        GetTableNamesRequest
1232   */
1233  @Override
1234  public GetTableNamesResponse getTableNames(RpcController controller, GetTableNamesRequest req)
1235    throws ServiceException {
1236    try {
1237      server.checkServiceStarted();
1238
1239      final String regex = req.hasRegex() ? req.getRegex() : null;
1240      final String namespace = req.hasNamespace() ? req.getNamespace() : null;
1241      List<TableName> tableNames =
1242        server.listTableNames(namespace, regex, req.getIncludeSysTables());
1243
1244      GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder();
1245      if (tableNames != null && tableNames.size() > 0) {
1246        // Add the table names to the response
1247        for (TableName table : tableNames) {
1248          builder.addTableNames(ProtobufUtil.toProtoTableName(table));
1249        }
1250      }
1251      return builder.build();
1252    } catch (IOException e) {
1253      throw new ServiceException(e);
1254    }
1255  }
1256
1257  @Override
1258  public ListTableNamesByStateResponse listTableNamesByState(RpcController controller,
1259    ListTableNamesByStateRequest request) throws ServiceException {
1260    try {
1261      server.checkServiceStarted();
1262      List<TableName> tableNames = server.listTableNames(null, null, false);
1263      ListTableNamesByStateResponse.Builder builder = ListTableNamesByStateResponse.newBuilder();
1264      if (tableNames != null && tableNames.size() > 0) {
1265        // Add the disabled table names to the response
1266        TableState.State state =
1267          request.getIsEnabled() ? TableState.State.ENABLED : TableState.State.DISABLED;
1268        for (TableName table : tableNames) {
1269          if (server.getTableStateManager().isTableState(table, state)) {
1270            builder.addTableNames(ProtobufUtil.toProtoTableName(table));
1271          }
1272        }
1273      }
1274      return builder.build();
1275    } catch (IOException e) {
1276      throw new ServiceException(e);
1277    }
1278  }
1279
1280  @Override
1281  public GetTableStateResponse getTableState(RpcController controller, GetTableStateRequest request)
1282    throws ServiceException {
1283    try {
1284      server.checkServiceStarted();
1285      TableName tableName = ProtobufUtil.toTableName(request.getTableName());
1286      TableState ts = server.getTableStateManager().getTableState(tableName);
1287      GetTableStateResponse.Builder builder = GetTableStateResponse.newBuilder();
1288      builder.setTableState(ts.convert());
1289      return builder.build();
1290    } catch (IOException e) {
1291      throw new ServiceException(e);
1292    }
1293  }
1294
1295  @Override
1296  public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c,
1297    IsCatalogJanitorEnabledRequest req) throws ServiceException {
1298    return IsCatalogJanitorEnabledResponse.newBuilder().setValue(server.isCatalogJanitorEnabled())
1299      .build();
1300  }
1301
1302  @Override
1303  public IsCleanerChoreEnabledResponse isCleanerChoreEnabled(RpcController c,
1304    IsCleanerChoreEnabledRequest req) throws ServiceException {
1305    return IsCleanerChoreEnabledResponse.newBuilder().setValue(server.isCleanerChoreEnabled())
1306      .build();
1307  }
1308
1309  @Override
1310  public IsMasterRunningResponse isMasterRunning(RpcController c, IsMasterRunningRequest req)
1311    throws ServiceException {
1312    try {
1313      server.checkServiceStarted();
1314      return IsMasterRunningResponse.newBuilder().setIsMasterRunning(!server.isStopped()).build();
1315    } catch (IOException e) {
1316      throw new ServiceException(e);
1317    }
1318  }
1319
1320  /**
1321   * Checks if the specified procedure is done.
1322   * @return true if the procedure is done, false if the procedure is in the process of completing
1323   * @throws ServiceException if invalid procedure or failed procedure with progress failure reason.
1324   */
1325  @Override
1326  public IsProcedureDoneResponse isProcedureDone(RpcController controller,
1327    IsProcedureDoneRequest request) throws ServiceException {
1328    try {
1329      server.checkInitialized();
1330      ProcedureDescription desc = request.getProcedure();
1331      MasterProcedureManager mpm =
1332        server.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature());
1333      if (mpm == null) {
1334        throw new ServiceException("The procedure is not registered: " + desc.getSignature());
1335      }
1336      LOG.debug("Checking to see if procedure from request:" + desc.getSignature() + " is done");
1337
1338      IsProcedureDoneResponse.Builder builder = IsProcedureDoneResponse.newBuilder();
1339      boolean done = mpm.isProcedureDone(desc);
1340      builder.setDone(done);
1341      return builder.build();
1342    } catch (ForeignException e) {
1343      throw new ServiceException(e.getCause());
1344    } catch (IOException e) {
1345      throw new ServiceException(e);
1346    }
1347  }
1348
1349  /**
1350   * Checks if the specified snapshot is done.
1351   * @return true if the snapshot is in file system ready to use, false if the snapshot is in the
1352   *         process of completing
1353   * @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or a wrapped
1354   *                          HBaseSnapshotException with progress failure reason.
1355   */
1356  @Override
1357  public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
1358    IsSnapshotDoneRequest request) throws ServiceException {
1359    LOG.debug("Checking to see if snapshot from request:"
1360      + ClientSnapshotDescriptionUtils.toString(request.getSnapshot()) + " is done");
1361    try {
1362      server.checkInitialized();
1363      IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
1364      boolean done = server.snapshotManager.isSnapshotDone(request.getSnapshot());
1365      builder.setDone(done);
1366      return builder.build();
1367    } catch (ForeignException e) {
1368      throw new ServiceException(e.getCause());
1369    } catch (IOException e) {
1370      throw new ServiceException(e);
1371    }
1372  }
1373
1374  @Override
1375  public GetProcedureResultResponse getProcedureResult(RpcController controller,
1376    GetProcedureResultRequest request) throws ServiceException {
1377    LOG.debug("Checking to see if procedure is done pid={}", request.getProcId());
1378    try {
1379      server.checkInitialized();
1380      GetProcedureResultResponse.Builder builder = GetProcedureResultResponse.newBuilder();
1381      long procId = request.getProcId();
1382      ProcedureExecutor<?> executor = server.getMasterProcedureExecutor();
1383      Procedure<?> result = executor.getResultOrProcedure(procId);
1384      if (result != null) {
1385        builder.setSubmittedTime(result.getSubmittedTime());
1386        builder.setLastUpdate(result.getLastUpdate());
1387        if (executor.isFinished(procId)) {
1388          builder.setState(GetProcedureResultResponse.State.FINISHED);
1389          if (result.isFailed()) {
1390            IOException exception = MasterProcedureUtil.unwrapRemoteIOException(result);
1391            builder.setException(ForeignExceptionUtil.toProtoForeignException(exception));
1392          }
1393          byte[] resultData = result.getResult();
1394          if (resultData != null) {
1395            builder.setResult(UnsafeByteOperations.unsafeWrap(resultData));
1396          }
1397          server.getMasterProcedureExecutor().removeResult(request.getProcId());
1398        } else {
1399          builder.setState(GetProcedureResultResponse.State.RUNNING);
1400        }
1401      } else {
1402        builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
1403      }
1404      return builder.build();
1405    } catch (IOException e) {
1406      throw new ServiceException(e);
1407    }
1408  }
1409
1410  @Override
1411  public AbortProcedureResponse abortProcedure(RpcController rpcController,
1412    AbortProcedureRequest request) throws ServiceException {
1413    try {
1414      AbortProcedureResponse.Builder response = AbortProcedureResponse.newBuilder();
1415      boolean abortResult =
1416        server.abortProcedure(request.getProcId(), request.getMayInterruptIfRunning());
1417      response.setIsProcedureAborted(abortResult);
1418      return response.build();
1419    } catch (IOException e) {
1420      throw new ServiceException(e);
1421    }
1422  }
1423
1424  @Override
1425  public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController c,
1426    ListNamespaceDescriptorsRequest request) throws ServiceException {
1427    try {
1428      ListNamespaceDescriptorsResponse.Builder response =
1429        ListNamespaceDescriptorsResponse.newBuilder();
1430      for (NamespaceDescriptor ns : server.getNamespaces()) {
1431        response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
1432      }
1433      return response.build();
1434    } catch (IOException e) {
1435      throw new ServiceException(e);
1436    }
1437  }
1438
1439  @Override
1440  public GetProceduresResponse getProcedures(RpcController rpcController,
1441    GetProceduresRequest request) throws ServiceException {
1442    try {
1443      final GetProceduresResponse.Builder response = GetProceduresResponse.newBuilder();
1444      for (Procedure<?> p : server.getProcedures()) {
1445        response.addProcedure(ProcedureUtil.convertToProtoProcedure(p));
1446      }
1447      return response.build();
1448    } catch (IOException e) {
1449      throw new ServiceException(e);
1450    }
1451  }
1452
1453  @Override
1454  public GetLocksResponse getLocks(RpcController controller, GetLocksRequest request)
1455    throws ServiceException {
1456    try {
1457      final GetLocksResponse.Builder builder = GetLocksResponse.newBuilder();
1458
1459      for (LockedResource lockedResource : server.getLocks()) {
1460        builder.addLock(ProcedureUtil.convertToProtoLockedResource(lockedResource));
1461      }
1462
1463      return builder.build();
1464    } catch (IOException e) {
1465      throw new ServiceException(e);
1466    }
1467  }
1468
1469  @Override
1470  public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(RpcController c,
1471    ListTableDescriptorsByNamespaceRequest request) throws ServiceException {
1472    try {
1473      ListTableDescriptorsByNamespaceResponse.Builder b =
1474        ListTableDescriptorsByNamespaceResponse.newBuilder();
1475      for (TableDescriptor htd : server
1476        .listTableDescriptorsByNamespace(request.getNamespaceName())) {
1477        b.addTableSchema(ProtobufUtil.toTableSchema(htd));
1478      }
1479      return b.build();
1480    } catch (IOException e) {
1481      throw new ServiceException(e);
1482    }
1483  }
1484
1485  @Override
1486  public ListTableNamesByNamespaceResponse listTableNamesByNamespace(RpcController c,
1487    ListTableNamesByNamespaceRequest request) throws ServiceException {
1488    try {
1489      ListTableNamesByNamespaceResponse.Builder b = ListTableNamesByNamespaceResponse.newBuilder();
1490      for (TableName tableName : server.listTableNamesByNamespace(request.getNamespaceName())) {
1491        b.addTableName(ProtobufUtil.toProtoTableName(tableName));
1492      }
1493      return b.build();
1494    } catch (IOException e) {
1495      throw new ServiceException(e);
1496    }
1497  }
1498
1499  @Override
1500  public ModifyColumnResponse modifyColumn(RpcController controller, ModifyColumnRequest req)
1501    throws ServiceException {
1502    try {
1503      long procId = server.modifyColumn(ProtobufUtil.toTableName(req.getTableName()),
1504        ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()), req.getNonceGroup(),
1505        req.getNonce());
1506      if (procId == -1) {
1507        // This mean operation was not performed in server, so do not set any procId
1508        return ModifyColumnResponse.newBuilder().build();
1509      } else {
1510        return ModifyColumnResponse.newBuilder().setProcId(procId).build();
1511      }
1512    } catch (IOException ioe) {
1513      throw new ServiceException(ioe);
1514    }
1515  }
1516
1517  @Override
1518  public ModifyColumnStoreFileTrackerResponse modifyColumnStoreFileTracker(RpcController controller,
1519    ModifyColumnStoreFileTrackerRequest req) throws ServiceException {
1520    try {
1521      long procId =
1522        server.modifyColumnStoreFileTracker(ProtobufUtil.toTableName(req.getTableName()),
1523          req.getFamily().toByteArray(), req.getDstSft(), req.getNonceGroup(), req.getNonce());
1524      return ModifyColumnStoreFileTrackerResponse.newBuilder().setProcId(procId).build();
1525    } catch (IOException ioe) {
1526      throw new ServiceException(ioe);
1527    }
1528  }
1529
1530  @Override
1531  public ModifyNamespaceResponse modifyNamespace(RpcController controller,
1532    ModifyNamespaceRequest request) throws ServiceException {
1533    try {
1534      long procId =
1535        server.modifyNamespace(ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
1536          request.getNonceGroup(), request.getNonce());
1537      return ModifyNamespaceResponse.newBuilder().setProcId(procId).build();
1538    } catch (IOException e) {
1539      throw new ServiceException(e);
1540    }
1541  }
1542
1543  @Override
1544  public ModifyTableResponse modifyTable(RpcController controller, ModifyTableRequest req)
1545    throws ServiceException {
1546    try {
1547      long procId = server.modifyTable(ProtobufUtil.toTableName(req.getTableName()),
1548        ProtobufUtil.toTableDescriptor(req.getTableSchema()), req.getNonceGroup(), req.getNonce(),
1549        req.getReopenRegions());
1550      return ModifyTableResponse.newBuilder().setProcId(procId).build();
1551    } catch (IOException ioe) {
1552      throw new ServiceException(ioe);
1553    }
1554  }
1555
1556  @Override
1557  public ModifyTableStoreFileTrackerResponse modifyTableStoreFileTracker(RpcController controller,
1558    ModifyTableStoreFileTrackerRequest req) throws ServiceException {
1559    try {
1560      long procId = server.modifyTableStoreFileTracker(ProtobufUtil.toTableName(req.getTableName()),
1561        req.getDstSft(), req.getNonceGroup(), req.getNonce());
1562      return ModifyTableStoreFileTrackerResponse.newBuilder().setProcId(procId).build();
1563    } catch (IOException ioe) {
1564      throw new ServiceException(ioe);
1565    }
1566  }
1567
1568  @Override
1569  public MoveRegionResponse moveRegion(RpcController controller, MoveRegionRequest req)
1570    throws ServiceException {
1571    final byte[] encodedRegionName = req.getRegion().getValue().toByteArray();
1572    RegionSpecifierType type = req.getRegion().getType();
1573    final byte[] destServerName = (req.hasDestServerName())
1574      ? Bytes.toBytes(ProtobufUtil.toServerName(req.getDestServerName()).getServerName())
1575      : null;
1576    MoveRegionResponse mrr = MoveRegionResponse.newBuilder().build();
1577
1578    if (type != RegionSpecifierType.ENCODED_REGION_NAME) {
1579      LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME
1580        + " actual: " + type);
1581    }
1582
1583    try {
1584      server.checkInitialized();
1585      server.move(encodedRegionName, destServerName);
1586    } catch (IOException ioe) {
1587      throw new ServiceException(ioe);
1588    }
1589    return mrr;
1590  }
1591
1592  /**
1593   * Offline specified region from master's in-memory state. It will not attempt to reassign the
1594   * region as in unassign. This is a special method that should be used by experts or hbck.
1595   */
1596  @Override
1597  public OfflineRegionResponse offlineRegion(RpcController controller, OfflineRegionRequest request)
1598    throws ServiceException {
1599    try {
1600      server.checkInitialized();
1601
1602      final RegionSpecifierType type = request.getRegion().getType();
1603      if (type != RegionSpecifierType.REGION_NAME) {
1604        LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
1605          + " actual: " + type);
1606      }
1607
1608      final byte[] regionName = request.getRegion().getValue().toByteArray();
1609      final RegionInfo hri = server.getAssignmentManager().getRegionInfo(regionName);
1610      if (hri == null) {
1611        throw new UnknownRegionException(Bytes.toStringBinary(regionName));
1612      }
1613
1614      if (server.cpHost != null) {
1615        server.cpHost.preRegionOffline(hri);
1616      }
1617      LOG.info(server.getClientIdAuditPrefix() + " offline " + hri.getRegionNameAsString());
1618      server.getAssignmentManager().offlineRegion(hri);
1619      if (server.cpHost != null) {
1620        server.cpHost.postRegionOffline(hri);
1621      }
1622    } catch (IOException ioe) {
1623      throw new ServiceException(ioe);
1624    }
1625    return OfflineRegionResponse.newBuilder().build();
1626  }
1627
1628  /**
1629   * Execute Restore/Clone snapshot operation.
1630   * <p>
1631   * If the specified table exists a "Restore" is executed, replacing the table schema and directory
1632   * data with the content of the snapshot. The table must be disabled, or a
1633   * UnsupportedOperationException will be thrown.
1634   * <p>
1635   * If the table doesn't exist a "Clone" is executed, a new table is created using the schema at
1636   * the time of the snapshot, and the content of the snapshot.
1637   * <p>
1638   * The restore/clone operation does not require copying HFiles. Since HFiles are immutable the
1639   * table can point to and use the same files as the original one.
1640   */
1641  @Override
1642  public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
1643    RestoreSnapshotRequest request) throws ServiceException {
1644    try {
1645      long procId = server.restoreSnapshot(request.getSnapshot(), request.getNonceGroup(),
1646        request.getNonce(), request.getRestoreACL(), request.getCustomSFT());
1647      return RestoreSnapshotResponse.newBuilder().setProcId(procId).build();
1648    } catch (ForeignException e) {
1649      throw new ServiceException(e.getCause());
1650    } catch (IOException e) {
1651      throw new ServiceException(e);
1652    }
1653  }
1654
1655  @Override
1656  public SetSnapshotCleanupResponse switchSnapshotCleanup(RpcController controller,
1657    SetSnapshotCleanupRequest request) throws ServiceException {
1658    try {
1659      server.checkInitialized();
1660      final boolean enabled = request.getEnabled();
1661      final boolean isSynchronous = request.hasSynchronous() && request.getSynchronous();
1662      final boolean prevSnapshotCleanupRunning = this.switchSnapshotCleanup(enabled, isSynchronous);
1663      return SetSnapshotCleanupResponse.newBuilder()
1664        .setPrevSnapshotCleanup(prevSnapshotCleanupRunning).build();
1665    } catch (IOException e) {
1666      throw new ServiceException(e);
1667    }
1668  }
1669
1670  @Override
1671  public IsSnapshotCleanupEnabledResponse isSnapshotCleanupEnabled(RpcController controller,
1672    IsSnapshotCleanupEnabledRequest request) throws ServiceException {
1673    try {
1674      server.checkInitialized();
1675      final boolean isSnapshotCleanupEnabled = server.snapshotCleanupStateStore.get();
1676      return IsSnapshotCleanupEnabledResponse.newBuilder().setEnabled(isSnapshotCleanupEnabled)
1677        .build();
1678    } catch (IOException e) {
1679      throw new ServiceException(e);
1680    }
1681  }
1682
1683  /**
1684   * Turn on/off snapshot auto-cleanup based on TTL
1685   * @param enabledNewVal Set to <code>true</code> to enable, <code>false</code> to disable
1686   * @param synchronous   If <code>true</code>, it waits until current snapshot cleanup is
1687   *                      completed, if outstanding
1688   * @return previous snapshot auto-cleanup mode
1689   */
1690  private synchronized boolean switchSnapshotCleanup(final boolean enabledNewVal,
1691    final boolean synchronous) throws IOException {
1692    final boolean oldValue = server.snapshotCleanupStateStore.get();
1693    server.switchSnapshotCleanup(enabledNewVal, synchronous);
1694    LOG.info("{} Successfully set snapshot cleanup to {}", server.getClientIdAuditPrefix(),
1695      enabledNewVal);
1696    return oldValue;
1697  }
1698
1699  @Override
1700  public RunCatalogScanResponse runCatalogScan(RpcController c, RunCatalogScanRequest req)
1701    throws ServiceException {
1702    rpcPreCheck("runCatalogScan");
1703    try {
1704      return ResponseConverter.buildRunCatalogScanResponse(this.server.catalogJanitorChore.scan());
1705    } catch (IOException ioe) {
1706      throw new ServiceException(ioe);
1707    }
1708  }
1709
1710  @Override
1711  public RunCleanerChoreResponse runCleanerChore(RpcController c, RunCleanerChoreRequest req)
1712    throws ServiceException {
1713    rpcPreCheck("runCleanerChore");
1714    try {
1715      CompletableFuture<Boolean> fileCleanerFuture = server.getHFileCleaner().triggerCleanerNow();
1716      CompletableFuture<Boolean> logCleanerFuture = server.getLogCleaner().triggerCleanerNow();
1717      boolean result = fileCleanerFuture.get() && logCleanerFuture.get();
1718      return ResponseConverter.buildRunCleanerChoreResponse(result);
1719    } catch (InterruptedException e) {
1720      throw new ServiceException(e);
1721    } catch (ExecutionException e) {
1722      throw new ServiceException(e.getCause());
1723    }
1724  }
1725
1726  @Override
1727  public SetBalancerRunningResponse setBalancerRunning(RpcController c,
1728    SetBalancerRunningRequest req) throws ServiceException {
1729    try {
1730      server.checkInitialized();
1731      boolean prevValue = (req.getSynchronous())
1732        ? synchronousBalanceSwitch(req.getOn())
1733        : server.balanceSwitch(req.getOn());
1734      return SetBalancerRunningResponse.newBuilder().setPrevBalanceValue(prevValue).build();
1735    } catch (IOException ioe) {
1736      throw new ServiceException(ioe);
1737    }
1738  }
1739
1740  @Override
1741  public ShutdownResponse shutdown(RpcController controller, ShutdownRequest request)
1742    throws ServiceException {
1743    LOG.info(server.getClientIdAuditPrefix() + " shutdown");
1744    try {
1745      server.shutdown();
1746    } catch (IOException e) {
1747      LOG.error("Exception occurred in HMaster.shutdown()", e);
1748      throw new ServiceException(e);
1749    }
1750    return ShutdownResponse.newBuilder().build();
1751  }
1752
1753  /**
1754   * Triggers an asynchronous attempt to take a snapshot. {@inheritDoc}
1755   */
1756  @Override
1757  public SnapshotResponse snapshot(RpcController controller, SnapshotRequest request)
1758    throws ServiceException {
1759    try {
1760      server.checkInitialized();
1761      server.snapshotManager.checkSnapshotSupport();
1762
1763      LOG.info(server.getClientIdAuditPrefix() + " snapshot request for:"
1764        + ClientSnapshotDescriptionUtils.toString(request.getSnapshot()));
1765      // get the snapshot information
1766      SnapshotDescription snapshot =
1767        SnapshotDescriptionUtils.validate(request.getSnapshot(), server.getConfiguration());
1768      // send back the max amount of time the client should wait for the snapshot to complete
1769      long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(server.getConfiguration(),
1770        snapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
1771
1772      SnapshotResponse.Builder builder = SnapshotResponse.newBuilder().setExpectedTimeout(waitTime);
1773
1774      // If there is nonce group and nonce in the snapshot request, then the client can
1775      // handle snapshot procedure procId. And if enable the snapshot procedure, we
1776      // will do the snapshot work with proc-v2, otherwise we will fall back to zk proc.
1777      if (
1778        request.hasNonceGroup() && request.hasNonce()
1779          && server.snapshotManager.snapshotProcedureEnabled()
1780      ) {
1781        long nonceGroup = request.getNonceGroup();
1782        long nonce = request.getNonce();
1783        long procId = server.snapshotManager.takeSnapshot(snapshot, nonceGroup, nonce);
1784        return builder.setProcId(procId).build();
1785      } else {
1786        server.snapshotManager.takeSnapshot(snapshot);
1787        return builder.build();
1788      }
1789    } catch (ForeignException e) {
1790      throw new ServiceException(e.getCause());
1791    } catch (IOException e) {
1792      throw new ServiceException(e);
1793    }
1794  }
1795
1796  @Override
1797  public StopMasterResponse stopMaster(RpcController controller, StopMasterRequest request)
1798    throws ServiceException {
1799    LOG.info(server.getClientIdAuditPrefix() + " stop");
1800    try {
1801      server.stopMaster();
1802    } catch (IOException e) {
1803      LOG.error("Exception occurred while stopping master", e);
1804      throw new ServiceException(e);
1805    }
1806    return StopMasterResponse.newBuilder().build();
1807  }
1808
1809  @Override
1810  public IsInMaintenanceModeResponse isMasterInMaintenanceMode(final RpcController controller,
1811    final IsInMaintenanceModeRequest request) throws ServiceException {
1812    IsInMaintenanceModeResponse.Builder response = IsInMaintenanceModeResponse.newBuilder();
1813    response.setInMaintenanceMode(server.isInMaintenanceMode());
1814    return response.build();
1815  }
1816
1817  @Override
1818  public UnassignRegionResponse unassignRegion(RpcController controller, UnassignRegionRequest req)
1819    throws ServiceException {
1820    try {
1821      final byte[] regionName = req.getRegion().getValue().toByteArray();
1822      RegionSpecifierType type = req.getRegion().getType();
1823      UnassignRegionResponse urr = UnassignRegionResponse.newBuilder().build();
1824
1825      server.checkInitialized();
1826      if (type != RegionSpecifierType.REGION_NAME) {
1827        LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
1828          + " actual: " + type);
1829      }
1830      RegionStateNode rsn =
1831        server.getAssignmentManager().getRegionStates().getRegionStateNodeFromName(regionName);
1832      if (rsn == null) {
1833        throw new UnknownRegionException(Bytes.toString(regionName));
1834      }
1835
1836      RegionInfo hri = rsn.getRegionInfo();
1837      if (server.cpHost != null) {
1838        server.cpHost.preUnassign(hri);
1839      }
1840      LOG.debug(server.getClientIdAuditPrefix() + " unassign " + hri.getRegionNameAsString()
1841        + " in current location if it is online");
1842      server.getAssignmentManager().unassign(hri);
1843      if (server.cpHost != null) {
1844        server.cpHost.postUnassign(hri);
1845      }
1846
1847      return urr;
1848    } catch (IOException ioe) {
1849      throw new ServiceException(ioe);
1850    }
1851  }
1852
1853  @Override
1854  public ReportRegionStateTransitionResponse reportRegionStateTransition(RpcController c,
1855    ReportRegionStateTransitionRequest req) throws ServiceException {
1856    try {
1857      server.checkServiceStarted();
1858      for (RegionServerStatusProtos.RegionStateTransition transition : req.getTransitionList()) {
1859        long procId =
1860          transition.getProcIdCount() > 0 ? transition.getProcId(0) : Procedure.NO_PROC_ID;
1861        // -1 is less than any possible MasterActiveCode
1862        long initiatingMasterActiveTime = transition.hasInitiatingMasterActiveTime()
1863          ? transition.getInitiatingMasterActiveTime()
1864          : -1;
1865        throwOnOldMaster(procId, initiatingMasterActiveTime);
1866      }
1867      return server.getAssignmentManager().reportRegionStateTransition(req);
1868    } catch (IOException ioe) {
1869      throw new ServiceException(ioe);
1870    }
1871  }
1872
1873  @Override
1874  public SetQuotaResponse setQuota(RpcController c, SetQuotaRequest req) throws ServiceException {
1875    try {
1876      server.checkInitialized();
1877      SetQuotaResponse response = server.getMasterQuotaManager().setQuota(req);
1878      server.reloadRegionServerQuotas();
1879
1880      return response;
1881    } catch (Exception e) {
1882      throw new ServiceException(e);
1883    }
1884  }
1885
1886  @Override
1887  public MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(RpcController controller,
1888    MajorCompactionTimestampRequest request) throws ServiceException {
1889    MajorCompactionTimestampResponse.Builder response =
1890      MajorCompactionTimestampResponse.newBuilder();
1891    try {
1892      server.checkInitialized();
1893      response.setCompactionTimestamp(
1894        server.getLastMajorCompactionTimestamp(ProtobufUtil.toTableName(request.getTableName())));
1895    } catch (IOException e) {
1896      throw new ServiceException(e);
1897    }
1898    return response.build();
1899  }
1900
1901  @Override
1902  public MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion(
1903    RpcController controller, MajorCompactionTimestampForRegionRequest request)
1904    throws ServiceException {
1905    MajorCompactionTimestampResponse.Builder response =
1906      MajorCompactionTimestampResponse.newBuilder();
1907    try {
1908      server.checkInitialized();
1909      response.setCompactionTimestamp(server
1910        .getLastMajorCompactionTimestampForRegion(request.getRegion().getValue().toByteArray()));
1911    } catch (IOException e) {
1912      throw new ServiceException(e);
1913    }
1914    return response.build();
1915  }
1916
1917  @Override
1918  public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
1919    IsBalancerEnabledRequest request) throws ServiceException {
1920    IsBalancerEnabledResponse.Builder response = IsBalancerEnabledResponse.newBuilder();
1921    response.setEnabled(server.isBalancerOn());
1922    return response.build();
1923  }
1924
1925  @Override
1926  public SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(RpcController controller,
1927    SetSplitOrMergeEnabledRequest request) throws ServiceException {
1928    SetSplitOrMergeEnabledResponse.Builder response = SetSplitOrMergeEnabledResponse.newBuilder();
1929    try {
1930      server.checkInitialized();
1931      boolean newValue = request.getEnabled();
1932      for (MasterProtos.MasterSwitchType masterSwitchType : request.getSwitchTypesList()) {
1933        MasterSwitchType switchType = convert(masterSwitchType);
1934        boolean oldValue = server.isSplitOrMergeEnabled(switchType);
1935        response.addPrevValue(oldValue);
1936        if (server.cpHost != null) {
1937          server.cpHost.preSetSplitOrMergeEnabled(newValue, switchType);
1938        }
1939        server.getSplitOrMergeStateStore().setSplitOrMergeEnabled(newValue, switchType);
1940        if (server.cpHost != null) {
1941          server.cpHost.postSetSplitOrMergeEnabled(newValue, switchType);
1942        }
1943      }
1944    } catch (IOException e) {
1945      throw new ServiceException(e);
1946    }
1947    return response.build();
1948  }
1949
1950  @Override
1951  public IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(RpcController controller,
1952    IsSplitOrMergeEnabledRequest request) throws ServiceException {
1953    IsSplitOrMergeEnabledResponse.Builder response = IsSplitOrMergeEnabledResponse.newBuilder();
1954    response.setEnabled(server.isSplitOrMergeEnabled(convert(request.getSwitchType())));
1955    return response.build();
1956  }
1957
1958  @Override
1959  public NormalizeResponse normalize(RpcController controller, NormalizeRequest request)
1960    throws ServiceException {
1961    rpcPreCheck("normalize");
1962    try {
1963      final NormalizeTableFilterParams ntfp = new NormalizeTableFilterParams.Builder()
1964        .tableNames(ProtobufUtil.toTableNameList(request.getTableNamesList()))
1965        .regex(request.hasRegex() ? request.getRegex() : null)
1966        .namespace(request.hasNamespace() ? request.getNamespace() : null).build();
1967      return NormalizeResponse.newBuilder()
1968        // all API requests are considered priority requests.
1969        .setNormalizerRan(server.normalizeRegions(ntfp, true)).build();
1970    } catch (IOException ex) {
1971      throw new ServiceException(ex);
1972    }
1973  }
1974
1975  @Override
1976  public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller,
1977    SetNormalizerRunningRequest request) throws ServiceException {
1978    rpcPreCheck("setNormalizerRunning");
1979
1980    // Sets normalizer on/off flag in ZK.
1981    // TODO: this method is totally broken in terms of atomicity of actions and values read.
1982    // 1. The contract has this RPC returning the previous value. There isn't a ZKUtil method
1983    // that lets us retrieve the previous value as part of setting a new value, so we simply
1984    // perform a read before issuing the update. Thus we have a data race opportunity, between
1985    // when the `prevValue` is read and whatever is actually overwritten.
1986    // 2. Down in `setNormalizerOn`, the call to `createAndWatch` inside of the catch clause can
1987    // itself fail in the event that the znode already exists. Thus, another data race, between
1988    // when the initial `setData` call is notified of the absence of the target znode and the
1989    // subsequent `createAndWatch`, with another client creating said node.
1990    // That said, there's supposed to be only one active master and thus there's supposed to be
1991    // only one process with the authority to modify the value.
1992    final boolean prevValue = server.getRegionNormalizerManager().isNormalizerOn();
1993    final boolean newValue = request.getOn();
1994    try {
1995      server.getRegionNormalizerManager().setNormalizerOn(newValue);
1996    } catch (IOException e) {
1997      throw new ServiceException(e);
1998    }
1999    LOG.info("{} set normalizerSwitch={}", server.getClientIdAuditPrefix(), newValue);
2000    return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
2001  }
2002
2003  @Override
2004  public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller,
2005    IsNormalizerEnabledRequest request) {
2006    IsNormalizerEnabledResponse.Builder response = IsNormalizerEnabledResponse.newBuilder();
2007    response.setEnabled(server.isNormalizerOn());
2008    return response.build();
2009  }
2010
2011  /**
2012   * Returns the security capabilities in effect on the cluster
2013   */
2014  @Override
2015  public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller,
2016    SecurityCapabilitiesRequest request) throws ServiceException {
2017    SecurityCapabilitiesResponse.Builder response = SecurityCapabilitiesResponse.newBuilder();
2018    try {
2019      server.checkInitialized();
2020      Set<SecurityCapabilitiesResponse.Capability> capabilities = new HashSet<>();
2021      // Authentication
2022      if (User.isHBaseSecurityEnabled(server.getConfiguration())) {
2023        capabilities.add(SecurityCapabilitiesResponse.Capability.SECURE_AUTHENTICATION);
2024      } else {
2025        capabilities.add(SecurityCapabilitiesResponse.Capability.SIMPLE_AUTHENTICATION);
2026      }
2027      // A coprocessor that implements AccessControlService can provide AUTHORIZATION and
2028      // CELL_AUTHORIZATION
2029      if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
2030        if (AccessChecker.isAuthorizationSupported(server.getConfiguration())) {
2031          capabilities.add(SecurityCapabilitiesResponse.Capability.AUTHORIZATION);
2032        }
2033        if (AccessController.isCellAuthorizationSupported(server.getConfiguration())) {
2034          capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_AUTHORIZATION);
2035        }
2036      }
2037      // A coprocessor that implements VisibilityLabelsService can provide CELL_VISIBILITY.
2038      if (server.cpHost != null && hasVisibilityLabelsServiceCoprocessor(server.cpHost)) {
2039        if (VisibilityController.isCellAuthorizationSupported(server.getConfiguration())) {
2040          capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_VISIBILITY);
2041        }
2042      }
2043      response.addAllCapabilities(capabilities);
2044    } catch (IOException e) {
2045      throw new ServiceException(e);
2046    }
2047    return response.build();
2048  }
2049
2050  /**
2051   * Determines if there is a MasterCoprocessor deployed which implements
2052   * {@link AccessControlService.Interface}.
2053   */
2054  boolean hasAccessControlServiceCoprocessor(MasterCoprocessorHost cpHost) {
2055    return checkCoprocessorWithService(cpHost.findCoprocessors(MasterCoprocessor.class),
2056      AccessControlService.Interface.class);
2057  }
2058
2059  /**
2060   * Determines if there is a MasterCoprocessor deployed which implements
2061   * {@link VisibilityLabelsService.Interface}.
2062   */
2063  boolean hasVisibilityLabelsServiceCoprocessor(MasterCoprocessorHost cpHost) {
2064    return checkCoprocessorWithService(cpHost.findCoprocessors(MasterCoprocessor.class),
2065      VisibilityLabelsService.Interface.class);
2066  }
2067
2068  /**
2069   * Determines if there is a coprocessor implementation in the provided argument which extends or
2070   * implements the provided {@code service}.
2071   */
2072  boolean checkCoprocessorWithService(List<MasterCoprocessor> coprocessorsToCheck,
2073    Class<?> service) {
2074    if (coprocessorsToCheck == null || coprocessorsToCheck.isEmpty()) {
2075      return false;
2076    }
2077    for (MasterCoprocessor cp : coprocessorsToCheck) {
2078      if (service.isAssignableFrom(cp.getClass())) {
2079        return true;
2080      }
2081    }
2082    return false;
2083  }
2084
2085  private MasterSwitchType convert(MasterProtos.MasterSwitchType switchType) {
2086    switch (switchType) {
2087      case SPLIT:
2088        return MasterSwitchType.SPLIT;
2089      case MERGE:
2090        return MasterSwitchType.MERGE;
2091      default:
2092        break;
2093    }
2094    return null;
2095  }
2096
2097  @Override
2098  public AddReplicationPeerResponse addReplicationPeer(RpcController controller,
2099    AddReplicationPeerRequest request) throws ServiceException {
2100    try {
2101      long procId = server.addReplicationPeer(request.getPeerId(),
2102        ReplicationPeerConfigUtil.convert(request.getPeerConfig()),
2103        request.getPeerState().getState().equals(ReplicationState.State.ENABLED));
2104      return AddReplicationPeerResponse.newBuilder().setProcId(procId).build();
2105    } catch (ReplicationException | IOException e) {
2106      throw new ServiceException(e);
2107    }
2108  }
2109
2110  @Override
2111  public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller,
2112    RemoveReplicationPeerRequest request) throws ServiceException {
2113    try {
2114      long procId = server.removeReplicationPeer(request.getPeerId());
2115      return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build();
2116    } catch (ReplicationException | IOException e) {
2117      throw new ServiceException(e);
2118    }
2119  }
2120
2121  @Override
2122  public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller,
2123    EnableReplicationPeerRequest request) throws ServiceException {
2124    try {
2125      long procId = server.enableReplicationPeer(request.getPeerId());
2126      return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build();
2127    } catch (ReplicationException | IOException e) {
2128      throw new ServiceException(e);
2129    }
2130  }
2131
2132  @Override
2133  public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller,
2134    DisableReplicationPeerRequest request) throws ServiceException {
2135    try {
2136      long procId = server.disableReplicationPeer(request.getPeerId());
2137      return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build();
2138    } catch (ReplicationException | IOException e) {
2139      throw new ServiceException(e);
2140    }
2141  }
2142
2143  @Override
2144  public GetReplicationPeerConfigResponse getReplicationPeerConfig(RpcController controller,
2145    GetReplicationPeerConfigRequest request) throws ServiceException {
2146    GetReplicationPeerConfigResponse.Builder response =
2147      GetReplicationPeerConfigResponse.newBuilder();
2148    try {
2149      String peerId = request.getPeerId();
2150      ReplicationPeerConfig peerConfig = server.getReplicationPeerConfig(peerId);
2151      response.setPeerId(peerId);
2152      response.setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig));
2153    } catch (ReplicationException | IOException e) {
2154      throw new ServiceException(e);
2155    }
2156    return response.build();
2157  }
2158
2159  @Override
2160  public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller,
2161    UpdateReplicationPeerConfigRequest request) throws ServiceException {
2162    try {
2163      long procId = server.updateReplicationPeerConfig(request.getPeerId(),
2164        ReplicationPeerConfigUtil.convert(request.getPeerConfig()));
2165      return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build();
2166    } catch (ReplicationException | IOException e) {
2167      throw new ServiceException(e);
2168    }
2169  }
2170
2171  @Override
2172  public TransitReplicationPeerSyncReplicationStateResponse
2173    transitReplicationPeerSyncReplicationState(RpcController controller,
2174      TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
2175    try {
2176      long procId = server.transitReplicationPeerSyncReplicationState(request.getPeerId(),
2177        ReplicationPeerConfigUtil.toSyncReplicationState(request.getSyncReplicationState()));
2178      return TransitReplicationPeerSyncReplicationStateResponse.newBuilder().setProcId(procId)
2179        .build();
2180    } catch (ReplicationException | IOException e) {
2181      throw new ServiceException(e);
2182    }
2183  }
2184
2185  @Override
2186  public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
2187    ListReplicationPeersRequest request) throws ServiceException {
2188    ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder();
2189    try {
2190      List<ReplicationPeerDescription> peers =
2191        server.listReplicationPeers(request.hasRegex() ? request.getRegex() : null);
2192      for (ReplicationPeerDescription peer : peers) {
2193        response.addPeerDesc(ReplicationPeerConfigUtil.toProtoReplicationPeerDescription(peer));
2194      }
2195    } catch (ReplicationException | IOException e) {
2196      throw new ServiceException(e);
2197    }
2198    return response.build();
2199  }
2200
2201  @Override
2202  public GetReplicationPeerStateResponse isReplicationPeerEnabled(RpcController controller,
2203    GetReplicationPeerStateRequest request) throws ServiceException {
2204    boolean isEnabled;
2205    try {
2206      isEnabled = server.getReplicationPeerManager().getPeerState(request.getPeerId());
2207    } catch (ReplicationException ioe) {
2208      throw new ServiceException(ioe);
2209    }
2210    return GetReplicationPeerStateResponse.newBuilder().setIsEnabled(isEnabled).build();
2211  }
2212
2213  @Override
2214  public ReplicationPeerModificationSwitchResponse replicationPeerModificationSwitch(
2215    RpcController controller, ReplicationPeerModificationSwitchRequest request)
2216    throws ServiceException {
2217    try {
2218      server.checkInitialized();
2219      boolean prevValue = server.replicationPeerModificationSwitch(request.getOn());
2220      return ReplicationPeerModificationSwitchResponse.newBuilder().setPreviousValue(prevValue)
2221        .build();
2222    } catch (IOException ioe) {
2223      throw new ServiceException(ioe);
2224    }
2225  }
2226
2227  @Override
2228  public GetReplicationPeerModificationProceduresResponse getReplicationPeerModificationProcedures(
2229    RpcController controller, GetReplicationPeerModificationProceduresRequest request)
2230    throws ServiceException {
2231    try {
2232      server.checkInitialized();
2233      GetReplicationPeerModificationProceduresResponse.Builder builder =
2234        GetReplicationPeerModificationProceduresResponse.newBuilder();
2235      for (Procedure<?> proc : server.getProcedures()) {
2236        if (proc.isFinished()) {
2237          continue;
2238        }
2239        if (!(proc instanceof AbstractPeerNoLockProcedure)) {
2240          continue;
2241        }
2242        builder.addProcedure(ProcedureUtil.convertToProtoProcedure(proc));
2243      }
2244      return builder.build();
2245    } catch (IOException ioe) {
2246      throw new ServiceException(ioe);
2247    }
2248  }
2249
2250  @Override
2251  public IsReplicationPeerModificationEnabledResponse isReplicationPeerModificationEnabled(
2252    RpcController controller, IsReplicationPeerModificationEnabledRequest request)
2253    throws ServiceException {
2254    try {
2255      server.checkInitialized();
2256      return IsReplicationPeerModificationEnabledResponse.newBuilder()
2257        .setEnabled(server.isReplicationPeerModificationEnabled()).build();
2258    } catch (IOException ioe) {
2259      throw new ServiceException(ioe);
2260    }
2261  }
2262
2263  @Override
2264  public ListDecommissionedRegionServersResponse listDecommissionedRegionServers(
2265    RpcController controller, ListDecommissionedRegionServersRequest request)
2266    throws ServiceException {
2267    ListDecommissionedRegionServersResponse.Builder response =
2268      ListDecommissionedRegionServersResponse.newBuilder();
2269    try {
2270      server.checkInitialized();
2271      if (server.cpHost != null) {
2272        server.cpHost.preListDecommissionedRegionServers();
2273      }
2274      List<ServerName> servers = server.listDecommissionedRegionServers();
2275      response.addAllServerName((servers.stream().map(server -> ProtobufUtil.toServerName(server)))
2276        .collect(Collectors.toList()));
2277      if (server.cpHost != null) {
2278        server.cpHost.postListDecommissionedRegionServers();
2279      }
2280    } catch (IOException io) {
2281      throw new ServiceException(io);
2282    }
2283
2284    return response.build();
2285  }
2286
2287  @Override
2288  public DecommissionRegionServersResponse decommissionRegionServers(RpcController controller,
2289    DecommissionRegionServersRequest request) throws ServiceException {
2290    try {
2291      server.checkInitialized();
2292      List<ServerName> servers = request.getServerNameList().stream()
2293        .map(pbServer -> ProtobufUtil.toServerName(pbServer)).collect(Collectors.toList());
2294      boolean offload = request.getOffload();
2295      if (server.cpHost != null) {
2296        server.cpHost.preDecommissionRegionServers(servers, offload);
2297      }
2298      server.decommissionRegionServers(servers, offload);
2299      if (server.cpHost != null) {
2300        server.cpHost.postDecommissionRegionServers(servers, offload);
2301      }
2302    } catch (IOException io) {
2303      throw new ServiceException(io);
2304    }
2305
2306    return DecommissionRegionServersResponse.newBuilder().build();
2307  }
2308
2309  @Override
2310  public RecommissionRegionServerResponse recommissionRegionServer(RpcController controller,
2311    RecommissionRegionServerRequest request) throws ServiceException {
2312    try {
2313      server.checkInitialized();
2314      ServerName sn = ProtobufUtil.toServerName(request.getServerName());
2315      List<byte[]> encodedRegionNames = request.getRegionList().stream()
2316        .map(regionSpecifier -> regionSpecifier.getValue().toByteArray())
2317        .collect(Collectors.toList());
2318      if (server.cpHost != null) {
2319        server.cpHost.preRecommissionRegionServer(sn, encodedRegionNames);
2320      }
2321      server.recommissionRegionServer(sn, encodedRegionNames);
2322      if (server.cpHost != null) {
2323        server.cpHost.postRecommissionRegionServer(sn, encodedRegionNames);
2324      }
2325    } catch (IOException io) {
2326      throw new ServiceException(io);
2327    }
2328
2329    return RecommissionRegionServerResponse.newBuilder().build();
2330  }
2331
2332  @Override
2333  public LockResponse requestLock(RpcController controller, final LockRequest request)
2334    throws ServiceException {
2335    try {
2336      if (request.getDescription().isEmpty()) {
2337        throw new IllegalArgumentException("Empty description");
2338      }
2339      NonceProcedureRunnable npr;
2340      LockType type = LockType.valueOf(request.getLockType().name());
2341      if (request.getRegionInfoCount() > 0) {
2342        final RegionInfo[] regionInfos = new RegionInfo[request.getRegionInfoCount()];
2343        for (int i = 0; i < request.getRegionInfoCount(); ++i) {
2344          regionInfos[i] = ProtobufUtil.toRegionInfo(request.getRegionInfo(i));
2345        }
2346        npr = new NonceProcedureRunnable(server, request.getNonceGroup(), request.getNonce()) {
2347          @Override
2348          protected void run() throws IOException {
2349            setProcId(server.getLockManager().remoteLocks().requestRegionsLock(regionInfos,
2350              request.getDescription(), getNonceKey()));
2351          }
2352
2353          @Override
2354          protected String getDescription() {
2355            return "RequestLock";
2356          }
2357        };
2358      } else if (request.hasTableName()) {
2359        final TableName tableName = ProtobufUtil.toTableName(request.getTableName());
2360        npr = new NonceProcedureRunnable(server, request.getNonceGroup(), request.getNonce()) {
2361          @Override
2362          protected void run() throws IOException {
2363            setProcId(server.getLockManager().remoteLocks().requestTableLock(tableName, type,
2364              request.getDescription(), getNonceKey()));
2365          }
2366
2367          @Override
2368          protected String getDescription() {
2369            return "RequestLock";
2370          }
2371        };
2372      } else if (request.hasNamespace()) {
2373        npr = new NonceProcedureRunnable(server, request.getNonceGroup(), request.getNonce()) {
2374          @Override
2375          protected void run() throws IOException {
2376            setProcId(server.getLockManager().remoteLocks().requestNamespaceLock(
2377              request.getNamespace(), type, request.getDescription(), getNonceKey()));
2378          }
2379
2380          @Override
2381          protected String getDescription() {
2382            return "RequestLock";
2383          }
2384        };
2385      } else {
2386        throw new IllegalArgumentException("one of table/namespace/region should be specified");
2387      }
2388      long procId = MasterProcedureUtil.submitProcedure(npr);
2389      return LockResponse.newBuilder().setProcId(procId).build();
2390    } catch (IllegalArgumentException e) {
2391      LOG.warn("Exception when queuing lock", e);
2392      throw new ServiceException(new DoNotRetryIOException(e));
2393    } catch (IOException e) {
2394      LOG.warn("Exception when queuing lock", e);
2395      throw new ServiceException(e);
2396    }
2397  }
2398
2399  /**
2400   * @return LOCKED, if procedure is found and it has the lock; else UNLOCKED.
2401   * @throws ServiceException if given proc id is found but it is not a LockProcedure.
2402   */
2403  @Override
2404  public LockHeartbeatResponse lockHeartbeat(RpcController controller, LockHeartbeatRequest request)
2405    throws ServiceException {
2406    try {
2407      if (
2408        server.getLockManager().remoteLocks().lockHeartbeat(request.getProcId(),
2409          request.getKeepAlive())
2410      ) {
2411        return LockHeartbeatResponse.newBuilder()
2412          .setTimeoutMs(server.getConfiguration().getInt(LockProcedure.REMOTE_LOCKS_TIMEOUT_MS_CONF,
2413            LockProcedure.DEFAULT_REMOTE_LOCKS_TIMEOUT_MS))
2414          .setLockStatus(LockHeartbeatResponse.LockStatus.LOCKED).build();
2415      } else {
2416        return LockHeartbeatResponse.newBuilder()
2417          .setLockStatus(LockHeartbeatResponse.LockStatus.UNLOCKED).build();
2418      }
2419    } catch (IOException e) {
2420      throw new ServiceException(e);
2421    }
2422  }
2423
2424  @Override
2425  public RegionSpaceUseReportResponse reportRegionSpaceUse(RpcController controller,
2426    RegionSpaceUseReportRequest request) throws ServiceException {
2427    try {
2428      server.checkInitialized();
2429      if (!QuotaUtil.isQuotaEnabled(server.getConfiguration())) {
2430        return RegionSpaceUseReportResponse.newBuilder().build();
2431      }
2432      MasterQuotaManager quotaManager = this.server.getMasterQuotaManager();
2433      if (quotaManager != null) {
2434        final long now = EnvironmentEdgeManager.currentTime();
2435        for (RegionSpaceUse report : request.getSpaceUseList()) {
2436          quotaManager.addRegionSize(ProtobufUtil.toRegionInfo(report.getRegionInfo()),
2437            report.getRegionSize(), now);
2438        }
2439      } else {
2440        LOG.debug("Received region space usage report but HMaster is not ready to process it, "
2441          + "skipping");
2442      }
2443      return RegionSpaceUseReportResponse.newBuilder().build();
2444    } catch (Exception e) {
2445      throw new ServiceException(e);
2446    }
2447  }
2448
2449  @Override
2450  public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(RpcController controller,
2451    GetSpaceQuotaRegionSizesRequest request) throws ServiceException {
2452    try {
2453      server.checkInitialized();
2454      MasterQuotaManager quotaManager = this.server.getMasterQuotaManager();
2455      GetSpaceQuotaRegionSizesResponse.Builder builder =
2456        GetSpaceQuotaRegionSizesResponse.newBuilder();
2457      if (quotaManager != null) {
2458        Map<RegionInfo, Long> regionSizes = quotaManager.snapshotRegionSizes();
2459        Map<TableName, Long> regionSizesByTable = new HashMap<>();
2460        // Translate hregioninfo+long -> tablename+long
2461        for (Entry<RegionInfo, Long> entry : regionSizes.entrySet()) {
2462          final TableName tableName = entry.getKey().getTable();
2463          Long prevSize = regionSizesByTable.get(tableName);
2464          if (prevSize == null) {
2465            prevSize = 0L;
2466          }
2467          regionSizesByTable.put(tableName, prevSize + entry.getValue());
2468        }
2469        // Serialize them into the protobuf
2470        for (Entry<TableName, Long> tableSize : regionSizesByTable.entrySet()) {
2471          builder.addSizes(
2472            RegionSizes.newBuilder().setTableName(ProtobufUtil.toProtoTableName(tableSize.getKey()))
2473              .setSize(tableSize.getValue()).build());
2474        }
2475        return builder.build();
2476      } else {
2477        LOG.debug("Received space quota region size report but HMaster is not ready to process it,"
2478          + "skipping");
2479      }
2480      return builder.build();
2481    } catch (Exception e) {
2482      throw new ServiceException(e);
2483    }
2484  }
2485
2486  @Override
2487  public GetQuotaStatesResponse getQuotaStates(RpcController controller,
2488    GetQuotaStatesRequest request) throws ServiceException {
2489    try {
2490      server.checkInitialized();
2491      QuotaObserverChore quotaChore = this.server.getQuotaObserverChore();
2492      GetQuotaStatesResponse.Builder builder = GetQuotaStatesResponse.newBuilder();
2493      if (quotaChore != null) {
2494        // The "current" view of all tables with quotas
2495        Map<TableName, SpaceQuotaSnapshot> tableSnapshots = quotaChore.getTableQuotaSnapshots();
2496        for (Entry<TableName, SpaceQuotaSnapshot> entry : tableSnapshots.entrySet()) {
2497          builder.addTableSnapshots(TableQuotaSnapshot.newBuilder()
2498            .setTableName(ProtobufUtil.toProtoTableName(entry.getKey()))
2499            .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build());
2500        }
2501        // The "current" view of all namespaces with quotas
2502        Map<String, SpaceQuotaSnapshot> nsSnapshots = quotaChore.getNamespaceQuotaSnapshots();
2503        for (Entry<String, SpaceQuotaSnapshot> entry : nsSnapshots.entrySet()) {
2504          builder.addNsSnapshots(NamespaceQuotaSnapshot.newBuilder().setNamespace(entry.getKey())
2505            .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build());
2506        }
2507        return builder.build();
2508      }
2509      return builder.build();
2510    } catch (Exception e) {
2511      throw new ServiceException(e);
2512    }
2513  }
2514
2515  @Override
2516  public ClearDeadServersResponse clearDeadServers(RpcController controller,
2517    ClearDeadServersRequest request) throws ServiceException {
2518    LOG.debug(server.getClientIdAuditPrefix() + " clear dead region servers.");
2519    ClearDeadServersResponse.Builder response = ClearDeadServersResponse.newBuilder();
2520    try {
2521      server.checkInitialized();
2522      if (server.cpHost != null) {
2523        server.cpHost.preClearDeadServers();
2524      }
2525
2526      if (server.getServerManager().areDeadServersInProgress()) {
2527        LOG.debug("Some dead server is still under processing, won't clear the dead server list");
2528        response.addAllServerName(request.getServerNameList());
2529      } else {
2530        DeadServer deadServer = server.getServerManager().getDeadServers();
2531        Set<Address> clearedServers = new HashSet<>();
2532        for (HBaseProtos.ServerName pbServer : request.getServerNameList()) {
2533          ServerName serverName = ProtobufUtil.toServerName(pbServer);
2534          final boolean deadInProcess =
2535            server.getProcedures().stream().anyMatch(p -> (p instanceof ServerCrashProcedure)
2536              && ((ServerCrashProcedure) p).getServerName().equals(serverName));
2537          if (deadInProcess) {
2538            throw new ServiceException(
2539              String.format("Dead server '%s' is not 'dead' in fact...", serverName));
2540          }
2541
2542          if (!deadServer.removeDeadServer(serverName)) {
2543            response.addServerName(pbServer);
2544          } else {
2545            clearedServers.add(serverName.getAddress());
2546          }
2547        }
2548        server.getRSGroupInfoManager().removeServers(clearedServers);
2549        LOG.info("Remove decommissioned servers {} from RSGroup done", clearedServers);
2550      }
2551
2552      if (server.cpHost != null) {
2553        server.cpHost.postClearDeadServers(
2554          ProtobufUtil.toServerNameList(request.getServerNameList()),
2555          ProtobufUtil.toServerNameList(response.getServerNameList()));
2556      }
2557    } catch (IOException io) {
2558      throw new ServiceException(io);
2559    }
2560    return response.build();
2561  }
2562
2563  @Override
2564  public ReportProcedureDoneResponse reportProcedureDone(RpcController controller,
2565    ReportProcedureDoneRequest request) throws ServiceException {
2566    // Check Masters is up and ready for duty before progressing. Remote side will keep trying.
2567    try {
2568      this.server.checkServiceStarted();
2569      for (RemoteProcedureResult result : request.getResultList()) {
2570        // -1 is less than any possible MasterActiveCode
2571        long initiatingMasterActiveTime =
2572          result.hasInitiatingMasterActiveTime() ? result.getInitiatingMasterActiveTime() : -1;
2573        throwOnOldMaster(result.getProcId(), initiatingMasterActiveTime);
2574      }
2575    } catch (IOException ioe) {
2576      throw new ServiceException(ioe);
2577    }
2578    request.getResultList().forEach(result -> {
2579      if (result.getStatus() == RemoteProcedureResult.Status.SUCCESS) {
2580        byte[] remoteResultData =
2581          result.hasProcResultData() ? result.getProcResultData().toByteArray() : null;
2582        server.remoteProcedureCompleted(result.getProcId(), remoteResultData);
2583      } else {
2584        server.remoteProcedureFailed(result.getProcId(),
2585          RemoteProcedureException.fromProto(result.getError()));
2586      }
2587    });
2588    return ReportProcedureDoneResponse.getDefaultInstance();
2589  }
2590
2591  private void throwOnOldMaster(long procId, long initiatingMasterActiveTime)
2592    throws MasterNotRunningException {
2593    if (initiatingMasterActiveTime > server.getMasterActiveTime()) {
2594      // procedure is initiated by new active master but report received on master with older active
2595      // time
2596      LOG.warn(
2597        "Report for procId: {} and initiatingMasterAT {} received on master with activeTime {}",
2598        procId, initiatingMasterActiveTime, server.getMasterActiveTime());
2599      throw new MasterNotRunningException("Another master is active");
2600    }
2601  }
2602
2603  @Override
2604  public FileArchiveNotificationResponse reportFileArchival(RpcController controller,
2605    FileArchiveNotificationRequest request) throws ServiceException {
2606    try {
2607      server.checkInitialized();
2608      if (!QuotaUtil.isQuotaEnabled(server.getConfiguration())) {
2609        return FileArchiveNotificationResponse.newBuilder().build();
2610      }
2611      server.getMasterQuotaManager().processFileArchivals(request, server.getConnection(),
2612        server.getConfiguration(), server.getFileSystem());
2613      return FileArchiveNotificationResponse.newBuilder().build();
2614    } catch (Exception e) {
2615      throw new ServiceException(e);
2616    }
2617  }
2618
2619  // HBCK Services
2620
2621  @Override
2622  public RunHbckChoreResponse runHbckChore(RpcController c, RunHbckChoreRequest req)
2623    throws ServiceException {
2624    rpcPreCheck("runHbckChore");
2625    LOG.info("{} request HBCK chore to run", server.getClientIdAuditPrefix());
2626    HbckChore hbckChore = server.getHbckChore();
2627    boolean ran = hbckChore.runChore();
2628    return RunHbckChoreResponse.newBuilder().setRan(ran).build();
2629  }
2630
2631  /**
2632   * Update state of the table in meta only. This is required by hbck in some situations to cleanup
2633   * stuck assign/ unassign regions procedures for the table.
2634   * @return previous state of the table
2635   */
2636  @Override
2637  public GetTableStateResponse setTableStateInMeta(RpcController controller,
2638    SetTableStateInMetaRequest request) throws ServiceException {
2639    rpcPreCheck("setTableStateInMeta");
2640    TableName tn = ProtobufUtil.toTableName(request.getTableName());
2641    try {
2642      TableState prevState = this.server.getTableStateManager().getTableState(tn);
2643      TableState newState = TableState.convert(tn, request.getTableState());
2644      LOG.info("{} set table={} state from {} to {}", server.getClientIdAuditPrefix(), tn,
2645        prevState.getState(), newState.getState());
2646      this.server.getTableStateManager().setTableState(tn, newState.getState());
2647      return GetTableStateResponse.newBuilder().setTableState(prevState.convert()).build();
2648    } catch (Exception e) {
2649      throw new ServiceException(e);
2650    }
2651  }
2652
2653  /**
2654   * Update state of the region in meta only. This is required by hbck in some situations to cleanup
2655   * stuck assign/ unassign regions procedures for the table.
2656   * @return previous states of the regions
2657   */
2658  @Override
2659  public SetRegionStateInMetaResponse setRegionStateInMeta(RpcController controller,
2660    SetRegionStateInMetaRequest request) throws ServiceException {
2661    rpcPreCheck("setRegionStateInMeta");
2662    SetRegionStateInMetaResponse.Builder builder = SetRegionStateInMetaResponse.newBuilder();
2663    final AssignmentManager am = server.getAssignmentManager();
2664    try {
2665      for (RegionSpecifierAndState s : request.getStatesList()) {
2666        final RegionSpecifier spec = s.getRegionSpecifier();
2667        final RegionInfo targetRegionInfo = getRegionInfo(spec);
2668        final RegionState.State targetState = RegionState.State.convert(s.getState());
2669        final RegionState.State currentState = Optional.ofNullable(targetRegionInfo)
2670          .map(info -> am.getRegionStates().getRegionState(info)).map(RegionState::getState)
2671          .orElseThrow(
2672            () -> new ServiceException("No existing state known for region '" + spec + "'."));
2673        LOG.info("{} set region={} state from {} to {}", server.getClientIdAuditPrefix(),
2674          targetRegionInfo, currentState, targetState);
2675        if (currentState == targetState) {
2676          LOG.debug("Proposed state matches current state. {}, {}", targetRegionInfo, currentState);
2677          continue;
2678        }
2679        MetaTableAccessor.updateRegionState(server.getConnection(), targetRegionInfo, targetState);
2680        // Loads from meta again to refresh AM cache with the new region state
2681        am.populateRegionStatesFromMeta(targetRegionInfo);
2682        builder.addStates(RegionSpecifierAndState.newBuilder().setRegionSpecifier(spec)
2683          .setState(currentState.convert()));
2684      }
2685    } catch (IOException e) {
2686      throw new ServiceException(e);
2687    }
2688    return builder.build();
2689  }
2690
2691  /**
2692   * Get {@link RegionInfo} from Master using content of {@link RegionSpecifier} as key.
2693   * @return {@link RegionInfo} found by decoding {@code rs} or {@code null} if {@code rs} is
2694   *         unknown to the master.
2695   * @throws ServiceException If some error occurs while querying META or parsing results.
2696   */
2697  private RegionInfo getRegionInfo(HBaseProtos.RegionSpecifier rs) throws ServiceException {
2698    // TODO: this doesn't handle MOB regions. Should it? See the public method #getRegionInfo
2699    final AssignmentManager am = server.getAssignmentManager();
2700    final String encodedRegionName;
2701    final RegionInfo info;
2702    // first try resolving from the AM's caches.
2703    switch (rs.getType()) {
2704      case REGION_NAME:
2705        final byte[] regionName = rs.getValue().toByteArray();
2706        encodedRegionName = RegionInfo.encodeRegionName(regionName);
2707        info = am.getRegionInfo(regionName);
2708        break;
2709      case ENCODED_REGION_NAME:
2710        encodedRegionName = rs.getValue().toStringUtf8();
2711        info = am.getRegionInfo(encodedRegionName);
2712        break;
2713      default:
2714        throw new IllegalArgumentException("Unrecognized RegionSpecifierType " + rs.getType());
2715    }
2716    if (info != null) {
2717      return info;
2718    }
2719    // fall back to a meta scan and check the cache again.
2720    try {
2721      am.populateRegionStatesFromMeta(encodedRegionName);
2722    } catch (IOException e) {
2723      throw new ServiceException(e);
2724    }
2725    return am.getRegionInfo(encodedRegionName);
2726  }
2727
2728  /**
2729   * @throws ServiceException If no MasterProcedureExecutor
2730   */
2731  private void checkMasterProcedureExecutor() throws ServiceException {
2732    if (this.server.getMasterProcedureExecutor() == null) {
2733      throw new ServiceException("Master's ProcedureExecutor not initialized; retry later");
2734    }
2735  }
2736
2737  /**
2738   * A 'raw' version of assign that does bulk and can skirt Master state checks if override is set;
2739   * i.e. assigns can be forced during Master startup or if RegionState is unclean. Used by HBCK2.
2740   */
2741  @Override
2742  public MasterProtos.AssignsResponse assigns(RpcController controller,
2743    MasterProtos.AssignsRequest request) throws ServiceException {
2744    checkMasterProcedureExecutor();
2745    final ProcedureExecutor<MasterProcedureEnv> pe = server.getMasterProcedureExecutor();
2746    final AssignmentManager am = server.getAssignmentManager();
2747    MasterProtos.AssignsResponse.Builder responseBuilder =
2748      MasterProtos.AssignsResponse.newBuilder();
2749    final boolean override = request.getOverride();
2750    final boolean force = request.getForce();
2751    LOG.info("{} assigns, override={}", server.getClientIdAuditPrefix(), override);
2752    for (HBaseProtos.RegionSpecifier rs : request.getRegionList()) {
2753      final RegionInfo info = getRegionInfo(rs);
2754      if (info == null) {
2755        LOG.info("Unknown region {}", rs);
2756        continue;
2757      }
2758      responseBuilder.addPid(Optional.ofNullable(am.createOneAssignProcedure(info, override, force))
2759        .map(pe::submitProcedure).orElse(Procedure.NO_PROC_ID));
2760    }
2761    return responseBuilder.build();
2762  }
2763
2764  /**
2765   * A 'raw' version of unassign that does bulk and can skirt Master state checks if override is
2766   * set; i.e. unassigns can be forced during Master startup or if RegionState is unclean. Used by
2767   * HBCK2.
2768   */
2769  @Override
2770  public MasterProtos.UnassignsResponse unassigns(RpcController controller,
2771    MasterProtos.UnassignsRequest request) throws ServiceException {
2772    checkMasterProcedureExecutor();
2773    final ProcedureExecutor<MasterProcedureEnv> pe = server.getMasterProcedureExecutor();
2774    final AssignmentManager am = server.getAssignmentManager();
2775    MasterProtos.UnassignsResponse.Builder responseBuilder =
2776      MasterProtos.UnassignsResponse.newBuilder();
2777    final boolean override = request.getOverride();
2778    final boolean force = request.getForce();
2779    LOG.info("{} unassigns, override={}", server.getClientIdAuditPrefix(), override);
2780    for (HBaseProtos.RegionSpecifier rs : request.getRegionList()) {
2781      final RegionInfo info = getRegionInfo(rs);
2782      if (info == null) {
2783        LOG.info("Unknown region {}", rs);
2784        continue;
2785      }
2786      responseBuilder
2787        .addPid(Optional.ofNullable(am.createOneUnassignProcedure(info, override, force))
2788          .map(pe::submitProcedure).orElse(Procedure.NO_PROC_ID));
2789    }
2790    return responseBuilder.build();
2791  }
2792
2793  /**
2794   * Bypass specified procedure to completion. Procedure is marked completed but no actual work is
2795   * done from the current state/ step onwards. Parents of the procedure are also marked for bypass.
2796   * NOTE: this is a dangerous operation and may be used to unstuck buggy procedures. This may leave
2797   * system in incoherent state. This may need to be followed by some cleanup steps/ actions by
2798   * operator.
2799   * @return BypassProcedureToCompletionResponse indicating success or failure
2800   */
2801  @Override
2802  public MasterProtos.BypassProcedureResponse bypassProcedure(RpcController controller,
2803    MasterProtos.BypassProcedureRequest request) throws ServiceException {
2804    try {
2805      LOG.info("{} bypass procedures={}, waitTime={}, override={}, recursive={}",
2806        server.getClientIdAuditPrefix(), request.getProcIdList(), request.getWaitTime(),
2807        request.getOverride(), request.getRecursive());
2808      List<Boolean> ret =
2809        server.getMasterProcedureExecutor().bypassProcedure(request.getProcIdList(),
2810          request.getWaitTime(), request.getOverride(), request.getRecursive());
2811      return MasterProtos.BypassProcedureResponse.newBuilder().addAllBypassed(ret).build();
2812    } catch (IOException e) {
2813      throw new ServiceException(e);
2814    }
2815  }
2816
2817  @Override
2818  public MasterProtos.ScheduleServerCrashProcedureResponse scheduleServerCrashProcedure(
2819    RpcController controller, MasterProtos.ScheduleServerCrashProcedureRequest request)
2820    throws ServiceException {
2821    List<Long> pids = new ArrayList<>();
2822    for (HBaseProtos.ServerName sn : request.getServerNameList()) {
2823      ServerName serverName = ProtobufUtil.toServerName(sn);
2824      LOG.info("{} schedule ServerCrashProcedure for {}", this.server.getClientIdAuditPrefix(),
2825        serverName);
2826      if (shouldSubmitSCP(serverName)) {
2827        pids.add(this.server.getServerManager().expireServer(serverName, true));
2828      } else {
2829        pids.add(Procedure.NO_PROC_ID);
2830      }
2831    }
2832    return MasterProtos.ScheduleServerCrashProcedureResponse.newBuilder().addAllPid(pids).build();
2833  }
2834
2835  @Override
2836  public MasterProtos.ScheduleSCPsForUnknownServersResponse scheduleSCPsForUnknownServers(
2837    RpcController controller, MasterProtos.ScheduleSCPsForUnknownServersRequest request)
2838    throws ServiceException {
2839    List<Long> pids = new ArrayList<>();
2840    final Set<ServerName> serverNames = server.getAssignmentManager().getRegionStates()
2841      .getRegionStates().stream().map(RegionState::getServerName).collect(Collectors.toSet());
2842
2843    final Set<ServerName> unknownServerNames = serverNames.stream()
2844      .filter(sn -> server.getServerManager().isServerUnknown(sn)).collect(Collectors.toSet());
2845
2846    for (ServerName sn : unknownServerNames) {
2847      LOG.info("{} schedule ServerCrashProcedure for unknown {}",
2848        this.server.getClientIdAuditPrefix(), sn);
2849      if (shouldSubmitSCP(sn)) {
2850        pids.add(this.server.getServerManager().expireServer(sn, true));
2851      } else {
2852        pids.add(Procedure.NO_PROC_ID);
2853      }
2854    }
2855    return MasterProtos.ScheduleSCPsForUnknownServersResponse.newBuilder().addAllPid(pids).build();
2856  }
2857
2858  @Override
2859  public FixMetaResponse fixMeta(RpcController controller, FixMetaRequest request)
2860    throws ServiceException {
2861    rpcPreCheck("fixMeta");
2862    try {
2863      MetaFixer mf = new MetaFixer(this.server);
2864      mf.fix();
2865      return FixMetaResponse.newBuilder().build();
2866    } catch (IOException ioe) {
2867      throw new ServiceException(ioe);
2868    }
2869  }
2870
2871  @Override
2872  public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller,
2873    SwitchRpcThrottleRequest request) throws ServiceException {
2874    try {
2875      server.checkInitialized();
2876      return server.getMasterQuotaManager().switchRpcThrottle(request);
2877    } catch (Exception e) {
2878      throw new ServiceException(e);
2879    }
2880  }
2881
2882  @Override
2883  public MasterProtos.IsRpcThrottleEnabledResponse isRpcThrottleEnabled(RpcController controller,
2884    MasterProtos.IsRpcThrottleEnabledRequest request) throws ServiceException {
2885    try {
2886      server.checkInitialized();
2887      return server.getMasterQuotaManager().isRpcThrottleEnabled(request);
2888    } catch (Exception e) {
2889      throw new ServiceException(e);
2890    }
2891  }
2892
2893  @Override
2894  public SwitchExceedThrottleQuotaResponse switchExceedThrottleQuota(RpcController controller,
2895    SwitchExceedThrottleQuotaRequest request) throws ServiceException {
2896    try {
2897      server.checkInitialized();
2898      return server.getMasterQuotaManager().switchExceedThrottleQuota(request);
2899    } catch (Exception e) {
2900      throw new ServiceException(e);
2901    }
2902  }
2903
2904  @Override
2905  public GrantResponse grant(RpcController controller, GrantRequest request)
2906    throws ServiceException {
2907    try {
2908      server.checkInitialized();
2909      if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
2910        final UserPermission perm =
2911          ShadedAccessControlUtil.toUserPermission(request.getUserPermission());
2912        boolean mergeExistingPermissions = request.getMergeExistingPermissions();
2913        server.cpHost.preGrant(perm, mergeExistingPermissions);
2914        try (Table table = server.getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
2915          PermissionStorage.addUserPermission(getConfiguration(), perm, table,
2916            mergeExistingPermissions);
2917        }
2918        server.cpHost.postGrant(perm, mergeExistingPermissions);
2919        return GrantResponse.getDefaultInstance();
2920      } else {
2921        throw new DoNotRetryIOException(
2922          new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
2923      }
2924    } catch (IOException ioe) {
2925      throw new ServiceException(ioe);
2926    }
2927  }
2928
2929  @Override
2930  public RevokeResponse revoke(RpcController controller, RevokeRequest request)
2931    throws ServiceException {
2932    try {
2933      server.checkInitialized();
2934      if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
2935        final UserPermission userPermission =
2936          ShadedAccessControlUtil.toUserPermission(request.getUserPermission());
2937        server.cpHost.preRevoke(userPermission);
2938        try (Table table = server.getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
2939          PermissionStorage.removeUserPermission(server.getConfiguration(), userPermission, table);
2940        }
2941        server.cpHost.postRevoke(userPermission);
2942        return RevokeResponse.getDefaultInstance();
2943      } else {
2944        throw new DoNotRetryIOException(
2945          new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
2946      }
2947    } catch (IOException ioe) {
2948      throw new ServiceException(ioe);
2949    }
2950  }
2951
2952  @Override
2953  public GetUserPermissionsResponse getUserPermissions(RpcController controller,
2954    GetUserPermissionsRequest request) throws ServiceException {
2955    try {
2956      server.checkInitialized();
2957      if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
2958        final String userName = request.hasUserName() ? request.getUserName().toStringUtf8() : null;
2959        String namespace =
2960          request.hasNamespaceName() ? request.getNamespaceName().toStringUtf8() : null;
2961        TableName table =
2962          request.hasTableName() ? ProtobufUtil.toTableName(request.getTableName()) : null;
2963        byte[] cf = request.hasColumnFamily() ? request.getColumnFamily().toByteArray() : null;
2964        byte[] cq =
2965          request.hasColumnQualifier() ? request.getColumnQualifier().toByteArray() : null;
2966        Type permissionType = request.hasType() ? request.getType() : null;
2967        server.getMasterCoprocessorHost().preGetUserPermissions(userName, namespace, table, cf, cq);
2968
2969        List<UserPermission> perms = null;
2970        if (permissionType == Type.Table) {
2971          boolean filter = (cf != null || userName != null) ? true : false;
2972          perms = PermissionStorage.getUserTablePermissions(server.getConfiguration(), table, cf,
2973            cq, userName, filter);
2974        } else if (permissionType == Type.Namespace) {
2975          perms = PermissionStorage.getUserNamespacePermissions(server.getConfiguration(),
2976            namespace, userName, userName != null ? true : false);
2977        } else {
2978          perms = PermissionStorage.getUserPermissions(server.getConfiguration(), null, null, null,
2979            userName, userName != null ? true : false);
2980          // Skip super users when filter user is specified
2981          if (userName == null) {
2982            // Adding superusers explicitly to the result set as PermissionStorage do not store
2983            // them. Also using acl as table name to be inline with the results of global admin and
2984            // will help in avoiding any leakage of information about being superusers.
2985            for (String user : Superusers.getSuperUsers()) {
2986              perms.add(new UserPermission(user,
2987                Permission.newBuilder().withActions(Action.values()).build()));
2988            }
2989          }
2990        }
2991
2992        server.getMasterCoprocessorHost().postGetUserPermissions(userName, namespace, table, cf,
2993          cq);
2994        AccessControlProtos.GetUserPermissionsResponse response =
2995          ShadedAccessControlUtil.buildGetUserPermissionsResponse(perms);
2996        return response;
2997      } else {
2998        throw new DoNotRetryIOException(
2999          new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
3000      }
3001    } catch (IOException ioe) {
3002      throw new ServiceException(ioe);
3003    }
3004  }
3005
3006  @Override
3007  public HasUserPermissionsResponse hasUserPermissions(RpcController controller,
3008    HasUserPermissionsRequest request) throws ServiceException {
3009    try {
3010      server.checkInitialized();
3011      if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
3012        User caller = RpcServer.getRequestUser().orElse(null);
3013        String userName =
3014          request.hasUserName() ? request.getUserName().toStringUtf8() : caller.getShortName();
3015        List<Permission> permissions = new ArrayList<>();
3016        for (int i = 0; i < request.getPermissionCount(); i++) {
3017          permissions.add(ShadedAccessControlUtil.toPermission(request.getPermission(i)));
3018        }
3019        server.getMasterCoprocessorHost().preHasUserPermissions(userName, permissions);
3020        if (!caller.getShortName().equals(userName)) {
3021          List<String> groups = AccessChecker.getUserGroups(userName);
3022          caller = new InputUser(userName, groups.toArray(new String[groups.size()]));
3023        }
3024        List<Boolean> hasUserPermissions = new ArrayList<>();
3025        if (getAccessChecker() != null) {
3026          for (Permission permission : permissions) {
3027            boolean hasUserPermission =
3028              getAccessChecker().hasUserPermission(caller, "hasUserPermissions", permission);
3029            hasUserPermissions.add(hasUserPermission);
3030          }
3031        } else {
3032          for (int i = 0; i < permissions.size(); i++) {
3033            hasUserPermissions.add(true);
3034          }
3035        }
3036        server.getMasterCoprocessorHost().postHasUserPermissions(userName, permissions);
3037        HasUserPermissionsResponse.Builder builder =
3038          HasUserPermissionsResponse.newBuilder().addAllHasUserPermission(hasUserPermissions);
3039        return builder.build();
3040      } else {
3041        throw new DoNotRetryIOException(
3042          new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
3043      }
3044    } catch (IOException ioe) {
3045      throw new ServiceException(ioe);
3046    }
3047  }
3048
3049  private boolean shouldSubmitSCP(ServerName serverName) {
3050    // check if there is already a SCP of this server running
3051    List<Procedure<MasterProcedureEnv>> procedures =
3052      server.getMasterProcedureExecutor().getProcedures();
3053    for (Procedure<MasterProcedureEnv> procedure : procedures) {
3054      if (procedure instanceof ServerCrashProcedure) {
3055        if (
3056          serverName.compareTo(((ServerCrashProcedure) procedure).getServerName()) == 0
3057            && !procedure.isFinished()
3058        ) {
3059          LOG.info("there is already a SCP of this server {} running, pid {}", serverName,
3060            procedure.getProcId());
3061          return false;
3062        }
3063      }
3064    }
3065    return true;
3066  }
3067
3068  @Override
3069  public GetRSGroupInfoResponse getRSGroupInfo(RpcController controller,
3070    GetRSGroupInfoRequest request) throws ServiceException {
3071    String groupName = request.getRSGroupName();
3072    LOG.info(
3073      server.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, group=" + groupName);
3074    try {
3075      if (server.getMasterCoprocessorHost() != null) {
3076        server.getMasterCoprocessorHost().preGetRSGroupInfo(groupName);
3077      }
3078      RSGroupInfo rsGroupInfo = server.getRSGroupInfoManager().getRSGroup(groupName);
3079      GetRSGroupInfoResponse resp;
3080      if (rsGroupInfo != null) {
3081        resp = GetRSGroupInfoResponse.newBuilder()
3082          .setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo)).build();
3083      } else {
3084        resp = GetRSGroupInfoResponse.getDefaultInstance();
3085      }
3086      if (server.getMasterCoprocessorHost() != null) {
3087        server.getMasterCoprocessorHost().postGetRSGroupInfo(groupName);
3088      }
3089      return resp;
3090    } catch (IOException e) {
3091      throw new ServiceException(e);
3092    }
3093  }
3094
3095  @Override
3096  public GetRSGroupInfoOfTableResponse getRSGroupInfoOfTable(RpcController controller,
3097    GetRSGroupInfoOfTableRequest request) throws ServiceException {
3098    TableName tableName = ProtobufUtil.toTableName(request.getTableName());
3099    LOG.info(
3100      server.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, table=" + tableName);
3101    try {
3102      if (server.getMasterCoprocessorHost() != null) {
3103        server.getMasterCoprocessorHost().preGetRSGroupInfoOfTable(tableName);
3104      }
3105      GetRSGroupInfoOfTableResponse resp;
3106      TableDescriptor td = server.getTableDescriptors().get(tableName);
3107      if (td == null) {
3108        resp = GetRSGroupInfoOfTableResponse.getDefaultInstance();
3109      } else {
3110        RSGroupInfo rsGroupInfo =
3111          RSGroupUtil.getRSGroupInfo(server, server.getRSGroupInfoManager(), tableName)
3112            .orElse(server.getRSGroupInfoManager().getRSGroup(RSGroupInfo.DEFAULT_GROUP));
3113        resp = GetRSGroupInfoOfTableResponse.newBuilder()
3114          .setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo)).build();
3115      }
3116      if (server.getMasterCoprocessorHost() != null) {
3117        server.getMasterCoprocessorHost().postGetRSGroupInfoOfTable(tableName);
3118      }
3119      return resp;
3120    } catch (IOException e) {
3121      throw new ServiceException(e);
3122    }
3123  }
3124
3125  @Override
3126  public GetRSGroupInfoOfServerResponse getRSGroupInfoOfServer(RpcController controller,
3127    GetRSGroupInfoOfServerRequest request) throws ServiceException {
3128    Address hp =
3129      Address.fromParts(request.getServer().getHostName(), request.getServer().getPort());
3130    LOG.info(server.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server=" + hp);
3131    try {
3132      if (server.getMasterCoprocessorHost() != null) {
3133        server.getMasterCoprocessorHost().preGetRSGroupInfoOfServer(hp);
3134      }
3135      RSGroupInfo rsGroupInfo = server.getRSGroupInfoManager().getRSGroupOfServer(hp);
3136      GetRSGroupInfoOfServerResponse resp;
3137      if (rsGroupInfo != null) {
3138        resp = GetRSGroupInfoOfServerResponse.newBuilder()
3139          .setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo)).build();
3140      } else {
3141        resp = GetRSGroupInfoOfServerResponse.getDefaultInstance();
3142      }
3143      if (server.getMasterCoprocessorHost() != null) {
3144        server.getMasterCoprocessorHost().postGetRSGroupInfoOfServer(hp);
3145      }
3146      return resp;
3147    } catch (IOException e) {
3148      throw new ServiceException(e);
3149    }
3150  }
3151
3152  @Override
3153  public MoveServersResponse moveServers(RpcController controller, MoveServersRequest request)
3154    throws ServiceException {
3155    Set<Address> hostPorts = Sets.newHashSet();
3156    MoveServersResponse.Builder builder = MoveServersResponse.newBuilder();
3157    for (HBaseProtos.ServerName el : request.getServersList()) {
3158      hostPorts.add(Address.fromParts(el.getHostName(), el.getPort()));
3159    }
3160    LOG.info(server.getClientIdAuditPrefix() + " move servers " + hostPorts + " to rsgroup "
3161      + request.getTargetGroup());
3162    try {
3163      if (server.getMasterCoprocessorHost() != null) {
3164        server.getMasterCoprocessorHost().preMoveServers(hostPorts, request.getTargetGroup());
3165      }
3166      server.getRSGroupInfoManager().moveServers(hostPorts, request.getTargetGroup());
3167      if (server.getMasterCoprocessorHost() != null) {
3168        server.getMasterCoprocessorHost().postMoveServers(hostPorts, request.getTargetGroup());
3169      }
3170    } catch (IOException e) {
3171      throw new ServiceException(e);
3172    }
3173    return builder.build();
3174  }
3175
3176  @Override
3177  public AddRSGroupResponse addRSGroup(RpcController controller, AddRSGroupRequest request)
3178    throws ServiceException {
3179    AddRSGroupResponse.Builder builder = AddRSGroupResponse.newBuilder();
3180    LOG.info(server.getClientIdAuditPrefix() + " add rsgroup " + request.getRSGroupName());
3181    try {
3182      if (server.getMasterCoprocessorHost() != null) {
3183        server.getMasterCoprocessorHost().preAddRSGroup(request.getRSGroupName());
3184      }
3185      server.getRSGroupInfoManager().addRSGroup(new RSGroupInfo(request.getRSGroupName()));
3186      if (server.getMasterCoprocessorHost() != null) {
3187        server.getMasterCoprocessorHost().postAddRSGroup(request.getRSGroupName());
3188      }
3189    } catch (IOException e) {
3190      throw new ServiceException(e);
3191    }
3192    return builder.build();
3193  }
3194
3195  @Override
3196  public RemoveRSGroupResponse removeRSGroup(RpcController controller, RemoveRSGroupRequest request)
3197    throws ServiceException {
3198    RemoveRSGroupResponse.Builder builder = RemoveRSGroupResponse.newBuilder();
3199    LOG.info(server.getClientIdAuditPrefix() + " remove rsgroup " + request.getRSGroupName());
3200    try {
3201      if (server.getMasterCoprocessorHost() != null) {
3202        server.getMasterCoprocessorHost().preRemoveRSGroup(request.getRSGroupName());
3203      }
3204      server.getRSGroupInfoManager().removeRSGroup(request.getRSGroupName());
3205      if (server.getMasterCoprocessorHost() != null) {
3206        server.getMasterCoprocessorHost().postRemoveRSGroup(request.getRSGroupName());
3207      }
3208    } catch (IOException e) {
3209      throw new ServiceException(e);
3210    }
3211    return builder.build();
3212  }
3213
3214  @Override
3215  public BalanceRSGroupResponse balanceRSGroup(RpcController controller,
3216    BalanceRSGroupRequest request) throws ServiceException {
3217    BalanceRequest balanceRequest = ProtobufUtil.toBalanceRequest(request);
3218
3219    BalanceRSGroupResponse.Builder builder =
3220      BalanceRSGroupResponse.newBuilder().setBalanceRan(false);
3221
3222    LOG.info(
3223      server.getClientIdAuditPrefix() + " balance rsgroup, group=" + request.getRSGroupName());
3224    try {
3225      if (server.getMasterCoprocessorHost() != null) {
3226        server.getMasterCoprocessorHost().preBalanceRSGroup(request.getRSGroupName(),
3227          balanceRequest);
3228      }
3229      BalanceResponse response =
3230        server.getRSGroupInfoManager().balanceRSGroup(request.getRSGroupName(), balanceRequest);
3231      ProtobufUtil.populateBalanceRSGroupResponse(builder, response);
3232      if (server.getMasterCoprocessorHost() != null) {
3233        server.getMasterCoprocessorHost().postBalanceRSGroup(request.getRSGroupName(),
3234          balanceRequest, response);
3235      }
3236    } catch (IOException e) {
3237      throw new ServiceException(e);
3238    }
3239    return builder.build();
3240  }
3241
3242  @Override
3243  public ListRSGroupInfosResponse listRSGroupInfos(RpcController controller,
3244    ListRSGroupInfosRequest request) throws ServiceException {
3245    ListRSGroupInfosResponse.Builder builder = ListRSGroupInfosResponse.newBuilder();
3246    LOG.info(server.getClientIdAuditPrefix() + " list rsgroup");
3247    try {
3248      if (server.getMasterCoprocessorHost() != null) {
3249        server.getMasterCoprocessorHost().preListRSGroups();
3250      }
3251      List<RSGroupInfo> rsGroupInfos = server.getRSGroupInfoManager().listRSGroups().stream()
3252        .map(RSGroupInfo::new).collect(Collectors.toList());
3253      Map<String, RSGroupInfo> name2Info = new HashMap<>();
3254      List<TableDescriptor> needToFill =
3255        new ArrayList<>(server.getTableDescriptors().getAll().values());
3256      for (RSGroupInfo rsGroupInfo : rsGroupInfos) {
3257        name2Info.put(rsGroupInfo.getName(), rsGroupInfo);
3258        for (TableDescriptor td : server.getTableDescriptors().getAll().values()) {
3259          if (rsGroupInfo.containsTable(td.getTableName())) {
3260            needToFill.remove(td);
3261          }
3262        }
3263      }
3264      for (TableDescriptor td : needToFill) {
3265        String groupName = td.getRegionServerGroup().orElse(RSGroupInfo.DEFAULT_GROUP);
3266        RSGroupInfo rsGroupInfo = name2Info.get(groupName);
3267        if (rsGroupInfo != null) {
3268          rsGroupInfo.addTable(td.getTableName());
3269        }
3270      }
3271      for (RSGroupInfo rsGroupInfo : rsGroupInfos) {
3272        // TODO: this can be done at once outside this loop, do not need to scan all every time.
3273        builder.addRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo));
3274      }
3275      if (server.getMasterCoprocessorHost() != null) {
3276        server.getMasterCoprocessorHost().postListRSGroups();
3277      }
3278    } catch (IOException e) {
3279      throw new ServiceException(e);
3280    }
3281    return builder.build();
3282  }
3283
3284  @Override
3285  public RemoveServersResponse removeServers(RpcController controller, RemoveServersRequest request)
3286    throws ServiceException {
3287    RemoveServersResponse.Builder builder = RemoveServersResponse.newBuilder();
3288    Set<Address> servers = Sets.newHashSet();
3289    for (HBaseProtos.ServerName el : request.getServersList()) {
3290      servers.add(Address.fromParts(el.getHostName(), el.getPort()));
3291    }
3292    LOG.info(
3293      server.getClientIdAuditPrefix() + " remove decommissioned servers from rsgroup: " + servers);
3294    try {
3295      if (server.getMasterCoprocessorHost() != null) {
3296        server.getMasterCoprocessorHost().preRemoveServers(servers);
3297      }
3298      server.getRSGroupInfoManager().removeServers(servers);
3299      if (server.getMasterCoprocessorHost() != null) {
3300        server.getMasterCoprocessorHost().postRemoveServers(servers);
3301      }
3302    } catch (IOException e) {
3303      throw new ServiceException(e);
3304    }
3305    return builder.build();
3306  }
3307
3308  @Override
3309  public ListTablesInRSGroupResponse listTablesInRSGroup(RpcController controller,
3310    ListTablesInRSGroupRequest request) throws ServiceException {
3311    ListTablesInRSGroupResponse.Builder builder = ListTablesInRSGroupResponse.newBuilder();
3312    String groupName = request.getGroupName();
3313    LOG.info(server.getClientIdAuditPrefix() + " list tables in rsgroup " + groupName);
3314    try {
3315      if (server.getMasterCoprocessorHost() != null) {
3316        server.getMasterCoprocessorHost().preListTablesInRSGroup(groupName);
3317      }
3318      RSGroupUtil.listTablesInRSGroup(server, groupName).stream()
3319        .map(ProtobufUtil::toProtoTableName).forEach(builder::addTableName);
3320      if (server.getMasterCoprocessorHost() != null) {
3321        server.getMasterCoprocessorHost().postListTablesInRSGroup(groupName);
3322      }
3323    } catch (IOException e) {
3324      throw new ServiceException(e);
3325    }
3326    return builder.build();
3327  }
3328
3329  @Override
3330  public GetConfiguredNamespacesAndTablesInRSGroupResponse
3331    getConfiguredNamespacesAndTablesInRSGroup(RpcController controller,
3332      GetConfiguredNamespacesAndTablesInRSGroupRequest request) throws ServiceException {
3333    GetConfiguredNamespacesAndTablesInRSGroupResponse.Builder builder =
3334      GetConfiguredNamespacesAndTablesInRSGroupResponse.newBuilder();
3335    String groupName = request.getGroupName();
3336    LOG.info(server.getClientIdAuditPrefix() + " get configured namespaces and tables in rsgroup "
3337      + groupName);
3338    try {
3339      if (server.getMasterCoprocessorHost() != null) {
3340        server.getMasterCoprocessorHost().preGetConfiguredNamespacesAndTablesInRSGroup(groupName);
3341      }
3342      for (NamespaceDescriptor nd : server.getClusterSchema().getNamespaces()) {
3343        if (groupName.equals(nd.getConfigurationValue(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP))) {
3344          builder.addNamespace(nd.getName());
3345        }
3346      }
3347      for (TableDescriptor td : server.getTableDescriptors().getAll().values()) {
3348        if (td.getRegionServerGroup().map(g -> g.equals(groupName)).orElse(false)) {
3349          builder.addTableName(ProtobufUtil.toProtoTableName(td.getTableName()));
3350        }
3351      }
3352      if (server.getMasterCoprocessorHost() != null) {
3353        server.getMasterCoprocessorHost().postGetConfiguredNamespacesAndTablesInRSGroup(groupName);
3354      }
3355    } catch (IOException e) {
3356      throw new ServiceException(e);
3357    }
3358    return builder.build();
3359  }
3360
3361  @Override
3362  public RenameRSGroupResponse renameRSGroup(RpcController controller, RenameRSGroupRequest request)
3363    throws ServiceException {
3364    RenameRSGroupResponse.Builder builder = RenameRSGroupResponse.newBuilder();
3365    String oldRSGroup = request.getOldRsgroupName();
3366    String newRSGroup = request.getNewRsgroupName();
3367    LOG.info("{} rename rsgroup from {} to {} ", server.getClientIdAuditPrefix(), oldRSGroup,
3368      newRSGroup);
3369    try {
3370      if (server.getMasterCoprocessorHost() != null) {
3371        server.getMasterCoprocessorHost().preRenameRSGroup(oldRSGroup, newRSGroup);
3372      }
3373      server.getRSGroupInfoManager().renameRSGroup(oldRSGroup, newRSGroup);
3374      if (server.getMasterCoprocessorHost() != null) {
3375        server.getMasterCoprocessorHost().postRenameRSGroup(oldRSGroup, newRSGroup);
3376      }
3377    } catch (IOException e) {
3378      throw new ServiceException(e);
3379    }
3380    return builder.build();
3381  }
3382
3383  @Override
3384  public UpdateRSGroupConfigResponse updateRSGroupConfig(RpcController controller,
3385    UpdateRSGroupConfigRequest request) throws ServiceException {
3386    UpdateRSGroupConfigResponse.Builder builder = UpdateRSGroupConfigResponse.newBuilder();
3387    String groupName = request.getGroupName();
3388    Map<String, String> configuration = new HashMap<>();
3389    request.getConfigurationList().forEach(p -> configuration.put(p.getName(), p.getValue()));
3390    LOG.info("{} update rsgroup {} configuration {}", server.getClientIdAuditPrefix(), groupName,
3391      configuration);
3392    try {
3393      if (server.getMasterCoprocessorHost() != null) {
3394        server.getMasterCoprocessorHost().preUpdateRSGroupConfig(groupName, configuration);
3395      }
3396      server.getRSGroupInfoManager().updateRSGroupConfig(groupName, configuration);
3397      if (server.getMasterCoprocessorHost() != null) {
3398        server.getMasterCoprocessorHost().postUpdateRSGroupConfig(groupName, configuration);
3399      }
3400    } catch (IOException e) {
3401      throw new ServiceException(e);
3402    }
3403    return builder.build();
3404  }
3405
3406  @Override
3407  public HBaseProtos.LogEntry getLogEntries(RpcController controller,
3408    HBaseProtos.LogRequest request) throws ServiceException {
3409    try {
3410      final String logClassName = request.getLogClassName();
3411      Class<?> logClass = Class.forName(logClassName).asSubclass(Message.class);
3412      Method method = logClass.getMethod("parseFrom", ByteString.class);
3413      if (logClassName.contains("BalancerDecisionsRequest")) {
3414        MasterProtos.BalancerDecisionsRequest balancerDecisionsRequest =
3415          (MasterProtos.BalancerDecisionsRequest) method.invoke(null, request.getLogMessage());
3416        MasterProtos.BalancerDecisionsResponse balancerDecisionsResponse =
3417          getBalancerDecisions(balancerDecisionsRequest);
3418        return HBaseProtos.LogEntry.newBuilder()
3419          .setLogClassName(balancerDecisionsResponse.getClass().getName())
3420          .setLogMessage(balancerDecisionsResponse.toByteString()).build();
3421      } else if (logClassName.contains("BalancerRejectionsRequest")) {
3422        MasterProtos.BalancerRejectionsRequest balancerRejectionsRequest =
3423          (MasterProtos.BalancerRejectionsRequest) method.invoke(null, request.getLogMessage());
3424        MasterProtos.BalancerRejectionsResponse balancerRejectionsResponse =
3425          getBalancerRejections(balancerRejectionsRequest);
3426        return HBaseProtos.LogEntry.newBuilder()
3427          .setLogClassName(balancerRejectionsResponse.getClass().getName())
3428          .setLogMessage(balancerRejectionsResponse.toByteString()).build();
3429      }
3430    } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
3431      | InvocationTargetException e) {
3432      LOG.error("Error while retrieving log entries.", e);
3433      throw new ServiceException(e);
3434    }
3435    throw new ServiceException("Invalid request params");
3436  }
3437
3438  private MasterProtos.BalancerDecisionsResponse
3439    getBalancerDecisions(MasterProtos.BalancerDecisionsRequest request) {
3440    final NamedQueueRecorder namedQueueRecorder = this.server.getNamedQueueRecorder();
3441    if (namedQueueRecorder == null) {
3442      return MasterProtos.BalancerDecisionsResponse.newBuilder()
3443        .addAllBalancerDecision(Collections.emptyList()).build();
3444    }
3445    final NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
3446    namedQueueGetRequest.setNamedQueueEvent(BalancerDecisionDetails.BALANCER_DECISION_EVENT);
3447    namedQueueGetRequest.setBalancerDecisionsRequest(request);
3448    NamedQueueGetResponse namedQueueGetResponse =
3449      namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
3450    List<RecentLogs.BalancerDecision> balancerDecisions = namedQueueGetResponse != null
3451      ? namedQueueGetResponse.getBalancerDecisions()
3452      : Collections.emptyList();
3453    return MasterProtos.BalancerDecisionsResponse.newBuilder()
3454      .addAllBalancerDecision(balancerDecisions).build();
3455  }
3456
3457  private MasterProtos.BalancerRejectionsResponse
3458    getBalancerRejections(MasterProtos.BalancerRejectionsRequest request) {
3459    final NamedQueueRecorder namedQueueRecorder = this.server.getNamedQueueRecorder();
3460    if (namedQueueRecorder == null) {
3461      return MasterProtos.BalancerRejectionsResponse.newBuilder()
3462        .addAllBalancerRejection(Collections.emptyList()).build();
3463    }
3464    final NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
3465    namedQueueGetRequest.setNamedQueueEvent(BalancerRejectionDetails.BALANCER_REJECTION_EVENT);
3466    namedQueueGetRequest.setBalancerRejectionsRequest(request);
3467    NamedQueueGetResponse namedQueueGetResponse =
3468      namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
3469    List<RecentLogs.BalancerRejection> balancerRejections = namedQueueGetResponse != null
3470      ? namedQueueGetResponse.getBalancerRejections()
3471      : Collections.emptyList();
3472    return MasterProtos.BalancerRejectionsResponse.newBuilder()
3473      .addAllBalancerRejection(balancerRejections).build();
3474  }
3475
3476  @Override
3477  @QosPriority(priority = HConstants.ADMIN_QOS)
3478  public GetRegionInfoResponse getRegionInfo(final RpcController controller,
3479    final GetRegionInfoRequest request) throws ServiceException {
3480    final GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
3481    final RegionInfo info = getRegionInfo(request.getRegion());
3482    if (info != null) {
3483      builder.setRegionInfo(ProtobufUtil.toRegionInfo(info));
3484    } else {
3485      // Is it a MOB name? These work differently.
3486      byte[] regionName = request.getRegion().getValue().toByteArray();
3487      TableName tableName = RegionInfo.getTable(regionName);
3488      if (MobUtils.isMobRegionName(tableName, regionName)) {
3489        // a dummy region info contains the compaction state.
3490        RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
3491        builder.setRegionInfo(ProtobufUtil.toRegionInfo(mobRegionInfo));
3492        if (request.hasCompactionState() && request.getCompactionState()) {
3493          builder.setCompactionState(server.getMobCompactionState(tableName));
3494        }
3495      } else {
3496        // If unknown RegionInfo and not a MOB region, it is unknown.
3497        throw new ServiceException(new UnknownRegionException(Bytes.toString(regionName)));
3498      }
3499    }
3500    return builder.build();
3501  }
3502
3503  @Override
3504  public GetStoreFileResponse getStoreFile(RpcController controller, GetStoreFileRequest request)
3505    throws ServiceException {
3506    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3507  }
3508
3509  @Override
3510  public GetOnlineRegionResponse getOnlineRegion(RpcController controller,
3511    GetOnlineRegionRequest request) throws ServiceException {
3512    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3513  }
3514
3515  @Override
3516  public OpenRegionResponse openRegion(RpcController controller, OpenRegionRequest request)
3517    throws ServiceException {
3518    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3519  }
3520
3521  @Override
3522  public WarmupRegionResponse warmupRegion(RpcController controller, WarmupRegionRequest request)
3523    throws ServiceException {
3524    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3525  }
3526
3527  @Override
3528  public CloseRegionResponse closeRegion(RpcController controller, CloseRegionRequest request)
3529    throws ServiceException {
3530    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3531  }
3532
3533  @Override
3534  public FlushRegionResponse flushRegion(RpcController controller, FlushRegionRequest request)
3535    throws ServiceException {
3536    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3537  }
3538
3539  @Override
3540  public CompactionSwitchResponse compactionSwitch(RpcController controller,
3541    CompactionSwitchRequest request) throws ServiceException {
3542    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3543  }
3544
3545  @Override
3546  public CompactRegionResponse compactRegion(RpcController controller, CompactRegionRequest request)
3547    throws ServiceException {
3548    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3549  }
3550
3551  @Override
3552  public ReplicateWALEntryResponse replicateWALEntry(RpcController controller,
3553    ReplicateWALEntryRequest request) throws ServiceException {
3554    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3555  }
3556
3557  @Override
3558  public ReplicateWALEntryResponse replay(RpcController controller,
3559    ReplicateWALEntryRequest request) throws ServiceException {
3560    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3561  }
3562
3563  @Override
3564  public RollWALWriterResponse rollWALWriter(RpcController controller, RollWALWriterRequest request)
3565    throws ServiceException {
3566    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3567  }
3568
3569  @Override
3570  public GetServerInfoResponse getServerInfo(RpcController controller, GetServerInfoRequest request)
3571    throws ServiceException {
3572    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3573  }
3574
3575  @Override
3576  public StopServerResponse stopServer(RpcController controller, StopServerRequest request)
3577    throws ServiceException {
3578    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3579  }
3580
3581  @Override
3582  public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,
3583    UpdateFavoredNodesRequest request) throws ServiceException {
3584    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3585  }
3586
3587  @Override
3588  public GetRegionLoadResponse getRegionLoad(RpcController controller, GetRegionLoadRequest request)
3589    throws ServiceException {
3590    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3591  }
3592
3593  @Override
3594  public ClearCompactionQueuesResponse clearCompactionQueues(RpcController controller,
3595    ClearCompactionQueuesRequest request) throws ServiceException {
3596    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3597  }
3598
3599  @Override
3600  public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,
3601    ClearRegionBlockCacheRequest request) throws ServiceException {
3602    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3603  }
3604
3605  @Override
3606  public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(RpcController controller,
3607    GetSpaceQuotaSnapshotsRequest request) throws ServiceException {
3608    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3609  }
3610
3611  @Override
3612  public ExecuteProceduresResponse executeProcedures(RpcController controller,
3613    ExecuteProceduresRequest request) throws ServiceException {
3614    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3615  }
3616
3617  @Override
3618  public GetCachedFilesListResponse getCachedFilesList(RpcController controller,
3619    GetCachedFilesListRequest request) throws ServiceException {
3620    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3621  }
3622
3623  @Override
3624  public GetLiveRegionServersResponse getLiveRegionServers(RpcController controller,
3625    GetLiveRegionServersRequest request) throws ServiceException {
3626    List<ServerName> regionServers = new ArrayList<>(server.getLiveRegionServers());
3627    Collections.shuffle(regionServers, ThreadLocalRandom.current());
3628    GetLiveRegionServersResponse.Builder builder =
3629      GetLiveRegionServersResponse.newBuilder().setTotal(regionServers.size());
3630    regionServers.stream().limit(request.getCount()).map(ProtobufUtil::toServerName)
3631      .forEach(builder::addServer);
3632    return builder.build();
3633  }
3634
3635  @Override
3636  public ReplicateWALEntryResponse replicateToReplica(RpcController controller,
3637    ReplicateWALEntryRequest request) throws ServiceException {
3638    throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3639  }
3640
3641  @Override
3642  public FlushMasterStoreResponse flushMasterStore(RpcController controller,
3643    FlushMasterStoreRequest request) throws ServiceException {
3644    rpcPreCheck("flushMasterStore");
3645    try {
3646      server.flushMasterStore();
3647    } catch (IOException ioe) {
3648      throw new ServiceException(ioe);
3649    }
3650    return FlushMasterStoreResponse.newBuilder().build();
3651  }
3652
3653  @Override
3654  public FlushTableResponse flushTable(RpcController controller, FlushTableRequest req)
3655    throws ServiceException {
3656    TableName tableName = ProtobufUtil.toTableName(req.getTableName());
3657    List<byte[]> columnFamilies = req.getColumnFamilyCount() > 0
3658      ? req.getColumnFamilyList().stream().filter(cf -> !cf.isEmpty()).map(ByteString::toByteArray)
3659        .collect(Collectors.toList())
3660      : null;
3661    try {
3662      long procId =
3663        server.flushTable(tableName, columnFamilies, req.getNonceGroup(), req.getNonce());
3664      return FlushTableResponse.newBuilder().setProcId(procId).build();
3665    } catch (IOException ioe) {
3666      throw new ServiceException(ioe);
3667    }
3668  }
3669
3670  @Override
3671  public RollAllWALWritersResponse rollAllWALWriters(RpcController rpcController,
3672    RollAllWALWritersRequest request) throws ServiceException {
3673    try {
3674      long procId = server.rollAllWALWriters(request.getNonceGroup(), request.getNonce());
3675      return RollAllWALWritersResponse.newBuilder().setProcId(procId).build();
3676    } catch (IOException ioe) {
3677      throw new ServiceException(ioe);
3678    }
3679  }
3680}