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.net.BindException;
025import java.net.InetAddress;
026import java.net.InetSocketAddress;
027import java.util.ArrayList;
028import java.util.HashMap;
029import java.util.HashSet;
030import java.util.List;
031import java.util.Map;
032import java.util.Map.Entry;
033import java.util.Set;
034import java.util.function.BiFunction;
035import java.util.stream.Collectors;
036import org.apache.hadoop.conf.Configuration;
037import org.apache.hadoop.fs.Path;
038import org.apache.hadoop.hbase.ClusterMetricsBuilder;
039import org.apache.hadoop.hbase.DoNotRetryIOException;
040import org.apache.hadoop.hbase.HConstants;
041import org.apache.hadoop.hbase.MetaTableAccessor;
042import org.apache.hadoop.hbase.NamespaceDescriptor;
043import org.apache.hadoop.hbase.Server;
044import org.apache.hadoop.hbase.ServerMetrics;
045import org.apache.hadoop.hbase.ServerMetricsBuilder;
046import org.apache.hadoop.hbase.ServerName;
047import org.apache.hadoop.hbase.TableName;
048import org.apache.hadoop.hbase.UnknownRegionException;
049import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
050import org.apache.hadoop.hbase.client.MasterSwitchType;
051import org.apache.hadoop.hbase.client.RegionInfo;
052import org.apache.hadoop.hbase.client.RegionInfoBuilder;
053import org.apache.hadoop.hbase.client.TableDescriptor;
054import org.apache.hadoop.hbase.client.TableState;
055import org.apache.hadoop.hbase.client.VersionInfoUtil;
056import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
057import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
058import org.apache.hadoop.hbase.errorhandling.ForeignException;
059import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
060import org.apache.hadoop.hbase.io.hfile.HFile;
061import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
062import org.apache.hadoop.hbase.ipc.PriorityFunction;
063import org.apache.hadoop.hbase.ipc.QosPriority;
064import org.apache.hadoop.hbase.ipc.RpcServer;
065import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
066import org.apache.hadoop.hbase.ipc.RpcServerFactory;
067import org.apache.hadoop.hbase.ipc.RpcServerInterface;
068import org.apache.hadoop.hbase.ipc.ServerRpcController;
069import org.apache.hadoop.hbase.master.assignment.RegionStates;
070import org.apache.hadoop.hbase.master.locking.LockProcedure;
071import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
072import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
073import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
074import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
075import org.apache.hadoop.hbase.mob.MobUtils;
076import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
077import org.apache.hadoop.hbase.procedure2.LockType;
078import org.apache.hadoop.hbase.procedure2.LockedResource;
079import org.apache.hadoop.hbase.procedure2.Procedure;
080import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
081import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
082import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
083import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
084import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
085import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
086import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
087import org.apache.hadoop.hbase.quotas.QuotaUtil;
088import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
089import org.apache.hadoop.hbase.regionserver.RSRpcServices;
090import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
091import org.apache.hadoop.hbase.replication.ReplicationException;
092import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
093import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
094import org.apache.hadoop.hbase.security.User;
095import org.apache.hadoop.hbase.security.access.AccessChecker;
096import org.apache.hadoop.hbase.security.access.AccessController;
097import org.apache.hadoop.hbase.security.access.Permission;
098import org.apache.hadoop.hbase.security.visibility.VisibilityController;
099import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
100import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
101import org.apache.hadoop.hbase.util.Bytes;
102import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
103import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
104import org.apache.hadoop.hbase.util.Pair;
105import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
106import org.apache.yetus.audience.InterfaceAudience;
107import org.apache.zookeeper.KeeperException;
108import org.slf4j.Logger;
109import org.slf4j.LoggerFactory;
110
111import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
112import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
113import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
114
115import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
116import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
117import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
118import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
119import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
120import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
121import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
122import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
123import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
124import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
125import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
126import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
127import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
128import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo;
129import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
130import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
131import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
132import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
133import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
134import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
135import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
136import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
137import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
138import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
139import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
140import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
141import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
142import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
143import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
144import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse;
145import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
146import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
147import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
148import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
149import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersRequest;
150import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersResponse;
151import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
152import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
153import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
154import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
155import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
156import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
157import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
158import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
159import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
160import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
161import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
162import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
163import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
164import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
165import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
166import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
167import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
168import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
169import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
170import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
171import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
172import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
173import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
174import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
175import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
176import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
177import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
178import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
179import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
180import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
181import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
182import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
183import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
184import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
185import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
186import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
187import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
188import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
189import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService;
190import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
191import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
192import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
193import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
194import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
195import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse;
196import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
197import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
198import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
199import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
200import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
201import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
202import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
203import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
204import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
205import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
206import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
207import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
208import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest;
209import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse;
210import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
211import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
212import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
213import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
214import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
215import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
216import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
217import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
218import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
219import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
220import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
221import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
222import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
223import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
224import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
225import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
226import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
227import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
228import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
229import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
230import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
231import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
232import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
233import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
234import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest;
235import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse;
236import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
237import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
238import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
239import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
240import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
241import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
242import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreRequest;
243import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreResponse;
244import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
245import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
246import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
247import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
248import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
249import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse;
250import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
251import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
252import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
253import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
254import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
255import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
256import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest;
257import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
258import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
259import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
260import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
261import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
262import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse;
263import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
264import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse;
265import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
266import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
267import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
268import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
269import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
270import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
271import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot;
272import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot;
273import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
274import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
275import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
276import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationRequest;
277import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationResponse;
278import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
279import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
280import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
281import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse;
282import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
283import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
284import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
285import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
286import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
287import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse;
288import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult;
289import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
290import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse;
291import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
292import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
293import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
294import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
295import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
296import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
297import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
298import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
299import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest;
300import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
301import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
302import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
303import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
304import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
305import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
306import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
307import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState;
308import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
309import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
310import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
311
312/**
313 * Implements the master RPC services.
314 */
315@InterfaceAudience.Private
316@SuppressWarnings("deprecation")
317public class MasterRpcServices extends RSRpcServices
318      implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
319        LockService.BlockingInterface, HbckService.BlockingInterface {
320  private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
321
322  private final HMaster master;
323
324  /**
325   * @return Subset of configuration to pass initializing regionservers: e.g.
326   * the filesystem to use and root directory to use.
327   */
328  private RegionServerStartupResponse.Builder createConfigurationSubset() {
329    RegionServerStartupResponse.Builder resp = addConfig(
330      RegionServerStartupResponse.newBuilder(), HConstants.HBASE_DIR);
331    resp = addConfig(resp, "fs.defaultFS");
332    return addConfig(resp, "hbase.master.info.port");
333  }
334
335  private RegionServerStartupResponse.Builder addConfig(
336      final RegionServerStartupResponse.Builder resp, final String key) {
337    NameStringPair.Builder entry = NameStringPair.newBuilder()
338      .setName(key)
339      .setValue(master.getConfiguration().get(key));
340    resp.addMapEntries(entry.build());
341    return resp;
342  }
343
344  public MasterRpcServices(HMaster m) throws IOException {
345    super(m);
346    master = m;
347  }
348
349  @Override
350  protected Class<?> getRpcSchedulerFactoryClass() {
351    Configuration conf = getConfiguration();
352    if (conf != null) {
353      return conf.getClass(MASTER_RPC_SCHEDULER_FACTORY_CLASS, super.getRpcSchedulerFactoryClass());
354    } else {
355      return super.getRpcSchedulerFactoryClass();
356    }
357  }
358
359  @Override
360  protected RpcServerInterface createRpcServer(Server server, Configuration conf,
361      RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name)
362      throws IOException {
363    // RpcServer at HM by default enable ByteBufferPool iff HM having user table region in it
364    boolean reservoirEnabled = conf.getBoolean(RESERVOIR_ENABLED_KEY,
365        LoadBalancer.isMasterCanHostUserRegions(conf));
366    try {
367      return RpcServerFactory.createRpcServer(server, name, getServices(),
368          bindAddress, // use final bindAddress for this server.
369          conf, rpcSchedulerFactory.create(conf, this, server), reservoirEnabled);
370    } catch (BindException be) {
371      throw new IOException(be.getMessage() + ". To switch ports use the '"
372          + HConstants.MASTER_PORT + "' configuration property.",
373          be.getCause() != null ? be.getCause() : be);
374    }
375  }
376
377  @Override
378  protected PriorityFunction createPriority() {
379    return new MasterAnnotationReadingPriorityFunction(this);
380  }
381
382  /**
383   * Checks for the following pre-checks in order:
384   * <ol>
385   *   <li>Master is initialized</li>
386   *   <li>Rpc caller has admin permissions</li>
387   * </ol>
388   * @param requestName name of rpc request. Used in reporting failures to provide context.
389   * @throws ServiceException If any of the above listed pre-check fails.
390   */
391  private void rpcPreCheck(String requestName) throws ServiceException {
392    try {
393      master.checkInitialized();
394      requirePermission(requestName, Permission.Action.ADMIN);
395    } catch (IOException ioe) {
396      throw new ServiceException(ioe);
397    }
398  }
399
400  enum BalanceSwitchMode {
401    SYNC,
402    ASYNC
403  }
404
405  /**
406   * Assigns balancer switch according to BalanceSwitchMode
407   * @param b new balancer switch
408   * @param mode BalanceSwitchMode
409   * @return old balancer switch
410   */
411  boolean switchBalancer(final boolean b, BalanceSwitchMode mode) throws IOException {
412    boolean oldValue = master.loadBalancerTracker.isBalancerOn();
413    boolean newValue = b;
414    try {
415      if (master.cpHost != null) {
416        master.cpHost.preBalanceSwitch(newValue);
417      }
418      try {
419        if (mode == BalanceSwitchMode.SYNC) {
420          synchronized (master.getLoadBalancer()) {
421            master.loadBalancerTracker.setBalancerOn(newValue);
422          }
423        } else {
424          master.loadBalancerTracker.setBalancerOn(newValue);
425        }
426      } catch (KeeperException ke) {
427        throw new IOException(ke);
428      }
429      LOG.info(master.getClientIdAuditPrefix() + " set balanceSwitch=" + newValue);
430      if (master.cpHost != null) {
431        master.cpHost.postBalanceSwitch(oldValue, newValue);
432      }
433      master.getLoadBalancer().updateBalancerStatus(newValue);
434    } catch (IOException ioe) {
435      LOG.warn("Error flipping balance switch", ioe);
436    }
437    return oldValue;
438  }
439
440  boolean synchronousBalanceSwitch(final boolean b) throws IOException {
441    return switchBalancer(b, BalanceSwitchMode.SYNC);
442  }
443
444  /**
445   * @return list of blocking services and their security info classes that this server supports
446   */
447  @Override
448  protected List<BlockingServiceAndInterface> getServices() {
449    List<BlockingServiceAndInterface> bssi = new ArrayList<>(5);
450    bssi.add(new BlockingServiceAndInterface(
451      MasterService.newReflectiveBlockingService(this),
452      MasterService.BlockingInterface.class));
453    bssi.add(new BlockingServiceAndInterface(
454      RegionServerStatusService.newReflectiveBlockingService(this),
455      RegionServerStatusService.BlockingInterface.class));
456    bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
457        LockService.BlockingInterface.class));
458    bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this),
459        HbckService.BlockingInterface.class));
460    bssi.addAll(super.getServices());
461    return bssi;
462  }
463
464  @Override
465  @QosPriority(priority = HConstants.ADMIN_QOS)
466  public GetLastFlushedSequenceIdResponse getLastFlushedSequenceId(RpcController controller,
467      GetLastFlushedSequenceIdRequest request) throws ServiceException {
468    try {
469      master.checkServiceStarted();
470    } catch (IOException ioe) {
471      throw new ServiceException(ioe);
472    }
473    byte[] encodedRegionName = request.getRegionName().toByteArray();
474    RegionStoreSequenceIds ids = master.getServerManager()
475      .getLastFlushedSequenceId(encodedRegionName);
476    return ResponseConverter.buildGetLastFlushedSequenceIdResponse(ids);
477  }
478
479  @Override
480  public RegionServerReportResponse regionServerReport(RpcController controller,
481      RegionServerReportRequest request) throws ServiceException {
482    try {
483      master.checkServiceStarted();
484      int versionNumber = 0;
485      String version = "0.0.0";
486      VersionInfo versionInfo = VersionInfoUtil.getCurrentClientVersionInfo();
487      if (versionInfo != null) {
488        version = versionInfo.getVersion();
489        versionNumber = VersionInfoUtil.getVersionNumber(versionInfo);
490      }
491      ClusterStatusProtos.ServerLoad sl = request.getLoad();
492      ServerName serverName = ProtobufUtil.toServerName(request.getServer());
493      ServerMetrics oldLoad = master.getServerManager().getLoad(serverName);
494      ServerMetrics newLoad =
495        ServerMetricsBuilder.toServerMetrics(serverName, versionNumber, version, sl);
496      master.getServerManager().regionServerReport(serverName, newLoad);
497      master.getAssignmentManager().reportOnlineRegions(serverName,
498        newLoad.getRegionMetrics().keySet());
499      if (sl != null && master.metricsMaster != null) {
500        // Up our metrics.
501        master.metricsMaster.incrementRequests(
502          sl.getTotalNumberOfRequests() - (oldLoad != null ? oldLoad.getRequestCount() : 0));
503      }
504    } catch (IOException ioe) {
505      throw new ServiceException(ioe);
506    }
507    return RegionServerReportResponse.newBuilder().build();
508  }
509
510  @Override
511  public RegionServerStartupResponse regionServerStartup(RpcController controller,
512      RegionServerStartupRequest request) throws ServiceException {
513    // Register with server manager
514    try {
515      master.checkServiceStarted();
516      int versionNumber = 0;
517      String version = "0.0.0";
518      VersionInfo versionInfo = VersionInfoUtil.getCurrentClientVersionInfo();
519      if (versionInfo != null) {
520        version = versionInfo.getVersion();
521        versionNumber = VersionInfoUtil.getVersionNumber(versionInfo);
522      }
523      InetAddress ia = master.getRemoteInetAddress(request.getPort(), request.getServerStartCode());
524      // if regionserver passed hostname to use,
525      // then use it instead of doing a reverse DNS lookup
526      ServerName rs =
527        master.getServerManager().regionServerStartup(request, versionNumber, version, ia);
528
529      // Send back some config info
530      RegionServerStartupResponse.Builder resp = createConfigurationSubset();
531      NameStringPair.Builder entry = NameStringPair.newBuilder()
532        .setName(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER).setValue(rs.getHostname());
533      resp.addMapEntries(entry.build());
534
535      return resp.build();
536    } catch (IOException ioe) {
537      throw new ServiceException(ioe);
538    }
539  }
540
541  @Override
542  public ReportRSFatalErrorResponse reportRSFatalError(
543      RpcController controller, ReportRSFatalErrorRequest request) throws ServiceException {
544    String errorText = request.getErrorMessage();
545    ServerName sn = ProtobufUtil.toServerName(request.getServer());
546    String msg = sn + " reported a fatal error:\n" + errorText;
547    LOG.warn(msg);
548    master.rsFatals.add(msg);
549    return ReportRSFatalErrorResponse.newBuilder().build();
550  }
551
552  @Override
553  public AddColumnResponse addColumn(RpcController controller,
554      AddColumnRequest req) throws ServiceException {
555    try {
556      long procId = master.addColumn(
557          ProtobufUtil.toTableName(req.getTableName()),
558          ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()),
559          req.getNonceGroup(),
560          req.getNonce());
561      if (procId == -1) {
562        // This mean operation was not performed in server, so do not set any procId
563        return AddColumnResponse.newBuilder().build();
564      } else {
565        return AddColumnResponse.newBuilder().setProcId(procId).build();
566      }
567    } catch (IOException ioe) {
568      throw new ServiceException(ioe);
569    }
570  }
571
572  @Override
573  public AssignRegionResponse assignRegion(RpcController controller,
574      AssignRegionRequest req) throws ServiceException {
575    try {
576      master.checkInitialized();
577
578      final RegionSpecifierType type = req.getRegion().getType();
579      if (type != RegionSpecifierType.REGION_NAME) {
580        LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
581          + " actual: " + type);
582      }
583
584      final byte[] regionName = req.getRegion().getValue().toByteArray();
585      final RegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName);
586      if (regionInfo == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
587
588      final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
589      if (master.cpHost != null) {
590        master.cpHost.preAssign(regionInfo);
591      }
592      LOG.info(master.getClientIdAuditPrefix() + " assign " + regionInfo.getRegionNameAsString());
593      master.getAssignmentManager().assign(regionInfo);
594      if (master.cpHost != null) {
595        master.cpHost.postAssign(regionInfo);
596      }
597      return arr;
598    } catch (IOException ioe) {
599      throw new ServiceException(ioe);
600    }
601  }
602
603
604  @Override
605  public BalanceResponse balance(RpcController controller,
606      BalanceRequest request) throws ServiceException {
607    try {
608      return BalanceResponse.newBuilder().setBalancerRan(master.balance(
609        request.hasForce()? request.getForce(): false)).build();
610    } catch (IOException ex) {
611      throw new ServiceException(ex);
612    }
613  }
614
615  @Override
616  public CreateNamespaceResponse createNamespace(RpcController controller,
617     CreateNamespaceRequest request) throws ServiceException {
618    try {
619      long procId = master.createNamespace(
620        ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
621        request.getNonceGroup(),
622        request.getNonce());
623      return CreateNamespaceResponse.newBuilder().setProcId(procId).build();
624    } catch (IOException e) {
625      throw new ServiceException(e);
626    }
627  }
628
629  @Override
630  public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
631  throws ServiceException {
632    TableDescriptor tableDescriptor = ProtobufUtil.toTableDescriptor(req.getTableSchema());
633    byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
634    try {
635      long procId =
636          master.createTable(tableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce());
637      LOG.info(master.getClientIdAuditPrefix() + " procedure request for creating table: " +
638              req.getTableSchema().getTableName() + " procId is: " + procId);
639      return CreateTableResponse.newBuilder().setProcId(procId).build();
640    } catch (IOException ioe) {
641      throw new ServiceException(ioe);
642    }
643  }
644
645  @Override
646  public DeleteColumnResponse deleteColumn(RpcController controller,
647      DeleteColumnRequest req) throws ServiceException {
648    try {
649      long procId = master.deleteColumn(
650        ProtobufUtil.toTableName(req.getTableName()),
651        req.getColumnName().toByteArray(),
652        req.getNonceGroup(),
653        req.getNonce());
654      if (procId == -1) {
655        // This mean operation was not performed in server, so do not set any procId
656        return DeleteColumnResponse.newBuilder().build();
657      } else {
658        return DeleteColumnResponse.newBuilder().setProcId(procId).build();
659      }
660    } catch (IOException ioe) {
661      throw new ServiceException(ioe);
662    }
663  }
664
665  @Override
666  public DeleteNamespaceResponse deleteNamespace(RpcController controller,
667      DeleteNamespaceRequest request) throws ServiceException {
668    try {
669      long procId = master.deleteNamespace(
670        request.getNamespaceName(),
671        request.getNonceGroup(),
672        request.getNonce());
673      return DeleteNamespaceResponse.newBuilder().setProcId(procId).build();
674    } catch (IOException e) {
675      throw new ServiceException(e);
676    }
677  }
678
679  /**
680   * Execute Delete Snapshot operation.
681   * @return DeleteSnapshotResponse (a protobuf wrapped void) if the snapshot existed and was
682   *    deleted properly.
683   * @throws ServiceException wrapping SnapshotDoesNotExistException if specified snapshot did not
684   *    exist.
685   */
686  @Override
687  public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
688      DeleteSnapshotRequest request) throws ServiceException {
689    try {
690      master.checkInitialized();
691      master.snapshotManager.checkSnapshotSupport();
692
693      LOG.info(master.getClientIdAuditPrefix() + " delete " + request.getSnapshot());
694      master.snapshotManager.deleteSnapshot(request.getSnapshot());
695      return DeleteSnapshotResponse.newBuilder().build();
696    } catch (IOException e) {
697      throw new ServiceException(e);
698    }
699  }
700
701  @Override
702  public DeleteTableResponse deleteTable(RpcController controller,
703      DeleteTableRequest request) throws ServiceException {
704    try {
705      long procId = master.deleteTable(ProtobufUtil.toTableName(
706          request.getTableName()), request.getNonceGroup(), request.getNonce());
707      return DeleteTableResponse.newBuilder().setProcId(procId).build();
708    } catch (IOException ioe) {
709      throw new ServiceException(ioe);
710    }
711  }
712
713  @Override
714  public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
715      throws ServiceException {
716    try {
717      long procId = master.truncateTable(
718        ProtobufUtil.toTableName(request.getTableName()),
719        request.getPreserveSplits(),
720        request.getNonceGroup(),
721        request.getNonce());
722      return TruncateTableResponse.newBuilder().setProcId(procId).build();
723    } catch (IOException ioe) {
724      throw new ServiceException(ioe);
725    }
726  }
727
728  @Override
729  public DisableTableResponse disableTable(RpcController controller,
730      DisableTableRequest request) throws ServiceException {
731    try {
732      long procId = master.disableTable(
733        ProtobufUtil.toTableName(request.getTableName()),
734        request.getNonceGroup(),
735        request.getNonce());
736      return DisableTableResponse.newBuilder().setProcId(procId).build();
737    } catch (IOException ioe) {
738      throw new ServiceException(ioe);
739    }
740  }
741
742  @Override
743  public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
744      EnableCatalogJanitorRequest req) throws ServiceException {
745    rpcPreCheck("enableCatalogJanitor");
746    return EnableCatalogJanitorResponse.newBuilder().setPrevValue(
747      master.catalogJanitorChore.setEnabled(req.getEnable())).build();
748  }
749
750  @Override
751  public SetCleanerChoreRunningResponse setCleanerChoreRunning(
752    RpcController c, SetCleanerChoreRunningRequest req) throws ServiceException {
753    rpcPreCheck("setCleanerChoreRunning");
754
755    boolean prevValue =
756      master.getLogCleaner().getEnabled() && master.getHFileCleaner().getEnabled();
757    master.getLogCleaner().setEnabled(req.getOn());
758    master.getHFileCleaner().setEnabled(req.getOn());
759    return SetCleanerChoreRunningResponse.newBuilder().setPrevValue(prevValue).build();
760  }
761
762  @Override
763  public EnableTableResponse enableTable(RpcController controller,
764      EnableTableRequest request) throws ServiceException {
765    try {
766      long procId = master.enableTable(
767        ProtobufUtil.toTableName(request.getTableName()),
768        request.getNonceGroup(),
769        request.getNonce());
770      return EnableTableResponse.newBuilder().setProcId(procId).build();
771    } catch (IOException ioe) {
772      throw new ServiceException(ioe);
773    }
774  }
775
776  @Override
777  public MergeTableRegionsResponse mergeTableRegions(
778      RpcController c, MergeTableRegionsRequest request) throws ServiceException {
779    try {
780      master.checkInitialized();
781    } catch (IOException ioe) {
782      throw new ServiceException(ioe);
783    }
784
785    RegionStates regionStates = master.getAssignmentManager().getRegionStates();
786
787    assert(request.getRegionCount() == 2);
788    RegionInfo[] regionsToMerge = new RegionInfo[request.getRegionCount()];
789    for (int i = 0; i < request.getRegionCount(); i++) {
790      final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray();
791      if (request.getRegion(i).getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
792        LOG.warn("MergeRegions specifier type: expected: "
793          + RegionSpecifierType.ENCODED_REGION_NAME + " actual: region " + i + " ="
794          + request.getRegion(i).getType());
795      }
796      RegionState regionState = regionStates.getRegionState(Bytes.toString(encodedNameOfRegion));
797      if (regionState == null) {
798        throw new ServiceException(
799          new UnknownRegionException(Bytes.toStringBinary(encodedNameOfRegion)));
800      }
801      regionsToMerge[i] = regionState.getRegion();
802    }
803
804    try {
805      long procId = master.mergeRegions(
806        regionsToMerge,
807        request.getForcible(),
808        request.getNonceGroup(),
809        request.getNonce());
810      return MergeTableRegionsResponse.newBuilder().setProcId(procId).build();
811    } catch (IOException ioe) {
812      throw new ServiceException(ioe);
813    }
814  }
815
816  @Override
817  public SplitTableRegionResponse splitRegion(final RpcController controller,
818      final SplitTableRegionRequest request) throws ServiceException {
819    try {
820      long procId = master.splitRegion(
821        ProtobufUtil.toRegionInfo(request.getRegionInfo()),
822        request.hasSplitRow() ? request.getSplitRow().toByteArray() : null,
823        request.getNonceGroup(),
824        request.getNonce());
825      return SplitTableRegionResponse.newBuilder().setProcId(procId).build();
826    } catch (IOException ie) {
827      throw new ServiceException(ie);
828    }
829  }
830
831  @Override
832  public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
833      final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {
834    rpcPreCheck("execMasterService");
835    try {
836      ServerRpcController execController = new ServerRpcController();
837      ClientProtos.CoprocessorServiceCall call = request.getCall();
838      String serviceName = call.getServiceName();
839      String methodName = call.getMethodName();
840      if (!master.coprocessorServiceHandlers.containsKey(serviceName)) {
841        throw new UnknownProtocolException(null,
842          "No registered Master Coprocessor Endpoint found for " + serviceName +
843          ". Has it been enabled?");
844      }
845
846      com.google.protobuf.Service service = master.coprocessorServiceHandlers.get(serviceName);
847      com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
848      com.google.protobuf.Descriptors.MethodDescriptor methodDesc =
849          CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
850
851      com.google.protobuf.Message execRequest =
852          CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
853      final com.google.protobuf.Message.Builder responseBuilder =
854          service.getResponsePrototype(methodDesc).newBuilderForType();
855      service.callMethod(methodDesc, execController, execRequest,
856        (message) -> {
857          if (message != null) {
858            responseBuilder.mergeFrom(message);
859          }
860        });
861      com.google.protobuf.Message execResult = responseBuilder.build();
862      if (execController.getFailedOn() != null) {
863        throw execController.getFailedOn();
864      }
865      return CoprocessorRpcUtils.getResponse(execResult, HConstants.EMPTY_BYTE_ARRAY);
866    } catch (IOException ie) {
867      throw new ServiceException(ie);
868    }
869  }
870
871  /**
872   * Triggers an asynchronous attempt to run a distributed procedure.
873   * {@inheritDoc}
874   */
875  @Override
876  public ExecProcedureResponse execProcedure(RpcController controller,
877      ExecProcedureRequest request) throws ServiceException {
878    try {
879      master.checkInitialized();
880      ProcedureDescription desc = request.getProcedure();
881      MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
882        desc.getSignature());
883      if (mpm == null) {
884        throw new ServiceException(new DoNotRetryIOException("The procedure is not registered: "
885          + desc.getSignature()));
886      }
887      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
888      mpm.checkPermissions(desc, accessChecker, RpcServer.getRequestUser().orElse(null));
889      mpm.execProcedure(desc);
890      // send back the max amount of time the client should wait for the procedure
891      // to complete
892      long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME;
893      return ExecProcedureResponse.newBuilder().setExpectedTimeout(
894        waitTime).build();
895    } catch (ForeignException e) {
896      throw new ServiceException(e.getCause());
897    } catch (IOException e) {
898      throw new ServiceException(e);
899    }
900  }
901
902  /**
903   * Triggers a synchronous attempt to run a distributed procedure and sets
904   * return data in response.
905   * {@inheritDoc}
906   */
907  @Override
908  public ExecProcedureResponse execProcedureWithRet(RpcController controller,
909      ExecProcedureRequest request) throws ServiceException {
910    rpcPreCheck("execProcedureWithRet");
911    try {
912      ProcedureDescription desc = request.getProcedure();
913      MasterProcedureManager mpm =
914        master.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature());
915      if (mpm == null) {
916        throw new ServiceException("The procedure is not registered: " + desc.getSignature());
917      }
918      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
919      byte[] data = mpm.execProcedureWithRet(desc);
920      ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder();
921      // set return data if available
922      if (data != null) {
923        builder.setReturnData(UnsafeByteOperations.unsafeWrap(data));
924      }
925      return builder.build();
926    } catch (IOException e) {
927      throw new ServiceException(e);
928    }
929  }
930
931  @Override
932  public GetClusterStatusResponse getClusterStatus(RpcController controller,
933      GetClusterStatusRequest req) throws ServiceException {
934    GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
935    try {
936      // We used to check if Master was up at this point but let this call proceed even if
937      // Master is initializing... else we shut out stuff like hbck2 tool from making progress
938      // since it queries this method to figure cluster version. hbck2 wants to be able to work
939      // against Master even if it is 'initializing' so it can do fixup.
940      response.setClusterStatus(ClusterMetricsBuilder.toClusterStatus(
941        master.getClusterMetrics(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
942    } catch (IOException e) {
943      throw new ServiceException(e);
944    }
945    return response.build();
946  }
947
948  /**
949   * List the currently available/stored snapshots. Any in-progress snapshots are ignored
950   */
951  @Override
952  public GetCompletedSnapshotsResponse getCompletedSnapshots(RpcController controller,
953      GetCompletedSnapshotsRequest request) throws ServiceException {
954    try {
955      master.checkInitialized();
956      GetCompletedSnapshotsResponse.Builder builder = GetCompletedSnapshotsResponse.newBuilder();
957      List<SnapshotDescription> snapshots = master.snapshotManager.getCompletedSnapshots();
958
959      // convert to protobuf
960      for (SnapshotDescription snapshot : snapshots) {
961        builder.addSnapshots(snapshot);
962      }
963      return builder.build();
964    } catch (IOException e) {
965      throw new ServiceException(e);
966    }
967  }
968
969  @Override
970  public GetNamespaceDescriptorResponse getNamespaceDescriptor(
971      RpcController controller, GetNamespaceDescriptorRequest request)
972      throws ServiceException {
973    try {
974      return GetNamespaceDescriptorResponse.newBuilder()
975        .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
976            master.getNamespace(request.getNamespaceName())))
977        .build();
978    } catch (IOException e) {
979      throw new ServiceException(e);
980    }
981  }
982
983  /**
984   * Get the number of regions of the table that have been updated by the alter.
985   *
986   * @return Pair indicating the number of regions updated Pair.getFirst is the
987   *         regions that are yet to be updated Pair.getSecond is the total number
988   *         of regions of the table
989   * @throws ServiceException
990   */
991  @Override
992  public GetSchemaAlterStatusResponse getSchemaAlterStatus(
993      RpcController controller, GetSchemaAlterStatusRequest req) throws ServiceException {
994    // TODO: currently, we query using the table name on the client side. this
995    // may overlap with other table operations or the table operation may
996    // have completed before querying this API. We need to refactor to a
997    // transaction system in the future to avoid these ambiguities.
998    TableName tableName = ProtobufUtil.toTableName(req.getTableName());
999
1000    try {
1001      master.checkInitialized();
1002      Pair<Integer,Integer> pair = master.getAssignmentManager().getReopenStatus(tableName);
1003      GetSchemaAlterStatusResponse.Builder ret = GetSchemaAlterStatusResponse.newBuilder();
1004      ret.setYetToUpdateRegions(pair.getFirst());
1005      ret.setTotalRegions(pair.getSecond());
1006      return ret.build();
1007    } catch (IOException ioe) {
1008      throw new ServiceException(ioe);
1009    }
1010  }
1011
1012  /**
1013   * Get list of TableDescriptors for requested tables.
1014   * @param c Unused (set to null).
1015   * @param req GetTableDescriptorsRequest that contains:
1016   * - tableNames: requested tables, or if empty, all are requested
1017   * @return GetTableDescriptorsResponse
1018   * @throws ServiceException
1019   */
1020  @Override
1021  public GetTableDescriptorsResponse getTableDescriptors(RpcController c,
1022      GetTableDescriptorsRequest req) throws ServiceException {
1023    try {
1024      master.checkInitialized();
1025
1026      final String regex = req.hasRegex() ? req.getRegex() : null;
1027      final String namespace = req.hasNamespace() ? req.getNamespace() : null;
1028      List<TableName> tableNameList = null;
1029      if (req.getTableNamesCount() > 0) {
1030        tableNameList = new ArrayList<TableName>(req.getTableNamesCount());
1031        for (HBaseProtos.TableName tableNamePB: req.getTableNamesList()) {
1032          tableNameList.add(ProtobufUtil.toTableName(tableNamePB));
1033        }
1034      }
1035
1036      List<TableDescriptor> descriptors = master.listTableDescriptors(namespace, regex,
1037          tableNameList, req.getIncludeSysTables());
1038
1039      GetTableDescriptorsResponse.Builder builder = GetTableDescriptorsResponse.newBuilder();
1040      if (descriptors != null && descriptors.size() > 0) {
1041        // Add the table descriptors to the response
1042        for (TableDescriptor htd: descriptors) {
1043          builder.addTableSchema(ProtobufUtil.toTableSchema(htd));
1044        }
1045      }
1046      return builder.build();
1047    } catch (IOException ioe) {
1048      throw new ServiceException(ioe);
1049    }
1050  }
1051
1052  /**
1053   * Get list of userspace table names
1054   * @param controller Unused (set to null).
1055   * @param req GetTableNamesRequest
1056   * @return GetTableNamesResponse
1057   * @throws ServiceException
1058   */
1059  @Override
1060  public GetTableNamesResponse getTableNames(RpcController controller,
1061      GetTableNamesRequest req) throws ServiceException {
1062    try {
1063      master.checkServiceStarted();
1064
1065      final String regex = req.hasRegex() ? req.getRegex() : null;
1066      final String namespace = req.hasNamespace() ? req.getNamespace() : null;
1067      List<TableName> tableNames = master.listTableNames(namespace, regex,
1068          req.getIncludeSysTables());
1069
1070      GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder();
1071      if (tableNames != null && tableNames.size() > 0) {
1072        // Add the table names to the response
1073        for (TableName table: tableNames) {
1074          builder.addTableNames(ProtobufUtil.toProtoTableName(table));
1075        }
1076      }
1077      return builder.build();
1078    } catch (IOException e) {
1079      throw new ServiceException(e);
1080    }
1081  }
1082
1083  @Override
1084  public GetTableStateResponse getTableState(RpcController controller,
1085      GetTableStateRequest request) throws ServiceException {
1086    try {
1087      master.checkServiceStarted();
1088      TableName tableName = ProtobufUtil.toTableName(request.getTableName());
1089      TableState ts = master.getTableStateManager().getTableState(tableName);
1090      GetTableStateResponse.Builder builder = GetTableStateResponse.newBuilder();
1091      builder.setTableState(ts.convert());
1092      return builder.build();
1093    } catch (IOException e) {
1094      throw new ServiceException(e);
1095    }
1096  }
1097
1098  @Override
1099  public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c,
1100      IsCatalogJanitorEnabledRequest req) throws ServiceException {
1101    return IsCatalogJanitorEnabledResponse.newBuilder().setValue(
1102      master.isCatalogJanitorEnabled()).build();
1103  }
1104
1105  @Override
1106  public IsCleanerChoreEnabledResponse isCleanerChoreEnabled(RpcController c,
1107                                                             IsCleanerChoreEnabledRequest req)
1108    throws ServiceException {
1109    return IsCleanerChoreEnabledResponse.newBuilder().setValue(master.isCleanerChoreEnabled())
1110                                        .build();
1111  }
1112
1113  @Override
1114  public IsMasterRunningResponse isMasterRunning(RpcController c,
1115      IsMasterRunningRequest req) throws ServiceException {
1116    try {
1117      master.checkServiceStarted();
1118      return IsMasterRunningResponse.newBuilder().setIsMasterRunning(
1119        !master.isStopped()).build();
1120    } catch (IOException e) {
1121      throw new ServiceException(e);
1122    }
1123  }
1124
1125  /**
1126   * Checks if the specified procedure is done.
1127   * @return true if the procedure is done, false if the procedure is in the process of completing
1128   * @throws ServiceException if invalid procedure or failed procedure with progress failure reason.
1129   */
1130  @Override
1131  public IsProcedureDoneResponse isProcedureDone(RpcController controller,
1132      IsProcedureDoneRequest request) throws ServiceException {
1133    try {
1134      master.checkInitialized();
1135      ProcedureDescription desc = request.getProcedure();
1136      MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
1137        desc.getSignature());
1138      if (mpm == null) {
1139        throw new ServiceException("The procedure is not registered: "
1140          + desc.getSignature());
1141      }
1142      LOG.debug("Checking to see if procedure from request:"
1143        + desc.getSignature() + " is done");
1144
1145      IsProcedureDoneResponse.Builder builder =
1146        IsProcedureDoneResponse.newBuilder();
1147      boolean done = mpm.isProcedureDone(desc);
1148      builder.setDone(done);
1149      return builder.build();
1150    } catch (ForeignException e) {
1151      throw new ServiceException(e.getCause());
1152    } catch (IOException e) {
1153      throw new ServiceException(e);
1154    }
1155  }
1156
1157  /**
1158   * Checks if the specified snapshot is done.
1159   * @return true if the snapshot is in file system ready to use,
1160   *   false if the snapshot is in the process of completing
1161   * @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or
1162   *  a wrapped HBaseSnapshotException with progress failure reason.
1163   */
1164  @Override
1165  public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
1166      IsSnapshotDoneRequest request) throws ServiceException {
1167    LOG.debug("Checking to see if snapshot from request:" +
1168      ClientSnapshotDescriptionUtils.toString(request.getSnapshot()) + " is done");
1169    try {
1170      master.checkInitialized();
1171      IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
1172      boolean done = master.snapshotManager.isSnapshotDone(request.getSnapshot());
1173      builder.setDone(done);
1174      return builder.build();
1175    } catch (ForeignException e) {
1176      throw new ServiceException(e.getCause());
1177    } catch (IOException e) {
1178      throw new ServiceException(e);
1179    }
1180  }
1181
1182  @Override
1183  public GetProcedureResultResponse getProcedureResult(RpcController controller,
1184      GetProcedureResultRequest request) throws ServiceException {
1185    LOG.debug("Checking to see if procedure is done pid=" + request.getProcId());
1186    try {
1187      master.checkInitialized();
1188      GetProcedureResultResponse.Builder builder = GetProcedureResultResponse.newBuilder();
1189      long procId = request.getProcId();
1190      ProcedureExecutor<?> executor = master.getMasterProcedureExecutor();
1191      Procedure<?> result = executor.getResultOrProcedure(procId);
1192      if (result != null) {
1193        builder.setSubmittedTime(result.getSubmittedTime());
1194        builder.setLastUpdate(result.getLastUpdate());
1195        if (executor.isFinished(procId)) {
1196          builder.setState(GetProcedureResultResponse.State.FINISHED);
1197          if (result.isFailed()) {
1198            IOException exception =
1199                MasterProcedureUtil.unwrapRemoteIOException(result);
1200            builder.setException(ForeignExceptionUtil.toProtoForeignException(exception));
1201          }
1202          byte[] resultData = result.getResult();
1203          if (resultData != null) {
1204            builder.setResult(UnsafeByteOperations.unsafeWrap(resultData));
1205          }
1206          master.getMasterProcedureExecutor().removeResult(request.getProcId());
1207        } else {
1208          builder.setState(GetProcedureResultResponse.State.RUNNING);
1209        }
1210      } else {
1211        builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
1212      }
1213      return builder.build();
1214    } catch (IOException e) {
1215      throw new ServiceException(e);
1216    }
1217  }
1218
1219  @Override
1220  public AbortProcedureResponse abortProcedure(
1221      RpcController rpcController, AbortProcedureRequest request) throws ServiceException {
1222    try {
1223      AbortProcedureResponse.Builder response = AbortProcedureResponse.newBuilder();
1224      boolean abortResult =
1225          master.abortProcedure(request.getProcId(), request.getMayInterruptIfRunning());
1226      response.setIsProcedureAborted(abortResult);
1227      return response.build();
1228    } catch (IOException e) {
1229      throw new ServiceException(e);
1230    }
1231  }
1232
1233  @Override
1234  public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController c,
1235      ListNamespaceDescriptorsRequest request) throws ServiceException {
1236    try {
1237      ListNamespaceDescriptorsResponse.Builder response =
1238        ListNamespaceDescriptorsResponse.newBuilder();
1239      for(NamespaceDescriptor ns: master.getNamespaces()) {
1240        response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
1241      }
1242      return response.build();
1243    } catch (IOException e) {
1244      throw new ServiceException(e);
1245    }
1246  }
1247
1248  @Override
1249  public GetProceduresResponse getProcedures(
1250      RpcController rpcController,
1251      GetProceduresRequest request) throws ServiceException {
1252    try {
1253      final GetProceduresResponse.Builder response = GetProceduresResponse.newBuilder();
1254      for (Procedure<?> p: master.getProcedures()) {
1255        response.addProcedure(ProcedureUtil.convertToProtoProcedure(p));
1256      }
1257      return response.build();
1258    } catch (IOException e) {
1259      throw new ServiceException(e);
1260    }
1261  }
1262
1263  @Override
1264  public GetLocksResponse getLocks(
1265      RpcController controller,
1266      GetLocksRequest request) throws ServiceException {
1267    try {
1268      final GetLocksResponse.Builder builder = GetLocksResponse.newBuilder();
1269
1270      for (LockedResource lockedResource: master.getLocks()) {
1271        builder.addLock(ProcedureUtil.convertToProtoLockedResource(lockedResource));
1272      }
1273
1274      return builder.build();
1275    } catch (IOException e) {
1276      throw new ServiceException(e);
1277    }
1278  }
1279
1280  @Override
1281  public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(RpcController c,
1282      ListTableDescriptorsByNamespaceRequest request) throws ServiceException {
1283    try {
1284      ListTableDescriptorsByNamespaceResponse.Builder b =
1285          ListTableDescriptorsByNamespaceResponse.newBuilder();
1286      for (TableDescriptor htd : master
1287          .listTableDescriptorsByNamespace(request.getNamespaceName())) {
1288        b.addTableSchema(ProtobufUtil.toTableSchema(htd));
1289      }
1290      return b.build();
1291    } catch (IOException e) {
1292      throw new ServiceException(e);
1293    }
1294  }
1295
1296  @Override
1297  public ListTableNamesByNamespaceResponse listTableNamesByNamespace(RpcController c,
1298      ListTableNamesByNamespaceRequest request) throws ServiceException {
1299    try {
1300      ListTableNamesByNamespaceResponse.Builder b =
1301        ListTableNamesByNamespaceResponse.newBuilder();
1302      for (TableName tableName: master.listTableNamesByNamespace(request.getNamespaceName())) {
1303        b.addTableName(ProtobufUtil.toProtoTableName(tableName));
1304      }
1305      return b.build();
1306    } catch (IOException e) {
1307      throw new ServiceException(e);
1308    }
1309  }
1310
1311  @Override
1312  public ModifyColumnResponse modifyColumn(RpcController controller,
1313      ModifyColumnRequest req) throws ServiceException {
1314    try {
1315      long procId = master.modifyColumn(
1316        ProtobufUtil.toTableName(req.getTableName()),
1317        ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()),
1318        req.getNonceGroup(),
1319        req.getNonce());
1320      if (procId == -1) {
1321        // This mean operation was not performed in server, so do not set any procId
1322        return ModifyColumnResponse.newBuilder().build();
1323      } else {
1324        return ModifyColumnResponse.newBuilder().setProcId(procId).build();
1325      }
1326    } catch (IOException ioe) {
1327      throw new ServiceException(ioe);
1328    }
1329  }
1330
1331  @Override
1332  public ModifyNamespaceResponse modifyNamespace(RpcController controller,
1333      ModifyNamespaceRequest request) throws ServiceException {
1334    try {
1335      long procId = master.modifyNamespace(
1336        ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
1337        request.getNonceGroup(),
1338        request.getNonce());
1339      return ModifyNamespaceResponse.newBuilder().setProcId(procId).build();
1340    } catch (IOException e) {
1341      throw new ServiceException(e);
1342    }
1343  }
1344
1345  @Override
1346  public ModifyTableResponse modifyTable(RpcController controller,
1347      ModifyTableRequest req) throws ServiceException {
1348    try {
1349      long procId = master.modifyTable(
1350        ProtobufUtil.toTableName(req.getTableName()),
1351        ProtobufUtil.toTableDescriptor(req.getTableSchema()),
1352        req.getNonceGroup(),
1353        req.getNonce());
1354      return ModifyTableResponse.newBuilder().setProcId(procId).build();
1355    } catch (IOException ioe) {
1356      throw new ServiceException(ioe);
1357    }
1358  }
1359
1360  @Override
1361  public MoveRegionResponse moveRegion(RpcController controller,
1362      MoveRegionRequest req) throws ServiceException {
1363    final byte [] encodedRegionName = req.getRegion().getValue().toByteArray();
1364    RegionSpecifierType type = req.getRegion().getType();
1365    final byte [] destServerName = (req.hasDestServerName())?
1366      Bytes.toBytes(ProtobufUtil.toServerName(req.getDestServerName()).getServerName()):null;
1367    MoveRegionResponse mrr = MoveRegionResponse.newBuilder().build();
1368
1369    if (type != RegionSpecifierType.ENCODED_REGION_NAME) {
1370      LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME
1371        + " actual: " + type);
1372    }
1373
1374    try {
1375      master.checkInitialized();
1376      master.move(encodedRegionName, destServerName);
1377    } catch (IOException ioe) {
1378      throw new ServiceException(ioe);
1379    }
1380    return mrr;
1381  }
1382
1383  /**
1384   * Offline specified region from master's in-memory state. It will not attempt to
1385   * reassign the region as in unassign.
1386   *
1387   * This is a special method that should be used by experts or hbck.
1388   *
1389   */
1390  @Override
1391  public OfflineRegionResponse offlineRegion(RpcController controller,
1392      OfflineRegionRequest request) throws ServiceException {
1393    try {
1394      master.checkInitialized();
1395
1396      final RegionSpecifierType type = request.getRegion().getType();
1397      if (type != RegionSpecifierType.REGION_NAME) {
1398        LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
1399          + " actual: " + type);
1400      }
1401
1402      final byte[] regionName = request.getRegion().getValue().toByteArray();
1403      final RegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
1404      if (hri == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
1405
1406      if (master.cpHost != null) {
1407        master.cpHost.preRegionOffline(hri);
1408      }
1409      LOG.info(master.getClientIdAuditPrefix() + " offline " + hri.getRegionNameAsString());
1410      master.getAssignmentManager().offlineRegion(hri);
1411      if (master.cpHost != null) {
1412        master.cpHost.postRegionOffline(hri);
1413      }
1414    } catch (IOException ioe) {
1415      throw new ServiceException(ioe);
1416    }
1417    return OfflineRegionResponse.newBuilder().build();
1418  }
1419
1420  /**
1421   * Execute Restore/Clone snapshot operation.
1422   *
1423   * <p>If the specified table exists a "Restore" is executed, replacing the table
1424   * schema and directory data with the content of the snapshot.
1425   * The table must be disabled, or a UnsupportedOperationException will be thrown.
1426   *
1427   * <p>If the table doesn't exist a "Clone" is executed, a new table is created
1428   * using the schema at the time of the snapshot, and the content of the snapshot.
1429   *
1430   * <p>The restore/clone operation does not require copying HFiles. Since HFiles
1431   * are immutable the table can point to and use the same files as the original one.
1432   */
1433  @Override
1434  public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
1435      RestoreSnapshotRequest request) throws ServiceException {
1436    try {
1437      long procId = master.restoreSnapshot(request.getSnapshot(), request.getNonceGroup(),
1438        request.getNonce(), request.getRestoreACL());
1439      return RestoreSnapshotResponse.newBuilder().setProcId(procId).build();
1440    } catch (ForeignException e) {
1441      throw new ServiceException(e.getCause());
1442    } catch (IOException e) {
1443      throw new ServiceException(e);
1444    }
1445  }
1446
1447  @Override
1448  public RunCatalogScanResponse runCatalogScan(RpcController c,
1449      RunCatalogScanRequest req) throws ServiceException {
1450    rpcPreCheck("runCatalogScan");
1451    try {
1452      return ResponseConverter.buildRunCatalogScanResponse(
1453          this.master.catalogJanitorChore.scan());
1454    } catch (IOException ioe) {
1455      throw new ServiceException(ioe);
1456    }
1457  }
1458
1459  @Override
1460  public RunCleanerChoreResponse runCleanerChore(RpcController c, RunCleanerChoreRequest req)
1461    throws ServiceException {
1462    rpcPreCheck("runCleanerChore");
1463    boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
1464    return ResponseConverter.buildRunCleanerChoreResponse(result);
1465  }
1466
1467  @Override
1468  public SetBalancerRunningResponse setBalancerRunning(RpcController c,
1469      SetBalancerRunningRequest req) throws ServiceException {
1470    try {
1471      master.checkInitialized();
1472      boolean prevValue = (req.getSynchronous())?
1473        synchronousBalanceSwitch(req.getOn()) : master.balanceSwitch(req.getOn());
1474      return SetBalancerRunningResponse.newBuilder().setPrevBalanceValue(prevValue).build();
1475    } catch (IOException ioe) {
1476      throw new ServiceException(ioe);
1477    }
1478  }
1479
1480  @Override
1481  public ShutdownResponse shutdown(RpcController controller,
1482      ShutdownRequest request) throws ServiceException {
1483    LOG.info(master.getClientIdAuditPrefix() + " shutdown");
1484    try {
1485      master.shutdown();
1486    } catch (IOException e) {
1487      LOG.error("Exception occurred in HMaster.shutdown()", e);
1488      throw new ServiceException(e);
1489    }
1490    return ShutdownResponse.newBuilder().build();
1491  }
1492
1493  /**
1494   * Triggers an asynchronous attempt to take a snapshot.
1495   * {@inheritDoc}
1496   */
1497  @Override
1498  public SnapshotResponse snapshot(RpcController controller,
1499      SnapshotRequest request) throws ServiceException {
1500    try {
1501      master.checkInitialized();
1502      master.snapshotManager.checkSnapshotSupport();
1503
1504      LOG.info(master.getClientIdAuditPrefix() + " snapshot request for:" +
1505        ClientSnapshotDescriptionUtils.toString(request.getSnapshot()));
1506      // get the snapshot information
1507      SnapshotDescription snapshot = SnapshotDescriptionUtils.validate(
1508        request.getSnapshot(), master.getConfiguration());
1509      master.snapshotManager.takeSnapshot(snapshot);
1510
1511      // send back the max amount of time the client should wait for the snapshot to complete
1512      long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(master.getConfiguration(),
1513        snapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
1514      return SnapshotResponse.newBuilder().setExpectedTimeout(waitTime).build();
1515    } catch (ForeignException e) {
1516      throw new ServiceException(e.getCause());
1517    } catch (IOException e) {
1518      throw new ServiceException(e);
1519    }
1520  }
1521
1522  @Override
1523  public StopMasterResponse stopMaster(RpcController controller,
1524      StopMasterRequest request) throws ServiceException {
1525    LOG.info(master.getClientIdAuditPrefix() + " stop");
1526    try {
1527      master.stopMaster();
1528    } catch (IOException e) {
1529      LOG.error("Exception occurred while stopping master", e);
1530      throw new ServiceException(e);
1531    }
1532    return StopMasterResponse.newBuilder().build();
1533  }
1534
1535  @Override
1536  public IsInMaintenanceModeResponse isMasterInMaintenanceMode(
1537      final RpcController controller,
1538      final IsInMaintenanceModeRequest request) throws ServiceException {
1539    IsInMaintenanceModeResponse.Builder response = IsInMaintenanceModeResponse.newBuilder();
1540    response.setInMaintenanceMode(master.isInMaintenanceMode());
1541    return response.build();
1542  }
1543
1544  @Override
1545  public UnassignRegionResponse unassignRegion(RpcController controller,
1546      UnassignRegionRequest req) throws ServiceException {
1547    try {
1548      final byte [] regionName = req.getRegion().getValue().toByteArray();
1549      RegionSpecifierType type = req.getRegion().getType();
1550      final boolean force = req.getForce();
1551      UnassignRegionResponse urr = UnassignRegionResponse.newBuilder().build();
1552
1553      master.checkInitialized();
1554      if (type != RegionSpecifierType.REGION_NAME) {
1555        LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
1556          + " actual: " + type);
1557      }
1558      Pair<RegionInfo, ServerName> pair =
1559        MetaTableAccessor.getRegion(master.getConnection(), regionName);
1560      if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(),regionName)) {
1561        pair = new Pair<>(RegionInfoBuilder.FIRST_META_REGIONINFO,
1562            master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper()));
1563      }
1564      if (pair == null) {
1565        throw new UnknownRegionException(Bytes.toString(regionName));
1566      }
1567
1568      RegionInfo hri = pair.getFirst();
1569      if (master.cpHost != null) {
1570        master.cpHost.preUnassign(hri, force);
1571      }
1572      LOG.debug(master.getClientIdAuditPrefix() + " unassign " + hri.getRegionNameAsString()
1573          + " in current location if it is online and reassign.force=" + force);
1574      master.getAssignmentManager().unassign(hri);
1575      if (master.cpHost != null) {
1576        master.cpHost.postUnassign(hri, force);
1577      }
1578
1579      return urr;
1580    } catch (IOException ioe) {
1581      throw new ServiceException(ioe);
1582    }
1583  }
1584
1585  @Override
1586  public ReportRegionStateTransitionResponse reportRegionStateTransition(RpcController c,
1587      ReportRegionStateTransitionRequest req) throws ServiceException {
1588    try {
1589      master.checkServiceStarted();
1590      return master.getAssignmentManager().reportRegionStateTransition(req);
1591    } catch (IOException ioe) {
1592      throw new ServiceException(ioe);
1593    }
1594  }
1595
1596  @Override
1597  public SetQuotaResponse setQuota(RpcController c, SetQuotaRequest req)
1598      throws ServiceException {
1599    try {
1600      master.checkInitialized();
1601      return master.getMasterQuotaManager().setQuota(req);
1602    } catch (Exception e) {
1603      throw new ServiceException(e);
1604    }
1605  }
1606
1607  @Override
1608  public MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(RpcController controller,
1609      MajorCompactionTimestampRequest request) throws ServiceException {
1610    MajorCompactionTimestampResponse.Builder response =
1611        MajorCompactionTimestampResponse.newBuilder();
1612    try {
1613      master.checkInitialized();
1614      response.setCompactionTimestamp(master.getLastMajorCompactionTimestamp(ProtobufUtil
1615          .toTableName(request.getTableName())));
1616    } catch (IOException e) {
1617      throw new ServiceException(e);
1618    }
1619    return response.build();
1620  }
1621
1622  @Override
1623  public MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion(
1624      RpcController controller, MajorCompactionTimestampForRegionRequest request)
1625      throws ServiceException {
1626    MajorCompactionTimestampResponse.Builder response =
1627        MajorCompactionTimestampResponse.newBuilder();
1628    try {
1629      master.checkInitialized();
1630      response.setCompactionTimestamp(master.getLastMajorCompactionTimestampForRegion(request
1631          .getRegion().getValue().toByteArray()));
1632    } catch (IOException e) {
1633      throw new ServiceException(e);
1634    }
1635    return response.build();
1636  }
1637
1638  /**
1639   * Compact a region on the master.
1640   *
1641   * @param controller the RPC controller
1642   * @param request the request
1643   * @throws ServiceException
1644   */
1645  @Override
1646  @QosPriority(priority=HConstants.ADMIN_QOS)
1647  public CompactRegionResponse compactRegion(final RpcController controller,
1648    final CompactRegionRequest request) throws ServiceException {
1649    try {
1650      master.checkInitialized();
1651      byte[] regionName = request.getRegion().getValue().toByteArray();
1652      TableName tableName = RegionInfo.getTable(regionName);
1653      // if the region is a mob region, do the mob file compaction.
1654      if (MobUtils.isMobRegionName(tableName, regionName)) {
1655        checkHFileFormatVersionForMob();
1656        return compactMob(request, tableName);
1657      } else {
1658        return super.compactRegion(controller, request);
1659      }
1660    } catch (IOException ie) {
1661      throw new ServiceException(ie);
1662    }
1663  }
1664
1665  /**
1666   * check configured hfile format version before to do compaction
1667   * @throws IOException throw IOException
1668   */
1669  private void checkHFileFormatVersionForMob() throws IOException {
1670    if (HFile.getFormatVersion(master.getConfiguration()) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
1671      LOG.error("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
1672          + " is required for MOB compaction. Compaction will not run.");
1673      throw new IOException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
1674          + " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY
1675          + " accordingly.");
1676    }
1677  }
1678
1679  /**
1680   * This method implements Admin getRegionInfo. On RegionServer, it is
1681   * able to return RegionInfo and detail. On Master, it just returns
1682   * RegionInfo. On Master it has been hijacked to return Mob detail.
1683   * Master implementation is good for querying full region name if
1684   * you only have the encoded name (useful around region replicas
1685   * for example which do not have a row in hbase:meta).
1686   */
1687  @Override
1688  @QosPriority(priority=HConstants.ADMIN_QOS)
1689  public GetRegionInfoResponse getRegionInfo(final RpcController controller,
1690    final GetRegionInfoRequest request) throws ServiceException {
1691    RegionInfo ri = null;
1692    try {
1693      ri = getRegionInfo(request.getRegion());
1694    } catch(UnknownRegionException ure) {
1695      throw new ServiceException(ure);
1696    }
1697    GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
1698    if (ri != null) {
1699      builder.setRegionInfo(ProtobufUtil.toRegionInfo(ri));
1700    } else {
1701      // Is it a MOB name? These work differently.
1702      byte [] regionName = request.getRegion().getValue().toByteArray();
1703      TableName tableName = RegionInfo.getTable(regionName);
1704      if (MobUtils.isMobRegionName(tableName, regionName)) {
1705        // a dummy region info contains the compaction state.
1706        RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
1707        builder.setRegionInfo(ProtobufUtil.toRegionInfo(mobRegionInfo));
1708        if (request.hasCompactionState() && request.getCompactionState()) {
1709          builder.setCompactionState(master.getMobCompactionState(tableName));
1710        }
1711      } else {
1712        // If unknown RegionInfo and not a MOB region, it is unknown.
1713        throw new ServiceException(new UnknownRegionException(Bytes.toString(regionName)));
1714      }
1715    }
1716    return builder.build();
1717  }
1718
1719  /**
1720   * Compacts the mob files in the current table.
1721   * @param request the request.
1722   * @param tableName the current table name.
1723   * @return The response of the mob file compaction.
1724   * @throws IOException
1725   */
1726  private CompactRegionResponse compactMob(final CompactRegionRequest request,
1727    TableName tableName) throws IOException {
1728    if (!master.getTableStateManager().isTableState(tableName, TableState.State.ENABLED)) {
1729      throw new DoNotRetryIOException("Table " + tableName + " is not enabled");
1730    }
1731    boolean allFiles = false;
1732    List<ColumnFamilyDescriptor> compactedColumns = new ArrayList<>();
1733    ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies();
1734    byte[] family = null;
1735    if (request.hasFamily()) {
1736      family = request.getFamily().toByteArray();
1737      for (ColumnFamilyDescriptor hcd : hcds) {
1738        if (Bytes.equals(family, hcd.getName())) {
1739          if (!hcd.isMobEnabled()) {
1740            LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family");
1741            throw new DoNotRetryIOException("Column family " + hcd.getNameAsString()
1742                    + " is not a mob column family");
1743          }
1744          compactedColumns.add(hcd);
1745        }
1746      }
1747    } else {
1748      for (ColumnFamilyDescriptor hcd : hcds) {
1749        if (hcd.isMobEnabled()) {
1750          compactedColumns.add(hcd);
1751        }
1752      }
1753    }
1754    if (compactedColumns.isEmpty()) {
1755      LOG.error("No mob column families are assigned in the mob compaction");
1756      throw new DoNotRetryIOException(
1757              "No mob column families are assigned in the mob compaction");
1758    }
1759    if (request.hasMajor() && request.getMajor()) {
1760      allFiles = true;
1761    }
1762    String familyLogMsg = (family != null) ? Bytes.toString(family) : "";
1763    if (LOG.isTraceEnabled()) {
1764      LOG.trace("User-triggered mob compaction requested for table: "
1765              + tableName.getNameAsString() + " for column family: " + familyLogMsg);
1766    }
1767    master.requestMobCompaction(tableName, compactedColumns, allFiles);
1768    return CompactRegionResponse.newBuilder().build();
1769  }
1770
1771  @Override
1772  public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
1773      IsBalancerEnabledRequest request) throws ServiceException {
1774    IsBalancerEnabledResponse.Builder response = IsBalancerEnabledResponse.newBuilder();
1775    response.setEnabled(master.isBalancerOn());
1776    return response.build();
1777  }
1778
1779  @Override
1780  public SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(RpcController controller,
1781    SetSplitOrMergeEnabledRequest request) throws ServiceException {
1782    SetSplitOrMergeEnabledResponse.Builder response = SetSplitOrMergeEnabledResponse.newBuilder();
1783    try {
1784      master.checkInitialized();
1785      boolean newValue = request.getEnabled();
1786      for (MasterProtos.MasterSwitchType masterSwitchType: request.getSwitchTypesList()) {
1787        MasterSwitchType switchType = convert(masterSwitchType);
1788        boolean oldValue = master.isSplitOrMergeEnabled(switchType);
1789        response.addPrevValue(oldValue);
1790        if (master.cpHost != null) {
1791          master.cpHost.preSetSplitOrMergeEnabled(newValue, switchType);
1792        }
1793        master.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue, switchType);
1794        if (master.cpHost != null) {
1795          master.cpHost.postSetSplitOrMergeEnabled(newValue, switchType);
1796        }
1797      }
1798    } catch (IOException e) {
1799      throw new ServiceException(e);
1800    } catch (KeeperException e) {
1801      throw new ServiceException(e);
1802    }
1803    return response.build();
1804  }
1805
1806  @Override
1807  public IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(RpcController controller,
1808    IsSplitOrMergeEnabledRequest request) throws ServiceException {
1809    IsSplitOrMergeEnabledResponse.Builder response = IsSplitOrMergeEnabledResponse.newBuilder();
1810    response.setEnabled(master.isSplitOrMergeEnabled(convert(request.getSwitchType())));
1811    return response.build();
1812  }
1813
1814  @Override
1815  public NormalizeResponse normalize(RpcController controller,
1816      NormalizeRequest request) throws ServiceException {
1817    rpcPreCheck("normalize");
1818    try {
1819      return NormalizeResponse.newBuilder().setNormalizerRan(master.normalizeRegions()).build();
1820    } catch (IOException ex) {
1821      throw new ServiceException(ex);
1822    }
1823  }
1824
1825  @Override
1826  public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller,
1827      SetNormalizerRunningRequest request) throws ServiceException {
1828    rpcPreCheck("setNormalizerRunning");
1829
1830    // Sets normalizer on/off flag in ZK.
1831    boolean prevValue = master.getRegionNormalizerTracker().isNormalizerOn();
1832    boolean newValue = request.getOn();
1833    try {
1834      master.getRegionNormalizerTracker().setNormalizerOn(newValue);
1835    } catch (KeeperException ke) {
1836      LOG.warn("Error flipping normalizer switch", ke);
1837    }
1838    LOG.info("{} set normalizerSwitch={}", master.getClientIdAuditPrefix(), newValue);
1839    return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
1840  }
1841
1842  @Override
1843  public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller,
1844      IsNormalizerEnabledRequest request) throws ServiceException {
1845    IsNormalizerEnabledResponse.Builder response = IsNormalizerEnabledResponse.newBuilder();
1846    response.setEnabled(master.isNormalizerOn());
1847    return response.build();
1848  }
1849
1850  /**
1851   * Returns the security capabilities in effect on the cluster
1852   */
1853  @Override
1854  public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller,
1855      SecurityCapabilitiesRequest request) throws ServiceException {
1856    SecurityCapabilitiesResponse.Builder response = SecurityCapabilitiesResponse.newBuilder();
1857    try {
1858      master.checkInitialized();
1859      Set<SecurityCapabilitiesResponse.Capability> capabilities = new HashSet<>();
1860      // Authentication
1861      if (User.isHBaseSecurityEnabled(master.getConfiguration())) {
1862        capabilities.add(SecurityCapabilitiesResponse.Capability.SECURE_AUTHENTICATION);
1863      } else {
1864        capabilities.add(SecurityCapabilitiesResponse.Capability.SIMPLE_AUTHENTICATION);
1865      }
1866      // A coprocessor that implements AccessControlService can provide AUTHORIZATION and
1867      // CELL_AUTHORIZATION
1868      if (master.cpHost != null && hasAccessControlServiceCoprocessor(master.cpHost)) {
1869        if (AccessChecker.isAuthorizationSupported(master.getConfiguration())) {
1870          capabilities.add(SecurityCapabilitiesResponse.Capability.AUTHORIZATION);
1871        }
1872        if (AccessController.isCellAuthorizationSupported(master.getConfiguration())) {
1873          capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_AUTHORIZATION);
1874        }
1875      }
1876      // A coprocessor that implements VisibilityLabelsService can provide CELL_VISIBILITY.
1877      if (master.cpHost != null && hasVisibilityLabelsServiceCoprocessor(master.cpHost)) {
1878        if (VisibilityController.isCellAuthorizationSupported(master.getConfiguration())) {
1879          capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_VISIBILITY);
1880        }
1881      }
1882      response.addAllCapabilities(capabilities);
1883    } catch (IOException e) {
1884      throw new ServiceException(e);
1885    }
1886    return response.build();
1887  }
1888
1889  /**
1890   * Determines if there is a MasterCoprocessor deployed which implements
1891   * {@link org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface}.
1892   */
1893  boolean hasAccessControlServiceCoprocessor(MasterCoprocessorHost cpHost) {
1894    return checkCoprocessorWithService(
1895        cpHost.findCoprocessors(MasterCoprocessor.class), AccessControlService.Interface.class);
1896  }
1897
1898  /**
1899   * Determines if there is a MasterCoprocessor deployed which implements
1900   * {@link org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface}.
1901   */
1902  boolean hasVisibilityLabelsServiceCoprocessor(MasterCoprocessorHost cpHost) {
1903    return checkCoprocessorWithService(
1904        cpHost.findCoprocessors(MasterCoprocessor.class),
1905        VisibilityLabelsService.Interface.class);
1906  }
1907
1908  /**
1909   * Determines if there is a coprocessor implementation in the provided argument which extends
1910   * or implements the provided {@code service}.
1911   */
1912  boolean checkCoprocessorWithService(
1913      List<MasterCoprocessor> coprocessorsToCheck, Class<?> service) {
1914    if (coprocessorsToCheck == null || coprocessorsToCheck.isEmpty()) {
1915      return false;
1916    }
1917    for (MasterCoprocessor cp : coprocessorsToCheck) {
1918      if (service.isAssignableFrom(cp.getClass())) {
1919        return true;
1920      }
1921    }
1922    return false;
1923  }
1924
1925  private MasterSwitchType convert(MasterProtos.MasterSwitchType switchType) {
1926    switch (switchType) {
1927      case SPLIT:
1928        return MasterSwitchType.SPLIT;
1929      case MERGE:
1930        return MasterSwitchType.MERGE;
1931      default:
1932        break;
1933    }
1934    return null;
1935  }
1936
1937  @Override
1938  public AddReplicationPeerResponse addReplicationPeer(RpcController controller,
1939      AddReplicationPeerRequest request) throws ServiceException {
1940    try {
1941      long procId = master.addReplicationPeer(request.getPeerId(),
1942        ReplicationPeerConfigUtil.convert(request.getPeerConfig()),
1943        request.getPeerState().getState().equals(ReplicationState.State.ENABLED));
1944      return AddReplicationPeerResponse.newBuilder().setProcId(procId).build();
1945    } catch (ReplicationException | IOException e) {
1946      throw new ServiceException(e);
1947    }
1948  }
1949
1950  @Override
1951  public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller,
1952      RemoveReplicationPeerRequest request) throws ServiceException {
1953    try {
1954      long procId = master.removeReplicationPeer(request.getPeerId());
1955      return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build();
1956    } catch (ReplicationException | IOException e) {
1957      throw new ServiceException(e);
1958    }
1959  }
1960
1961  @Override
1962  public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller,
1963      EnableReplicationPeerRequest request) throws ServiceException {
1964    try {
1965      long procId = master.enableReplicationPeer(request.getPeerId());
1966      return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build();
1967    } catch (ReplicationException | IOException e) {
1968      throw new ServiceException(e);
1969    }
1970  }
1971
1972  @Override
1973  public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller,
1974      DisableReplicationPeerRequest request) throws ServiceException {
1975    try {
1976      long procId = master.disableReplicationPeer(request.getPeerId());
1977      return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build();
1978    } catch (ReplicationException | IOException e) {
1979      throw new ServiceException(e);
1980    }
1981  }
1982
1983  @Override
1984  public GetReplicationPeerConfigResponse getReplicationPeerConfig(RpcController controller,
1985      GetReplicationPeerConfigRequest request) throws ServiceException {
1986    GetReplicationPeerConfigResponse.Builder response = GetReplicationPeerConfigResponse
1987        .newBuilder();
1988    try {
1989      String peerId = request.getPeerId();
1990      ReplicationPeerConfig peerConfig = master.getReplicationPeerConfig(peerId);
1991      response.setPeerId(peerId);
1992      response.setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig));
1993    } catch (ReplicationException | IOException e) {
1994      throw new ServiceException(e);
1995    }
1996    return response.build();
1997  }
1998
1999  @Override
2000  public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller,
2001      UpdateReplicationPeerConfigRequest request) throws ServiceException {
2002    try {
2003      long procId = master.updateReplicationPeerConfig(request.getPeerId(),
2004        ReplicationPeerConfigUtil.convert(request.getPeerConfig()));
2005      return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build();
2006    } catch (ReplicationException | IOException e) {
2007      throw new ServiceException(e);
2008    }
2009  }
2010
2011  @Override
2012  public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
2013      ListReplicationPeersRequest request) throws ServiceException {
2014    ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder();
2015    try {
2016      List<ReplicationPeerDescription> peers = master
2017          .listReplicationPeers(request.hasRegex() ? request.getRegex() : null);
2018      for (ReplicationPeerDescription peer : peers) {
2019        response.addPeerDesc(ReplicationPeerConfigUtil.toProtoReplicationPeerDescription(peer));
2020      }
2021    } catch (ReplicationException | IOException e) {
2022      throw new ServiceException(e);
2023    }
2024    return response.build();
2025  }
2026
2027  @Override
2028  public ListDecommissionedRegionServersResponse listDecommissionedRegionServers(
2029      RpcController controller, ListDecommissionedRegionServersRequest request)
2030      throws ServiceException {
2031    ListDecommissionedRegionServersResponse.Builder response =
2032        ListDecommissionedRegionServersResponse.newBuilder();
2033    try {
2034      master.checkInitialized();
2035      if (master.cpHost != null) {
2036        master.cpHost.preListDecommissionedRegionServers();
2037      }
2038      List<ServerName> servers = master.listDecommissionedRegionServers();
2039      response.addAllServerName((servers.stream().map(server -> ProtobufUtil.toServerName(server)))
2040          .collect(Collectors.toList()));
2041      if (master.cpHost != null) {
2042        master.cpHost.postListDecommissionedRegionServers();
2043      }
2044    } catch (IOException io) {
2045      throw new ServiceException(io);
2046    }
2047
2048    return response.build();
2049  }
2050
2051  @Override
2052  public DecommissionRegionServersResponse decommissionRegionServers(RpcController controller,
2053      DecommissionRegionServersRequest request) throws ServiceException {
2054    try {
2055      master.checkInitialized();
2056      List<ServerName> servers = request.getServerNameList().stream()
2057          .map(pbServer -> ProtobufUtil.toServerName(pbServer)).collect(Collectors.toList());
2058      boolean offload = request.getOffload();
2059      if (master.cpHost != null) {
2060        master.cpHost.preDecommissionRegionServers(servers, offload);
2061      }
2062      master.decommissionRegionServers(servers, offload);
2063      if (master.cpHost != null) {
2064        master.cpHost.postDecommissionRegionServers(servers, offload);
2065      }
2066    } catch (IOException io) {
2067      throw new ServiceException(io);
2068    }
2069
2070    return DecommissionRegionServersResponse.newBuilder().build();
2071  }
2072
2073  @Override
2074  public RecommissionRegionServerResponse recommissionRegionServer(RpcController controller,
2075      RecommissionRegionServerRequest request) throws ServiceException {
2076    try {
2077      master.checkInitialized();
2078      ServerName server = ProtobufUtil.toServerName(request.getServerName());
2079      List<byte[]> encodedRegionNames = request.getRegionList().stream()
2080          .map(regionSpecifier -> regionSpecifier.getValue().toByteArray())
2081          .collect(Collectors.toList());
2082      if (master.cpHost != null) {
2083        master.cpHost.preRecommissionRegionServer(server, encodedRegionNames);
2084      }
2085      master.recommissionRegionServer(server, encodedRegionNames);
2086      if (master.cpHost != null) {
2087        master.cpHost.postRecommissionRegionServer(server, encodedRegionNames);
2088      }
2089    } catch (IOException io) {
2090      throw new ServiceException(io);
2091    }
2092
2093    return RecommissionRegionServerResponse.newBuilder().build();
2094  }
2095
2096  @Override
2097  public LockResponse requestLock(RpcController controller, final LockRequest request)
2098      throws ServiceException {
2099    try {
2100      if (request.getDescription().isEmpty()) {
2101        throw new IllegalArgumentException("Empty description");
2102      }
2103      NonceProcedureRunnable npr;
2104      LockType type = LockType.valueOf(request.getLockType().name());
2105      if (request.getRegionInfoCount() > 0) {
2106        final RegionInfo[] regionInfos = new RegionInfo[request.getRegionInfoCount()];
2107        for (int i = 0; i < request.getRegionInfoCount(); ++i) {
2108          regionInfos[i] = ProtobufUtil.toRegionInfo(request.getRegionInfo(i));
2109        }
2110        npr = new NonceProcedureRunnable(master, request.getNonceGroup(), request.getNonce()) {
2111          @Override
2112          protected void run() throws IOException {
2113            setProcId(master.getLockManager().remoteLocks().requestRegionsLock(regionInfos,
2114                request.getDescription(), getNonceKey()));
2115          }
2116
2117          @Override
2118          protected String getDescription() {
2119            return "RequestLock";
2120          }
2121        };
2122      } else if (request.hasTableName()) {
2123        final TableName tableName = ProtobufUtil.toTableName(request.getTableName());
2124        npr = new NonceProcedureRunnable(master, request.getNonceGroup(), request.getNonce()) {
2125          @Override
2126          protected void run() throws IOException {
2127            setProcId(master.getLockManager().remoteLocks().requestTableLock(tableName, type,
2128                request.getDescription(), getNonceKey()));
2129          }
2130
2131          @Override
2132          protected String getDescription() {
2133            return "RequestLock";
2134          }
2135        };
2136      } else if (request.hasNamespace()) {
2137        npr = new NonceProcedureRunnable(master, request.getNonceGroup(), request.getNonce()) {
2138          @Override
2139          protected void run() throws IOException {
2140            setProcId(master.getLockManager().remoteLocks().requestNamespaceLock(
2141                request.getNamespace(), type, request.getDescription(), getNonceKey()));
2142          }
2143
2144          @Override
2145          protected String getDescription() {
2146            return "RequestLock";
2147          }
2148        };
2149      } else {
2150        throw new IllegalArgumentException("one of table/namespace/region should be specified");
2151      }
2152      long procId = MasterProcedureUtil.submitProcedure(npr);
2153      return LockResponse.newBuilder().setProcId(procId).build();
2154    } catch (IllegalArgumentException e) {
2155      LOG.warn("Exception when queuing lock", e);
2156      throw new ServiceException(new DoNotRetryIOException(e));
2157    } catch (IOException e) {
2158      LOG.warn("Exception when queuing lock", e);
2159      throw new ServiceException(e);
2160    }
2161  }
2162
2163  /**
2164   * @return LOCKED, if procedure is found and it has the lock; else UNLOCKED.
2165   * @throws ServiceException if given proc id is found but it is not a LockProcedure.
2166   */
2167  @Override
2168  public LockHeartbeatResponse lockHeartbeat(RpcController controller, LockHeartbeatRequest request)
2169      throws ServiceException {
2170    try {
2171      if (master.getLockManager().remoteLocks().lockHeartbeat(request.getProcId(),
2172          request.getKeepAlive())) {
2173        return LockHeartbeatResponse.newBuilder().setTimeoutMs(
2174            master.getConfiguration().getInt(LockProcedure.REMOTE_LOCKS_TIMEOUT_MS_CONF,
2175                LockProcedure.DEFAULT_REMOTE_LOCKS_TIMEOUT_MS))
2176            .setLockStatus(LockHeartbeatResponse.LockStatus.LOCKED).build();
2177      } else {
2178        return LockHeartbeatResponse.newBuilder()
2179            .setLockStatus(LockHeartbeatResponse.LockStatus.UNLOCKED).build();
2180      }
2181    } catch (IOException e) {
2182      throw new ServiceException(e);
2183    }
2184  }
2185
2186  @Override
2187  public RegionSpaceUseReportResponse reportRegionSpaceUse(RpcController controller,
2188      RegionSpaceUseReportRequest request) throws ServiceException {
2189    try {
2190      master.checkInitialized();
2191      if (!QuotaUtil.isQuotaEnabled(master.getConfiguration())) {
2192        return RegionSpaceUseReportResponse.newBuilder().build();
2193      }
2194      MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
2195      if (quotaManager != null) {
2196        final long now = EnvironmentEdgeManager.currentTime();
2197        for (RegionSpaceUse report : request.getSpaceUseList()) {
2198          quotaManager.addRegionSize(ProtobufUtil.toRegionInfo(report.getRegionInfo()),
2199            report.getRegionSize(), now);
2200        }
2201      } else {
2202        LOG.debug(
2203          "Received region space usage report but HMaster is not ready to process it, skipping");
2204      }
2205      return RegionSpaceUseReportResponse.newBuilder().build();
2206    } catch (Exception e) {
2207      throw new ServiceException(e);
2208    }
2209  }
2210
2211  @Override
2212  public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
2213      RpcController controller, GetSpaceQuotaRegionSizesRequest request) throws ServiceException {
2214    try {
2215      master.checkInitialized();
2216      MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
2217      GetSpaceQuotaRegionSizesResponse.Builder builder =
2218          GetSpaceQuotaRegionSizesResponse.newBuilder();
2219      if (quotaManager != null) {
2220        Map<RegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
2221        Map<TableName,Long> regionSizesByTable = new HashMap<>();
2222        // Translate hregioninfo+long -> tablename+long
2223        for (Entry<RegionInfo,Long> entry : regionSizes.entrySet()) {
2224          final TableName tableName = entry.getKey().getTable();
2225          Long prevSize = regionSizesByTable.get(tableName);
2226          if (prevSize == null) {
2227            prevSize = 0L;
2228          }
2229          regionSizesByTable.put(tableName, prevSize + entry.getValue());
2230        }
2231        // Serialize them into the protobuf
2232        for (Entry<TableName,Long> tableSize : regionSizesByTable.entrySet()) {
2233          builder.addSizes(RegionSizes.newBuilder()
2234              .setTableName(ProtobufUtil.toProtoTableName(tableSize.getKey()))
2235              .setSize(tableSize.getValue()).build());
2236        }
2237        return builder.build();
2238      } else {
2239        LOG.debug(
2240          "Received space quota region size report but HMaster is not ready to process it, skipping");
2241      }
2242      return builder.build();
2243    } catch (Exception e) {
2244      throw new ServiceException(e);
2245    }
2246  }
2247
2248  @Override
2249  public GetQuotaStatesResponse getQuotaStates(
2250      RpcController controller, GetQuotaStatesRequest request) throws ServiceException {
2251    try {
2252      master.checkInitialized();
2253      QuotaObserverChore quotaChore = this.master.getQuotaObserverChore();
2254      GetQuotaStatesResponse.Builder builder = GetQuotaStatesResponse.newBuilder();
2255      if (quotaChore != null) {
2256        // The "current" view of all tables with quotas
2257        Map<TableName, SpaceQuotaSnapshot> tableSnapshots = quotaChore.getTableQuotaSnapshots();
2258        for (Entry<TableName, SpaceQuotaSnapshot> entry : tableSnapshots.entrySet()) {
2259          builder.addTableSnapshots(
2260              TableQuotaSnapshot.newBuilder()
2261                  .setTableName(ProtobufUtil.toProtoTableName(entry.getKey()))
2262                  .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build());
2263        }
2264        // The "current" view of all namespaces with quotas
2265        Map<String, SpaceQuotaSnapshot> nsSnapshots = quotaChore.getNamespaceQuotaSnapshots();
2266        for (Entry<String, SpaceQuotaSnapshot> entry : nsSnapshots.entrySet()) {
2267          builder.addNsSnapshots(
2268              NamespaceQuotaSnapshot.newBuilder()
2269                  .setNamespace(entry.getKey())
2270                  .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build());
2271        }
2272        return builder.build();
2273      }
2274      return builder.build();
2275    } catch (Exception e) {
2276      throw new ServiceException(e);
2277    }
2278  }
2279
2280  @Override
2281  public ClearDeadServersResponse clearDeadServers(RpcController controller,
2282      ClearDeadServersRequest request) throws ServiceException {
2283    LOG.debug(master.getClientIdAuditPrefix() + " clear dead region servers.");
2284    ClearDeadServersResponse.Builder response = ClearDeadServersResponse.newBuilder();
2285    try {
2286      master.checkInitialized();
2287      if (master.cpHost != null) {
2288        master.cpHost.preClearDeadServers();
2289      }
2290
2291      if (master.getServerManager().areDeadServersInProgress()) {
2292        LOG.debug("Some dead server is still under processing, won't clear the dead server list");
2293        response.addAllServerName(request.getServerNameList());
2294      } else {
2295        for (HBaseProtos.ServerName pbServer : request.getServerNameList()) {
2296          if (!master.getServerManager().getDeadServers()
2297                  .removeDeadServer(ProtobufUtil.toServerName(pbServer))) {
2298            response.addServerName(pbServer);
2299          }
2300        }
2301      }
2302
2303      if (master.cpHost != null) {
2304        master.cpHost.postClearDeadServers(
2305            ProtobufUtil.toServerNameList(request.getServerNameList()),
2306            ProtobufUtil.toServerNameList(response.getServerNameList()));
2307      }
2308    } catch (IOException io) {
2309      throw new ServiceException(io);
2310    }
2311    return response.build();
2312  }
2313
2314  @Override
2315  public ReportProcedureDoneResponse reportProcedureDone(RpcController controller,
2316      ReportProcedureDoneRequest request) throws ServiceException {
2317    request.getResultList().forEach(result -> {
2318      if (result.getStatus() == RemoteProcedureResult.Status.SUCCESS) {
2319        master.remoteProcedureCompleted(result.getProcId());
2320      } else {
2321        master.remoteProcedureFailed(result.getProcId(),
2322          RemoteProcedureException.fromProto(result.getError()));
2323      }
2324    });
2325    return ReportProcedureDoneResponse.getDefaultInstance();
2326  }
2327
2328  // HBCK Services
2329
2330  @Override
2331  public RunHbckChoreResponse runHbckChore(RpcController c, RunHbckChoreRequest req)
2332      throws ServiceException {
2333    rpcPreCheck("runHbckChore");
2334    LOG.info("{} request HBCK chore to run", master.getClientIdAuditPrefix());
2335    HbckChore hbckChore = master.getHbckChore();
2336    boolean ran = hbckChore.runChore();
2337    return RunHbckChoreResponse.newBuilder().setRan(ran).build();
2338  }
2339
2340  /**
2341   * Update state of the table in meta only. This is required by hbck in some situations to cleanup
2342   * stuck assign/ unassign regions procedures for the table.
2343   *
2344   * @return previous state of the table
2345   */
2346  @Override
2347  public GetTableStateResponse setTableStateInMeta(RpcController controller,
2348      SetTableStateInMetaRequest request) throws ServiceException {
2349    TableName tn = ProtobufUtil.toTableName(request.getTableName());
2350    try {
2351      TableState prevState = this.master.getTableStateManager().getTableState(tn);
2352      TableState newState = TableState.convert(tn, request.getTableState());
2353      LOG.info("{} set table={} state from {} to {}", master.getClientIdAuditPrefix(),
2354          tn, prevState.getState(), newState.getState());
2355      this.master.getTableStateManager().setTableState(tn, newState.getState());
2356      return GetTableStateResponse.newBuilder().setTableState(prevState.convert()).build();
2357    } catch (Exception e) {
2358      throw new ServiceException(e);
2359    }
2360  }
2361
2362  /**
2363   * Get RegionInfo from Master using content of RegionSpecifier as key.
2364   * @return RegionInfo found by decoding <code>rs</code> or null if none found
2365   */
2366  private RegionInfo getRegionInfo(HBaseProtos.RegionSpecifier rs) throws UnknownRegionException {
2367    RegionInfo ri = null;
2368    switch(rs.getType()) {
2369      case REGION_NAME:
2370        final byte[] regionName = rs.getValue().toByteArray();
2371        ri = this.master.getAssignmentManager().getRegionInfo(regionName);
2372        break;
2373      case ENCODED_REGION_NAME:
2374        String encodedRegionName = Bytes.toString(rs.getValue().toByteArray());
2375        RegionState regionState = this.master.getAssignmentManager().getRegionStates().
2376            getRegionState(encodedRegionName);
2377        ri = regionState == null? null: regionState.getRegion();
2378        break;
2379      default:
2380        break;
2381    }
2382    return ri;
2383  }
2384
2385  /**
2386   * Submit the Procedure that gets created by <code>f</code>
2387   * @return pid of the submitted Procedure.
2388   */
2389  private long submitProcedure(HBaseProtos.RegionSpecifier rs, boolean override,
2390      BiFunction<RegionInfo, Boolean, Procedure> f)
2391    throws UnknownRegionException {
2392    RegionInfo ri = getRegionInfo(rs);
2393    long pid = Procedure.NO_PROC_ID;
2394    if (ri == null) {
2395      LOG.warn("No RegionInfo found to match {}", rs);
2396    } else {
2397      pid = this.master.getMasterProcedureExecutor().submitProcedure(f.apply(ri, override));
2398    }
2399    return pid;
2400  }
2401
2402  /**
2403   * A 'raw' version of assign that does bulk and skirts Master state checks (assigns can be made
2404   * during Master startup). For use by Hbck2.
2405   */
2406  @Override
2407  public MasterProtos.AssignsResponse assigns(RpcController controller,
2408      MasterProtos.AssignsRequest request)
2409    throws ServiceException {
2410    if (this.master.getMasterProcedureExecutor() == null) {
2411      throw new ServiceException("Master's ProcedureExecutor not initialized; retry later");
2412    }
2413    MasterProtos.AssignsResponse.Builder responseBuilder =
2414        MasterProtos.AssignsResponse.newBuilder();
2415    try {
2416      boolean override = request.getOverride();
2417      LOG.info("{} assigns, override={}", master.getClientIdAuditPrefix(), override);
2418      for (HBaseProtos.RegionSpecifier rs: request.getRegionList()) {
2419        long pid = submitProcedure(rs, override,
2420          (r, b) -> this.master.getAssignmentManager().createAssignProcedure(r, b));
2421        responseBuilder.addPid(pid);
2422      }
2423      return responseBuilder.build();
2424    } catch (IOException ioe) {
2425      throw new ServiceException(ioe);
2426    }
2427  }
2428
2429  /**
2430   * A 'raw' version of unassign that does bulk and skirts Master state checks (unassigns can be
2431   * made during Master startup). For use by Hbck2.
2432   */
2433  @Override
2434  public MasterProtos.UnassignsResponse unassigns(RpcController controller,
2435      MasterProtos.UnassignsRequest request)
2436      throws ServiceException {
2437    if (this.master.getMasterProcedureExecutor() == null) {
2438      throw new ServiceException("Master's ProcedureExecutor not initialized; retry later");
2439    }
2440    MasterProtos.UnassignsResponse.Builder responseBuilder =
2441        MasterProtos.UnassignsResponse.newBuilder();
2442    try {
2443      boolean override = request.getOverride();
2444      LOG.info("{} unassigns, override={}", master.getClientIdAuditPrefix(), override);
2445      for (HBaseProtos.RegionSpecifier rs: request.getRegionList()) {
2446        long pid = submitProcedure(rs, override,
2447          (r, b) -> this.master.getAssignmentManager().createUnassignProcedure(r, b));
2448        responseBuilder.addPid(pid);
2449      }
2450      return responseBuilder.build();
2451    } catch (IOException ioe) {
2452      throw new ServiceException(ioe);
2453    }
2454  }
2455
2456  /**
2457   * Bypass specified procedure to completion. Procedure is marked completed but no actual work
2458   * is done from the current state/ step onwards. Parents of the procedure are also marked for
2459   * bypass.
2460   *
2461   * NOTE: this is a dangerous operation and may be used to unstuck buggy procedures. This may
2462   * leave system in inconherent state. This may need to be followed by some cleanup steps/
2463   * actions by operator.
2464   *
2465   * @return BypassProcedureToCompletionResponse indicating success or failure
2466   */
2467  @Override
2468  public MasterProtos.BypassProcedureResponse bypassProcedure(RpcController controller,
2469      MasterProtos.BypassProcedureRequest request) throws ServiceException {
2470    try {
2471      LOG.info("{} bypass procedures={}, waitTime={}, override={}, recursive={}",
2472          master.getClientIdAuditPrefix(), request.getProcIdList(), request.getWaitTime(),
2473          request.getOverride(), request.getRecursive());
2474      List<Boolean> ret =
2475          master.getMasterProcedureExecutor().bypassProcedure(request.getProcIdList(),
2476          request.getWaitTime(), request.getOverride(), request.getRecursive());
2477      return MasterProtos.BypassProcedureResponse.newBuilder().addAllBypassed(ret).build();
2478    } catch (IOException e) {
2479      throw new ServiceException(e);
2480    }
2481  }
2482
2483  @Override
2484  public MasterProtos.ScheduleServerCrashProcedureResponse scheduleServerCrashProcedure(
2485      RpcController controller, MasterProtos.ScheduleServerCrashProcedureRequest request)
2486      throws ServiceException {
2487    List<HBaseProtos.ServerName> serverNames = request.getServerNameList();
2488    List<Long> pids = new ArrayList<>();
2489    try {
2490      for (HBaseProtos.ServerName serverName : serverNames) {
2491        ServerName server = ProtobufUtil.toServerName(serverName);
2492        LOG.info("{} schedule ServerCrashProcedure for {}",
2493            master.getClientIdAuditPrefix(), server);
2494        if (shouldSubmitSCP(server)) {
2495          master.getServerManager().moveFromOnlineToDeadServers(server);
2496          ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
2497          pids.add(procExec.submitProcedure(new ServerCrashProcedure(procExec.getEnvironment(),
2498            server, true, containMetaWals(server))));
2499        } else {
2500          pids.add(-1L);
2501        }
2502      }
2503      return MasterProtos.ScheduleServerCrashProcedureResponse.newBuilder().addAllPid(pids).build();
2504    } catch (IOException e) {
2505      throw new ServiceException(e);
2506    }
2507  }
2508
2509  @Override
2510  public FileArchiveNotificationResponse reportFileArchival(RpcController controller,
2511      FileArchiveNotificationRequest request) throws ServiceException {
2512    try {
2513      master.checkInitialized();
2514      if (!QuotaUtil.isQuotaEnabled(master.getConfiguration())) {
2515        return FileArchiveNotificationResponse.newBuilder().build();
2516      }
2517      master.getMasterQuotaManager().processFileArchivals(request, master.getConnection(),
2518          master.getConfiguration(), master.getFileSystem());
2519      return FileArchiveNotificationResponse.newBuilder().build();
2520    } catch (Exception e) {
2521      throw new ServiceException(e);
2522    }
2523  }
2524
2525  private boolean containMetaWals(ServerName serverName) throws IOException {
2526    Path logDir = new Path(master.getWALRootDir(),
2527        AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
2528    Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
2529    Path checkDir = master.getFileSystem().exists(splitDir) ? splitDir : logDir;
2530    try {
2531      return master.getFileSystem().listStatus(checkDir, META_FILTER).length > 0;
2532    } catch (FileNotFoundException fnfe) {
2533      // If no files, then we don't contain metas; was failing schedule of
2534      // SCP because this was FNFE'ing when no server dirs ('Unknown Server').
2535      LOG.warn("No dir for WALs for {}; continuing", serverName.toString());
2536      return false;
2537    }
2538  }
2539
2540  private boolean shouldSubmitSCP(ServerName serverName) {
2541    // check if there is already a SCP of this server running
2542    List<Procedure<MasterProcedureEnv>> procedures =
2543        master.getMasterProcedureExecutor().getProcedures();
2544    for (Procedure<MasterProcedureEnv> procedure : procedures) {
2545      if (procedure instanceof ServerCrashProcedure) {
2546        if (serverName.compareTo(((ServerCrashProcedure) procedure).getServerName()) == 0
2547            && !procedure.isFinished()) {
2548          LOG.info("there is already a SCP of this server {} running, pid {}", serverName,
2549            procedure.getProcId());
2550          return false;
2551        }
2552      }
2553    }
2554    return true;
2555  }
2556
2557  @Override
2558  public FixMetaResponse fixMeta(RpcController controller, FixMetaRequest request)
2559      throws ServiceException {
2560    try {
2561      MetaFixer mf = new MetaFixer(this.master);
2562      mf.fix();
2563      return FixMetaResponse.newBuilder().build();
2564    } catch (IOException ioe) {
2565      throw new ServiceException(ioe);
2566    }
2567  }
2568}