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