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