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