HBASE-26587 Introduce a new Admin API to change SFT implementation (#4030)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / master / MasterRpcServices.java
blobd53a27161f55aed2c9c8720d7dad279de691d979
1 /*
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.master;
20 import java.io.IOException;
21 import java.lang.reflect.InvocationTargetException;
22 import java.lang.reflect.Method;
23 import java.net.InetAddress;
24 import java.util.ArrayList;
25 import java.util.Collections;
26 import java.util.HashMap;
27 import java.util.HashSet;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.Map.Entry;
31 import java.util.Set;
32 import java.util.concurrent.ThreadLocalRandom;
33 import java.util.stream.Collectors;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
36 import org.apache.hadoop.hbase.DoNotRetryIOException;
37 import org.apache.hadoop.hbase.HBaseRpcServicesBase;
38 import org.apache.hadoop.hbase.HConstants;
39 import org.apache.hadoop.hbase.MetaTableAccessor;
40 import org.apache.hadoop.hbase.NamespaceDescriptor;
41 import org.apache.hadoop.hbase.ServerMetrics;
42 import org.apache.hadoop.hbase.ServerMetricsBuilder;
43 import org.apache.hadoop.hbase.ServerName;
44 import org.apache.hadoop.hbase.TableName;
45 import org.apache.hadoop.hbase.UnknownRegionException;
46 import org.apache.hadoop.hbase.client.BalanceRequest;
47 import org.apache.hadoop.hbase.client.BalanceResponse;
48 import org.apache.hadoop.hbase.client.MasterSwitchType;
49 import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
50 import org.apache.hadoop.hbase.client.Put;
51 import org.apache.hadoop.hbase.client.RegionInfo;
52 import org.apache.hadoop.hbase.client.Table;
53 import org.apache.hadoop.hbase.client.TableDescriptor;
54 import org.apache.hadoop.hbase.client.TableState;
55 import org.apache.hadoop.hbase.client.VersionInfoUtil;
56 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
57 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
58 import org.apache.hadoop.hbase.errorhandling.ForeignException;
59 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
60 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
61 import org.apache.hadoop.hbase.ipc.PriorityFunction;
62 import org.apache.hadoop.hbase.ipc.QosPriority;
63 import org.apache.hadoop.hbase.ipc.RpcServer;
64 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
65 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
66 import org.apache.hadoop.hbase.ipc.ServerRpcController;
67 import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
68 import org.apache.hadoop.hbase.master.assignment.RegionStates;
69 import org.apache.hadoop.hbase.master.janitor.MetaFixer;
70 import org.apache.hadoop.hbase.master.locking.LockProcedure;
71 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
72 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
73 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
74 import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
75 import org.apache.hadoop.hbase.mob.MobUtils;
76 import org.apache.hadoop.hbase.namequeues.BalancerDecisionDetails;
77 import org.apache.hadoop.hbase.namequeues.BalancerRejectionDetails;
78 import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
79 import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
80 import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
81 import org.apache.hadoop.hbase.net.Address;
82 import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
83 import org.apache.hadoop.hbase.procedure2.LockType;
84 import org.apache.hadoop.hbase.procedure2.LockedResource;
85 import org.apache.hadoop.hbase.procedure2.Procedure;
86 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
87 import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
88 import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
89 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
90 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
91 import org.apache.hadoop.hbase.quotas.QuotaUtil;
92 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
93 import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
94 import org.apache.hadoop.hbase.replication.ReplicationException;
95 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
96 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
97 import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
98 import org.apache.hadoop.hbase.rsgroup.RSGroupUtil;
99 import org.apache.hadoop.hbase.security.Superusers;
100 import org.apache.hadoop.hbase.security.User;
101 import org.apache.hadoop.hbase.security.access.AccessChecker;
102 import org.apache.hadoop.hbase.security.access.AccessChecker.InputUser;
103 import org.apache.hadoop.hbase.security.access.AccessController;
104 import org.apache.hadoop.hbase.security.access.Permission;
105 import org.apache.hadoop.hbase.security.access.Permission.Action;
106 import org.apache.hadoop.hbase.security.access.PermissionStorage;
107 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
108 import org.apache.hadoop.hbase.security.access.UserPermission;
109 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
110 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
111 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
112 import org.apache.hadoop.hbase.util.Bytes;
113 import org.apache.hadoop.hbase.util.DNS;
114 import org.apache.hadoop.hbase.util.DNS.ServerType;
115 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
116 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
117 import org.apache.hadoop.hbase.util.Pair;
118 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
119 import org.apache.yetus.audience.InterfaceAudience;
120 import org.apache.zookeeper.KeeperException;
121 import org.slf4j.Logger;
122 import org.slf4j.LoggerFactory;
124 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
125 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
126 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
127 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
128 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
129 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
130 import org.apache.hbase.thirdparty.com.google.protobuf.Service;
131 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
132 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
134 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
135 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
136 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
137 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService;
138 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest;
139 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
140 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
141 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantResponse;
142 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsRequest;
143 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsResponse;
144 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type;
145 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest;
146 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeResponse;
147 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
148 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
149 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
150 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
151 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
152 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
153 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
154 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
155 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
156 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest;
157 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
158 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
159 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
160 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
161 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
162 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
163 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
164 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
165 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
166 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
167 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
168 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
169 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
170 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
171 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
172 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
173 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
174 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
175 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
176 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
177 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse;
178 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest;
179 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse;
180 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
181 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
182 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
183 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse;
184 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
185 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
186 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
187 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
188 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
189 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
190 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
191 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
192 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo;
193 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
194 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
195 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
196 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
197 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
198 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
199 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
200 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
201 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
202 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
203 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
204 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
205 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
206 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse;
207 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
208 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
209 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
210 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
211 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersRequest;
212 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersResponse;
213 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
214 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
215 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
216 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
217 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
218 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
219 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
220 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
221 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
222 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
223 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
224 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
225 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
226 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
227 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
228 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
229 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
230 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
231 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
232 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
233 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
234 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
235 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
236 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
237 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
238 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
239 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
240 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
241 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
242 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
243 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
244 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
245 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
246 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
247 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
248 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
249 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
250 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
251 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService;
252 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
253 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
254 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
255 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
256 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
257 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse;
258 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
259 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
260 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
261 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
262 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
263 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
264 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
265 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
266 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledRequest;
267 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledResponse;
268 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
269 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
270 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
271 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
272 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest;
273 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse;
274 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
275 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
276 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesRequest;
277 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesResponse;
278 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
279 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
280 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
281 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
282 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
283 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
284 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
285 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
286 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
287 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
288 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
289 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
290 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest;
291 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse;
292 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
293 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
294 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
295 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
296 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest;
297 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse;
298 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
299 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
300 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
301 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
302 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
303 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
304 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest;
305 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse;
306 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RegionSpecifierAndState;
307 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
308 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
309 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
310 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
311 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
312 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
313 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreRequest;
314 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreResponse;
315 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
316 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
317 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
318 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
319 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
320 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse;
321 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
322 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
323 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
324 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
325 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetRegionStateInMetaRequest;
326 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetRegionStateInMetaResponse;
327 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSnapshotCleanupRequest;
328 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSnapshotCleanupResponse;
329 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
330 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
331 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest;
332 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
333 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
334 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
335 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
336 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
337 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse;
338 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
339 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse;
340 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaRequest;
341 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaResponse;
342 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleRequest;
343 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleResponse;
344 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
345 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
346 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
347 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
348 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
349 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
350 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot;
351 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot;
352 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
353 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
354 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
355 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
356 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
357 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.AddRSGroupRequest;
358 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.AddRSGroupResponse;
359 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.BalanceRSGroupRequest;
360 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.BalanceRSGroupResponse;
361 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetConfiguredNamespacesAndTablesInRSGroupRequest;
362 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetConfiguredNamespacesAndTablesInRSGroupResponse;
363 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoOfServerRequest;
364 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoOfServerResponse;
365 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoOfTableRequest;
366 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoOfTableResponse;
367 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoRequest;
368 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoResponse;
369 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosRequest;
370 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosResponse;
371 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListTablesInRSGroupRequest;
372 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListTablesInRSGroupResponse;
373 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersRequest;
374 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersResponse;
375 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
376 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
377 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
378 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersResponse;
379 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RenameRSGroupRequest;
380 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RenameRSGroupResponse;
381 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.UpdateRSGroupConfigRequest;
382 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.UpdateRSGroupConfigResponse;
383 import org.apache.hadoop.hbase.shaded.protobuf.generated.RecentLogs;
384 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationRequest;
385 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationResponse;
386 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
387 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
388 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLiveRegionServersRequest;
389 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLiveRegionServersResponse;
390 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
391 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse;
392 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
393 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
394 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
395 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
396 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
397 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse;
398 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult;
399 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
400 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse;
401 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
402 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
403 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
404 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
405 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService;
406 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
407 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
408 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
409 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
410 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest;
411 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
412 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
413 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
414 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
415 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
416 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
417 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
418 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState;
419 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
420 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
421 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
422 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
423 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
424 import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
427 * Implements the master RPC services.
429 @InterfaceAudience.Private
430 public class MasterRpcServices extends HBaseRpcServicesBase<HMaster>
431 implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
432 LockService.BlockingInterface, HbckService.BlockingInterface {
434 private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
435 private static final Logger AUDITLOG =
436 LoggerFactory.getLogger("SecurityLogger."+MasterRpcServices.class.getName());
438 /** RPC scheduler to use for the master. */
439 public static final String MASTER_RPC_SCHEDULER_FACTORY_CLASS =
440 "hbase.master.rpc.scheduler.factory.class";
443 * @return Subset of configuration to pass initializing regionservers: e.g.
444 * the filesystem to use and root directory to use.
446 private RegionServerStartupResponse.Builder createConfigurationSubset() {
447 RegionServerStartupResponse.Builder resp = addConfig(
448 RegionServerStartupResponse.newBuilder(), HConstants.HBASE_DIR);
449 resp = addConfig(resp, "fs.defaultFS");
450 return addConfig(resp, "hbase.master.info.port");
453 private RegionServerStartupResponse.Builder addConfig(
454 final RegionServerStartupResponse.Builder resp, final String key) {
455 NameStringPair.Builder entry = NameStringPair.newBuilder()
456 .setName(key)
457 .setValue(server.getConfiguration().get(key));
458 resp.addMapEntries(entry.build());
459 return resp;
462 public MasterRpcServices(HMaster m) throws IOException {
463 super(m, m.getProcessName());
466 @Override
467 protected boolean defaultReservoirEnabled() {
468 return false;
471 @Override
472 protected ServerType getDNSServerType() {
473 return DNS.ServerType.MASTER;
476 @Override
477 protected String getHostname(Configuration conf, String defaultHostname) {
478 return conf.get("hbase.master.ipc.address", defaultHostname);
481 @Override
482 protected String getPortConfigName() {
483 return HConstants.MASTER_PORT;
486 @Override
487 protected int getDefaultPort() {
488 return HConstants.DEFAULT_MASTER_PORT;
491 @Override
492 protected Class<?> getRpcSchedulerFactoryClass(Configuration conf) {
493 return conf.getClass(MASTER_RPC_SCHEDULER_FACTORY_CLASS, SimpleRpcSchedulerFactory.class);
496 @Override
497 protected PriorityFunction createPriority() {
498 return new MasterAnnotationReadingPriorityFunction(this);
502 * Checks for the following pre-checks in order:
503 * <ol>
504 * <li>Master is initialized</li>
505 * <li>Rpc caller has admin permissions</li>
506 * </ol>
507 * @param requestName name of rpc request. Used in reporting failures to provide context.
508 * @throws ServiceException If any of the above listed pre-check fails.
510 private void rpcPreCheck(String requestName) throws ServiceException {
511 try {
512 server.checkInitialized();
513 requirePermission(requestName, Permission.Action.ADMIN);
514 } catch (IOException ioe) {
515 throw new ServiceException(ioe);
519 enum BalanceSwitchMode {
520 SYNC,
521 ASYNC
525 * Assigns balancer switch according to BalanceSwitchMode
526 * @param b new balancer switch
527 * @param mode BalanceSwitchMode
528 * @return old balancer switch
530 boolean switchBalancer(final boolean b, BalanceSwitchMode mode) throws IOException {
531 boolean oldValue = server.loadBalancerTracker.isBalancerOn();
532 boolean newValue = b;
533 try {
534 if (server.cpHost != null) {
535 server.cpHost.preBalanceSwitch(newValue);
537 try {
538 if (mode == BalanceSwitchMode.SYNC) {
539 synchronized (server.getLoadBalancer()) {
540 server.loadBalancerTracker.setBalancerOn(newValue);
542 } else {
543 server.loadBalancerTracker.setBalancerOn(newValue);
545 } catch (KeeperException ke) {
546 throw new IOException(ke);
548 LOG.info(server.getClientIdAuditPrefix() + " set balanceSwitch=" + newValue);
549 if (server.cpHost != null) {
550 server.cpHost.postBalanceSwitch(oldValue, newValue);
552 server.getLoadBalancer().updateBalancerStatus(newValue);
553 } catch (IOException ioe) {
554 LOG.warn("Error flipping balance switch", ioe);
556 return oldValue;
559 boolean synchronousBalanceSwitch(final boolean b) throws IOException {
560 return switchBalancer(b, BalanceSwitchMode.SYNC);
564 * @return list of blocking services and their security info classes that this server supports
566 @Override
567 protected List<BlockingServiceAndInterface> getServices() {
568 List<BlockingServiceAndInterface> bssi = new ArrayList<>(5);
569 bssi.add(new BlockingServiceAndInterface(
570 MasterService.newReflectiveBlockingService(this),
571 MasterService.BlockingInterface.class));
572 bssi.add(new BlockingServiceAndInterface(
573 RegionServerStatusService.newReflectiveBlockingService(this),
574 RegionServerStatusService.BlockingInterface.class));
575 bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
576 LockService.BlockingInterface.class));
577 bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this),
578 HbckService.BlockingInterface.class));
579 bssi.add(new BlockingServiceAndInterface(ClientMetaService.newReflectiveBlockingService(this),
580 ClientMetaService.BlockingInterface.class));
581 bssi.add(new BlockingServiceAndInterface(AdminService.newReflectiveBlockingService(this),
582 AdminService.BlockingInterface.class));
583 return bssi;
586 void start(ZKWatcher zkWatcher) {
587 internalStart(zkWatcher);
590 void stop() {
591 internalStop();
594 @Override
595 @QosPriority(priority = HConstants.ADMIN_QOS)
596 public GetLastFlushedSequenceIdResponse getLastFlushedSequenceId(RpcController controller,
597 GetLastFlushedSequenceIdRequest request) throws ServiceException {
598 try {
599 server.checkServiceStarted();
600 } catch (IOException ioe) {
601 throw new ServiceException(ioe);
603 byte[] encodedRegionName = request.getRegionName().toByteArray();
604 RegionStoreSequenceIds ids = server.getServerManager()
605 .getLastFlushedSequenceId(encodedRegionName);
606 return ResponseConverter.buildGetLastFlushedSequenceIdResponse(ids);
609 @Override
610 public RegionServerReportResponse regionServerReport(RpcController controller,
611 RegionServerReportRequest request) throws ServiceException {
612 try {
613 server.checkServiceStarted();
614 int versionNumber = 0;
615 String version = "0.0.0";
616 VersionInfo versionInfo = VersionInfoUtil.getCurrentClientVersionInfo();
617 if (versionInfo != null) {
618 version = versionInfo.getVersion();
619 versionNumber = VersionInfoUtil.getVersionNumber(versionInfo);
621 ClusterStatusProtos.ServerLoad sl = request.getLoad();
622 ServerName serverName = ProtobufUtil.toServerName(request.getServer());
623 ServerMetrics oldLoad = server.getServerManager().getLoad(serverName);
624 ServerMetrics newLoad =
625 ServerMetricsBuilder.toServerMetrics(serverName, versionNumber, version, sl);
626 server.getServerManager().regionServerReport(serverName, newLoad);
627 server.getAssignmentManager().reportOnlineRegions(serverName,
628 newLoad.getRegionMetrics().keySet());
629 if (sl != null && server.metricsMaster != null) {
630 // Up our metrics.
631 server.metricsMaster.incrementRequests(
632 sl.getTotalNumberOfRequests() -
633 (oldLoad != null ? oldLoad.getRequestCount() : 0));
634 server.metricsMaster.incrementReadRequests(sl.getReadRequestsCount() -
635 (oldLoad != null ? oldLoad.getReadRequestsCount() : 0));
636 server.metricsMaster.incrementWriteRequests(sl.getWriteRequestsCount() -
637 (oldLoad != null ? oldLoad.getWriteRequestsCount() : 0));
639 } catch (IOException ioe) {
640 throw new ServiceException(ioe);
642 return RegionServerReportResponse.newBuilder().build();
645 @Override
646 public RegionServerStartupResponse regionServerStartup(RpcController controller,
647 RegionServerStartupRequest request) throws ServiceException {
648 // Register with server manager
649 try {
650 server.checkServiceStarted();
651 int versionNumber = 0;
652 String version = "0.0.0";
653 VersionInfo versionInfo = VersionInfoUtil.getCurrentClientVersionInfo();
654 if (versionInfo != null) {
655 version = versionInfo.getVersion();
656 versionNumber = VersionInfoUtil.getVersionNumber(versionInfo);
658 InetAddress ia = server.getRemoteInetAddress(request.getPort(), request.getServerStartCode());
659 // if regionserver passed hostname to use,
660 // then use it instead of doing a reverse DNS lookup
661 ServerName rs =
662 server.getServerManager().regionServerStartup(request, versionNumber, version, ia);
664 // Send back some config info
665 RegionServerStartupResponse.Builder resp = createConfigurationSubset();
666 NameStringPair.Builder entry = NameStringPair.newBuilder()
667 .setName(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER).setValue(rs.getHostname());
668 resp.addMapEntries(entry.build());
670 return resp.build();
671 } catch (IOException ioe) {
672 throw new ServiceException(ioe);
676 @Override
677 public ReportRSFatalErrorResponse reportRSFatalError(
678 RpcController controller, ReportRSFatalErrorRequest request) throws ServiceException {
679 String errorText = request.getErrorMessage();
680 ServerName sn = ProtobufUtil.toServerName(request.getServer());
681 String msg = sn + " reported a fatal error:\n" + errorText;
682 LOG.warn(msg);
683 server.rsFatals.add(msg);
684 return ReportRSFatalErrorResponse.newBuilder().build();
687 @Override
688 public AddColumnResponse addColumn(RpcController controller,
689 AddColumnRequest req) throws ServiceException {
690 try {
691 long procId = server.addColumn(
692 ProtobufUtil.toTableName(req.getTableName()),
693 ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()),
694 req.getNonceGroup(),
695 req.getNonce());
696 if (procId == -1) {
697 // This mean operation was not performed in server, so do not set any procId
698 return AddColumnResponse.newBuilder().build();
699 } else {
700 return AddColumnResponse.newBuilder().setProcId(procId).build();
702 } catch (IOException ioe) {
703 throw new ServiceException(ioe);
707 @Override
708 public AssignRegionResponse assignRegion(RpcController controller,
709 AssignRegionRequest req) throws ServiceException {
710 try {
711 server.checkInitialized();
713 final RegionSpecifierType type = req.getRegion().getType();
714 if (type != RegionSpecifierType.REGION_NAME) {
715 LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
716 + " actual: " + type);
719 final byte[] regionName = req.getRegion().getValue().toByteArray();
720 final RegionInfo regionInfo = server.getAssignmentManager().getRegionInfo(regionName);
721 if (regionInfo == null) {
722 throw new UnknownRegionException(Bytes.toStringBinary(regionName));
725 final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
726 if (server.cpHost != null) {
727 server.cpHost.preAssign(regionInfo);
729 LOG.info(server.getClientIdAuditPrefix() + " assign " + regionInfo.getRegionNameAsString());
730 server.getAssignmentManager().assign(regionInfo);
731 if (server.cpHost != null) {
732 server.cpHost.postAssign(regionInfo);
734 return arr;
735 } catch (IOException ioe) {
736 throw new ServiceException(ioe);
740 @Override
741 public MasterProtos.BalanceResponse balance(RpcController controller,
742 MasterProtos.BalanceRequest request) throws ServiceException {
743 try {
744 return ProtobufUtil.toBalanceResponse(server.balance(ProtobufUtil.toBalanceRequest(request)));
745 } catch (IOException ex) {
746 throw new ServiceException(ex);
750 @Override
751 public CreateNamespaceResponse createNamespace(RpcController controller,
752 CreateNamespaceRequest request) throws ServiceException {
753 try {
754 long procId = server.createNamespace(
755 ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
756 request.getNonceGroup(),
757 request.getNonce());
758 return CreateNamespaceResponse.newBuilder().setProcId(procId).build();
759 } catch (IOException e) {
760 throw new ServiceException(e);
764 @Override
765 public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
766 throws ServiceException {
767 TableDescriptor tableDescriptor = ProtobufUtil.toTableDescriptor(req.getTableSchema());
768 byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
769 try {
770 long procId =
771 server.createTable(tableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce());
772 LOG.info(server.getClientIdAuditPrefix() + " procedure request for creating table: " +
773 req.getTableSchema().getTableName() + " procId is: " + procId);
774 return CreateTableResponse.newBuilder().setProcId(procId).build();
775 } catch (IOException ioe) {
776 throw new ServiceException(ioe);
780 @Override
781 public DeleteColumnResponse deleteColumn(RpcController controller,
782 DeleteColumnRequest req) throws ServiceException {
783 try {
784 long procId = server.deleteColumn(
785 ProtobufUtil.toTableName(req.getTableName()),
786 req.getColumnName().toByteArray(),
787 req.getNonceGroup(),
788 req.getNonce());
789 if (procId == -1) {
790 // This mean operation was not performed in server, so do not set any procId
791 return DeleteColumnResponse.newBuilder().build();
792 } else {
793 return DeleteColumnResponse.newBuilder().setProcId(procId).build();
795 } catch (IOException ioe) {
796 throw new ServiceException(ioe);
800 @Override
801 public DeleteNamespaceResponse deleteNamespace(RpcController controller,
802 DeleteNamespaceRequest request) throws ServiceException {
803 try {
804 long procId = server.deleteNamespace(
805 request.getNamespaceName(),
806 request.getNonceGroup(),
807 request.getNonce());
808 return DeleteNamespaceResponse.newBuilder().setProcId(procId).build();
809 } catch (IOException e) {
810 throw new ServiceException(e);
815 * Execute Delete Snapshot operation.
816 * @return DeleteSnapshotResponse (a protobuf wrapped void) if the snapshot existed and was
817 * deleted properly.
818 * @throws ServiceException wrapping SnapshotDoesNotExistException if specified snapshot did not
819 * exist.
821 @Override
822 public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
823 DeleteSnapshotRequest request) throws ServiceException {
824 try {
825 server.checkInitialized();
826 server.snapshotManager.checkSnapshotSupport();
828 LOG.info(server.getClientIdAuditPrefix() + " delete " + request.getSnapshot());
829 server.snapshotManager.deleteSnapshot(request.getSnapshot());
830 return DeleteSnapshotResponse.newBuilder().build();
831 } catch (IOException e) {
832 throw new ServiceException(e);
836 @Override
837 public DeleteTableResponse deleteTable(RpcController controller,
838 DeleteTableRequest request) throws ServiceException {
839 try {
840 long procId = server.deleteTable(ProtobufUtil.toTableName(
841 request.getTableName()), request.getNonceGroup(), request.getNonce());
842 return DeleteTableResponse.newBuilder().setProcId(procId).build();
843 } catch (IOException ioe) {
844 throw new ServiceException(ioe);
848 @Override
849 public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
850 throws ServiceException {
851 try {
852 long procId = server.truncateTable(
853 ProtobufUtil.toTableName(request.getTableName()),
854 request.getPreserveSplits(),
855 request.getNonceGroup(),
856 request.getNonce());
857 return TruncateTableResponse.newBuilder().setProcId(procId).build();
858 } catch (IOException ioe) {
859 throw new ServiceException(ioe);
863 @Override
864 public DisableTableResponse disableTable(RpcController controller,
865 DisableTableRequest request) throws ServiceException {
866 try {
867 long procId = server.disableTable(
868 ProtobufUtil.toTableName(request.getTableName()),
869 request.getNonceGroup(),
870 request.getNonce());
871 return DisableTableResponse.newBuilder().setProcId(procId).build();
872 } catch (IOException ioe) {
873 throw new ServiceException(ioe);
877 @Override
878 public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
879 EnableCatalogJanitorRequest req) throws ServiceException {
880 rpcPreCheck("enableCatalogJanitor");
881 return EnableCatalogJanitorResponse.newBuilder().setPrevValue(
882 server.catalogJanitorChore.setEnabled(req.getEnable())).build();
885 @Override
886 public SetCleanerChoreRunningResponse setCleanerChoreRunning(
887 RpcController c, SetCleanerChoreRunningRequest req) throws ServiceException {
888 rpcPreCheck("setCleanerChoreRunning");
890 boolean prevValue =
891 server.getLogCleaner().getEnabled() && server.getHFileCleaner().getEnabled();
892 server.getLogCleaner().setEnabled(req.getOn());
893 server.getHFileCleaner().setEnabled(req.getOn());
894 return SetCleanerChoreRunningResponse.newBuilder().setPrevValue(prevValue).build();
897 @Override
898 public EnableTableResponse enableTable(RpcController controller,
899 EnableTableRequest request) throws ServiceException {
900 try {
901 long procId = server.enableTable(
902 ProtobufUtil.toTableName(request.getTableName()),
903 request.getNonceGroup(),
904 request.getNonce());
905 return EnableTableResponse.newBuilder().setProcId(procId).build();
906 } catch (IOException ioe) {
907 throw new ServiceException(ioe);
911 @Override
912 public MergeTableRegionsResponse mergeTableRegions(
913 RpcController c, MergeTableRegionsRequest request) throws ServiceException {
914 try {
915 server.checkInitialized();
916 } catch (IOException ioe) {
917 throw new ServiceException(ioe);
920 RegionStates regionStates = server.getAssignmentManager().getRegionStates();
922 RegionInfo[] regionsToMerge = new RegionInfo[request.getRegionCount()];
923 for (int i = 0; i < request.getRegionCount(); i++) {
924 final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray();
925 if (request.getRegion(i).getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
926 LOG.warn("MergeRegions specifier type: expected: "
927 + RegionSpecifierType.ENCODED_REGION_NAME + " actual: region " + i + " ="
928 + request.getRegion(i).getType());
930 RegionState regionState = regionStates.getRegionState(Bytes.toString(encodedNameOfRegion));
931 if (regionState == null) {
932 throw new ServiceException(
933 new UnknownRegionException(Bytes.toStringBinary(encodedNameOfRegion)));
935 regionsToMerge[i] = regionState.getRegion();
938 try {
939 long procId = server.mergeRegions(
940 regionsToMerge,
941 request.getForcible(),
942 request.getNonceGroup(),
943 request.getNonce());
944 return MergeTableRegionsResponse.newBuilder().setProcId(procId).build();
945 } catch (IOException ioe) {
946 throw new ServiceException(ioe);
950 @Override
951 public SplitTableRegionResponse splitRegion(final RpcController controller,
952 final SplitTableRegionRequest request) throws ServiceException {
953 try {
954 long procId = server.splitRegion(
955 ProtobufUtil.toRegionInfo(request.getRegionInfo()),
956 request.hasSplitRow() ? request.getSplitRow().toByteArray() : null,
957 request.getNonceGroup(),
958 request.getNonce());
959 return SplitTableRegionResponse.newBuilder().setProcId(procId).build();
960 } catch (IOException ie) {
961 throw new ServiceException(ie);
965 @Override
966 public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
967 final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {
968 rpcPreCheck("execMasterService");
969 try {
970 ServerRpcController execController = new ServerRpcController();
971 ClientProtos.CoprocessorServiceCall call = request.getCall();
972 String serviceName = call.getServiceName();
973 String methodName = call.getMethodName();
974 if (!server.coprocessorServiceHandlers.containsKey(serviceName)) {
975 throw new UnknownProtocolException(null,
976 "No registered Master Coprocessor Endpoint found for " + serviceName +
977 ". Has it been enabled?");
980 Service service = server.coprocessorServiceHandlers.get(serviceName);
981 ServiceDescriptor serviceDesc = service.getDescriptorForType();
982 MethodDescriptor methodDesc =
983 CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
985 Message execRequest =
986 CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
987 final Message.Builder responseBuilder =
988 service.getResponsePrototype(methodDesc).newBuilderForType();
989 service.callMethod(methodDesc, execController, execRequest,
990 (message) -> {
991 if (message != null) {
992 responseBuilder.mergeFrom(message);
995 Message execResult = responseBuilder.build();
996 if (execController.getFailedOn() != null) {
997 throw execController.getFailedOn();
1000 String remoteAddress = RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("");
1001 User caller = RpcServer.getRequestUser().orElse(null);
1002 AUDITLOG.info("User {} (remote address: {}) master service request for {}.{}", caller,
1003 remoteAddress, serviceName, methodName);
1005 return CoprocessorRpcUtils.getResponse(execResult, HConstants.EMPTY_BYTE_ARRAY);
1006 } catch (IOException ie) {
1007 throw new ServiceException(ie);
1012 * Triggers an asynchronous attempt to run a distributed procedure.
1013 * {@inheritDoc}
1015 @Override
1016 public ExecProcedureResponse execProcedure(RpcController controller,
1017 ExecProcedureRequest request) throws ServiceException {
1018 try {
1019 server.checkInitialized();
1020 ProcedureDescription desc = request.getProcedure();
1021 MasterProcedureManager mpm = server.getMasterProcedureManagerHost().getProcedureManager(
1022 desc.getSignature());
1023 if (mpm == null) {
1024 throw new ServiceException(new DoNotRetryIOException("The procedure is not registered: "
1025 + desc.getSignature()));
1027 LOG.info(server.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
1028 mpm.checkPermissions(desc, getAccessChecker(), RpcServer.getRequestUser().orElse(null));
1029 mpm.execProcedure(desc);
1030 // send back the max amount of time the client should wait for the procedure
1031 // to complete
1032 long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME;
1033 return ExecProcedureResponse.newBuilder().setExpectedTimeout(
1034 waitTime).build();
1035 } catch (ForeignException e) {
1036 throw new ServiceException(e.getCause());
1037 } catch (IOException e) {
1038 throw new ServiceException(e);
1043 * Triggers a synchronous attempt to run a distributed procedure and sets
1044 * return data in response.
1045 * {@inheritDoc}
1047 @Override
1048 public ExecProcedureResponse execProcedureWithRet(RpcController controller,
1049 ExecProcedureRequest request) throws ServiceException {
1050 rpcPreCheck("execProcedureWithRet");
1051 try {
1052 ProcedureDescription desc = request.getProcedure();
1053 MasterProcedureManager mpm =
1054 server.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature());
1055 if (mpm == null) {
1056 throw new ServiceException("The procedure is not registered: " + desc.getSignature());
1058 LOG.info(server.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
1059 byte[] data = mpm.execProcedureWithRet(desc);
1060 ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder();
1061 // set return data if available
1062 if (data != null) {
1063 builder.setReturnData(UnsafeByteOperations.unsafeWrap(data));
1065 return builder.build();
1066 } catch (IOException e) {
1067 throw new ServiceException(e);
1071 @Override
1072 public GetClusterStatusResponse getClusterStatus(RpcController controller,
1073 GetClusterStatusRequest req) throws ServiceException {
1074 GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
1075 try {
1076 // We used to check if Master was up at this point but let this call proceed even if
1077 // Master is initializing... else we shut out stuff like hbck2 tool from making progress
1078 // since it queries this method to figure cluster version. hbck2 wants to be able to work
1079 // against Master even if it is 'initializing' so it can do fixup.
1080 response.setClusterStatus(ClusterMetricsBuilder.toClusterStatus(
1081 server.getClusterMetrics(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
1082 } catch (IOException e) {
1083 throw new ServiceException(e);
1085 return response.build();
1089 * List the currently available/stored snapshots. Any in-progress snapshots are ignored
1091 @Override
1092 public GetCompletedSnapshotsResponse getCompletedSnapshots(RpcController controller,
1093 GetCompletedSnapshotsRequest request) throws ServiceException {
1094 try {
1095 server.checkInitialized();
1096 GetCompletedSnapshotsResponse.Builder builder = GetCompletedSnapshotsResponse.newBuilder();
1097 List<SnapshotDescription> snapshots = server.snapshotManager.getCompletedSnapshots();
1099 // convert to protobuf
1100 for (SnapshotDescription snapshot : snapshots) {
1101 builder.addSnapshots(snapshot);
1103 return builder.build();
1104 } catch (IOException e) {
1105 throw new ServiceException(e);
1109 @Override
1110 public ListNamespacesResponse listNamespaces(
1111 RpcController controller, ListNamespacesRequest request)
1112 throws ServiceException {
1113 try {
1114 return ListNamespacesResponse.newBuilder()
1115 .addAllNamespaceName(server.listNamespaces())
1116 .build();
1117 } catch (IOException e) {
1118 throw new ServiceException(e);
1122 @Override
1123 public GetNamespaceDescriptorResponse getNamespaceDescriptor(
1124 RpcController controller, GetNamespaceDescriptorRequest request)
1125 throws ServiceException {
1126 try {
1127 return GetNamespaceDescriptorResponse.newBuilder()
1128 .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
1129 server.getNamespace(request.getNamespaceName())))
1130 .build();
1131 } catch (IOException e) {
1132 throw new ServiceException(e);
1137 * Get the number of regions of the table that have been updated by the alter.
1139 * @return Pair indicating the number of regions updated Pair.getFirst is the
1140 * regions that are yet to be updated Pair.getSecond is the total number
1141 * of regions of the table
1142 * @throws ServiceException
1144 @Override
1145 public GetSchemaAlterStatusResponse getSchemaAlterStatus(
1146 RpcController controller, GetSchemaAlterStatusRequest req) throws ServiceException {
1147 // TODO: currently, we query using the table name on the client side. this
1148 // may overlap with other table operations or the table operation may
1149 // have completed before querying this API. We need to refactor to a
1150 // transaction system in the future to avoid these ambiguities.
1151 TableName tableName = ProtobufUtil.toTableName(req.getTableName());
1153 try {
1154 server.checkInitialized();
1155 Pair<Integer,Integer> pair = server.getAssignmentManager().getReopenStatus(tableName);
1156 GetSchemaAlterStatusResponse.Builder ret = GetSchemaAlterStatusResponse.newBuilder();
1157 ret.setYetToUpdateRegions(pair.getFirst());
1158 ret.setTotalRegions(pair.getSecond());
1159 return ret.build();
1160 } catch (IOException ioe) {
1161 throw new ServiceException(ioe);
1166 * Get list of TableDescriptors for requested tables.
1167 * @param c Unused (set to null).
1168 * @param req GetTableDescriptorsRequest that contains:
1169 * - tableNames: requested tables, or if empty, all are requested.
1170 * @return GetTableDescriptorsResponse
1171 * @throws ServiceException
1173 @Override
1174 public GetTableDescriptorsResponse getTableDescriptors(RpcController c,
1175 GetTableDescriptorsRequest req) throws ServiceException {
1176 try {
1177 server.checkInitialized();
1179 final String regex = req.hasRegex() ? req.getRegex() : null;
1180 final String namespace = req.hasNamespace() ? req.getNamespace() : null;
1181 List<TableName> tableNameList = null;
1182 if (req.getTableNamesCount() > 0) {
1183 tableNameList = new ArrayList<TableName>(req.getTableNamesCount());
1184 for (HBaseProtos.TableName tableNamePB: req.getTableNamesList()) {
1185 tableNameList.add(ProtobufUtil.toTableName(tableNamePB));
1189 List<TableDescriptor> descriptors = server.listTableDescriptors(namespace, regex,
1190 tableNameList, req.getIncludeSysTables());
1192 GetTableDescriptorsResponse.Builder builder = GetTableDescriptorsResponse.newBuilder();
1193 if (descriptors != null && descriptors.size() > 0) {
1194 // Add the table descriptors to the response
1195 for (TableDescriptor htd: descriptors) {
1196 builder.addTableSchema(ProtobufUtil.toTableSchema(htd));
1199 return builder.build();
1200 } catch (IOException ioe) {
1201 throw new ServiceException(ioe);
1206 * Get list of userspace table names
1207 * @param controller Unused (set to null).
1208 * @param req GetTableNamesRequest
1209 * @return GetTableNamesResponse
1210 * @throws ServiceException
1212 @Override
1213 public GetTableNamesResponse getTableNames(RpcController controller,
1214 GetTableNamesRequest req) throws ServiceException {
1215 try {
1216 server.checkServiceStarted();
1218 final String regex = req.hasRegex() ? req.getRegex() : null;
1219 final String namespace = req.hasNamespace() ? req.getNamespace() : null;
1220 List<TableName> tableNames = server.listTableNames(namespace, regex,
1221 req.getIncludeSysTables());
1223 GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder();
1224 if (tableNames != null && tableNames.size() > 0) {
1225 // Add the table names to the response
1226 for (TableName table: tableNames) {
1227 builder.addTableNames(ProtobufUtil.toProtoTableName(table));
1230 return builder.build();
1231 } catch (IOException e) {
1232 throw new ServiceException(e);
1236 @Override
1237 public GetTableStateResponse getTableState(RpcController controller,
1238 GetTableStateRequest request) throws ServiceException {
1239 try {
1240 server.checkServiceStarted();
1241 TableName tableName = ProtobufUtil.toTableName(request.getTableName());
1242 TableState ts = server.getTableStateManager().getTableState(tableName);
1243 GetTableStateResponse.Builder builder = GetTableStateResponse.newBuilder();
1244 builder.setTableState(ts.convert());
1245 return builder.build();
1246 } catch (IOException e) {
1247 throw new ServiceException(e);
1251 @Override
1252 public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c,
1253 IsCatalogJanitorEnabledRequest req) throws ServiceException {
1254 return IsCatalogJanitorEnabledResponse.newBuilder().setValue(
1255 server.isCatalogJanitorEnabled()).build();
1258 @Override
1259 public IsCleanerChoreEnabledResponse isCleanerChoreEnabled(RpcController c,
1260 IsCleanerChoreEnabledRequest req)
1261 throws ServiceException {
1262 return IsCleanerChoreEnabledResponse.newBuilder().setValue(server.isCleanerChoreEnabled())
1263 .build();
1266 @Override
1267 public IsMasterRunningResponse isMasterRunning(RpcController c,
1268 IsMasterRunningRequest req) throws ServiceException {
1269 try {
1270 server.checkServiceStarted();
1271 return IsMasterRunningResponse.newBuilder().setIsMasterRunning(
1272 !server.isStopped()).build();
1273 } catch (IOException e) {
1274 throw new ServiceException(e);
1279 * Checks if the specified procedure is done.
1280 * @return true if the procedure is done, false if the procedure is in the process of completing
1281 * @throws ServiceException if invalid procedure or failed procedure with progress failure reason.
1283 @Override
1284 public IsProcedureDoneResponse isProcedureDone(RpcController controller,
1285 IsProcedureDoneRequest request) throws ServiceException {
1286 try {
1287 server.checkInitialized();
1288 ProcedureDescription desc = request.getProcedure();
1289 MasterProcedureManager mpm = server.getMasterProcedureManagerHost().getProcedureManager(
1290 desc.getSignature());
1291 if (mpm == null) {
1292 throw new ServiceException("The procedure is not registered: "
1293 + desc.getSignature());
1295 LOG.debug("Checking to see if procedure from request:"
1296 + desc.getSignature() + " is done");
1298 IsProcedureDoneResponse.Builder builder =
1299 IsProcedureDoneResponse.newBuilder();
1300 boolean done = mpm.isProcedureDone(desc);
1301 builder.setDone(done);
1302 return builder.build();
1303 } catch (ForeignException e) {
1304 throw new ServiceException(e.getCause());
1305 } catch (IOException e) {
1306 throw new ServiceException(e);
1311 * Checks if the specified snapshot is done.
1312 * @return true if the snapshot is in file system ready to use,
1313 * false if the snapshot is in the process of completing
1314 * @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or
1315 * a wrapped HBaseSnapshotException with progress failure reason.
1317 @Override
1318 public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
1319 IsSnapshotDoneRequest request) throws ServiceException {
1320 LOG.debug("Checking to see if snapshot from request:" +
1321 ClientSnapshotDescriptionUtils.toString(request.getSnapshot()) + " is done");
1322 try {
1323 server.checkInitialized();
1324 IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
1325 boolean done = server.snapshotManager.isSnapshotDone(request.getSnapshot());
1326 builder.setDone(done);
1327 return builder.build();
1328 } catch (ForeignException e) {
1329 throw new ServiceException(e.getCause());
1330 } catch (IOException e) {
1331 throw new ServiceException(e);
1335 @Override
1336 public GetProcedureResultResponse getProcedureResult(RpcController controller,
1337 GetProcedureResultRequest request) throws ServiceException {
1338 LOG.debug("Checking to see if procedure is done pid=" + request.getProcId());
1339 try {
1340 server.checkInitialized();
1341 GetProcedureResultResponse.Builder builder = GetProcedureResultResponse.newBuilder();
1342 long procId = request.getProcId();
1343 ProcedureExecutor<?> executor = server.getMasterProcedureExecutor();
1344 Procedure<?> result = executor.getResultOrProcedure(procId);
1345 if (result != null) {
1346 builder.setSubmittedTime(result.getSubmittedTime());
1347 builder.setLastUpdate(result.getLastUpdate());
1348 if (executor.isFinished(procId)) {
1349 builder.setState(GetProcedureResultResponse.State.FINISHED);
1350 if (result.isFailed()) {
1351 IOException exception =
1352 MasterProcedureUtil.unwrapRemoteIOException(result);
1353 builder.setException(ForeignExceptionUtil.toProtoForeignException(exception));
1355 byte[] resultData = result.getResult();
1356 if (resultData != null) {
1357 builder.setResult(UnsafeByteOperations.unsafeWrap(resultData));
1359 server.getMasterProcedureExecutor().removeResult(request.getProcId());
1360 } else {
1361 builder.setState(GetProcedureResultResponse.State.RUNNING);
1363 } else {
1364 builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
1366 return builder.build();
1367 } catch (IOException e) {
1368 throw new ServiceException(e);
1372 @Override
1373 public AbortProcedureResponse abortProcedure(
1374 RpcController rpcController, AbortProcedureRequest request) throws ServiceException {
1375 try {
1376 AbortProcedureResponse.Builder response = AbortProcedureResponse.newBuilder();
1377 boolean abortResult =
1378 server.abortProcedure(request.getProcId(), request.getMayInterruptIfRunning());
1379 response.setIsProcedureAborted(abortResult);
1380 return response.build();
1381 } catch (IOException e) {
1382 throw new ServiceException(e);
1386 @Override
1387 public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController c,
1388 ListNamespaceDescriptorsRequest request) throws ServiceException {
1389 try {
1390 ListNamespaceDescriptorsResponse.Builder response =
1391 ListNamespaceDescriptorsResponse.newBuilder();
1392 for(NamespaceDescriptor ns: server.getNamespaces()) {
1393 response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
1395 return response.build();
1396 } catch (IOException e) {
1397 throw new ServiceException(e);
1401 @Override
1402 public GetProceduresResponse getProcedures(
1403 RpcController rpcController,
1404 GetProceduresRequest request) throws ServiceException {
1405 try {
1406 final GetProceduresResponse.Builder response = GetProceduresResponse.newBuilder();
1407 for (Procedure<?> p: server.getProcedures()) {
1408 response.addProcedure(ProcedureUtil.convertToProtoProcedure(p));
1410 return response.build();
1411 } catch (IOException e) {
1412 throw new ServiceException(e);
1416 @Override
1417 public GetLocksResponse getLocks(
1418 RpcController controller,
1419 GetLocksRequest request) throws ServiceException {
1420 try {
1421 final GetLocksResponse.Builder builder = GetLocksResponse.newBuilder();
1423 for (LockedResource lockedResource: server.getLocks()) {
1424 builder.addLock(ProcedureUtil.convertToProtoLockedResource(lockedResource));
1427 return builder.build();
1428 } catch (IOException e) {
1429 throw new ServiceException(e);
1433 @Override
1434 public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(RpcController c,
1435 ListTableDescriptorsByNamespaceRequest request) throws ServiceException {
1436 try {
1437 ListTableDescriptorsByNamespaceResponse.Builder b =
1438 ListTableDescriptorsByNamespaceResponse.newBuilder();
1439 for (TableDescriptor htd : server
1440 .listTableDescriptorsByNamespace(request.getNamespaceName())) {
1441 b.addTableSchema(ProtobufUtil.toTableSchema(htd));
1443 return b.build();
1444 } catch (IOException e) {
1445 throw new ServiceException(e);
1449 @Override
1450 public ListTableNamesByNamespaceResponse listTableNamesByNamespace(RpcController c,
1451 ListTableNamesByNamespaceRequest request) throws ServiceException {
1452 try {
1453 ListTableNamesByNamespaceResponse.Builder b =
1454 ListTableNamesByNamespaceResponse.newBuilder();
1455 for (TableName tableName: server.listTableNamesByNamespace(request.getNamespaceName())) {
1456 b.addTableName(ProtobufUtil.toProtoTableName(tableName));
1458 return b.build();
1459 } catch (IOException e) {
1460 throw new ServiceException(e);
1464 @Override
1465 public ModifyColumnResponse modifyColumn(RpcController controller,
1466 ModifyColumnRequest req) throws ServiceException {
1467 try {
1468 long procId = server.modifyColumn(
1469 ProtobufUtil.toTableName(req.getTableName()),
1470 ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()),
1471 req.getNonceGroup(),
1472 req.getNonce());
1473 if (procId == -1) {
1474 // This mean operation was not performed in server, so do not set any procId
1475 return ModifyColumnResponse.newBuilder().build();
1476 } else {
1477 return ModifyColumnResponse.newBuilder().setProcId(procId).build();
1479 } catch (IOException ioe) {
1480 throw new ServiceException(ioe);
1484 @Override
1485 public ModifyColumnStoreFileTrackerResponse modifyColumnStoreFileTracker(
1486 RpcController controller, ModifyColumnStoreFileTrackerRequest req)
1487 throws ServiceException {
1488 try {
1489 long procId =
1490 server.modifyColumnStoreFileTracker(ProtobufUtil.toTableName(req.getTableName()),
1491 req.getFamily().toByteArray(), req.getDstSft(), req.getNonceGroup(), req.getNonce());
1492 return ModifyColumnStoreFileTrackerResponse.newBuilder().setProcId(procId).build();
1493 } catch (IOException ioe) {
1494 throw new ServiceException(ioe);
1498 @Override
1499 public ModifyNamespaceResponse modifyNamespace(RpcController controller,
1500 ModifyNamespaceRequest request) throws ServiceException {
1501 try {
1502 long procId = server.modifyNamespace(
1503 ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
1504 request.getNonceGroup(),
1505 request.getNonce());
1506 return ModifyNamespaceResponse.newBuilder().setProcId(procId).build();
1507 } catch (IOException e) {
1508 throw new ServiceException(e);
1512 @Override
1513 public ModifyTableResponse modifyTable(RpcController controller,
1514 ModifyTableRequest req) throws ServiceException {
1515 try {
1516 long procId = server.modifyTable(
1517 ProtobufUtil.toTableName(req.getTableName()),
1518 ProtobufUtil.toTableDescriptor(req.getTableSchema()),
1519 req.getNonceGroup(),
1520 req.getNonce());
1521 return ModifyTableResponse.newBuilder().setProcId(procId).build();
1522 } catch (IOException ioe) {
1523 throw new ServiceException(ioe);
1527 @Override
1528 public ModifyTableStoreFileTrackerResponse modifyTableStoreFileTracker(RpcController controller,
1529 ModifyTableStoreFileTrackerRequest req) throws ServiceException {
1530 try {
1531 long procId = server.modifyTableStoreFileTracker(ProtobufUtil.toTableName(req.getTableName()),
1532 req.getDstSft(), req.getNonceGroup(), req.getNonce());
1533 return ModifyTableStoreFileTrackerResponse.newBuilder().setProcId(procId).build();
1534 } catch (IOException ioe) {
1535 throw new ServiceException(ioe);
1539 @Override
1540 public MoveRegionResponse moveRegion(RpcController controller,
1541 MoveRegionRequest req) throws ServiceException {
1542 final byte [] encodedRegionName = req.getRegion().getValue().toByteArray();
1543 RegionSpecifierType type = req.getRegion().getType();
1544 final byte [] destServerName = (req.hasDestServerName())?
1545 Bytes.toBytes(ProtobufUtil.toServerName(req.getDestServerName()).getServerName()):null;
1546 MoveRegionResponse mrr = MoveRegionResponse.newBuilder().build();
1548 if (type != RegionSpecifierType.ENCODED_REGION_NAME) {
1549 LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME
1550 + " actual: " + type);
1553 try {
1554 server.checkInitialized();
1555 server.move(encodedRegionName, destServerName);
1556 } catch (IOException ioe) {
1557 throw new ServiceException(ioe);
1559 return mrr;
1563 * Offline specified region from master's in-memory state. It will not attempt to
1564 * reassign the region as in unassign.
1566 * This is a special method that should be used by experts or hbck.
1569 @Override
1570 public OfflineRegionResponse offlineRegion(RpcController controller,
1571 OfflineRegionRequest request) throws ServiceException {
1572 try {
1573 server.checkInitialized();
1575 final RegionSpecifierType type = request.getRegion().getType();
1576 if (type != RegionSpecifierType.REGION_NAME) {
1577 LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
1578 + " actual: " + type);
1581 final byte[] regionName = request.getRegion().getValue().toByteArray();
1582 final RegionInfo hri = server.getAssignmentManager().getRegionInfo(regionName);
1583 if (hri == null) {
1584 throw new UnknownRegionException(Bytes.toStringBinary(regionName));
1587 if (server.cpHost != null) {
1588 server.cpHost.preRegionOffline(hri);
1590 LOG.info(server.getClientIdAuditPrefix() + " offline " + hri.getRegionNameAsString());
1591 server.getAssignmentManager().offlineRegion(hri);
1592 if (server.cpHost != null) {
1593 server.cpHost.postRegionOffline(hri);
1595 } catch (IOException ioe) {
1596 throw new ServiceException(ioe);
1598 return OfflineRegionResponse.newBuilder().build();
1602 * Execute Restore/Clone snapshot operation.
1604 * <p>If the specified table exists a "Restore" is executed, replacing the table
1605 * schema and directory data with the content of the snapshot.
1606 * The table must be disabled, or a UnsupportedOperationException will be thrown.
1608 * <p>If the table doesn't exist a "Clone" is executed, a new table is created
1609 * using the schema at the time of the snapshot, and the content of the snapshot.
1611 * <p>The restore/clone operation does not require copying HFiles. Since HFiles
1612 * are immutable the table can point to and use the same files as the original one.
1614 @Override
1615 public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
1616 RestoreSnapshotRequest request) throws ServiceException {
1617 try {
1618 long procId = server.restoreSnapshot(request.getSnapshot(), request.getNonceGroup(),
1619 request.getNonce(), request.getRestoreACL(), request.getCustomSFT());
1620 return RestoreSnapshotResponse.newBuilder().setProcId(procId).build();
1621 } catch (ForeignException e) {
1622 throw new ServiceException(e.getCause());
1623 } catch (IOException e) {
1624 throw new ServiceException(e);
1628 @Override
1629 public SetSnapshotCleanupResponse switchSnapshotCleanup(
1630 RpcController controller, SetSnapshotCleanupRequest request)
1631 throws ServiceException {
1632 try {
1633 server.checkInitialized();
1634 final boolean enabled = request.getEnabled();
1635 final boolean isSynchronous = request.hasSynchronous() && request.getSynchronous();
1636 final boolean prevSnapshotCleanupRunning = this.switchSnapshotCleanup(enabled, isSynchronous);
1637 return SetSnapshotCleanupResponse.newBuilder()
1638 .setPrevSnapshotCleanup(prevSnapshotCleanupRunning).build();
1639 } catch (IOException e) {
1640 throw new ServiceException(e);
1644 @Override
1645 public IsSnapshotCleanupEnabledResponse isSnapshotCleanupEnabled(
1646 RpcController controller, IsSnapshotCleanupEnabledRequest request)
1647 throws ServiceException {
1648 try {
1649 server.checkInitialized();
1650 final boolean isSnapshotCleanupEnabled = server.snapshotCleanupTracker
1651 .isSnapshotCleanupEnabled();
1652 return IsSnapshotCleanupEnabledResponse.newBuilder()
1653 .setEnabled(isSnapshotCleanupEnabled).build();
1654 } catch (IOException e) {
1655 throw new ServiceException(e);
1660 * Turn on/off snapshot auto-cleanup based on TTL
1662 * @param enabledNewVal Set to <code>true</code> to enable, <code>false</code> to disable
1663 * @param synchronous If <code>true</code>, it waits until current snapshot cleanup is completed,
1664 * if outstanding
1665 * @return previous snapshot auto-cleanup mode
1667 private synchronized boolean switchSnapshotCleanup(final boolean enabledNewVal,
1668 final boolean synchronous) {
1669 final boolean oldValue = server.snapshotCleanupTracker.isSnapshotCleanupEnabled();
1670 server.switchSnapshotCleanup(enabledNewVal, synchronous);
1671 LOG.info("{} Successfully set snapshot cleanup to {}", server.getClientIdAuditPrefix(),
1672 enabledNewVal);
1673 return oldValue;
1677 @Override
1678 public RunCatalogScanResponse runCatalogScan(RpcController c,
1679 RunCatalogScanRequest req) throws ServiceException {
1680 rpcPreCheck("runCatalogScan");
1681 try {
1682 return ResponseConverter.buildRunCatalogScanResponse(
1683 this.server.catalogJanitorChore.scan());
1684 } catch (IOException ioe) {
1685 throw new ServiceException(ioe);
1689 @Override
1690 public RunCleanerChoreResponse runCleanerChore(RpcController c, RunCleanerChoreRequest req)
1691 throws ServiceException {
1692 rpcPreCheck("runCleanerChore");
1693 boolean result = server.getHFileCleaner().runCleaner() && server.getLogCleaner().runCleaner();
1694 return ResponseConverter.buildRunCleanerChoreResponse(result);
1697 @Override
1698 public SetBalancerRunningResponse setBalancerRunning(RpcController c,
1699 SetBalancerRunningRequest req) throws ServiceException {
1700 try {
1701 server.checkInitialized();
1702 boolean prevValue = (req.getSynchronous())?
1703 synchronousBalanceSwitch(req.getOn()) : server.balanceSwitch(req.getOn());
1704 return SetBalancerRunningResponse.newBuilder().setPrevBalanceValue(prevValue).build();
1705 } catch (IOException ioe) {
1706 throw new ServiceException(ioe);
1710 @Override
1711 public ShutdownResponse shutdown(RpcController controller,
1712 ShutdownRequest request) throws ServiceException {
1713 LOG.info(server.getClientIdAuditPrefix() + " shutdown");
1714 try {
1715 server.shutdown();
1716 } catch (IOException e) {
1717 LOG.error("Exception occurred in HMaster.shutdown()", e);
1718 throw new ServiceException(e);
1720 return ShutdownResponse.newBuilder().build();
1724 * Triggers an asynchronous attempt to take a snapshot.
1725 * {@inheritDoc}
1727 @Override
1728 public SnapshotResponse snapshot(RpcController controller,
1729 SnapshotRequest request) throws ServiceException {
1730 try {
1731 server.checkInitialized();
1732 server.snapshotManager.checkSnapshotSupport();
1734 LOG.info(server.getClientIdAuditPrefix() + " snapshot request for:" +
1735 ClientSnapshotDescriptionUtils.toString(request.getSnapshot()));
1736 // get the snapshot information
1737 SnapshotDescription snapshot = SnapshotDescriptionUtils.validate(
1738 request.getSnapshot(), server.getConfiguration());
1739 server.snapshotManager.takeSnapshot(snapshot);
1741 // send back the max amount of time the client should wait for the snapshot to complete
1742 long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(server.getConfiguration(),
1743 snapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
1744 return SnapshotResponse.newBuilder().setExpectedTimeout(waitTime).build();
1745 } catch (ForeignException e) {
1746 throw new ServiceException(e.getCause());
1747 } catch (IOException e) {
1748 throw new ServiceException(e);
1752 @Override
1753 public StopMasterResponse stopMaster(RpcController controller,
1754 StopMasterRequest request) throws ServiceException {
1755 LOG.info(server.getClientIdAuditPrefix() + " stop");
1756 try {
1757 server.stopMaster();
1758 } catch (IOException e) {
1759 LOG.error("Exception occurred while stopping master", e);
1760 throw new ServiceException(e);
1762 return StopMasterResponse.newBuilder().build();
1765 @Override
1766 public IsInMaintenanceModeResponse isMasterInMaintenanceMode(
1767 final RpcController controller,
1768 final IsInMaintenanceModeRequest request) throws ServiceException {
1769 IsInMaintenanceModeResponse.Builder response = IsInMaintenanceModeResponse.newBuilder();
1770 response.setInMaintenanceMode(server.isInMaintenanceMode());
1771 return response.build();
1774 @Override
1775 public UnassignRegionResponse unassignRegion(RpcController controller,
1776 UnassignRegionRequest req) throws ServiceException {
1777 try {
1778 final byte [] regionName = req.getRegion().getValue().toByteArray();
1779 RegionSpecifierType type = req.getRegion().getType();
1780 UnassignRegionResponse urr = UnassignRegionResponse.newBuilder().build();
1782 server.checkInitialized();
1783 if (type != RegionSpecifierType.REGION_NAME) {
1784 LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
1785 + " actual: " + type);
1787 RegionStateNode rsn =
1788 server.getAssignmentManager().getRegionStates().getRegionStateNodeFromName(regionName);
1789 if (rsn == null) {
1790 throw new UnknownRegionException(Bytes.toString(regionName));
1793 RegionInfo hri = rsn.getRegionInfo();
1794 if (server.cpHost != null) {
1795 server.cpHost.preUnassign(hri);
1797 LOG.debug(server.getClientIdAuditPrefix() + " unassign " + hri.getRegionNameAsString()
1798 + " in current location if it is online");
1799 server.getAssignmentManager().unassign(hri);
1800 if (server.cpHost != null) {
1801 server.cpHost.postUnassign(hri);
1804 return urr;
1805 } catch (IOException ioe) {
1806 throw new ServiceException(ioe);
1810 @Override
1811 public ReportRegionStateTransitionResponse reportRegionStateTransition(RpcController c,
1812 ReportRegionStateTransitionRequest req) throws ServiceException {
1813 try {
1814 server.checkServiceStarted();
1815 return server.getAssignmentManager().reportRegionStateTransition(req);
1816 } catch (IOException ioe) {
1817 throw new ServiceException(ioe);
1821 @Override
1822 public SetQuotaResponse setQuota(RpcController c, SetQuotaRequest req)
1823 throws ServiceException {
1824 try {
1825 server.checkInitialized();
1826 return server.getMasterQuotaManager().setQuota(req);
1827 } catch (Exception e) {
1828 throw new ServiceException(e);
1832 @Override
1833 public MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(RpcController controller,
1834 MajorCompactionTimestampRequest request) throws ServiceException {
1835 MajorCompactionTimestampResponse.Builder response =
1836 MajorCompactionTimestampResponse.newBuilder();
1837 try {
1838 server.checkInitialized();
1839 response.setCompactionTimestamp(server.getLastMajorCompactionTimestamp(ProtobufUtil
1840 .toTableName(request.getTableName())));
1841 } catch (IOException e) {
1842 throw new ServiceException(e);
1844 return response.build();
1847 @Override
1848 public MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion(
1849 RpcController controller, MajorCompactionTimestampForRegionRequest request)
1850 throws ServiceException {
1851 MajorCompactionTimestampResponse.Builder response =
1852 MajorCompactionTimestampResponse.newBuilder();
1853 try {
1854 server.checkInitialized();
1855 response.setCompactionTimestamp(server.getLastMajorCompactionTimestampForRegion(request
1856 .getRegion().getValue().toByteArray()));
1857 } catch (IOException e) {
1858 throw new ServiceException(e);
1860 return response.build();
1863 @Override
1864 public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
1865 IsBalancerEnabledRequest request) throws ServiceException {
1866 IsBalancerEnabledResponse.Builder response = IsBalancerEnabledResponse.newBuilder();
1867 response.setEnabled(server.isBalancerOn());
1868 return response.build();
1871 @Override
1872 public SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(RpcController controller,
1873 SetSplitOrMergeEnabledRequest request) throws ServiceException {
1874 SetSplitOrMergeEnabledResponse.Builder response = SetSplitOrMergeEnabledResponse.newBuilder();
1875 try {
1876 server.checkInitialized();
1877 boolean newValue = request.getEnabled();
1878 for (MasterProtos.MasterSwitchType masterSwitchType: request.getSwitchTypesList()) {
1879 MasterSwitchType switchType = convert(masterSwitchType);
1880 boolean oldValue = server.isSplitOrMergeEnabled(switchType);
1881 response.addPrevValue(oldValue);
1882 if (server.cpHost != null) {
1883 server.cpHost.preSetSplitOrMergeEnabled(newValue, switchType);
1885 server.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue, switchType);
1886 if (server.cpHost != null) {
1887 server.cpHost.postSetSplitOrMergeEnabled(newValue, switchType);
1890 } catch (IOException | KeeperException e) {
1891 throw new ServiceException(e);
1893 return response.build();
1896 @Override
1897 public IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(RpcController controller,
1898 IsSplitOrMergeEnabledRequest request) throws ServiceException {
1899 IsSplitOrMergeEnabledResponse.Builder response = IsSplitOrMergeEnabledResponse.newBuilder();
1900 response.setEnabled(server.isSplitOrMergeEnabled(convert(request.getSwitchType())));
1901 return response.build();
1904 @Override
1905 public NormalizeResponse normalize(RpcController controller,
1906 NormalizeRequest request) throws ServiceException {
1907 rpcPreCheck("normalize");
1908 try {
1909 final NormalizeTableFilterParams ntfp = new NormalizeTableFilterParams.Builder()
1910 .tableNames(ProtobufUtil.toTableNameList(request.getTableNamesList()))
1911 .regex(request.hasRegex() ? request.getRegex() : null)
1912 .namespace(request.hasNamespace() ? request.getNamespace() : null)
1913 .build();
1914 return NormalizeResponse.newBuilder()
1915 // all API requests are considered priority requests.
1916 .setNormalizerRan(server.normalizeRegions(ntfp, true))
1917 .build();
1918 } catch (IOException ex) {
1919 throw new ServiceException(ex);
1923 @Override
1924 public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller,
1925 SetNormalizerRunningRequest request) throws ServiceException {
1926 rpcPreCheck("setNormalizerRunning");
1928 // Sets normalizer on/off flag in ZK.
1929 // TODO: this method is totally broken in terms of atomicity of actions and values read.
1930 // 1. The contract has this RPC returning the previous value. There isn't a ZKUtil method
1931 // that lets us retrieve the previous value as part of setting a new value, so we simply
1932 // perform a read before issuing the update. Thus we have a data race opportunity, between
1933 // when the `prevValue` is read and whatever is actually overwritten.
1934 // 2. Down in `setNormalizerOn`, the call to `createAndWatch` inside of the catch clause can
1935 // itself fail in the event that the znode already exists. Thus, another data race, between
1936 // when the initial `setData` call is notified of the absence of the target znode and the
1937 // subsequent `createAndWatch`, with another client creating said node.
1938 // That said, there's supposed to be only one active master and thus there's supposed to be
1939 // only one process with the authority to modify the value.
1940 final boolean prevValue = server.getRegionNormalizerManager().isNormalizerOn();
1941 final boolean newValue = request.getOn();
1942 server.getRegionNormalizerManager().setNormalizerOn(newValue);
1943 LOG.info("{} set normalizerSwitch={}", server.getClientIdAuditPrefix(), newValue);
1944 return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
1947 @Override
1948 public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller,
1949 IsNormalizerEnabledRequest request) {
1950 IsNormalizerEnabledResponse.Builder response = IsNormalizerEnabledResponse.newBuilder();
1951 response.setEnabled(server.isNormalizerOn());
1952 return response.build();
1956 * Returns the security capabilities in effect on the cluster
1958 @Override
1959 public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller,
1960 SecurityCapabilitiesRequest request) throws ServiceException {
1961 SecurityCapabilitiesResponse.Builder response = SecurityCapabilitiesResponse.newBuilder();
1962 try {
1963 server.checkInitialized();
1964 Set<SecurityCapabilitiesResponse.Capability> capabilities = new HashSet<>();
1965 // Authentication
1966 if (User.isHBaseSecurityEnabled(server.getConfiguration())) {
1967 capabilities.add(SecurityCapabilitiesResponse.Capability.SECURE_AUTHENTICATION);
1968 } else {
1969 capabilities.add(SecurityCapabilitiesResponse.Capability.SIMPLE_AUTHENTICATION);
1971 // A coprocessor that implements AccessControlService can provide AUTHORIZATION and
1972 // CELL_AUTHORIZATION
1973 if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
1974 if (AccessChecker.isAuthorizationSupported(server.getConfiguration())) {
1975 capabilities.add(SecurityCapabilitiesResponse.Capability.AUTHORIZATION);
1977 if (AccessController.isCellAuthorizationSupported(server.getConfiguration())) {
1978 capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_AUTHORIZATION);
1981 // A coprocessor that implements VisibilityLabelsService can provide CELL_VISIBILITY.
1982 if (server.cpHost != null && hasVisibilityLabelsServiceCoprocessor(server.cpHost)) {
1983 if (VisibilityController.isCellAuthorizationSupported(server.getConfiguration())) {
1984 capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_VISIBILITY);
1987 response.addAllCapabilities(capabilities);
1988 } catch (IOException e) {
1989 throw new ServiceException(e);
1991 return response.build();
1995 * Determines if there is a MasterCoprocessor deployed which implements
1996 * {@link AccessControlService.Interface}.
1998 boolean hasAccessControlServiceCoprocessor(MasterCoprocessorHost cpHost) {
1999 return checkCoprocessorWithService(cpHost.findCoprocessors(MasterCoprocessor.class),
2000 AccessControlService.Interface.class);
2004 * Determines if there is a MasterCoprocessor deployed which implements
2005 * {@link VisibilityLabelsService.Interface}.
2007 boolean hasVisibilityLabelsServiceCoprocessor(MasterCoprocessorHost cpHost) {
2008 return checkCoprocessorWithService(cpHost.findCoprocessors(MasterCoprocessor.class),
2009 VisibilityLabelsService.Interface.class);
2013 * Determines if there is a coprocessor implementation in the provided argument which extends
2014 * or implements the provided {@code service}.
2016 boolean checkCoprocessorWithService(
2017 List<MasterCoprocessor> coprocessorsToCheck, Class<?> service) {
2018 if (coprocessorsToCheck == null || coprocessorsToCheck.isEmpty()) {
2019 return false;
2021 for (MasterCoprocessor cp : coprocessorsToCheck) {
2022 if (service.isAssignableFrom(cp.getClass())) {
2023 return true;
2026 return false;
2029 private MasterSwitchType convert(MasterProtos.MasterSwitchType switchType) {
2030 switch (switchType) {
2031 case SPLIT:
2032 return MasterSwitchType.SPLIT;
2033 case MERGE:
2034 return MasterSwitchType.MERGE;
2035 default:
2036 break;
2038 return null;
2041 @Override
2042 public AddReplicationPeerResponse addReplicationPeer(RpcController controller,
2043 AddReplicationPeerRequest request) throws ServiceException {
2044 try {
2045 long procId = server.addReplicationPeer(request.getPeerId(),
2046 ReplicationPeerConfigUtil.convert(request.getPeerConfig()),
2047 request.getPeerState().getState().equals(ReplicationState.State.ENABLED));
2048 return AddReplicationPeerResponse.newBuilder().setProcId(procId).build();
2049 } catch (ReplicationException | IOException e) {
2050 throw new ServiceException(e);
2054 @Override
2055 public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller,
2056 RemoveReplicationPeerRequest request) throws ServiceException {
2057 try {
2058 long procId = server.removeReplicationPeer(request.getPeerId());
2059 return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build();
2060 } catch (ReplicationException | IOException e) {
2061 throw new ServiceException(e);
2065 @Override
2066 public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller,
2067 EnableReplicationPeerRequest request) throws ServiceException {
2068 try {
2069 long procId = server.enableReplicationPeer(request.getPeerId());
2070 return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build();
2071 } catch (ReplicationException | IOException e) {
2072 throw new ServiceException(e);
2076 @Override
2077 public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller,
2078 DisableReplicationPeerRequest request) throws ServiceException {
2079 try {
2080 long procId = server.disableReplicationPeer(request.getPeerId());
2081 return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build();
2082 } catch (ReplicationException | IOException e) {
2083 throw new ServiceException(e);
2087 @Override
2088 public GetReplicationPeerConfigResponse getReplicationPeerConfig(RpcController controller,
2089 GetReplicationPeerConfigRequest request) throws ServiceException {
2090 GetReplicationPeerConfigResponse.Builder response = GetReplicationPeerConfigResponse
2091 .newBuilder();
2092 try {
2093 String peerId = request.getPeerId();
2094 ReplicationPeerConfig peerConfig = server.getReplicationPeerConfig(peerId);
2095 response.setPeerId(peerId);
2096 response.setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig));
2097 } catch (ReplicationException | IOException e) {
2098 throw new ServiceException(e);
2100 return response.build();
2103 @Override
2104 public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller,
2105 UpdateReplicationPeerConfigRequest request) throws ServiceException {
2106 try {
2107 long procId = server.updateReplicationPeerConfig(request.getPeerId(),
2108 ReplicationPeerConfigUtil.convert(request.getPeerConfig()));
2109 return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build();
2110 } catch (ReplicationException | IOException e) {
2111 throw new ServiceException(e);
2115 @Override
2116 public TransitReplicationPeerSyncReplicationStateResponse
2117 transitReplicationPeerSyncReplicationState(RpcController controller,
2118 TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
2119 try {
2120 long procId = server.transitReplicationPeerSyncReplicationState(request.getPeerId(),
2121 ReplicationPeerConfigUtil.toSyncReplicationState(request.getSyncReplicationState()));
2122 return TransitReplicationPeerSyncReplicationStateResponse.newBuilder().setProcId(procId)
2123 .build();
2124 } catch (ReplicationException | IOException e) {
2125 throw new ServiceException(e);
2129 @Override
2130 public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
2131 ListReplicationPeersRequest request) throws ServiceException {
2132 ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder();
2133 try {
2134 List<ReplicationPeerDescription> peers = server
2135 .listReplicationPeers(request.hasRegex() ? request.getRegex() : null);
2136 for (ReplicationPeerDescription peer : peers) {
2137 response.addPeerDesc(ReplicationPeerConfigUtil.toProtoReplicationPeerDescription(peer));
2139 } catch (ReplicationException | IOException e) {
2140 throw new ServiceException(e);
2142 return response.build();
2145 @Override
2146 public ListDecommissionedRegionServersResponse listDecommissionedRegionServers(
2147 RpcController controller, ListDecommissionedRegionServersRequest request)
2148 throws ServiceException {
2149 ListDecommissionedRegionServersResponse.Builder response =
2150 ListDecommissionedRegionServersResponse.newBuilder();
2151 try {
2152 server.checkInitialized();
2153 if (server.cpHost != null) {
2154 server.cpHost.preListDecommissionedRegionServers();
2156 List<ServerName> servers = server.listDecommissionedRegionServers();
2157 response.addAllServerName((servers.stream().map(server -> ProtobufUtil.toServerName(server)))
2158 .collect(Collectors.toList()));
2159 if (server.cpHost != null) {
2160 server.cpHost.postListDecommissionedRegionServers();
2162 } catch (IOException io) {
2163 throw new ServiceException(io);
2166 return response.build();
2169 @Override
2170 public DecommissionRegionServersResponse decommissionRegionServers(RpcController controller,
2171 DecommissionRegionServersRequest request) throws ServiceException {
2172 try {
2173 server.checkInitialized();
2174 List<ServerName> servers = request.getServerNameList().stream()
2175 .map(pbServer -> ProtobufUtil.toServerName(pbServer)).collect(Collectors.toList());
2176 boolean offload = request.getOffload();
2177 if (server.cpHost != null) {
2178 server.cpHost.preDecommissionRegionServers(servers, offload);
2180 server.decommissionRegionServers(servers, offload);
2181 if (server.cpHost != null) {
2182 server.cpHost.postDecommissionRegionServers(servers, offload);
2184 } catch (IOException io) {
2185 throw new ServiceException(io);
2188 return DecommissionRegionServersResponse.newBuilder().build();
2191 @Override
2192 public RecommissionRegionServerResponse recommissionRegionServer(RpcController controller,
2193 RecommissionRegionServerRequest request) throws ServiceException {
2194 try {
2195 server.checkInitialized();
2196 ServerName sn = ProtobufUtil.toServerName(request.getServerName());
2197 List<byte[]> encodedRegionNames = request.getRegionList().stream()
2198 .map(regionSpecifier -> regionSpecifier.getValue().toByteArray())
2199 .collect(Collectors.toList());
2200 if (server.cpHost != null) {
2201 server.cpHost.preRecommissionRegionServer(sn, encodedRegionNames);
2203 server.recommissionRegionServer(sn, encodedRegionNames);
2204 if (server.cpHost != null) {
2205 server.cpHost.postRecommissionRegionServer(sn, encodedRegionNames);
2207 } catch (IOException io) {
2208 throw new ServiceException(io);
2211 return RecommissionRegionServerResponse.newBuilder().build();
2214 @Override
2215 public LockResponse requestLock(RpcController controller, final LockRequest request)
2216 throws ServiceException {
2217 try {
2218 if (request.getDescription().isEmpty()) {
2219 throw new IllegalArgumentException("Empty description");
2221 NonceProcedureRunnable npr;
2222 LockType type = LockType.valueOf(request.getLockType().name());
2223 if (request.getRegionInfoCount() > 0) {
2224 final RegionInfo[] regionInfos = new RegionInfo[request.getRegionInfoCount()];
2225 for (int i = 0; i < request.getRegionInfoCount(); ++i) {
2226 regionInfos[i] = ProtobufUtil.toRegionInfo(request.getRegionInfo(i));
2228 npr = new NonceProcedureRunnable(server, request.getNonceGroup(), request.getNonce()) {
2229 @Override
2230 protected void run() throws IOException {
2231 setProcId(server.getLockManager().remoteLocks().requestRegionsLock(regionInfos,
2232 request.getDescription(), getNonceKey()));
2235 @Override
2236 protected String getDescription() {
2237 return "RequestLock";
2240 } else if (request.hasTableName()) {
2241 final TableName tableName = ProtobufUtil.toTableName(request.getTableName());
2242 npr = new NonceProcedureRunnable(server, request.getNonceGroup(), request.getNonce()) {
2243 @Override
2244 protected void run() throws IOException {
2245 setProcId(server.getLockManager().remoteLocks().requestTableLock(tableName, type,
2246 request.getDescription(), getNonceKey()));
2249 @Override
2250 protected String getDescription() {
2251 return "RequestLock";
2254 } else if (request.hasNamespace()) {
2255 npr = new NonceProcedureRunnable(server, request.getNonceGroup(), request.getNonce()) {
2256 @Override
2257 protected void run() throws IOException {
2258 setProcId(server.getLockManager().remoteLocks().requestNamespaceLock(
2259 request.getNamespace(), type, request.getDescription(), getNonceKey()));
2262 @Override
2263 protected String getDescription() {
2264 return "RequestLock";
2267 } else {
2268 throw new IllegalArgumentException("one of table/namespace/region should be specified");
2270 long procId = MasterProcedureUtil.submitProcedure(npr);
2271 return LockResponse.newBuilder().setProcId(procId).build();
2272 } catch (IllegalArgumentException e) {
2273 LOG.warn("Exception when queuing lock", e);
2274 throw new ServiceException(new DoNotRetryIOException(e));
2275 } catch (IOException e) {
2276 LOG.warn("Exception when queuing lock", e);
2277 throw new ServiceException(e);
2282 * @return LOCKED, if procedure is found and it has the lock; else UNLOCKED.
2283 * @throws ServiceException if given proc id is found but it is not a LockProcedure.
2285 @Override
2286 public LockHeartbeatResponse lockHeartbeat(RpcController controller, LockHeartbeatRequest request)
2287 throws ServiceException {
2288 try {
2289 if (server.getLockManager().remoteLocks().lockHeartbeat(request.getProcId(),
2290 request.getKeepAlive())) {
2291 return LockHeartbeatResponse.newBuilder().setTimeoutMs(
2292 server.getConfiguration().getInt(LockProcedure.REMOTE_LOCKS_TIMEOUT_MS_CONF,
2293 LockProcedure.DEFAULT_REMOTE_LOCKS_TIMEOUT_MS))
2294 .setLockStatus(LockHeartbeatResponse.LockStatus.LOCKED).build();
2295 } else {
2296 return LockHeartbeatResponse.newBuilder()
2297 .setLockStatus(LockHeartbeatResponse.LockStatus.UNLOCKED).build();
2299 } catch (IOException e) {
2300 throw new ServiceException(e);
2304 @Override
2305 public RegionSpaceUseReportResponse reportRegionSpaceUse(RpcController controller,
2306 RegionSpaceUseReportRequest request) throws ServiceException {
2307 try {
2308 server.checkInitialized();
2309 if (!QuotaUtil.isQuotaEnabled(server.getConfiguration())) {
2310 return RegionSpaceUseReportResponse.newBuilder().build();
2312 MasterQuotaManager quotaManager = this.server.getMasterQuotaManager();
2313 if (quotaManager != null) {
2314 final long now = EnvironmentEdgeManager.currentTime();
2315 for (RegionSpaceUse report : request.getSpaceUseList()) {
2316 quotaManager.addRegionSize(ProtobufUtil.toRegionInfo(report.getRegionInfo()),
2317 report.getRegionSize(), now);
2319 } else {
2320 LOG.debug("Received region space usage report but HMaster is not ready to process it, "
2321 + "skipping");
2323 return RegionSpaceUseReportResponse.newBuilder().build();
2324 } catch (Exception e) {
2325 throw new ServiceException(e);
2329 @Override
2330 public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
2331 RpcController controller, GetSpaceQuotaRegionSizesRequest request) throws ServiceException {
2332 try {
2333 server.checkInitialized();
2334 MasterQuotaManager quotaManager = this.server.getMasterQuotaManager();
2335 GetSpaceQuotaRegionSizesResponse.Builder builder =
2336 GetSpaceQuotaRegionSizesResponse.newBuilder();
2337 if (quotaManager != null) {
2338 Map<RegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
2339 Map<TableName,Long> regionSizesByTable = new HashMap<>();
2340 // Translate hregioninfo+long -> tablename+long
2341 for (Entry<RegionInfo,Long> entry : regionSizes.entrySet()) {
2342 final TableName tableName = entry.getKey().getTable();
2343 Long prevSize = regionSizesByTable.get(tableName);
2344 if (prevSize == null) {
2345 prevSize = 0L;
2347 regionSizesByTable.put(tableName, prevSize + entry.getValue());
2349 // Serialize them into the protobuf
2350 for (Entry<TableName,Long> tableSize : regionSizesByTable.entrySet()) {
2351 builder.addSizes(RegionSizes.newBuilder()
2352 .setTableName(ProtobufUtil.toProtoTableName(tableSize.getKey()))
2353 .setSize(tableSize.getValue()).build());
2355 return builder.build();
2356 } else {
2357 LOG.debug("Received space quota region size report but HMaster is not ready to process it,"
2358 + "skipping");
2360 return builder.build();
2361 } catch (Exception e) {
2362 throw new ServiceException(e);
2366 @Override
2367 public GetQuotaStatesResponse getQuotaStates(
2368 RpcController controller, GetQuotaStatesRequest request) throws ServiceException {
2369 try {
2370 server.checkInitialized();
2371 QuotaObserverChore quotaChore = this.server.getQuotaObserverChore();
2372 GetQuotaStatesResponse.Builder builder = GetQuotaStatesResponse.newBuilder();
2373 if (quotaChore != null) {
2374 // The "current" view of all tables with quotas
2375 Map<TableName, SpaceQuotaSnapshot> tableSnapshots = quotaChore.getTableQuotaSnapshots();
2376 for (Entry<TableName, SpaceQuotaSnapshot> entry : tableSnapshots.entrySet()) {
2377 builder.addTableSnapshots(
2378 TableQuotaSnapshot.newBuilder()
2379 .setTableName(ProtobufUtil.toProtoTableName(entry.getKey()))
2380 .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build());
2382 // The "current" view of all namespaces with quotas
2383 Map<String, SpaceQuotaSnapshot> nsSnapshots = quotaChore.getNamespaceQuotaSnapshots();
2384 for (Entry<String, SpaceQuotaSnapshot> entry : nsSnapshots.entrySet()) {
2385 builder.addNsSnapshots(
2386 NamespaceQuotaSnapshot.newBuilder()
2387 .setNamespace(entry.getKey())
2388 .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(entry.getValue())).build());
2390 return builder.build();
2392 return builder.build();
2393 } catch (Exception e) {
2394 throw new ServiceException(e);
2398 @Override
2399 public ClearDeadServersResponse clearDeadServers(RpcController controller,
2400 ClearDeadServersRequest request) throws ServiceException {
2401 LOG.debug(server.getClientIdAuditPrefix() + " clear dead region servers.");
2402 ClearDeadServersResponse.Builder response = ClearDeadServersResponse.newBuilder();
2403 try {
2404 server.checkInitialized();
2405 if (server.cpHost != null) {
2406 server.cpHost.preClearDeadServers();
2409 if (server.getServerManager().areDeadServersInProgress()) {
2410 LOG.debug("Some dead server is still under processing, won't clear the dead server list");
2411 response.addAllServerName(request.getServerNameList());
2412 } else {
2413 DeadServer deadServer = server.getServerManager().getDeadServers();
2414 Set<Address> clearedServers = new HashSet<>();
2415 for (HBaseProtos.ServerName pbServer : request.getServerNameList()) {
2416 ServerName serverName = ProtobufUtil.toServerName(pbServer);
2417 final boolean deadInProcess = server.getProcedures().stream().anyMatch(
2418 p -> (p instanceof ServerCrashProcedure)
2419 && ((ServerCrashProcedure) p).getServerName().equals(serverName));
2420 if (deadInProcess) {
2421 throw new ServiceException(
2422 String.format("Dead server '%s' is not 'dead' in fact...", serverName));
2425 if (!deadServer.removeDeadServer(serverName)) {
2426 response.addServerName(pbServer);
2427 } else {
2428 clearedServers.add(serverName.getAddress());
2431 server.getRSGroupInfoManager().removeServers(clearedServers);
2432 LOG.info("Remove decommissioned servers {} from RSGroup done", clearedServers);
2435 if (server.cpHost != null) {
2436 server.cpHost.postClearDeadServers(
2437 ProtobufUtil.toServerNameList(request.getServerNameList()),
2438 ProtobufUtil.toServerNameList(response.getServerNameList()));
2440 } catch (IOException io) {
2441 throw new ServiceException(io);
2443 return response.build();
2446 @Override
2447 public ReportProcedureDoneResponse reportProcedureDone(RpcController controller,
2448 ReportProcedureDoneRequest request) throws ServiceException {
2449 // Check Masters is up and ready for duty before progressing. Remote side will keep trying.
2450 try {
2451 this.server.checkServiceStarted();
2452 } catch (ServerNotRunningYetException snrye) {
2453 throw new ServiceException(snrye);
2455 request.getResultList().forEach(result -> {
2456 if (result.getStatus() == RemoteProcedureResult.Status.SUCCESS) {
2457 server.remoteProcedureCompleted(result.getProcId());
2458 } else {
2459 server.remoteProcedureFailed(result.getProcId(),
2460 RemoteProcedureException.fromProto(result.getError()));
2463 return ReportProcedureDoneResponse.getDefaultInstance();
2466 @Override
2467 public FileArchiveNotificationResponse reportFileArchival(RpcController controller,
2468 FileArchiveNotificationRequest request) throws ServiceException {
2469 try {
2470 server.checkInitialized();
2471 if (!QuotaUtil.isQuotaEnabled(server.getConfiguration())) {
2472 return FileArchiveNotificationResponse.newBuilder().build();
2474 server.getMasterQuotaManager().processFileArchivals(request, server.getConnection(),
2475 server.getConfiguration(), server.getFileSystem());
2476 return FileArchiveNotificationResponse.newBuilder().build();
2477 } catch (Exception e) {
2478 throw new ServiceException(e);
2482 // HBCK Services
2484 @Override
2485 public RunHbckChoreResponse runHbckChore(RpcController c, RunHbckChoreRequest req)
2486 throws ServiceException {
2487 rpcPreCheck("runHbckChore");
2488 LOG.info("{} request HBCK chore to run", server.getClientIdAuditPrefix());
2489 HbckChore hbckChore = server.getHbckChore();
2490 boolean ran = hbckChore.runChore();
2491 return RunHbckChoreResponse.newBuilder().setRan(ran).build();
2495 * Update state of the table in meta only. This is required by hbck in some situations to cleanup
2496 * stuck assign/ unassign regions procedures for the table.
2498 * @return previous state of the table
2500 @Override
2501 public GetTableStateResponse setTableStateInMeta(RpcController controller,
2502 SetTableStateInMetaRequest request) throws ServiceException {
2503 rpcPreCheck("setTableStateInMeta");
2504 TableName tn = ProtobufUtil.toTableName(request.getTableName());
2505 try {
2506 TableState prevState = this.server.getTableStateManager().getTableState(tn);
2507 TableState newState = TableState.convert(tn, request.getTableState());
2508 LOG.info("{} set table={} state from {} to {}", server.getClientIdAuditPrefix(),
2509 tn, prevState.getState(), newState.getState());
2510 this.server.getTableStateManager().setTableState(tn, newState.getState());
2511 return GetTableStateResponse.newBuilder().setTableState(prevState.convert()).build();
2512 } catch (Exception e) {
2513 throw new ServiceException(e);
2518 * Update state of the region in meta only. This is required by hbck in some situations to cleanup
2519 * stuck assign/ unassign regions procedures for the table.
2521 * @return previous states of the regions
2523 @Override
2524 public SetRegionStateInMetaResponse setRegionStateInMeta(RpcController controller,
2525 SetRegionStateInMetaRequest request) throws ServiceException {
2526 rpcPreCheck("setRegionStateInMeta");
2527 SetRegionStateInMetaResponse.Builder builder = SetRegionStateInMetaResponse.newBuilder();
2528 try {
2529 for (RegionSpecifierAndState s : request.getStatesList()) {
2530 RegionSpecifier spec = s.getRegionSpecifier();
2531 String encodedName;
2532 if (spec.getType() == RegionSpecifierType.ENCODED_REGION_NAME) {
2533 encodedName = spec.getValue().toStringUtf8();
2534 } else {
2535 // TODO: actually, a full region name can save a lot on meta scan, improve later.
2536 encodedName = RegionInfo.encodeRegionName(spec.getValue().toByteArray());
2538 RegionInfo info = this.server.getAssignmentManager().loadRegionFromMeta(encodedName);
2539 LOG.trace("region info loaded from meta table: {}", info);
2540 RegionState prevState =
2541 this.server.getAssignmentManager().getRegionStates().getRegionState(info);
2542 RegionState.State newState = RegionState.State.convert(s.getState());
2543 LOG.info("{} set region={} state from {} to {}", server.getClientIdAuditPrefix(), info,
2544 prevState.getState(), newState);
2545 Put metaPut = MetaTableAccessor.makePutFromRegionInfo(info,
2546 EnvironmentEdgeManager.currentTime());
2547 metaPut.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
2548 Bytes.toBytes(newState.name()));
2549 List<Put> putList = new ArrayList<>();
2550 putList.add(metaPut);
2551 MetaTableAccessor.putsToMetaTable(this.server.getConnection(), putList);
2552 // Loads from meta again to refresh AM cache with the new region state
2553 this.server.getAssignmentManager().loadRegionFromMeta(encodedName);
2554 builder.addStates(RegionSpecifierAndState.newBuilder().setRegionSpecifier(spec)
2555 .setState(prevState.getState().convert()));
2557 } catch (Exception e) {
2558 throw new ServiceException(e);
2560 return builder.build();
2564 * Get RegionInfo from Master using content of RegionSpecifier as key.
2565 * @return RegionInfo found by decoding <code>rs</code> or null if none found
2567 private RegionInfo getRegionInfo(HBaseProtos.RegionSpecifier rs) throws UnknownRegionException {
2568 RegionInfo ri = null;
2569 switch(rs.getType()) {
2570 case REGION_NAME:
2571 final byte[] regionName = rs.getValue().toByteArray();
2572 ri = this.server.getAssignmentManager().getRegionInfo(regionName);
2573 break;
2574 case ENCODED_REGION_NAME:
2575 String encodedRegionName = Bytes.toString(rs.getValue().toByteArray());
2576 RegionState regionState = this.server.getAssignmentManager().getRegionStates().
2577 getRegionState(encodedRegionName);
2578 ri = regionState == null ?
2579 this.server.getAssignmentManager().loadRegionFromMeta(encodedRegionName) :
2580 regionState.getRegion();
2581 break;
2582 default:
2583 break;
2585 return ri;
2589 * @throws ServiceException If no MasterProcedureExecutor
2591 private void checkMasterProcedureExecutor() throws ServiceException {
2592 if (this.server.getMasterProcedureExecutor() == null) {
2593 throw new ServiceException("Master's ProcedureExecutor not initialized; retry later");
2598 * A 'raw' version of assign that does bulk and can skirt Master state checks if override
2599 * is set; i.e. assigns can be forced during Master startup or if RegionState is unclean.
2600 * Used by HBCK2.
2602 @Override
2603 public MasterProtos.AssignsResponse assigns(RpcController controller,
2604 MasterProtos.AssignsRequest request) throws ServiceException {
2605 checkMasterProcedureExecutor();
2606 MasterProtos.AssignsResponse.Builder responseBuilder =
2607 MasterProtos.AssignsResponse.newBuilder();
2608 try {
2609 boolean override = request.getOverride();
2610 LOG.info("{} assigns, override={}", server.getClientIdAuditPrefix(), override);
2611 for (HBaseProtos.RegionSpecifier rs: request.getRegionList()) {
2612 long pid = Procedure.NO_PROC_ID;
2613 RegionInfo ri = getRegionInfo(rs);
2614 if (ri == null) {
2615 LOG.info("Unknown={}", rs);
2616 } else {
2617 Procedure p = this.server.getAssignmentManager().createOneAssignProcedure(ri, override);
2618 if (p != null) {
2619 pid = this.server.getMasterProcedureExecutor().submitProcedure(p);
2622 responseBuilder.addPid(pid);
2624 return responseBuilder.build();
2625 } catch (IOException ioe) {
2626 throw new ServiceException(ioe);
2631 * A 'raw' version of unassign that does bulk and can skirt Master state checks if override
2632 * is set; i.e. unassigns can be forced during Master startup or if RegionState is unclean.
2633 * Used by HBCK2.
2635 @Override
2636 public MasterProtos.UnassignsResponse unassigns(RpcController controller,
2637 MasterProtos.UnassignsRequest request) throws ServiceException {
2638 checkMasterProcedureExecutor();
2639 MasterProtos.UnassignsResponse.Builder responseBuilder =
2640 MasterProtos.UnassignsResponse.newBuilder();
2641 try {
2642 boolean override = request.getOverride();
2643 LOG.info("{} unassigns, override={}", server.getClientIdAuditPrefix(), override);
2644 for (HBaseProtos.RegionSpecifier rs: request.getRegionList()) {
2645 long pid = Procedure.NO_PROC_ID;
2646 RegionInfo ri = getRegionInfo(rs);
2647 if (ri == null) {
2648 LOG.info("Unknown={}", rs);
2649 } else {
2650 Procedure p = this.server.getAssignmentManager().createOneUnassignProcedure(ri, override);
2651 if (p != null) {
2652 pid = this.server.getMasterProcedureExecutor().submitProcedure(p);
2655 responseBuilder.addPid(pid);
2657 return responseBuilder.build();
2658 } catch (IOException ioe) {
2659 throw new ServiceException(ioe);
2664 * Bypass specified procedure to completion. Procedure is marked completed but no actual work
2665 * is done from the current state/ step onwards. Parents of the procedure are also marked for
2666 * bypass.
2668 * NOTE: this is a dangerous operation and may be used to unstuck buggy procedures. This may
2669 * leave system in inconherent state. This may need to be followed by some cleanup steps/
2670 * actions by operator.
2672 * @return BypassProcedureToCompletionResponse indicating success or failure
2674 @Override
2675 public MasterProtos.BypassProcedureResponse bypassProcedure(RpcController controller,
2676 MasterProtos.BypassProcedureRequest request) throws ServiceException {
2677 try {
2678 LOG.info("{} bypass procedures={}, waitTime={}, override={}, recursive={}",
2679 server.getClientIdAuditPrefix(), request.getProcIdList(), request.getWaitTime(),
2680 request.getOverride(), request.getRecursive());
2681 List<Boolean> ret =
2682 server.getMasterProcedureExecutor().bypassProcedure(request.getProcIdList(),
2683 request.getWaitTime(), request.getOverride(), request.getRecursive());
2684 return MasterProtos.BypassProcedureResponse.newBuilder().addAllBypassed(ret).build();
2685 } catch (IOException e) {
2686 throw new ServiceException(e);
2690 @Override
2691 public MasterProtos.ScheduleServerCrashProcedureResponse scheduleServerCrashProcedure(
2692 RpcController controller, MasterProtos.ScheduleServerCrashProcedureRequest request)
2693 throws ServiceException {
2694 List<Long> pids = new ArrayList<>();
2695 for (HBaseProtos.ServerName sn: request.getServerNameList()) {
2696 ServerName serverName = ProtobufUtil.toServerName(sn);
2697 LOG.info("{} schedule ServerCrashProcedure for {}",
2698 this.server.getClientIdAuditPrefix(), serverName);
2699 if (shouldSubmitSCP(serverName)) {
2700 pids.add(this.server.getServerManager().expireServer(serverName, true));
2701 } else {
2702 pids.add(Procedure.NO_PROC_ID);
2705 return MasterProtos.ScheduleServerCrashProcedureResponse.newBuilder().addAllPid(pids).build();
2708 @Override
2709 public MasterProtos.ScheduleSCPsForUnknownServersResponse scheduleSCPsForUnknownServers(
2710 RpcController controller, MasterProtos.ScheduleSCPsForUnknownServersRequest request)
2711 throws ServiceException {
2712 List<Long> pids = new ArrayList<>();
2713 final Set<ServerName> serverNames =
2714 server.getAssignmentManager().getRegionStates().getRegionStates().stream()
2715 .map(RegionState::getServerName).collect(Collectors.toSet());
2717 final Set<ServerName> unknownServerNames = serverNames.stream()
2718 .filter(sn -> server.getServerManager().isServerUnknown(sn)).collect(Collectors.toSet());
2720 for (ServerName sn: unknownServerNames) {
2721 LOG.info("{} schedule ServerCrashProcedure for unknown {}",
2722 this.server.getClientIdAuditPrefix(), sn);
2723 if (shouldSubmitSCP(sn)) {
2724 pids.add(this.server.getServerManager().expireServer(sn, true));
2725 } else {
2726 pids.add(Procedure.NO_PROC_ID);
2729 return MasterProtos.ScheduleSCPsForUnknownServersResponse.newBuilder().addAllPid(pids).build();
2732 @Override
2733 public FixMetaResponse fixMeta(RpcController controller, FixMetaRequest request)
2734 throws ServiceException {
2735 rpcPreCheck("fixMeta");
2736 try {
2737 MetaFixer mf = new MetaFixer(this.server);
2738 mf.fix();
2739 return FixMetaResponse.newBuilder().build();
2740 } catch (IOException ioe) {
2741 throw new ServiceException(ioe);
2745 @Override
2746 public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller,
2747 SwitchRpcThrottleRequest request) throws ServiceException {
2748 try {
2749 server.checkInitialized();
2750 return server.getMasterQuotaManager().switchRpcThrottle(request);
2751 } catch (Exception e) {
2752 throw new ServiceException(e);
2756 @Override
2757 public MasterProtos.IsRpcThrottleEnabledResponse isRpcThrottleEnabled(RpcController controller,
2758 MasterProtos.IsRpcThrottleEnabledRequest request) throws ServiceException {
2759 try {
2760 server.checkInitialized();
2761 return server.getMasterQuotaManager().isRpcThrottleEnabled(request);
2762 } catch (Exception e) {
2763 throw new ServiceException(e);
2767 @Override
2768 public SwitchExceedThrottleQuotaResponse switchExceedThrottleQuota(RpcController controller,
2769 SwitchExceedThrottleQuotaRequest request) throws ServiceException {
2770 try {
2771 server.checkInitialized();
2772 return server.getMasterQuotaManager().switchExceedThrottleQuota(request);
2773 } catch (Exception e) {
2774 throw new ServiceException(e);
2778 @Override
2779 public GrantResponse grant(RpcController controller, GrantRequest request)
2780 throws ServiceException {
2781 try {
2782 server.checkInitialized();
2783 if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
2784 final UserPermission perm =
2785 ShadedAccessControlUtil.toUserPermission(request.getUserPermission());
2786 boolean mergeExistingPermissions = request.getMergeExistingPermissions();
2787 server.cpHost.preGrant(perm, mergeExistingPermissions);
2788 try (Table table = server.getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
2789 PermissionStorage.addUserPermission(getConfiguration(), perm, table,
2790 mergeExistingPermissions);
2792 server.cpHost.postGrant(perm, mergeExistingPermissions);
2793 User caller = RpcServer.getRequestUser().orElse(null);
2794 if (AUDITLOG.isTraceEnabled()) {
2795 // audit log should store permission changes in addition to auth results
2796 String remoteAddress = RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("");
2797 AUDITLOG.trace("User {} (remote address: {}) granted permission {}", caller,
2798 remoteAddress, perm);
2800 return GrantResponse.getDefaultInstance();
2801 } else {
2802 throw new DoNotRetryIOException(
2803 new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
2805 } catch (IOException ioe) {
2806 throw new ServiceException(ioe);
2810 @Override
2811 public RevokeResponse revoke(RpcController controller, RevokeRequest request)
2812 throws ServiceException {
2813 try {
2814 server.checkInitialized();
2815 if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
2816 final UserPermission userPermission =
2817 ShadedAccessControlUtil.toUserPermission(request.getUserPermission());
2818 server.cpHost.preRevoke(userPermission);
2819 try (Table table = server.getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
2820 PermissionStorage.removeUserPermission(server.getConfiguration(), userPermission, table);
2822 server.cpHost.postRevoke(userPermission);
2823 User caller = RpcServer.getRequestUser().orElse(null);
2824 if (AUDITLOG.isTraceEnabled()) {
2825 // audit log should record all permission changes
2826 String remoteAddress = RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("");
2827 AUDITLOG.trace("User {} (remote address: {}) revoked permission {}", caller,
2828 remoteAddress, userPermission);
2830 return RevokeResponse.getDefaultInstance();
2831 } else {
2832 throw new DoNotRetryIOException(
2833 new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
2835 } catch (IOException ioe) {
2836 throw new ServiceException(ioe);
2840 @Override
2841 public GetUserPermissionsResponse getUserPermissions(RpcController controller,
2842 GetUserPermissionsRequest request) throws ServiceException {
2843 try {
2844 server.checkInitialized();
2845 if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
2846 final String userName = request.hasUserName() ? request.getUserName().toStringUtf8() : null;
2847 String namespace =
2848 request.hasNamespaceName() ? request.getNamespaceName().toStringUtf8() : null;
2849 TableName table =
2850 request.hasTableName() ? ProtobufUtil.toTableName(request.getTableName()) : null;
2851 byte[] cf = request.hasColumnFamily() ? request.getColumnFamily().toByteArray() : null;
2852 byte[] cq =
2853 request.hasColumnQualifier() ? request.getColumnQualifier().toByteArray() : null;
2854 Type permissionType = request.hasType() ? request.getType() : null;
2855 server.getMasterCoprocessorHost().preGetUserPermissions(userName, namespace, table, cf, cq);
2857 List<UserPermission> perms = null;
2858 if (permissionType == Type.Table) {
2859 boolean filter = (cf != null || userName != null) ? true : false;
2860 perms = PermissionStorage.getUserTablePermissions(server.getConfiguration(), table, cf,
2861 cq, userName, filter);
2862 } else if (permissionType == Type.Namespace) {
2863 perms = PermissionStorage.getUserNamespacePermissions(server.getConfiguration(),
2864 namespace, userName, userName != null ? true : false);
2865 } else {
2866 perms = PermissionStorage.getUserPermissions(server.getConfiguration(), null, null, null,
2867 userName, userName != null ? true : false);
2868 // Skip super users when filter user is specified
2869 if (userName == null) {
2870 // Adding superusers explicitly to the result set as PermissionStorage do not store
2871 // them. Also using acl as table name to be inline with the results of global admin and
2872 // will help in avoiding any leakage of information about being superusers.
2873 for (String user : Superusers.getSuperUsers()) {
2874 perms.add(new UserPermission(user,
2875 Permission.newBuilder().withActions(Action.values()).build()));
2880 server.getMasterCoprocessorHost().postGetUserPermissions(userName, namespace, table, cf,
2881 cq);
2882 AccessControlProtos.GetUserPermissionsResponse response =
2883 ShadedAccessControlUtil.buildGetUserPermissionsResponse(perms);
2884 return response;
2885 } else {
2886 throw new DoNotRetryIOException(
2887 new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
2889 } catch (IOException ioe) {
2890 throw new ServiceException(ioe);
2894 @Override
2895 public HasUserPermissionsResponse hasUserPermissions(RpcController controller,
2896 HasUserPermissionsRequest request) throws ServiceException {
2897 try {
2898 server.checkInitialized();
2899 if (server.cpHost != null && hasAccessControlServiceCoprocessor(server.cpHost)) {
2900 User caller = RpcServer.getRequestUser().orElse(null);
2901 String userName =
2902 request.hasUserName() ? request.getUserName().toStringUtf8() : caller.getShortName();
2903 List<Permission> permissions = new ArrayList<>();
2904 for (int i = 0; i < request.getPermissionCount(); i++) {
2905 permissions.add(ShadedAccessControlUtil.toPermission(request.getPermission(i)));
2907 server.getMasterCoprocessorHost().preHasUserPermissions(userName, permissions);
2908 if (!caller.getShortName().equals(userName)) {
2909 List<String> groups = AccessChecker.getUserGroups(userName);
2910 caller = new InputUser(userName, groups.toArray(new String[groups.size()]));
2912 List<Boolean> hasUserPermissions = new ArrayList<>();
2913 if (getAccessChecker() != null) {
2914 for (Permission permission : permissions) {
2915 boolean hasUserPermission =
2916 getAccessChecker().hasUserPermission(caller, "hasUserPermissions", permission);
2917 hasUserPermissions.add(hasUserPermission);
2919 } else {
2920 for (int i = 0; i < permissions.size(); i++) {
2921 hasUserPermissions.add(true);
2924 server.getMasterCoprocessorHost().postHasUserPermissions(userName, permissions);
2925 HasUserPermissionsResponse.Builder builder =
2926 HasUserPermissionsResponse.newBuilder().addAllHasUserPermission(hasUserPermissions);
2927 return builder.build();
2928 } else {
2929 throw new DoNotRetryIOException(
2930 new UnsupportedOperationException(AccessController.class.getName() + " is not loaded"));
2932 } catch (IOException ioe) {
2933 throw new ServiceException(ioe);
2937 private boolean shouldSubmitSCP(ServerName serverName) {
2938 // check if there is already a SCP of this server running
2939 List<Procedure<MasterProcedureEnv>> procedures =
2940 server.getMasterProcedureExecutor().getProcedures();
2941 for (Procedure<MasterProcedureEnv> procedure : procedures) {
2942 if (procedure instanceof ServerCrashProcedure) {
2943 if (serverName.compareTo(((ServerCrashProcedure) procedure).getServerName()) == 0
2944 && !procedure.isFinished()) {
2945 LOG.info("there is already a SCP of this server {} running, pid {}", serverName,
2946 procedure.getProcId());
2947 return false;
2951 return true;
2954 @Override
2955 public GetRSGroupInfoResponse getRSGroupInfo(RpcController controller,
2956 GetRSGroupInfoRequest request) throws ServiceException {
2957 String groupName = request.getRSGroupName();
2958 LOG.info(
2959 server.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, group=" + groupName);
2960 try {
2961 if (server.getMasterCoprocessorHost() != null) {
2962 server.getMasterCoprocessorHost().preGetRSGroupInfo(groupName);
2964 RSGroupInfo rsGroupInfo = server.getRSGroupInfoManager().getRSGroup(groupName);
2965 GetRSGroupInfoResponse resp;
2966 if (rsGroupInfo != null) {
2967 resp = GetRSGroupInfoResponse.newBuilder()
2968 .setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo)).build();
2969 } else {
2970 resp = GetRSGroupInfoResponse.getDefaultInstance();
2972 if (server.getMasterCoprocessorHost() != null) {
2973 server.getMasterCoprocessorHost().postGetRSGroupInfo(groupName);
2975 return resp;
2976 } catch (IOException e) {
2977 throw new ServiceException(e);
2981 @Override
2982 public GetRSGroupInfoOfTableResponse getRSGroupInfoOfTable(RpcController controller,
2983 GetRSGroupInfoOfTableRequest request) throws ServiceException {
2984 TableName tableName = ProtobufUtil.toTableName(request.getTableName());
2985 LOG.info(
2986 server.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, table=" + tableName);
2987 try {
2988 if (server.getMasterCoprocessorHost() != null) {
2989 server.getMasterCoprocessorHost().preGetRSGroupInfoOfTable(tableName);
2991 GetRSGroupInfoOfTableResponse resp;
2992 TableDescriptor td = server.getTableDescriptors().get(tableName);
2993 if (td == null) {
2994 resp = GetRSGroupInfoOfTableResponse.getDefaultInstance();
2995 } else {
2996 RSGroupInfo rsGroupInfo =
2997 RSGroupUtil.getRSGroupInfo(server, server.getRSGroupInfoManager(), tableName)
2998 .orElse(server.getRSGroupInfoManager().getRSGroup(RSGroupInfo.DEFAULT_GROUP));
2999 resp = GetRSGroupInfoOfTableResponse.newBuilder()
3000 .setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo)).build();
3002 if (server.getMasterCoprocessorHost() != null) {
3003 server.getMasterCoprocessorHost().postGetRSGroupInfoOfTable(tableName);
3005 return resp;
3006 } catch (IOException e) {
3007 throw new ServiceException(e);
3011 @Override
3012 public GetRSGroupInfoOfServerResponse getRSGroupInfoOfServer(RpcController controller,
3013 GetRSGroupInfoOfServerRequest request) throws ServiceException {
3014 Address hp =
3015 Address.fromParts(request.getServer().getHostName(), request.getServer().getPort());
3016 LOG.info(server.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server=" + hp);
3017 try {
3018 if (server.getMasterCoprocessorHost() != null) {
3019 server.getMasterCoprocessorHost().preGetRSGroupInfoOfServer(hp);
3021 RSGroupInfo rsGroupInfo = server.getRSGroupInfoManager().getRSGroupOfServer(hp);
3022 GetRSGroupInfoOfServerResponse resp;
3023 if (rsGroupInfo != null) {
3024 resp = GetRSGroupInfoOfServerResponse.newBuilder()
3025 .setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo)).build();
3026 } else {
3027 resp = GetRSGroupInfoOfServerResponse.getDefaultInstance();
3029 if (server.getMasterCoprocessorHost() != null) {
3030 server.getMasterCoprocessorHost().postGetRSGroupInfoOfServer(hp);
3032 return resp;
3033 } catch (IOException e) {
3034 throw new ServiceException(e);
3038 @Override
3039 public MoveServersResponse moveServers(RpcController controller, MoveServersRequest request)
3040 throws ServiceException {
3041 Set<Address> hostPorts = Sets.newHashSet();
3042 MoveServersResponse.Builder builder = MoveServersResponse.newBuilder();
3043 for (HBaseProtos.ServerName el : request.getServersList()) {
3044 hostPorts.add(Address.fromParts(el.getHostName(), el.getPort()));
3046 LOG.info(server.getClientIdAuditPrefix() + " move servers " + hostPorts + " to rsgroup " +
3047 request.getTargetGroup());
3048 try {
3049 if (server.getMasterCoprocessorHost() != null) {
3050 server.getMasterCoprocessorHost().preMoveServers(hostPorts, request.getTargetGroup());
3052 server.getRSGroupInfoManager().moveServers(hostPorts, request.getTargetGroup());
3053 if (server.getMasterCoprocessorHost() != null) {
3054 server.getMasterCoprocessorHost().postMoveServers(hostPorts, request.getTargetGroup());
3056 } catch (IOException e) {
3057 throw new ServiceException(e);
3059 return builder.build();
3062 @Override
3063 public AddRSGroupResponse addRSGroup(RpcController controller, AddRSGroupRequest request)
3064 throws ServiceException {
3065 AddRSGroupResponse.Builder builder = AddRSGroupResponse.newBuilder();
3066 LOG.info(server.getClientIdAuditPrefix() + " add rsgroup " + request.getRSGroupName());
3067 try {
3068 if (server.getMasterCoprocessorHost() != null) {
3069 server.getMasterCoprocessorHost().preAddRSGroup(request.getRSGroupName());
3071 server.getRSGroupInfoManager().addRSGroup(new RSGroupInfo(request.getRSGroupName()));
3072 if (server.getMasterCoprocessorHost() != null) {
3073 server.getMasterCoprocessorHost().postAddRSGroup(request.getRSGroupName());
3075 } catch (IOException e) {
3076 throw new ServiceException(e);
3078 return builder.build();
3081 @Override
3082 public RemoveRSGroupResponse removeRSGroup(RpcController controller, RemoveRSGroupRequest request)
3083 throws ServiceException {
3084 RemoveRSGroupResponse.Builder builder = RemoveRSGroupResponse.newBuilder();
3085 LOG.info(server.getClientIdAuditPrefix() + " remove rsgroup " + request.getRSGroupName());
3086 try {
3087 if (server.getMasterCoprocessorHost() != null) {
3088 server.getMasterCoprocessorHost().preRemoveRSGroup(request.getRSGroupName());
3090 server.getRSGroupInfoManager().removeRSGroup(request.getRSGroupName());
3091 if (server.getMasterCoprocessorHost() != null) {
3092 server.getMasterCoprocessorHost().postRemoveRSGroup(request.getRSGroupName());
3094 } catch (IOException e) {
3095 throw new ServiceException(e);
3097 return builder.build();
3100 @Override
3101 public BalanceRSGroupResponse balanceRSGroup(RpcController controller,
3102 BalanceRSGroupRequest request) throws ServiceException {
3103 BalanceRequest balanceRequest = ProtobufUtil.toBalanceRequest(request);
3105 BalanceRSGroupResponse.Builder builder = BalanceRSGroupResponse.newBuilder()
3106 .setBalanceRan(false);
3108 LOG.info(
3109 server.getClientIdAuditPrefix() + " balance rsgroup, group=" + request.getRSGroupName());
3110 try {
3111 if (server.getMasterCoprocessorHost() != null) {
3112 server.getMasterCoprocessorHost()
3113 .preBalanceRSGroup(request.getRSGroupName(), balanceRequest);
3115 BalanceResponse response =
3116 server.getRSGroupInfoManager().balanceRSGroup(request.getRSGroupName(), balanceRequest);
3117 ProtobufUtil.populateBalanceRSGroupResponse(builder, response);
3118 if (server.getMasterCoprocessorHost() != null) {
3119 server.getMasterCoprocessorHost()
3120 .postBalanceRSGroup(request.getRSGroupName(), balanceRequest, response);
3122 } catch (IOException e) {
3123 throw new ServiceException(e);
3125 return builder.build();
3128 @Override
3129 public ListRSGroupInfosResponse listRSGroupInfos(RpcController controller,
3130 ListRSGroupInfosRequest request) throws ServiceException {
3131 ListRSGroupInfosResponse.Builder builder = ListRSGroupInfosResponse.newBuilder();
3132 LOG.info(server.getClientIdAuditPrefix() + " list rsgroup");
3133 try {
3134 if (server.getMasterCoprocessorHost() != null) {
3135 server.getMasterCoprocessorHost().preListRSGroups();
3137 List<RSGroupInfo> rsGroupInfos = server.getRSGroupInfoManager().listRSGroups().stream()
3138 .map(RSGroupInfo::new).collect(Collectors.toList());
3139 Map<String, RSGroupInfo> name2Info = new HashMap<>();
3140 List<TableDescriptor> needToFill =
3141 new ArrayList<>(server.getTableDescriptors().getAll().values());
3142 for (RSGroupInfo rsGroupInfo : rsGroupInfos) {
3143 name2Info.put(rsGroupInfo.getName(), rsGroupInfo);
3144 for (TableDescriptor td : server.getTableDescriptors().getAll().values()) {
3145 if (rsGroupInfo.containsTable(td.getTableName())){
3146 needToFill.remove(td);
3150 for (TableDescriptor td : needToFill) {
3151 String groupName = td.getRegionServerGroup().orElse(RSGroupInfo.DEFAULT_GROUP);
3152 RSGroupInfo rsGroupInfo = name2Info.get(groupName);
3153 if (rsGroupInfo != null) {
3154 rsGroupInfo.addTable(td.getTableName());
3157 for (RSGroupInfo rsGroupInfo : rsGroupInfos) {
3158 // TODO: this can be done at once outside this loop, do not need to scan all every time.
3159 builder.addRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo));
3161 if (server.getMasterCoprocessorHost() != null) {
3162 server.getMasterCoprocessorHost().postListRSGroups();
3164 } catch (IOException e) {
3165 throw new ServiceException(e);
3167 return builder.build();
3170 @Override
3171 public RemoveServersResponse removeServers(RpcController controller,
3172 RemoveServersRequest request) throws ServiceException {
3173 RemoveServersResponse.Builder builder = RemoveServersResponse.newBuilder();
3174 Set<Address> servers = Sets.newHashSet();
3175 for (HBaseProtos.ServerName el : request.getServersList()) {
3176 servers.add(Address.fromParts(el.getHostName(), el.getPort()));
3178 LOG.info(server.getClientIdAuditPrefix() + " remove decommissioned servers from rsgroup: " +
3179 servers);
3180 try {
3181 if (server.getMasterCoprocessorHost() != null) {
3182 server.getMasterCoprocessorHost().preRemoveServers(servers);
3184 server.getRSGroupInfoManager().removeServers(servers);
3185 if (server.getMasterCoprocessorHost() != null) {
3186 server.getMasterCoprocessorHost().postRemoveServers(servers);
3188 } catch (IOException e) {
3189 throw new ServiceException(e);
3191 return builder.build();
3194 @Override
3195 public ListTablesInRSGroupResponse listTablesInRSGroup(RpcController controller,
3196 ListTablesInRSGroupRequest request) throws ServiceException {
3197 ListTablesInRSGroupResponse.Builder builder = ListTablesInRSGroupResponse.newBuilder();
3198 String groupName = request.getGroupName();
3199 LOG.info(server.getClientIdAuditPrefix() + " list tables in rsgroup " + groupName);
3200 try {
3201 if (server.getMasterCoprocessorHost() != null) {
3202 server.getMasterCoprocessorHost().preListTablesInRSGroup(groupName);
3204 RSGroupUtil.listTablesInRSGroup(server, groupName).stream()
3205 .map(ProtobufUtil::toProtoTableName).forEach(builder::addTableName);
3206 if (server.getMasterCoprocessorHost() != null) {
3207 server.getMasterCoprocessorHost().postListTablesInRSGroup(groupName);
3209 } catch (IOException e) {
3210 throw new ServiceException(e);
3212 return builder.build();
3215 @Override
3216 public GetConfiguredNamespacesAndTablesInRSGroupResponse
3217 getConfiguredNamespacesAndTablesInRSGroup(RpcController controller,
3218 GetConfiguredNamespacesAndTablesInRSGroupRequest request) throws ServiceException {
3219 GetConfiguredNamespacesAndTablesInRSGroupResponse.Builder builder =
3220 GetConfiguredNamespacesAndTablesInRSGroupResponse.newBuilder();
3221 String groupName = request.getGroupName();
3222 LOG.info(server.getClientIdAuditPrefix() + " get configured namespaces and tables in rsgroup " +
3223 groupName);
3224 try {
3225 if (server.getMasterCoprocessorHost() != null) {
3226 server.getMasterCoprocessorHost().preGetConfiguredNamespacesAndTablesInRSGroup(groupName);
3228 for (NamespaceDescriptor nd : server.getClusterSchema().getNamespaces()) {
3229 if (groupName.equals(nd.getConfigurationValue(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP))) {
3230 builder.addNamespace(nd.getName());
3233 for (TableDescriptor td : server.getTableDescriptors().getAll().values()) {
3234 if (td.getRegionServerGroup().map(g -> g.equals(groupName)).orElse(false)) {
3235 builder.addTableName(ProtobufUtil.toProtoTableName(td.getTableName()));
3238 if (server.getMasterCoprocessorHost() != null) {
3239 server.getMasterCoprocessorHost().postGetConfiguredNamespacesAndTablesInRSGroup(groupName);
3241 } catch (IOException e) {
3242 throw new ServiceException(e);
3244 return builder.build();
3247 @Override
3248 public RenameRSGroupResponse renameRSGroup(RpcController controller,
3249 RenameRSGroupRequest request) throws ServiceException {
3250 RenameRSGroupResponse.Builder builder = RenameRSGroupResponse.newBuilder();
3251 String oldRSGroup = request.getOldRsgroupName();
3252 String newRSGroup = request.getNewRsgroupName();
3253 LOG.info("{} rename rsgroup from {} to {} ",
3254 server.getClientIdAuditPrefix(), oldRSGroup, newRSGroup);
3255 try {
3256 if (server.getMasterCoprocessorHost() != null) {
3257 server.getMasterCoprocessorHost().preRenameRSGroup(oldRSGroup, newRSGroup);
3259 server.getRSGroupInfoManager().renameRSGroup(oldRSGroup, newRSGroup);
3260 if (server.getMasterCoprocessorHost() != null) {
3261 server.getMasterCoprocessorHost().postRenameRSGroup(oldRSGroup, newRSGroup);
3263 } catch (IOException e) {
3264 throw new ServiceException(e);
3266 return builder.build();
3269 @Override
3270 public UpdateRSGroupConfigResponse updateRSGroupConfig(RpcController controller,
3271 UpdateRSGroupConfigRequest request)
3272 throws ServiceException {
3273 UpdateRSGroupConfigResponse.Builder builder = UpdateRSGroupConfigResponse.newBuilder();
3274 String groupName = request.getGroupName();
3275 Map<String, String> configuration = new HashMap<>();
3276 request.getConfigurationList().forEach(p -> configuration.put(p.getName(), p.getValue()));
3277 LOG.info("{} update rsgroup {} configuration {}", server.getClientIdAuditPrefix(), groupName,
3278 configuration);
3279 try {
3280 if (server.getMasterCoprocessorHost() != null) {
3281 server.getMasterCoprocessorHost().preUpdateRSGroupConfig(groupName, configuration);
3283 server.getRSGroupInfoManager().updateRSGroupConfig(groupName, configuration);
3284 if (server.getMasterCoprocessorHost() != null) {
3285 server.getMasterCoprocessorHost().postUpdateRSGroupConfig(groupName, configuration);
3287 } catch (IOException e) {
3288 throw new ServiceException(e);
3290 return builder.build();
3293 @Override
3294 public HBaseProtos.LogEntry getLogEntries(RpcController controller,
3295 HBaseProtos.LogRequest request) throws ServiceException {
3296 try {
3297 final String logClassName = request.getLogClassName();
3298 Class<?> logClass = Class.forName(logClassName)
3299 .asSubclass(Message.class);
3300 Method method = logClass.getMethod("parseFrom", ByteString.class);
3301 if (logClassName.contains("BalancerDecisionsRequest")) {
3302 MasterProtos.BalancerDecisionsRequest balancerDecisionsRequest =
3303 (MasterProtos.BalancerDecisionsRequest) method
3304 .invoke(null, request.getLogMessage());
3305 MasterProtos.BalancerDecisionsResponse balancerDecisionsResponse =
3306 getBalancerDecisions(balancerDecisionsRequest);
3307 return HBaseProtos.LogEntry.newBuilder()
3308 .setLogClassName(balancerDecisionsResponse.getClass().getName())
3309 .setLogMessage(balancerDecisionsResponse.toByteString())
3310 .build();
3311 }else if (logClassName.contains("BalancerRejectionsRequest")){
3312 MasterProtos.BalancerRejectionsRequest balancerRejectionsRequest =
3313 (MasterProtos.BalancerRejectionsRequest) method
3314 .invoke(null, request.getLogMessage());
3315 MasterProtos.BalancerRejectionsResponse balancerRejectionsResponse =
3316 getBalancerRejections(balancerRejectionsRequest);
3317 return HBaseProtos.LogEntry.newBuilder()
3318 .setLogClassName(balancerRejectionsResponse.getClass().getName())
3319 .setLogMessage(balancerRejectionsResponse.toByteString())
3320 .build();
3322 } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
3323 | InvocationTargetException e) {
3324 LOG.error("Error while retrieving log entries.", e);
3325 throw new ServiceException(e);
3327 throw new ServiceException("Invalid request params");
3330 private MasterProtos.BalancerDecisionsResponse
3331 getBalancerDecisions(MasterProtos.BalancerDecisionsRequest request) {
3332 final NamedQueueRecorder namedQueueRecorder = this.server.getNamedQueueRecorder();
3333 if (namedQueueRecorder == null) {
3334 return MasterProtos.BalancerDecisionsResponse.newBuilder()
3335 .addAllBalancerDecision(Collections.emptyList()).build();
3337 final NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
3338 namedQueueGetRequest.setNamedQueueEvent(BalancerDecisionDetails.BALANCER_DECISION_EVENT);
3339 namedQueueGetRequest.setBalancerDecisionsRequest(request);
3340 NamedQueueGetResponse namedQueueGetResponse =
3341 namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
3342 List<RecentLogs.BalancerDecision> balancerDecisions = namedQueueGetResponse != null ?
3343 namedQueueGetResponse.getBalancerDecisions() :
3344 Collections.emptyList();
3345 return MasterProtos.BalancerDecisionsResponse.newBuilder()
3346 .addAllBalancerDecision(balancerDecisions).build();
3349 private MasterProtos.BalancerRejectionsResponse getBalancerRejections(
3350 MasterProtos.BalancerRejectionsRequest request) {
3351 final NamedQueueRecorder namedQueueRecorder = this.server.getNamedQueueRecorder();
3352 if (namedQueueRecorder == null) {
3353 return MasterProtos.BalancerRejectionsResponse.newBuilder()
3354 .addAllBalancerRejection(Collections.emptyList()).build();
3356 final NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
3357 namedQueueGetRequest.setNamedQueueEvent(BalancerRejectionDetails.BALANCER_REJECTION_EVENT);
3358 namedQueueGetRequest.setBalancerRejectionsRequest(request);
3359 NamedQueueGetResponse namedQueueGetResponse =
3360 namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
3361 List<RecentLogs.BalancerRejection> balancerRejections = namedQueueGetResponse != null ?
3362 namedQueueGetResponse.getBalancerRejections() :
3363 Collections.emptyList();
3364 return MasterProtos.BalancerRejectionsResponse.newBuilder()
3365 .addAllBalancerRejection(balancerRejections).build();
3368 @Override
3369 @QosPriority(priority=HConstants.ADMIN_QOS)
3370 public GetRegionInfoResponse getRegionInfo(final RpcController controller,
3371 final GetRegionInfoRequest request) throws ServiceException {
3372 RegionInfo ri = null;
3373 try {
3374 ri = getRegionInfo(request.getRegion());
3375 } catch(UnknownRegionException ure) {
3376 throw new ServiceException(ure);
3378 GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
3379 if (ri != null) {
3380 builder.setRegionInfo(ProtobufUtil.toRegionInfo(ri));
3381 } else {
3382 // Is it a MOB name? These work differently.
3383 byte [] regionName = request.getRegion().getValue().toByteArray();
3384 TableName tableName = RegionInfo.getTable(regionName);
3385 if (MobUtils.isMobRegionName(tableName, regionName)) {
3386 // a dummy region info contains the compaction state.
3387 RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
3388 builder.setRegionInfo(ProtobufUtil.toRegionInfo(mobRegionInfo));
3389 if (request.hasCompactionState() && request.getCompactionState()) {
3390 builder.setCompactionState(server.getMobCompactionState(tableName));
3392 } else {
3393 // If unknown RegionInfo and not a MOB region, it is unknown.
3394 throw new ServiceException(new UnknownRegionException(Bytes.toString(regionName)));
3397 return builder.build();
3400 @Override
3401 public GetStoreFileResponse getStoreFile(RpcController controller, GetStoreFileRequest request)
3402 throws ServiceException {
3403 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3406 @Override
3407 public GetOnlineRegionResponse getOnlineRegion(RpcController controller,
3408 GetOnlineRegionRequest request) throws ServiceException {
3409 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3412 @Override
3413 public OpenRegionResponse openRegion(RpcController controller, OpenRegionRequest request)
3414 throws ServiceException {
3415 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3418 @Override
3419 public WarmupRegionResponse warmupRegion(RpcController controller, WarmupRegionRequest request)
3420 throws ServiceException {
3421 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3424 @Override
3425 public CloseRegionResponse closeRegion(RpcController controller, CloseRegionRequest request)
3426 throws ServiceException {
3427 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3430 @Override
3431 public FlushRegionResponse flushRegion(RpcController controller, FlushRegionRequest request)
3432 throws ServiceException {
3433 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3436 @Override
3437 public CompactionSwitchResponse compactionSwitch(RpcController controller,
3438 CompactionSwitchRequest request) throws ServiceException {
3439 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3442 @Override
3443 public CompactRegionResponse compactRegion(RpcController controller, CompactRegionRequest request)
3444 throws ServiceException {
3445 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3448 @Override
3449 public ReplicateWALEntryResponse replicateWALEntry(RpcController controller,
3450 ReplicateWALEntryRequest request) throws ServiceException {
3451 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3454 @Override
3455 public ReplicateWALEntryResponse replay(RpcController controller,
3456 ReplicateWALEntryRequest request) throws ServiceException {
3457 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3460 @Override
3461 public RollWALWriterResponse rollWALWriter(RpcController controller, RollWALWriterRequest request)
3462 throws ServiceException {
3463 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3466 @Override
3467 public GetServerInfoResponse getServerInfo(RpcController controller, GetServerInfoRequest request)
3468 throws ServiceException {
3469 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3472 @Override
3473 public StopServerResponse stopServer(RpcController controller, StopServerRequest request)
3474 throws ServiceException {
3475 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3478 @Override
3479 public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,
3480 UpdateFavoredNodesRequest request) throws ServiceException {
3481 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3484 @Override
3485 public GetRegionLoadResponse getRegionLoad(RpcController controller, GetRegionLoadRequest request)
3486 throws ServiceException {
3487 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3490 @Override
3491 public ClearCompactionQueuesResponse clearCompactionQueues(RpcController controller,
3492 ClearCompactionQueuesRequest request) throws ServiceException {
3493 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3496 @Override
3497 public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,
3498 ClearRegionBlockCacheRequest request) throws ServiceException {
3499 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3502 @Override
3503 public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(RpcController controller,
3504 GetSpaceQuotaSnapshotsRequest request) throws ServiceException {
3505 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3508 @Override
3509 public ExecuteProceduresResponse executeProcedures(RpcController controller,
3510 ExecuteProceduresRequest request) throws ServiceException {
3511 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3514 @Override
3515 public GetLiveRegionServersResponse getLiveRegionServers(RpcController controller,
3516 GetLiveRegionServersRequest request) throws ServiceException {
3517 List<ServerName> regionServers = new ArrayList<>(server.getLiveRegionServers());
3518 Collections.shuffle(regionServers, ThreadLocalRandom.current());
3519 GetLiveRegionServersResponse.Builder builder =
3520 GetLiveRegionServersResponse.newBuilder().setTotal(regionServers.size());
3521 regionServers.stream().limit(request.getCount()).map(ProtobufUtil::toServerName)
3522 .forEach(builder::addServer);
3523 return builder.build();
3526 @Override
3527 public ReplicateWALEntryResponse replicateToReplica(RpcController controller,
3528 ReplicateWALEntryRequest request) throws ServiceException {
3529 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));