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