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
;
30 import java
.util
.Map
.Entry
;
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
.ModifyNamespaceRequest
;
291 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.ModifyNamespaceResponse
;
292 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.ModifyTableRequest
;
293 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.ModifyTableResponse
;
294 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.MoveRegionRequest
;
295 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.MoveRegionResponse
;
296 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.NormalizeRequest
;
297 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.NormalizeResponse
;
298 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.OfflineRegionRequest
;
299 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.OfflineRegionResponse
;
300 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RecommissionRegionServerRequest
;
301 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RecommissionRegionServerResponse
;
302 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RegionSpecifierAndState
;
303 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RestoreSnapshotRequest
;
304 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RestoreSnapshotResponse
;
305 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RunCatalogScanRequest
;
306 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RunCatalogScanResponse
;
307 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RunCleanerChoreRequest
;
308 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RunCleanerChoreResponse
;
309 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RunHbckChoreRequest
;
310 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.RunHbckChoreResponse
;
311 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SecurityCapabilitiesRequest
;
312 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SecurityCapabilitiesResponse
;
313 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetBalancerRunningRequest
;
314 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetBalancerRunningResponse
;
315 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetCleanerChoreRunningRequest
;
316 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetCleanerChoreRunningResponse
;
317 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetNormalizerRunningRequest
;
318 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetNormalizerRunningResponse
;
319 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetQuotaRequest
;
320 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetQuotaResponse
;
321 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetRegionStateInMetaRequest
;
322 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetRegionStateInMetaResponse
;
323 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetSnapshotCleanupRequest
;
324 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetSnapshotCleanupResponse
;
325 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetSplitOrMergeEnabledRequest
;
326 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetSplitOrMergeEnabledResponse
;
327 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SetTableStateInMetaRequest
;
328 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.ShutdownRequest
;
329 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.ShutdownResponse
;
330 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SnapshotRequest
;
331 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SnapshotResponse
;
332 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SplitTableRegionRequest
;
333 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SplitTableRegionResponse
;
334 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.StopMasterRequest
;
335 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.StopMasterResponse
;
336 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SwitchExceedThrottleQuotaRequest
;
337 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SwitchExceedThrottleQuotaResponse
;
338 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SwitchRpcThrottleRequest
;
339 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.SwitchRpcThrottleResponse
;
340 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.TruncateTableRequest
;
341 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.TruncateTableResponse
;
342 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.UnassignRegionRequest
;
343 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.MasterProtos
.UnassignRegionResponse
;
344 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetQuotaStatesRequest
;
345 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetQuotaStatesResponse
;
346 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetQuotaStatesResponse
.NamespaceQuotaSnapshot
;
347 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetQuotaStatesResponse
.TableQuotaSnapshot
;
348 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetSpaceQuotaRegionSizesRequest
;
349 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetSpaceQuotaRegionSizesResponse
;
350 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetSpaceQuotaRegionSizesResponse
.RegionSizes
;
351 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetSpaceQuotaSnapshotsRequest
;
352 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetSpaceQuotaSnapshotsResponse
;
353 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.AddRSGroupRequest
;
354 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.AddRSGroupResponse
;
355 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.BalanceRSGroupRequest
;
356 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.BalanceRSGroupResponse
;
357 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.GetConfiguredNamespacesAndTablesInRSGroupRequest
;
358 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.GetConfiguredNamespacesAndTablesInRSGroupResponse
;
359 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.GetRSGroupInfoOfServerRequest
;
360 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.GetRSGroupInfoOfServerResponse
;
361 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.GetRSGroupInfoOfTableRequest
;
362 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.GetRSGroupInfoOfTableResponse
;
363 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.GetRSGroupInfoRequest
;
364 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.GetRSGroupInfoResponse
;
365 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.ListRSGroupInfosRequest
;
366 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.ListRSGroupInfosResponse
;
367 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.ListTablesInRSGroupRequest
;
368 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.ListTablesInRSGroupResponse
;
369 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.MoveServersRequest
;
370 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.MoveServersResponse
;
371 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.RemoveRSGroupRequest
;
372 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.RemoveRSGroupResponse
;
373 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.RemoveServersRequest
;
374 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.RemoveServersResponse
;
375 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.RenameRSGroupRequest
;
376 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.RenameRSGroupResponse
;
377 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.UpdateRSGroupConfigRequest
;
378 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RSGroupAdminProtos
.UpdateRSGroupConfigResponse
;
379 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RecentLogs
;
380 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.FileArchiveNotificationRequest
;
381 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.FileArchiveNotificationResponse
;
382 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.GetLastFlushedSequenceIdRequest
;
383 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.GetLastFlushedSequenceIdResponse
;
384 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.GetLiveRegionServersRequest
;
385 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.GetLiveRegionServersResponse
;
386 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RegionServerReportRequest
;
387 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RegionServerReportResponse
;
388 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RegionServerStartupRequest
;
389 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RegionServerStartupResponse
;
390 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RegionServerStatusService
;
391 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RegionSpaceUse
;
392 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RegionSpaceUseReportRequest
;
393 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RegionSpaceUseReportResponse
;
394 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.RemoteProcedureResult
;
395 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.ReportProcedureDoneRequest
;
396 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.ReportProcedureDoneResponse
;
397 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.ReportRSFatalErrorRequest
;
398 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.ReportRSFatalErrorResponse
;
399 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.ReportRegionStateTransitionRequest
;
400 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegionServerStatusProtos
.ReportRegionStateTransitionResponse
;
401 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.RegistryProtos
.ClientMetaService
;
402 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.AddReplicationPeerRequest
;
403 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.AddReplicationPeerResponse
;
404 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.DisableReplicationPeerRequest
;
405 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.DisableReplicationPeerResponse
;
406 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.EnableReplicationPeerRequest
;
407 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.EnableReplicationPeerResponse
;
408 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.GetReplicationPeerConfigRequest
;
409 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.GetReplicationPeerConfigResponse
;
410 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.ListReplicationPeersRequest
;
411 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.ListReplicationPeersResponse
;
412 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.RemoveReplicationPeerRequest
;
413 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.RemoveReplicationPeerResponse
;
414 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.ReplicationState
;
415 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.TransitReplicationPeerSyncReplicationStateRequest
;
416 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.TransitReplicationPeerSyncReplicationStateResponse
;
417 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.UpdateReplicationPeerConfigRequest
;
418 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
.UpdateReplicationPeerConfigResponse
;
419 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.SnapshotProtos
.SnapshotDescription
;
420 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.VisibilityLabelsProtos
.VisibilityLabelsService
;
423 * Implements the master RPC services.
425 @InterfaceAudience.Private
426 public class MasterRpcServices
extends HBaseRpcServicesBase
<HMaster
>
427 implements MasterService
.BlockingInterface
, RegionServerStatusService
.BlockingInterface
,
428 LockService
.BlockingInterface
, HbckService
.BlockingInterface
{
430 private static final Logger LOG
= LoggerFactory
.getLogger(MasterRpcServices
.class.getName());
431 private static final Logger AUDITLOG
=
432 LoggerFactory
.getLogger("SecurityLogger."+MasterRpcServices
.class.getName());
434 /** RPC scheduler to use for the master. */
435 public static final String MASTER_RPC_SCHEDULER_FACTORY_CLASS
=
436 "hbase.master.rpc.scheduler.factory.class";
439 * @return Subset of configuration to pass initializing regionservers: e.g.
440 * the filesystem to use and root directory to use.
442 private RegionServerStartupResponse
.Builder
createConfigurationSubset() {
443 RegionServerStartupResponse
.Builder resp
= addConfig(
444 RegionServerStartupResponse
.newBuilder(), HConstants
.HBASE_DIR
);
445 resp
= addConfig(resp
, "fs.defaultFS");
446 return addConfig(resp
, "hbase.master.info.port");
449 private RegionServerStartupResponse
.Builder
addConfig(
450 final RegionServerStartupResponse
.Builder resp
, final String key
) {
451 NameStringPair
.Builder entry
= NameStringPair
.newBuilder()
453 .setValue(server
.getConfiguration().get(key
));
454 resp
.addMapEntries(entry
.build());
458 public MasterRpcServices(HMaster m
) throws IOException
{
459 super(m
, m
.getProcessName());
463 protected boolean defaultReservoirEnabled() {
468 protected ServerType
getDNSServerType() {
469 return DNS
.ServerType
.MASTER
;
473 protected String
getHostname(Configuration conf
, String defaultHostname
) {
474 return conf
.get("hbase.master.ipc.address", defaultHostname
);
478 protected String
getPortConfigName() {
479 return HConstants
.MASTER_PORT
;
483 protected int getDefaultPort() {
484 return HConstants
.DEFAULT_MASTER_PORT
;
488 protected Class
<?
> getRpcSchedulerFactoryClass(Configuration conf
) {
489 return conf
.getClass(MASTER_RPC_SCHEDULER_FACTORY_CLASS
, SimpleRpcSchedulerFactory
.class);
493 protected PriorityFunction
createPriority() {
494 return new MasterAnnotationReadingPriorityFunction(this);
498 * Checks for the following pre-checks in order:
500 * <li>Master is initialized</li>
501 * <li>Rpc caller has admin permissions</li>
503 * @param requestName name of rpc request. Used in reporting failures to provide context.
504 * @throws ServiceException If any of the above listed pre-check fails.
506 private void rpcPreCheck(String requestName
) throws ServiceException
{
508 server
.checkInitialized();
509 requirePermission(requestName
, Permission
.Action
.ADMIN
);
510 } catch (IOException ioe
) {
511 throw new ServiceException(ioe
);
515 enum BalanceSwitchMode
{
521 * Assigns balancer switch according to BalanceSwitchMode
522 * @param b new balancer switch
523 * @param mode BalanceSwitchMode
524 * @return old balancer switch
526 boolean switchBalancer(final boolean b
, BalanceSwitchMode mode
) throws IOException
{
527 boolean oldValue
= server
.loadBalancerTracker
.isBalancerOn();
528 boolean newValue
= b
;
530 if (server
.cpHost
!= null) {
531 server
.cpHost
.preBalanceSwitch(newValue
);
534 if (mode
== BalanceSwitchMode
.SYNC
) {
535 synchronized (server
.getLoadBalancer()) {
536 server
.loadBalancerTracker
.setBalancerOn(newValue
);
539 server
.loadBalancerTracker
.setBalancerOn(newValue
);
541 } catch (KeeperException ke
) {
542 throw new IOException(ke
);
544 LOG
.info(server
.getClientIdAuditPrefix() + " set balanceSwitch=" + newValue
);
545 if (server
.cpHost
!= null) {
546 server
.cpHost
.postBalanceSwitch(oldValue
, newValue
);
548 server
.getLoadBalancer().updateBalancerStatus(newValue
);
549 } catch (IOException ioe
) {
550 LOG
.warn("Error flipping balance switch", ioe
);
555 boolean synchronousBalanceSwitch(final boolean b
) throws IOException
{
556 return switchBalancer(b
, BalanceSwitchMode
.SYNC
);
560 * @return list of blocking services and their security info classes that this server supports
563 protected List
<BlockingServiceAndInterface
> getServices() {
564 List
<BlockingServiceAndInterface
> bssi
= new ArrayList
<>(5);
565 bssi
.add(new BlockingServiceAndInterface(
566 MasterService
.newReflectiveBlockingService(this),
567 MasterService
.BlockingInterface
.class));
568 bssi
.add(new BlockingServiceAndInterface(
569 RegionServerStatusService
.newReflectiveBlockingService(this),
570 RegionServerStatusService
.BlockingInterface
.class));
571 bssi
.add(new BlockingServiceAndInterface(LockService
.newReflectiveBlockingService(this),
572 LockService
.BlockingInterface
.class));
573 bssi
.add(new BlockingServiceAndInterface(HbckService
.newReflectiveBlockingService(this),
574 HbckService
.BlockingInterface
.class));
575 bssi
.add(new BlockingServiceAndInterface(ClientMetaService
.newReflectiveBlockingService(this),
576 ClientMetaService
.BlockingInterface
.class));
577 bssi
.add(new BlockingServiceAndInterface(AdminService
.newReflectiveBlockingService(this),
578 AdminService
.BlockingInterface
.class));
582 void start(ZKWatcher zkWatcher
) {
583 internalStart(zkWatcher
);
591 @QosPriority(priority
= HConstants
.ADMIN_QOS
)
592 public GetLastFlushedSequenceIdResponse
getLastFlushedSequenceId(RpcController controller
,
593 GetLastFlushedSequenceIdRequest request
) throws ServiceException
{
595 server
.checkServiceStarted();
596 } catch (IOException ioe
) {
597 throw new ServiceException(ioe
);
599 byte[] encodedRegionName
= request
.getRegionName().toByteArray();
600 RegionStoreSequenceIds ids
= server
.getServerManager()
601 .getLastFlushedSequenceId(encodedRegionName
);
602 return ResponseConverter
.buildGetLastFlushedSequenceIdResponse(ids
);
606 public RegionServerReportResponse
regionServerReport(RpcController controller
,
607 RegionServerReportRequest request
) throws ServiceException
{
609 server
.checkServiceStarted();
610 int versionNumber
= 0;
611 String version
= "0.0.0";
612 VersionInfo versionInfo
= VersionInfoUtil
.getCurrentClientVersionInfo();
613 if (versionInfo
!= null) {
614 version
= versionInfo
.getVersion();
615 versionNumber
= VersionInfoUtil
.getVersionNumber(versionInfo
);
617 ClusterStatusProtos
.ServerLoad sl
= request
.getLoad();
618 ServerName serverName
= ProtobufUtil
.toServerName(request
.getServer());
619 ServerMetrics oldLoad
= server
.getServerManager().getLoad(serverName
);
620 ServerMetrics newLoad
=
621 ServerMetricsBuilder
.toServerMetrics(serverName
, versionNumber
, version
, sl
);
622 server
.getServerManager().regionServerReport(serverName
, newLoad
);
623 server
.getAssignmentManager().reportOnlineRegions(serverName
,
624 newLoad
.getRegionMetrics().keySet());
625 if (sl
!= null && server
.metricsMaster
!= null) {
627 server
.metricsMaster
.incrementRequests(
628 sl
.getTotalNumberOfRequests() -
629 (oldLoad
!= null ? oldLoad
.getRequestCount() : 0));
630 server
.metricsMaster
.incrementReadRequests(sl
.getReadRequestsCount() -
631 (oldLoad
!= null ? oldLoad
.getReadRequestsCount() : 0));
632 server
.metricsMaster
.incrementWriteRequests(sl
.getWriteRequestsCount() -
633 (oldLoad
!= null ? oldLoad
.getWriteRequestsCount() : 0));
635 } catch (IOException ioe
) {
636 throw new ServiceException(ioe
);
638 return RegionServerReportResponse
.newBuilder().build();
642 public RegionServerStartupResponse
regionServerStartup(RpcController controller
,
643 RegionServerStartupRequest request
) throws ServiceException
{
644 // Register with server manager
646 server
.checkServiceStarted();
647 int versionNumber
= 0;
648 String version
= "0.0.0";
649 VersionInfo versionInfo
= VersionInfoUtil
.getCurrentClientVersionInfo();
650 if (versionInfo
!= null) {
651 version
= versionInfo
.getVersion();
652 versionNumber
= VersionInfoUtil
.getVersionNumber(versionInfo
);
654 InetAddress ia
= server
.getRemoteInetAddress(request
.getPort(), request
.getServerStartCode());
655 // if regionserver passed hostname to use,
656 // then use it instead of doing a reverse DNS lookup
658 server
.getServerManager().regionServerStartup(request
, versionNumber
, version
, ia
);
660 // Send back some config info
661 RegionServerStartupResponse
.Builder resp
= createConfigurationSubset();
662 NameStringPair
.Builder entry
= NameStringPair
.newBuilder()
663 .setName(HConstants
.KEY_FOR_HOSTNAME_SEEN_BY_MASTER
).setValue(rs
.getHostname());
664 resp
.addMapEntries(entry
.build());
667 } catch (IOException ioe
) {
668 throw new ServiceException(ioe
);
673 public ReportRSFatalErrorResponse
reportRSFatalError(
674 RpcController controller
, ReportRSFatalErrorRequest request
) throws ServiceException
{
675 String errorText
= request
.getErrorMessage();
676 ServerName sn
= ProtobufUtil
.toServerName(request
.getServer());
677 String msg
= sn
+ " reported a fatal error:\n" + errorText
;
679 server
.rsFatals
.add(msg
);
680 return ReportRSFatalErrorResponse
.newBuilder().build();
684 public AddColumnResponse
addColumn(RpcController controller
,
685 AddColumnRequest req
) throws ServiceException
{
687 long procId
= server
.addColumn(
688 ProtobufUtil
.toTableName(req
.getTableName()),
689 ProtobufUtil
.toColumnFamilyDescriptor(req
.getColumnFamilies()),
693 // This mean operation was not performed in server, so do not set any procId
694 return AddColumnResponse
.newBuilder().build();
696 return AddColumnResponse
.newBuilder().setProcId(procId
).build();
698 } catch (IOException ioe
) {
699 throw new ServiceException(ioe
);
704 public AssignRegionResponse
assignRegion(RpcController controller
,
705 AssignRegionRequest req
) throws ServiceException
{
707 server
.checkInitialized();
709 final RegionSpecifierType type
= req
.getRegion().getType();
710 if (type
!= RegionSpecifierType
.REGION_NAME
) {
711 LOG
.warn("assignRegion specifier type: expected: " + RegionSpecifierType
.REGION_NAME
712 + " actual: " + type
);
715 final byte[] regionName
= req
.getRegion().getValue().toByteArray();
716 final RegionInfo regionInfo
= server
.getAssignmentManager().getRegionInfo(regionName
);
717 if (regionInfo
== null) {
718 throw new UnknownRegionException(Bytes
.toStringBinary(regionName
));
721 final AssignRegionResponse arr
= AssignRegionResponse
.newBuilder().build();
722 if (server
.cpHost
!= null) {
723 server
.cpHost
.preAssign(regionInfo
);
725 LOG
.info(server
.getClientIdAuditPrefix() + " assign " + regionInfo
.getRegionNameAsString());
726 server
.getAssignmentManager().assign(regionInfo
);
727 if (server
.cpHost
!= null) {
728 server
.cpHost
.postAssign(regionInfo
);
731 } catch (IOException ioe
) {
732 throw new ServiceException(ioe
);
737 public MasterProtos
.BalanceResponse
balance(RpcController controller
,
738 MasterProtos
.BalanceRequest request
) throws ServiceException
{
740 return ProtobufUtil
.toBalanceResponse(server
.balance(ProtobufUtil
.toBalanceRequest(request
)));
741 } catch (IOException ex
) {
742 throw new ServiceException(ex
);
747 public CreateNamespaceResponse
createNamespace(RpcController controller
,
748 CreateNamespaceRequest request
) throws ServiceException
{
750 long procId
= server
.createNamespace(
751 ProtobufUtil
.toNamespaceDescriptor(request
.getNamespaceDescriptor()),
752 request
.getNonceGroup(),
754 return CreateNamespaceResponse
.newBuilder().setProcId(procId
).build();
755 } catch (IOException e
) {
756 throw new ServiceException(e
);
761 public CreateTableResponse
createTable(RpcController controller
, CreateTableRequest req
)
762 throws ServiceException
{
763 TableDescriptor tableDescriptor
= ProtobufUtil
.toTableDescriptor(req
.getTableSchema());
764 byte [][] splitKeys
= ProtobufUtil
.getSplitKeysArray(req
);
767 server
.createTable(tableDescriptor
, splitKeys
, req
.getNonceGroup(), req
.getNonce());
768 LOG
.info(server
.getClientIdAuditPrefix() + " procedure request for creating table: " +
769 req
.getTableSchema().getTableName() + " procId is: " + procId
);
770 return CreateTableResponse
.newBuilder().setProcId(procId
).build();
771 } catch (IOException ioe
) {
772 throw new ServiceException(ioe
);
777 public DeleteColumnResponse
deleteColumn(RpcController controller
,
778 DeleteColumnRequest req
) throws ServiceException
{
780 long procId
= server
.deleteColumn(
781 ProtobufUtil
.toTableName(req
.getTableName()),
782 req
.getColumnName().toByteArray(),
786 // This mean operation was not performed in server, so do not set any procId
787 return DeleteColumnResponse
.newBuilder().build();
789 return DeleteColumnResponse
.newBuilder().setProcId(procId
).build();
791 } catch (IOException ioe
) {
792 throw new ServiceException(ioe
);
797 public DeleteNamespaceResponse
deleteNamespace(RpcController controller
,
798 DeleteNamespaceRequest request
) throws ServiceException
{
800 long procId
= server
.deleteNamespace(
801 request
.getNamespaceName(),
802 request
.getNonceGroup(),
804 return DeleteNamespaceResponse
.newBuilder().setProcId(procId
).build();
805 } catch (IOException e
) {
806 throw new ServiceException(e
);
811 * Execute Delete Snapshot operation.
812 * @return DeleteSnapshotResponse (a protobuf wrapped void) if the snapshot existed and was
814 * @throws ServiceException wrapping SnapshotDoesNotExistException if specified snapshot did not
818 public DeleteSnapshotResponse
deleteSnapshot(RpcController controller
,
819 DeleteSnapshotRequest request
) throws ServiceException
{
821 server
.checkInitialized();
822 server
.snapshotManager
.checkSnapshotSupport();
824 LOG
.info(server
.getClientIdAuditPrefix() + " delete " + request
.getSnapshot());
825 server
.snapshotManager
.deleteSnapshot(request
.getSnapshot());
826 return DeleteSnapshotResponse
.newBuilder().build();
827 } catch (IOException e
) {
828 throw new ServiceException(e
);
833 public DeleteTableResponse
deleteTable(RpcController controller
,
834 DeleteTableRequest request
) throws ServiceException
{
836 long procId
= server
.deleteTable(ProtobufUtil
.toTableName(
837 request
.getTableName()), request
.getNonceGroup(), request
.getNonce());
838 return DeleteTableResponse
.newBuilder().setProcId(procId
).build();
839 } catch (IOException ioe
) {
840 throw new ServiceException(ioe
);
845 public TruncateTableResponse
truncateTable(RpcController controller
, TruncateTableRequest request
)
846 throws ServiceException
{
848 long procId
= server
.truncateTable(
849 ProtobufUtil
.toTableName(request
.getTableName()),
850 request
.getPreserveSplits(),
851 request
.getNonceGroup(),
853 return TruncateTableResponse
.newBuilder().setProcId(procId
).build();
854 } catch (IOException ioe
) {
855 throw new ServiceException(ioe
);
860 public DisableTableResponse
disableTable(RpcController controller
,
861 DisableTableRequest request
) throws ServiceException
{
863 long procId
= server
.disableTable(
864 ProtobufUtil
.toTableName(request
.getTableName()),
865 request
.getNonceGroup(),
867 return DisableTableResponse
.newBuilder().setProcId(procId
).build();
868 } catch (IOException ioe
) {
869 throw new ServiceException(ioe
);
874 public EnableCatalogJanitorResponse
enableCatalogJanitor(RpcController c
,
875 EnableCatalogJanitorRequest req
) throws ServiceException
{
876 rpcPreCheck("enableCatalogJanitor");
877 return EnableCatalogJanitorResponse
.newBuilder().setPrevValue(
878 server
.catalogJanitorChore
.setEnabled(req
.getEnable())).build();
882 public SetCleanerChoreRunningResponse
setCleanerChoreRunning(
883 RpcController c
, SetCleanerChoreRunningRequest req
) throws ServiceException
{
884 rpcPreCheck("setCleanerChoreRunning");
887 server
.getLogCleaner().getEnabled() && server
.getHFileCleaner().getEnabled();
888 server
.getLogCleaner().setEnabled(req
.getOn());
889 server
.getHFileCleaner().setEnabled(req
.getOn());
890 return SetCleanerChoreRunningResponse
.newBuilder().setPrevValue(prevValue
).build();
894 public EnableTableResponse
enableTable(RpcController controller
,
895 EnableTableRequest request
) throws ServiceException
{
897 long procId
= server
.enableTable(
898 ProtobufUtil
.toTableName(request
.getTableName()),
899 request
.getNonceGroup(),
901 return EnableTableResponse
.newBuilder().setProcId(procId
).build();
902 } catch (IOException ioe
) {
903 throw new ServiceException(ioe
);
908 public MergeTableRegionsResponse
mergeTableRegions(
909 RpcController c
, MergeTableRegionsRequest request
) throws ServiceException
{
911 server
.checkInitialized();
912 } catch (IOException ioe
) {
913 throw new ServiceException(ioe
);
916 RegionStates regionStates
= server
.getAssignmentManager().getRegionStates();
918 RegionInfo
[] regionsToMerge
= new RegionInfo
[request
.getRegionCount()];
919 for (int i
= 0; i
< request
.getRegionCount(); i
++) {
920 final byte[] encodedNameOfRegion
= request
.getRegion(i
).getValue().toByteArray();
921 if (request
.getRegion(i
).getType() != RegionSpecifierType
.ENCODED_REGION_NAME
) {
922 LOG
.warn("MergeRegions specifier type: expected: "
923 + RegionSpecifierType
.ENCODED_REGION_NAME
+ " actual: region " + i
+ " ="
924 + request
.getRegion(i
).getType());
926 RegionState regionState
= regionStates
.getRegionState(Bytes
.toString(encodedNameOfRegion
));
927 if (regionState
== null) {
928 throw new ServiceException(
929 new UnknownRegionException(Bytes
.toStringBinary(encodedNameOfRegion
)));
931 regionsToMerge
[i
] = regionState
.getRegion();
935 long procId
= server
.mergeRegions(
937 request
.getForcible(),
938 request
.getNonceGroup(),
940 return MergeTableRegionsResponse
.newBuilder().setProcId(procId
).build();
941 } catch (IOException ioe
) {
942 throw new ServiceException(ioe
);
947 public SplitTableRegionResponse
splitRegion(final RpcController controller
,
948 final SplitTableRegionRequest request
) throws ServiceException
{
950 long procId
= server
.splitRegion(
951 ProtobufUtil
.toRegionInfo(request
.getRegionInfo()),
952 request
.hasSplitRow() ? request
.getSplitRow().toByteArray() : null,
953 request
.getNonceGroup(),
955 return SplitTableRegionResponse
.newBuilder().setProcId(procId
).build();
956 } catch (IOException ie
) {
957 throw new ServiceException(ie
);
962 public ClientProtos
.CoprocessorServiceResponse
execMasterService(final RpcController controller
,
963 final ClientProtos
.CoprocessorServiceRequest request
) throws ServiceException
{
964 rpcPreCheck("execMasterService");
966 ServerRpcController execController
= new ServerRpcController();
967 ClientProtos
.CoprocessorServiceCall call
= request
.getCall();
968 String serviceName
= call
.getServiceName();
969 String methodName
= call
.getMethodName();
970 if (!server
.coprocessorServiceHandlers
.containsKey(serviceName
)) {
971 throw new UnknownProtocolException(null,
972 "No registered Master Coprocessor Endpoint found for " + serviceName
+
973 ". Has it been enabled?");
976 Service service
= server
.coprocessorServiceHandlers
.get(serviceName
);
977 ServiceDescriptor serviceDesc
= service
.getDescriptorForType();
978 MethodDescriptor methodDesc
=
979 CoprocessorRpcUtils
.getMethodDescriptor(methodName
, serviceDesc
);
981 Message execRequest
=
982 CoprocessorRpcUtils
.getRequest(service
, methodDesc
, call
.getRequest());
983 final Message
.Builder responseBuilder
=
984 service
.getResponsePrototype(methodDesc
).newBuilderForType();
985 service
.callMethod(methodDesc
, execController
, execRequest
,
987 if (message
!= null) {
988 responseBuilder
.mergeFrom(message
);
991 Message execResult
= responseBuilder
.build();
992 if (execController
.getFailedOn() != null) {
993 throw execController
.getFailedOn();
996 String remoteAddress
= RpcServer
.getRemoteAddress().map(InetAddress
::toString
).orElse("");
997 User caller
= RpcServer
.getRequestUser().orElse(null);
998 AUDITLOG
.info("User {} (remote address: {}) master service request for {}.{}", caller
,
999 remoteAddress
, serviceName
, methodName
);
1001 return CoprocessorRpcUtils
.getResponse(execResult
, HConstants
.EMPTY_BYTE_ARRAY
);
1002 } catch (IOException ie
) {
1003 throw new ServiceException(ie
);
1008 * Triggers an asynchronous attempt to run a distributed procedure.
1012 public ExecProcedureResponse
execProcedure(RpcController controller
,
1013 ExecProcedureRequest request
) throws ServiceException
{
1015 server
.checkInitialized();
1016 ProcedureDescription desc
= request
.getProcedure();
1017 MasterProcedureManager mpm
= server
.getMasterProcedureManagerHost().getProcedureManager(
1018 desc
.getSignature());
1020 throw new ServiceException(new DoNotRetryIOException("The procedure is not registered: "
1021 + desc
.getSignature()));
1023 LOG
.info(server
.getClientIdAuditPrefix() + " procedure request for: " + desc
.getSignature());
1024 mpm
.checkPermissions(desc
, getAccessChecker(), RpcServer
.getRequestUser().orElse(null));
1025 mpm
.execProcedure(desc
);
1026 // send back the max amount of time the client should wait for the procedure
1028 long waitTime
= SnapshotDescriptionUtils
.DEFAULT_MAX_WAIT_TIME
;
1029 return ExecProcedureResponse
.newBuilder().setExpectedTimeout(
1031 } catch (ForeignException e
) {
1032 throw new ServiceException(e
.getCause());
1033 } catch (IOException e
) {
1034 throw new ServiceException(e
);
1039 * Triggers a synchronous attempt to run a distributed procedure and sets
1040 * return data in response.
1044 public ExecProcedureResponse
execProcedureWithRet(RpcController controller
,
1045 ExecProcedureRequest request
) throws ServiceException
{
1046 rpcPreCheck("execProcedureWithRet");
1048 ProcedureDescription desc
= request
.getProcedure();
1049 MasterProcedureManager mpm
=
1050 server
.getMasterProcedureManagerHost().getProcedureManager(desc
.getSignature());
1052 throw new ServiceException("The procedure is not registered: " + desc
.getSignature());
1054 LOG
.info(server
.getClientIdAuditPrefix() + " procedure request for: " + desc
.getSignature());
1055 byte[] data
= mpm
.execProcedureWithRet(desc
);
1056 ExecProcedureResponse
.Builder builder
= ExecProcedureResponse
.newBuilder();
1057 // set return data if available
1059 builder
.setReturnData(UnsafeByteOperations
.unsafeWrap(data
));
1061 return builder
.build();
1062 } catch (IOException e
) {
1063 throw new ServiceException(e
);
1068 public GetClusterStatusResponse
getClusterStatus(RpcController controller
,
1069 GetClusterStatusRequest req
) throws ServiceException
{
1070 GetClusterStatusResponse
.Builder response
= GetClusterStatusResponse
.newBuilder();
1072 // We used to check if Master was up at this point but let this call proceed even if
1073 // Master is initializing... else we shut out stuff like hbck2 tool from making progress
1074 // since it queries this method to figure cluster version. hbck2 wants to be able to work
1075 // against Master even if it is 'initializing' so it can do fixup.
1076 response
.setClusterStatus(ClusterMetricsBuilder
.toClusterStatus(
1077 server
.getClusterMetrics(ClusterMetricsBuilder
.toOptions(req
.getOptionsList()))));
1078 } catch (IOException e
) {
1079 throw new ServiceException(e
);
1081 return response
.build();
1085 * List the currently available/stored snapshots. Any in-progress snapshots are ignored
1088 public GetCompletedSnapshotsResponse
getCompletedSnapshots(RpcController controller
,
1089 GetCompletedSnapshotsRequest request
) throws ServiceException
{
1091 server
.checkInitialized();
1092 GetCompletedSnapshotsResponse
.Builder builder
= GetCompletedSnapshotsResponse
.newBuilder();
1093 List
<SnapshotDescription
> snapshots
= server
.snapshotManager
.getCompletedSnapshots();
1095 // convert to protobuf
1096 for (SnapshotDescription snapshot
: snapshots
) {
1097 builder
.addSnapshots(snapshot
);
1099 return builder
.build();
1100 } catch (IOException e
) {
1101 throw new ServiceException(e
);
1106 public ListNamespacesResponse
listNamespaces(
1107 RpcController controller
, ListNamespacesRequest request
)
1108 throws ServiceException
{
1110 return ListNamespacesResponse
.newBuilder()
1111 .addAllNamespaceName(server
.listNamespaces())
1113 } catch (IOException e
) {
1114 throw new ServiceException(e
);
1119 public GetNamespaceDescriptorResponse
getNamespaceDescriptor(
1120 RpcController controller
, GetNamespaceDescriptorRequest request
)
1121 throws ServiceException
{
1123 return GetNamespaceDescriptorResponse
.newBuilder()
1124 .setNamespaceDescriptor(ProtobufUtil
.toProtoNamespaceDescriptor(
1125 server
.getNamespace(request
.getNamespaceName())))
1127 } catch (IOException e
) {
1128 throw new ServiceException(e
);
1133 * Get the number of regions of the table that have been updated by the alter.
1135 * @return Pair indicating the number of regions updated Pair.getFirst is the
1136 * regions that are yet to be updated Pair.getSecond is the total number
1137 * of regions of the table
1138 * @throws ServiceException
1141 public GetSchemaAlterStatusResponse
getSchemaAlterStatus(
1142 RpcController controller
, GetSchemaAlterStatusRequest req
) throws ServiceException
{
1143 // TODO: currently, we query using the table name on the client side. this
1144 // may overlap with other table operations or the table operation may
1145 // have completed before querying this API. We need to refactor to a
1146 // transaction system in the future to avoid these ambiguities.
1147 TableName tableName
= ProtobufUtil
.toTableName(req
.getTableName());
1150 server
.checkInitialized();
1151 Pair
<Integer
,Integer
> pair
= server
.getAssignmentManager().getReopenStatus(tableName
);
1152 GetSchemaAlterStatusResponse
.Builder ret
= GetSchemaAlterStatusResponse
.newBuilder();
1153 ret
.setYetToUpdateRegions(pair
.getFirst());
1154 ret
.setTotalRegions(pair
.getSecond());
1156 } catch (IOException ioe
) {
1157 throw new ServiceException(ioe
);
1162 * Get list of TableDescriptors for requested tables.
1163 * @param c Unused (set to null).
1164 * @param req GetTableDescriptorsRequest that contains:
1165 * - tableNames: requested tables, or if empty, all are requested.
1166 * @return GetTableDescriptorsResponse
1167 * @throws ServiceException
1170 public GetTableDescriptorsResponse
getTableDescriptors(RpcController c
,
1171 GetTableDescriptorsRequest req
) throws ServiceException
{
1173 server
.checkInitialized();
1175 final String regex
= req
.hasRegex() ? req
.getRegex() : null;
1176 final String namespace
= req
.hasNamespace() ? req
.getNamespace() : null;
1177 List
<TableName
> tableNameList
= null;
1178 if (req
.getTableNamesCount() > 0) {
1179 tableNameList
= new ArrayList
<TableName
>(req
.getTableNamesCount());
1180 for (HBaseProtos
.TableName tableNamePB
: req
.getTableNamesList()) {
1181 tableNameList
.add(ProtobufUtil
.toTableName(tableNamePB
));
1185 List
<TableDescriptor
> descriptors
= server
.listTableDescriptors(namespace
, regex
,
1186 tableNameList
, req
.getIncludeSysTables());
1188 GetTableDescriptorsResponse
.Builder builder
= GetTableDescriptorsResponse
.newBuilder();
1189 if (descriptors
!= null && descriptors
.size() > 0) {
1190 // Add the table descriptors to the response
1191 for (TableDescriptor htd
: descriptors
) {
1192 builder
.addTableSchema(ProtobufUtil
.toTableSchema(htd
));
1195 return builder
.build();
1196 } catch (IOException ioe
) {
1197 throw new ServiceException(ioe
);
1202 * Get list of userspace table names
1203 * @param controller Unused (set to null).
1204 * @param req GetTableNamesRequest
1205 * @return GetTableNamesResponse
1206 * @throws ServiceException
1209 public GetTableNamesResponse
getTableNames(RpcController controller
,
1210 GetTableNamesRequest req
) throws ServiceException
{
1212 server
.checkServiceStarted();
1214 final String regex
= req
.hasRegex() ? req
.getRegex() : null;
1215 final String namespace
= req
.hasNamespace() ? req
.getNamespace() : null;
1216 List
<TableName
> tableNames
= server
.listTableNames(namespace
, regex
,
1217 req
.getIncludeSysTables());
1219 GetTableNamesResponse
.Builder builder
= GetTableNamesResponse
.newBuilder();
1220 if (tableNames
!= null && tableNames
.size() > 0) {
1221 // Add the table names to the response
1222 for (TableName table
: tableNames
) {
1223 builder
.addTableNames(ProtobufUtil
.toProtoTableName(table
));
1226 return builder
.build();
1227 } catch (IOException e
) {
1228 throw new ServiceException(e
);
1233 public GetTableStateResponse
getTableState(RpcController controller
,
1234 GetTableStateRequest request
) throws ServiceException
{
1236 server
.checkServiceStarted();
1237 TableName tableName
= ProtobufUtil
.toTableName(request
.getTableName());
1238 TableState ts
= server
.getTableStateManager().getTableState(tableName
);
1239 GetTableStateResponse
.Builder builder
= GetTableStateResponse
.newBuilder();
1240 builder
.setTableState(ts
.convert());
1241 return builder
.build();
1242 } catch (IOException e
) {
1243 throw new ServiceException(e
);
1248 public IsCatalogJanitorEnabledResponse
isCatalogJanitorEnabled(RpcController c
,
1249 IsCatalogJanitorEnabledRequest req
) throws ServiceException
{
1250 return IsCatalogJanitorEnabledResponse
.newBuilder().setValue(
1251 server
.isCatalogJanitorEnabled()).build();
1255 public IsCleanerChoreEnabledResponse
isCleanerChoreEnabled(RpcController c
,
1256 IsCleanerChoreEnabledRequest req
)
1257 throws ServiceException
{
1258 return IsCleanerChoreEnabledResponse
.newBuilder().setValue(server
.isCleanerChoreEnabled())
1263 public IsMasterRunningResponse
isMasterRunning(RpcController c
,
1264 IsMasterRunningRequest req
) throws ServiceException
{
1266 server
.checkServiceStarted();
1267 return IsMasterRunningResponse
.newBuilder().setIsMasterRunning(
1268 !server
.isStopped()).build();
1269 } catch (IOException e
) {
1270 throw new ServiceException(e
);
1275 * Checks if the specified procedure is done.
1276 * @return true if the procedure is done, false if the procedure is in the process of completing
1277 * @throws ServiceException if invalid procedure or failed procedure with progress failure reason.
1280 public IsProcedureDoneResponse
isProcedureDone(RpcController controller
,
1281 IsProcedureDoneRequest request
) throws ServiceException
{
1283 server
.checkInitialized();
1284 ProcedureDescription desc
= request
.getProcedure();
1285 MasterProcedureManager mpm
= server
.getMasterProcedureManagerHost().getProcedureManager(
1286 desc
.getSignature());
1288 throw new ServiceException("The procedure is not registered: "
1289 + desc
.getSignature());
1291 LOG
.debug("Checking to see if procedure from request:"
1292 + desc
.getSignature() + " is done");
1294 IsProcedureDoneResponse
.Builder builder
=
1295 IsProcedureDoneResponse
.newBuilder();
1296 boolean done
= mpm
.isProcedureDone(desc
);
1297 builder
.setDone(done
);
1298 return builder
.build();
1299 } catch (ForeignException e
) {
1300 throw new ServiceException(e
.getCause());
1301 } catch (IOException e
) {
1302 throw new ServiceException(e
);
1307 * Checks if the specified snapshot is done.
1308 * @return true if the snapshot is in file system ready to use,
1309 * false if the snapshot is in the process of completing
1310 * @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or
1311 * a wrapped HBaseSnapshotException with progress failure reason.
1314 public IsSnapshotDoneResponse
isSnapshotDone(RpcController controller
,
1315 IsSnapshotDoneRequest request
) throws ServiceException
{
1316 LOG
.debug("Checking to see if snapshot from request:" +
1317 ClientSnapshotDescriptionUtils
.toString(request
.getSnapshot()) + " is done");
1319 server
.checkInitialized();
1320 IsSnapshotDoneResponse
.Builder builder
= IsSnapshotDoneResponse
.newBuilder();
1321 boolean done
= server
.snapshotManager
.isSnapshotDone(request
.getSnapshot());
1322 builder
.setDone(done
);
1323 return builder
.build();
1324 } catch (ForeignException e
) {
1325 throw new ServiceException(e
.getCause());
1326 } catch (IOException e
) {
1327 throw new ServiceException(e
);
1332 public GetProcedureResultResponse
getProcedureResult(RpcController controller
,
1333 GetProcedureResultRequest request
) throws ServiceException
{
1334 LOG
.debug("Checking to see if procedure is done pid=" + request
.getProcId());
1336 server
.checkInitialized();
1337 GetProcedureResultResponse
.Builder builder
= GetProcedureResultResponse
.newBuilder();
1338 long procId
= request
.getProcId();
1339 ProcedureExecutor
<?
> executor
= server
.getMasterProcedureExecutor();
1340 Procedure
<?
> result
= executor
.getResultOrProcedure(procId
);
1341 if (result
!= null) {
1342 builder
.setSubmittedTime(result
.getSubmittedTime());
1343 builder
.setLastUpdate(result
.getLastUpdate());
1344 if (executor
.isFinished(procId
)) {
1345 builder
.setState(GetProcedureResultResponse
.State
.FINISHED
);
1346 if (result
.isFailed()) {
1347 IOException exception
=
1348 MasterProcedureUtil
.unwrapRemoteIOException(result
);
1349 builder
.setException(ForeignExceptionUtil
.toProtoForeignException(exception
));
1351 byte[] resultData
= result
.getResult();
1352 if (resultData
!= null) {
1353 builder
.setResult(UnsafeByteOperations
.unsafeWrap(resultData
));
1355 server
.getMasterProcedureExecutor().removeResult(request
.getProcId());
1357 builder
.setState(GetProcedureResultResponse
.State
.RUNNING
);
1360 builder
.setState(GetProcedureResultResponse
.State
.NOT_FOUND
);
1362 return builder
.build();
1363 } catch (IOException e
) {
1364 throw new ServiceException(e
);
1369 public AbortProcedureResponse
abortProcedure(
1370 RpcController rpcController
, AbortProcedureRequest request
) throws ServiceException
{
1372 AbortProcedureResponse
.Builder response
= AbortProcedureResponse
.newBuilder();
1373 boolean abortResult
=
1374 server
.abortProcedure(request
.getProcId(), request
.getMayInterruptIfRunning());
1375 response
.setIsProcedureAborted(abortResult
);
1376 return response
.build();
1377 } catch (IOException e
) {
1378 throw new ServiceException(e
);
1383 public ListNamespaceDescriptorsResponse
listNamespaceDescriptors(RpcController c
,
1384 ListNamespaceDescriptorsRequest request
) throws ServiceException
{
1386 ListNamespaceDescriptorsResponse
.Builder response
=
1387 ListNamespaceDescriptorsResponse
.newBuilder();
1388 for(NamespaceDescriptor ns
: server
.getNamespaces()) {
1389 response
.addNamespaceDescriptor(ProtobufUtil
.toProtoNamespaceDescriptor(ns
));
1391 return response
.build();
1392 } catch (IOException e
) {
1393 throw new ServiceException(e
);
1398 public GetProceduresResponse
getProcedures(
1399 RpcController rpcController
,
1400 GetProceduresRequest request
) throws ServiceException
{
1402 final GetProceduresResponse
.Builder response
= GetProceduresResponse
.newBuilder();
1403 for (Procedure
<?
> p
: server
.getProcedures()) {
1404 response
.addProcedure(ProcedureUtil
.convertToProtoProcedure(p
));
1406 return response
.build();
1407 } catch (IOException e
) {
1408 throw new ServiceException(e
);
1413 public GetLocksResponse
getLocks(
1414 RpcController controller
,
1415 GetLocksRequest request
) throws ServiceException
{
1417 final GetLocksResponse
.Builder builder
= GetLocksResponse
.newBuilder();
1419 for (LockedResource lockedResource
: server
.getLocks()) {
1420 builder
.addLock(ProcedureUtil
.convertToProtoLockedResource(lockedResource
));
1423 return builder
.build();
1424 } catch (IOException e
) {
1425 throw new ServiceException(e
);
1430 public ListTableDescriptorsByNamespaceResponse
listTableDescriptorsByNamespace(RpcController c
,
1431 ListTableDescriptorsByNamespaceRequest request
) throws ServiceException
{
1433 ListTableDescriptorsByNamespaceResponse
.Builder b
=
1434 ListTableDescriptorsByNamespaceResponse
.newBuilder();
1435 for (TableDescriptor htd
: server
1436 .listTableDescriptorsByNamespace(request
.getNamespaceName())) {
1437 b
.addTableSchema(ProtobufUtil
.toTableSchema(htd
));
1440 } catch (IOException e
) {
1441 throw new ServiceException(e
);
1446 public ListTableNamesByNamespaceResponse
listTableNamesByNamespace(RpcController c
,
1447 ListTableNamesByNamespaceRequest request
) throws ServiceException
{
1449 ListTableNamesByNamespaceResponse
.Builder b
=
1450 ListTableNamesByNamespaceResponse
.newBuilder();
1451 for (TableName tableName
: server
.listTableNamesByNamespace(request
.getNamespaceName())) {
1452 b
.addTableName(ProtobufUtil
.toProtoTableName(tableName
));
1455 } catch (IOException e
) {
1456 throw new ServiceException(e
);
1461 public ModifyColumnResponse
modifyColumn(RpcController controller
,
1462 ModifyColumnRequest req
) throws ServiceException
{
1464 long procId
= server
.modifyColumn(
1465 ProtobufUtil
.toTableName(req
.getTableName()),
1466 ProtobufUtil
.toColumnFamilyDescriptor(req
.getColumnFamilies()),
1467 req
.getNonceGroup(),
1470 // This mean operation was not performed in server, so do not set any procId
1471 return ModifyColumnResponse
.newBuilder().build();
1473 return ModifyColumnResponse
.newBuilder().setProcId(procId
).build();
1475 } catch (IOException ioe
) {
1476 throw new ServiceException(ioe
);
1481 public ModifyNamespaceResponse
modifyNamespace(RpcController controller
,
1482 ModifyNamespaceRequest request
) throws ServiceException
{
1484 long procId
= server
.modifyNamespace(
1485 ProtobufUtil
.toNamespaceDescriptor(request
.getNamespaceDescriptor()),
1486 request
.getNonceGroup(),
1487 request
.getNonce());
1488 return ModifyNamespaceResponse
.newBuilder().setProcId(procId
).build();
1489 } catch (IOException e
) {
1490 throw new ServiceException(e
);
1495 public ModifyTableResponse
modifyTable(RpcController controller
,
1496 ModifyTableRequest req
) throws ServiceException
{
1498 long procId
= server
.modifyTable(
1499 ProtobufUtil
.toTableName(req
.getTableName()),
1500 ProtobufUtil
.toTableDescriptor(req
.getTableSchema()),
1501 req
.getNonceGroup(),
1503 return ModifyTableResponse
.newBuilder().setProcId(procId
).build();
1504 } catch (IOException ioe
) {
1505 throw new ServiceException(ioe
);
1510 public MoveRegionResponse
moveRegion(RpcController controller
,
1511 MoveRegionRequest req
) throws ServiceException
{
1512 final byte [] encodedRegionName
= req
.getRegion().getValue().toByteArray();
1513 RegionSpecifierType type
= req
.getRegion().getType();
1514 final byte [] destServerName
= (req
.hasDestServerName())?
1515 Bytes
.toBytes(ProtobufUtil
.toServerName(req
.getDestServerName()).getServerName()):null;
1516 MoveRegionResponse mrr
= MoveRegionResponse
.newBuilder().build();
1518 if (type
!= RegionSpecifierType
.ENCODED_REGION_NAME
) {
1519 LOG
.warn("moveRegion specifier type: expected: " + RegionSpecifierType
.ENCODED_REGION_NAME
1520 + " actual: " + type
);
1524 server
.checkInitialized();
1525 server
.move(encodedRegionName
, destServerName
);
1526 } catch (IOException ioe
) {
1527 throw new ServiceException(ioe
);
1533 * Offline specified region from master's in-memory state. It will not attempt to
1534 * reassign the region as in unassign.
1536 * This is a special method that should be used by experts or hbck.
1540 public OfflineRegionResponse
offlineRegion(RpcController controller
,
1541 OfflineRegionRequest request
) throws ServiceException
{
1543 server
.checkInitialized();
1545 final RegionSpecifierType type
= request
.getRegion().getType();
1546 if (type
!= RegionSpecifierType
.REGION_NAME
) {
1547 LOG
.warn("moveRegion specifier type: expected: " + RegionSpecifierType
.REGION_NAME
1548 + " actual: " + type
);
1551 final byte[] regionName
= request
.getRegion().getValue().toByteArray();
1552 final RegionInfo hri
= server
.getAssignmentManager().getRegionInfo(regionName
);
1554 throw new UnknownRegionException(Bytes
.toStringBinary(regionName
));
1557 if (server
.cpHost
!= null) {
1558 server
.cpHost
.preRegionOffline(hri
);
1560 LOG
.info(server
.getClientIdAuditPrefix() + " offline " + hri
.getRegionNameAsString());
1561 server
.getAssignmentManager().offlineRegion(hri
);
1562 if (server
.cpHost
!= null) {
1563 server
.cpHost
.postRegionOffline(hri
);
1565 } catch (IOException ioe
) {
1566 throw new ServiceException(ioe
);
1568 return OfflineRegionResponse
.newBuilder().build();
1572 * Execute Restore/Clone snapshot operation.
1574 * <p>If the specified table exists a "Restore" is executed, replacing the table
1575 * schema and directory data with the content of the snapshot.
1576 * The table must be disabled, or a UnsupportedOperationException will be thrown.
1578 * <p>If the table doesn't exist a "Clone" is executed, a new table is created
1579 * using the schema at the time of the snapshot, and the content of the snapshot.
1581 * <p>The restore/clone operation does not require copying HFiles. Since HFiles
1582 * are immutable the table can point to and use the same files as the original one.
1585 public RestoreSnapshotResponse
restoreSnapshot(RpcController controller
,
1586 RestoreSnapshotRequest request
) throws ServiceException
{
1588 long procId
= server
.restoreSnapshot(request
.getSnapshot(), request
.getNonceGroup(),
1589 request
.getNonce(), request
.getRestoreACL(), request
.getCustomSFT());
1590 return RestoreSnapshotResponse
.newBuilder().setProcId(procId
).build();
1591 } catch (ForeignException e
) {
1592 throw new ServiceException(e
.getCause());
1593 } catch (IOException e
) {
1594 throw new ServiceException(e
);
1599 public SetSnapshotCleanupResponse
switchSnapshotCleanup(
1600 RpcController controller
, SetSnapshotCleanupRequest request
)
1601 throws ServiceException
{
1603 server
.checkInitialized();
1604 final boolean enabled
= request
.getEnabled();
1605 final boolean isSynchronous
= request
.hasSynchronous() && request
.getSynchronous();
1606 final boolean prevSnapshotCleanupRunning
= this.switchSnapshotCleanup(enabled
, isSynchronous
);
1607 return SetSnapshotCleanupResponse
.newBuilder()
1608 .setPrevSnapshotCleanup(prevSnapshotCleanupRunning
).build();
1609 } catch (IOException e
) {
1610 throw new ServiceException(e
);
1615 public IsSnapshotCleanupEnabledResponse
isSnapshotCleanupEnabled(
1616 RpcController controller
, IsSnapshotCleanupEnabledRequest request
)
1617 throws ServiceException
{
1619 server
.checkInitialized();
1620 final boolean isSnapshotCleanupEnabled
= server
.snapshotCleanupTracker
1621 .isSnapshotCleanupEnabled();
1622 return IsSnapshotCleanupEnabledResponse
.newBuilder()
1623 .setEnabled(isSnapshotCleanupEnabled
).build();
1624 } catch (IOException e
) {
1625 throw new ServiceException(e
);
1630 * Turn on/off snapshot auto-cleanup based on TTL
1632 * @param enabledNewVal Set to <code>true</code> to enable, <code>false</code> to disable
1633 * @param synchronous If <code>true</code>, it waits until current snapshot cleanup is completed,
1635 * @return previous snapshot auto-cleanup mode
1637 private synchronized boolean switchSnapshotCleanup(final boolean enabledNewVal
,
1638 final boolean synchronous
) {
1639 final boolean oldValue
= server
.snapshotCleanupTracker
.isSnapshotCleanupEnabled();
1640 server
.switchSnapshotCleanup(enabledNewVal
, synchronous
);
1641 LOG
.info("{} Successfully set snapshot cleanup to {}", server
.getClientIdAuditPrefix(),
1648 public RunCatalogScanResponse
runCatalogScan(RpcController c
,
1649 RunCatalogScanRequest req
) throws ServiceException
{
1650 rpcPreCheck("runCatalogScan");
1652 return ResponseConverter
.buildRunCatalogScanResponse(
1653 this.server
.catalogJanitorChore
.scan());
1654 } catch (IOException ioe
) {
1655 throw new ServiceException(ioe
);
1660 public RunCleanerChoreResponse
runCleanerChore(RpcController c
, RunCleanerChoreRequest req
)
1661 throws ServiceException
{
1662 rpcPreCheck("runCleanerChore");
1663 boolean result
= server
.getHFileCleaner().runCleaner() && server
.getLogCleaner().runCleaner();
1664 return ResponseConverter
.buildRunCleanerChoreResponse(result
);
1668 public SetBalancerRunningResponse
setBalancerRunning(RpcController c
,
1669 SetBalancerRunningRequest req
) throws ServiceException
{
1671 server
.checkInitialized();
1672 boolean prevValue
= (req
.getSynchronous())?
1673 synchronousBalanceSwitch(req
.getOn()) : server
.balanceSwitch(req
.getOn());
1674 return SetBalancerRunningResponse
.newBuilder().setPrevBalanceValue(prevValue
).build();
1675 } catch (IOException ioe
) {
1676 throw new ServiceException(ioe
);
1681 public ShutdownResponse
shutdown(RpcController controller
,
1682 ShutdownRequest request
) throws ServiceException
{
1683 LOG
.info(server
.getClientIdAuditPrefix() + " shutdown");
1686 } catch (IOException e
) {
1687 LOG
.error("Exception occurred in HMaster.shutdown()", e
);
1688 throw new ServiceException(e
);
1690 return ShutdownResponse
.newBuilder().build();
1694 * Triggers an asynchronous attempt to take a snapshot.
1698 public SnapshotResponse
snapshot(RpcController controller
,
1699 SnapshotRequest request
) throws ServiceException
{
1701 server
.checkInitialized();
1702 server
.snapshotManager
.checkSnapshotSupport();
1704 LOG
.info(server
.getClientIdAuditPrefix() + " snapshot request for:" +
1705 ClientSnapshotDescriptionUtils
.toString(request
.getSnapshot()));
1706 // get the snapshot information
1707 SnapshotDescription snapshot
= SnapshotDescriptionUtils
.validate(
1708 request
.getSnapshot(), server
.getConfiguration());
1709 server
.snapshotManager
.takeSnapshot(snapshot
);
1711 // send back the max amount of time the client should wait for the snapshot to complete
1712 long waitTime
= SnapshotDescriptionUtils
.getMaxMasterTimeout(server
.getConfiguration(),
1713 snapshot
.getType(), SnapshotDescriptionUtils
.DEFAULT_MAX_WAIT_TIME
);
1714 return SnapshotResponse
.newBuilder().setExpectedTimeout(waitTime
).build();
1715 } catch (ForeignException e
) {
1716 throw new ServiceException(e
.getCause());
1717 } catch (IOException e
) {
1718 throw new ServiceException(e
);
1723 public StopMasterResponse
stopMaster(RpcController controller
,
1724 StopMasterRequest request
) throws ServiceException
{
1725 LOG
.info(server
.getClientIdAuditPrefix() + " stop");
1727 server
.stopMaster();
1728 } catch (IOException e
) {
1729 LOG
.error("Exception occurred while stopping master", e
);
1730 throw new ServiceException(e
);
1732 return StopMasterResponse
.newBuilder().build();
1736 public IsInMaintenanceModeResponse
isMasterInMaintenanceMode(
1737 final RpcController controller
,
1738 final IsInMaintenanceModeRequest request
) throws ServiceException
{
1739 IsInMaintenanceModeResponse
.Builder response
= IsInMaintenanceModeResponse
.newBuilder();
1740 response
.setInMaintenanceMode(server
.isInMaintenanceMode());
1741 return response
.build();
1745 public UnassignRegionResponse
unassignRegion(RpcController controller
,
1746 UnassignRegionRequest req
) throws ServiceException
{
1748 final byte [] regionName
= req
.getRegion().getValue().toByteArray();
1749 RegionSpecifierType type
= req
.getRegion().getType();
1750 UnassignRegionResponse urr
= UnassignRegionResponse
.newBuilder().build();
1752 server
.checkInitialized();
1753 if (type
!= RegionSpecifierType
.REGION_NAME
) {
1754 LOG
.warn("unassignRegion specifier type: expected: " + RegionSpecifierType
.REGION_NAME
1755 + " actual: " + type
);
1757 RegionStateNode rsn
=
1758 server
.getAssignmentManager().getRegionStates().getRegionStateNodeFromName(regionName
);
1760 throw new UnknownRegionException(Bytes
.toString(regionName
));
1763 RegionInfo hri
= rsn
.getRegionInfo();
1764 if (server
.cpHost
!= null) {
1765 server
.cpHost
.preUnassign(hri
);
1767 LOG
.debug(server
.getClientIdAuditPrefix() + " unassign " + hri
.getRegionNameAsString()
1768 + " in current location if it is online");
1769 server
.getAssignmentManager().unassign(hri
);
1770 if (server
.cpHost
!= null) {
1771 server
.cpHost
.postUnassign(hri
);
1775 } catch (IOException ioe
) {
1776 throw new ServiceException(ioe
);
1781 public ReportRegionStateTransitionResponse
reportRegionStateTransition(RpcController c
,
1782 ReportRegionStateTransitionRequest req
) throws ServiceException
{
1784 server
.checkServiceStarted();
1785 return server
.getAssignmentManager().reportRegionStateTransition(req
);
1786 } catch (IOException ioe
) {
1787 throw new ServiceException(ioe
);
1792 public SetQuotaResponse
setQuota(RpcController c
, SetQuotaRequest req
)
1793 throws ServiceException
{
1795 server
.checkInitialized();
1796 return server
.getMasterQuotaManager().setQuota(req
);
1797 } catch (Exception e
) {
1798 throw new ServiceException(e
);
1803 public MajorCompactionTimestampResponse
getLastMajorCompactionTimestamp(RpcController controller
,
1804 MajorCompactionTimestampRequest request
) throws ServiceException
{
1805 MajorCompactionTimestampResponse
.Builder response
=
1806 MajorCompactionTimestampResponse
.newBuilder();
1808 server
.checkInitialized();
1809 response
.setCompactionTimestamp(server
.getLastMajorCompactionTimestamp(ProtobufUtil
1810 .toTableName(request
.getTableName())));
1811 } catch (IOException e
) {
1812 throw new ServiceException(e
);
1814 return response
.build();
1818 public MajorCompactionTimestampResponse
getLastMajorCompactionTimestampForRegion(
1819 RpcController controller
, MajorCompactionTimestampForRegionRequest request
)
1820 throws ServiceException
{
1821 MajorCompactionTimestampResponse
.Builder response
=
1822 MajorCompactionTimestampResponse
.newBuilder();
1824 server
.checkInitialized();
1825 response
.setCompactionTimestamp(server
.getLastMajorCompactionTimestampForRegion(request
1826 .getRegion().getValue().toByteArray()));
1827 } catch (IOException e
) {
1828 throw new ServiceException(e
);
1830 return response
.build();
1834 public IsBalancerEnabledResponse
isBalancerEnabled(RpcController controller
,
1835 IsBalancerEnabledRequest request
) throws ServiceException
{
1836 IsBalancerEnabledResponse
.Builder response
= IsBalancerEnabledResponse
.newBuilder();
1837 response
.setEnabled(server
.isBalancerOn());
1838 return response
.build();
1842 public SetSplitOrMergeEnabledResponse
setSplitOrMergeEnabled(RpcController controller
,
1843 SetSplitOrMergeEnabledRequest request
) throws ServiceException
{
1844 SetSplitOrMergeEnabledResponse
.Builder response
= SetSplitOrMergeEnabledResponse
.newBuilder();
1846 server
.checkInitialized();
1847 boolean newValue
= request
.getEnabled();
1848 for (MasterProtos
.MasterSwitchType masterSwitchType
: request
.getSwitchTypesList()) {
1849 MasterSwitchType switchType
= convert(masterSwitchType
);
1850 boolean oldValue
= server
.isSplitOrMergeEnabled(switchType
);
1851 response
.addPrevValue(oldValue
);
1852 if (server
.cpHost
!= null) {
1853 server
.cpHost
.preSetSplitOrMergeEnabled(newValue
, switchType
);
1855 server
.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue
, switchType
);
1856 if (server
.cpHost
!= null) {
1857 server
.cpHost
.postSetSplitOrMergeEnabled(newValue
, switchType
);
1860 } catch (IOException
| KeeperException e
) {
1861 throw new ServiceException(e
);
1863 return response
.build();
1867 public IsSplitOrMergeEnabledResponse
isSplitOrMergeEnabled(RpcController controller
,
1868 IsSplitOrMergeEnabledRequest request
) throws ServiceException
{
1869 IsSplitOrMergeEnabledResponse
.Builder response
= IsSplitOrMergeEnabledResponse
.newBuilder();
1870 response
.setEnabled(server
.isSplitOrMergeEnabled(convert(request
.getSwitchType())));
1871 return response
.build();
1875 public NormalizeResponse
normalize(RpcController controller
,
1876 NormalizeRequest request
) throws ServiceException
{
1877 rpcPreCheck("normalize");
1879 final NormalizeTableFilterParams ntfp
= new NormalizeTableFilterParams
.Builder()
1880 .tableNames(ProtobufUtil
.toTableNameList(request
.getTableNamesList()))
1881 .regex(request
.hasRegex() ? request
.getRegex() : null)
1882 .namespace(request
.hasNamespace() ? request
.getNamespace() : null)
1884 return NormalizeResponse
.newBuilder()
1885 // all API requests are considered priority requests.
1886 .setNormalizerRan(server
.normalizeRegions(ntfp
, true))
1888 } catch (IOException ex
) {
1889 throw new ServiceException(ex
);
1894 public SetNormalizerRunningResponse
setNormalizerRunning(RpcController controller
,
1895 SetNormalizerRunningRequest request
) throws ServiceException
{
1896 rpcPreCheck("setNormalizerRunning");
1898 // Sets normalizer on/off flag in ZK.
1899 // TODO: this method is totally broken in terms of atomicity of actions and values read.
1900 // 1. The contract has this RPC returning the previous value. There isn't a ZKUtil method
1901 // that lets us retrieve the previous value as part of setting a new value, so we simply
1902 // perform a read before issuing the update. Thus we have a data race opportunity, between
1903 // when the `prevValue` is read and whatever is actually overwritten.
1904 // 2. Down in `setNormalizerOn`, the call to `createAndWatch` inside of the catch clause can
1905 // itself fail in the event that the znode already exists. Thus, another data race, between
1906 // when the initial `setData` call is notified of the absence of the target znode and the
1907 // subsequent `createAndWatch`, with another client creating said node.
1908 // That said, there's supposed to be only one active master and thus there's supposed to be
1909 // only one process with the authority to modify the value.
1910 final boolean prevValue
= server
.getRegionNormalizerManager().isNormalizerOn();
1911 final boolean newValue
= request
.getOn();
1912 server
.getRegionNormalizerManager().setNormalizerOn(newValue
);
1913 LOG
.info("{} set normalizerSwitch={}", server
.getClientIdAuditPrefix(), newValue
);
1914 return SetNormalizerRunningResponse
.newBuilder().setPrevNormalizerValue(prevValue
).build();
1918 public IsNormalizerEnabledResponse
isNormalizerEnabled(RpcController controller
,
1919 IsNormalizerEnabledRequest request
) {
1920 IsNormalizerEnabledResponse
.Builder response
= IsNormalizerEnabledResponse
.newBuilder();
1921 response
.setEnabled(server
.isNormalizerOn());
1922 return response
.build();
1926 * Returns the security capabilities in effect on the cluster
1929 public SecurityCapabilitiesResponse
getSecurityCapabilities(RpcController controller
,
1930 SecurityCapabilitiesRequest request
) throws ServiceException
{
1931 SecurityCapabilitiesResponse
.Builder response
= SecurityCapabilitiesResponse
.newBuilder();
1933 server
.checkInitialized();
1934 Set
<SecurityCapabilitiesResponse
.Capability
> capabilities
= new HashSet
<>();
1936 if (User
.isHBaseSecurityEnabled(server
.getConfiguration())) {
1937 capabilities
.add(SecurityCapabilitiesResponse
.Capability
.SECURE_AUTHENTICATION
);
1939 capabilities
.add(SecurityCapabilitiesResponse
.Capability
.SIMPLE_AUTHENTICATION
);
1941 // A coprocessor that implements AccessControlService can provide AUTHORIZATION and
1942 // CELL_AUTHORIZATION
1943 if (server
.cpHost
!= null && hasAccessControlServiceCoprocessor(server
.cpHost
)) {
1944 if (AccessChecker
.isAuthorizationSupported(server
.getConfiguration())) {
1945 capabilities
.add(SecurityCapabilitiesResponse
.Capability
.AUTHORIZATION
);
1947 if (AccessController
.isCellAuthorizationSupported(server
.getConfiguration())) {
1948 capabilities
.add(SecurityCapabilitiesResponse
.Capability
.CELL_AUTHORIZATION
);
1951 // A coprocessor that implements VisibilityLabelsService can provide CELL_VISIBILITY.
1952 if (server
.cpHost
!= null && hasVisibilityLabelsServiceCoprocessor(server
.cpHost
)) {
1953 if (VisibilityController
.isCellAuthorizationSupported(server
.getConfiguration())) {
1954 capabilities
.add(SecurityCapabilitiesResponse
.Capability
.CELL_VISIBILITY
);
1957 response
.addAllCapabilities(capabilities
);
1958 } catch (IOException e
) {
1959 throw new ServiceException(e
);
1961 return response
.build();
1965 * Determines if there is a MasterCoprocessor deployed which implements
1966 * {@link AccessControlService.Interface}.
1968 boolean hasAccessControlServiceCoprocessor(MasterCoprocessorHost cpHost
) {
1969 return checkCoprocessorWithService(cpHost
.findCoprocessors(MasterCoprocessor
.class),
1970 AccessControlService
.Interface
.class);
1974 * Determines if there is a MasterCoprocessor deployed which implements
1975 * {@link VisibilityLabelsService.Interface}.
1977 boolean hasVisibilityLabelsServiceCoprocessor(MasterCoprocessorHost cpHost
) {
1978 return checkCoprocessorWithService(cpHost
.findCoprocessors(MasterCoprocessor
.class),
1979 VisibilityLabelsService
.Interface
.class);
1983 * Determines if there is a coprocessor implementation in the provided argument which extends
1984 * or implements the provided {@code service}.
1986 boolean checkCoprocessorWithService(
1987 List
<MasterCoprocessor
> coprocessorsToCheck
, Class
<?
> service
) {
1988 if (coprocessorsToCheck
== null || coprocessorsToCheck
.isEmpty()) {
1991 for (MasterCoprocessor cp
: coprocessorsToCheck
) {
1992 if (service
.isAssignableFrom(cp
.getClass())) {
1999 private MasterSwitchType
convert(MasterProtos
.MasterSwitchType switchType
) {
2000 switch (switchType
) {
2002 return MasterSwitchType
.SPLIT
;
2004 return MasterSwitchType
.MERGE
;
2012 public AddReplicationPeerResponse
addReplicationPeer(RpcController controller
,
2013 AddReplicationPeerRequest request
) throws ServiceException
{
2015 long procId
= server
.addReplicationPeer(request
.getPeerId(),
2016 ReplicationPeerConfigUtil
.convert(request
.getPeerConfig()),
2017 request
.getPeerState().getState().equals(ReplicationState
.State
.ENABLED
));
2018 return AddReplicationPeerResponse
.newBuilder().setProcId(procId
).build();
2019 } catch (ReplicationException
| IOException e
) {
2020 throw new ServiceException(e
);
2025 public RemoveReplicationPeerResponse
removeReplicationPeer(RpcController controller
,
2026 RemoveReplicationPeerRequest request
) throws ServiceException
{
2028 long procId
= server
.removeReplicationPeer(request
.getPeerId());
2029 return RemoveReplicationPeerResponse
.newBuilder().setProcId(procId
).build();
2030 } catch (ReplicationException
| IOException e
) {
2031 throw new ServiceException(e
);
2036 public EnableReplicationPeerResponse
enableReplicationPeer(RpcController controller
,
2037 EnableReplicationPeerRequest request
) throws ServiceException
{
2039 long procId
= server
.enableReplicationPeer(request
.getPeerId());
2040 return EnableReplicationPeerResponse
.newBuilder().setProcId(procId
).build();
2041 } catch (ReplicationException
| IOException e
) {
2042 throw new ServiceException(e
);
2047 public DisableReplicationPeerResponse
disableReplicationPeer(RpcController controller
,
2048 DisableReplicationPeerRequest request
) throws ServiceException
{
2050 long procId
= server
.disableReplicationPeer(request
.getPeerId());
2051 return DisableReplicationPeerResponse
.newBuilder().setProcId(procId
).build();
2052 } catch (ReplicationException
| IOException e
) {
2053 throw new ServiceException(e
);
2058 public GetReplicationPeerConfigResponse
getReplicationPeerConfig(RpcController controller
,
2059 GetReplicationPeerConfigRequest request
) throws ServiceException
{
2060 GetReplicationPeerConfigResponse
.Builder response
= GetReplicationPeerConfigResponse
2063 String peerId
= request
.getPeerId();
2064 ReplicationPeerConfig peerConfig
= server
.getReplicationPeerConfig(peerId
);
2065 response
.setPeerId(peerId
);
2066 response
.setPeerConfig(ReplicationPeerConfigUtil
.convert(peerConfig
));
2067 } catch (ReplicationException
| IOException e
) {
2068 throw new ServiceException(e
);
2070 return response
.build();
2074 public UpdateReplicationPeerConfigResponse
updateReplicationPeerConfig(RpcController controller
,
2075 UpdateReplicationPeerConfigRequest request
) throws ServiceException
{
2077 long procId
= server
.updateReplicationPeerConfig(request
.getPeerId(),
2078 ReplicationPeerConfigUtil
.convert(request
.getPeerConfig()));
2079 return UpdateReplicationPeerConfigResponse
.newBuilder().setProcId(procId
).build();
2080 } catch (ReplicationException
| IOException e
) {
2081 throw new ServiceException(e
);
2086 public TransitReplicationPeerSyncReplicationStateResponse
2087 transitReplicationPeerSyncReplicationState(RpcController controller
,
2088 TransitReplicationPeerSyncReplicationStateRequest request
) throws ServiceException
{
2090 long procId
= server
.transitReplicationPeerSyncReplicationState(request
.getPeerId(),
2091 ReplicationPeerConfigUtil
.toSyncReplicationState(request
.getSyncReplicationState()));
2092 return TransitReplicationPeerSyncReplicationStateResponse
.newBuilder().setProcId(procId
)
2094 } catch (ReplicationException
| IOException e
) {
2095 throw new ServiceException(e
);
2100 public ListReplicationPeersResponse
listReplicationPeers(RpcController controller
,
2101 ListReplicationPeersRequest request
) throws ServiceException
{
2102 ListReplicationPeersResponse
.Builder response
= ListReplicationPeersResponse
.newBuilder();
2104 List
<ReplicationPeerDescription
> peers
= server
2105 .listReplicationPeers(request
.hasRegex() ? request
.getRegex() : null);
2106 for (ReplicationPeerDescription peer
: peers
) {
2107 response
.addPeerDesc(ReplicationPeerConfigUtil
.toProtoReplicationPeerDescription(peer
));
2109 } catch (ReplicationException
| IOException e
) {
2110 throw new ServiceException(e
);
2112 return response
.build();
2116 public ListDecommissionedRegionServersResponse
listDecommissionedRegionServers(
2117 RpcController controller
, ListDecommissionedRegionServersRequest request
)
2118 throws ServiceException
{
2119 ListDecommissionedRegionServersResponse
.Builder response
=
2120 ListDecommissionedRegionServersResponse
.newBuilder();
2122 server
.checkInitialized();
2123 if (server
.cpHost
!= null) {
2124 server
.cpHost
.preListDecommissionedRegionServers();
2126 List
<ServerName
> servers
= server
.listDecommissionedRegionServers();
2127 response
.addAllServerName((servers
.stream().map(server
-> ProtobufUtil
.toServerName(server
)))
2128 .collect(Collectors
.toList()));
2129 if (server
.cpHost
!= null) {
2130 server
.cpHost
.postListDecommissionedRegionServers();
2132 } catch (IOException io
) {
2133 throw new ServiceException(io
);
2136 return response
.build();
2140 public DecommissionRegionServersResponse
decommissionRegionServers(RpcController controller
,
2141 DecommissionRegionServersRequest request
) throws ServiceException
{
2143 server
.checkInitialized();
2144 List
<ServerName
> servers
= request
.getServerNameList().stream()
2145 .map(pbServer
-> ProtobufUtil
.toServerName(pbServer
)).collect(Collectors
.toList());
2146 boolean offload
= request
.getOffload();
2147 if (server
.cpHost
!= null) {
2148 server
.cpHost
.preDecommissionRegionServers(servers
, offload
);
2150 server
.decommissionRegionServers(servers
, offload
);
2151 if (server
.cpHost
!= null) {
2152 server
.cpHost
.postDecommissionRegionServers(servers
, offload
);
2154 } catch (IOException io
) {
2155 throw new ServiceException(io
);
2158 return DecommissionRegionServersResponse
.newBuilder().build();
2162 public RecommissionRegionServerResponse
recommissionRegionServer(RpcController controller
,
2163 RecommissionRegionServerRequest request
) throws ServiceException
{
2165 server
.checkInitialized();
2166 ServerName sn
= ProtobufUtil
.toServerName(request
.getServerName());
2167 List
<byte[]> encodedRegionNames
= request
.getRegionList().stream()
2168 .map(regionSpecifier
-> regionSpecifier
.getValue().toByteArray())
2169 .collect(Collectors
.toList());
2170 if (server
.cpHost
!= null) {
2171 server
.cpHost
.preRecommissionRegionServer(sn
, encodedRegionNames
);
2173 server
.recommissionRegionServer(sn
, encodedRegionNames
);
2174 if (server
.cpHost
!= null) {
2175 server
.cpHost
.postRecommissionRegionServer(sn
, encodedRegionNames
);
2177 } catch (IOException io
) {
2178 throw new ServiceException(io
);
2181 return RecommissionRegionServerResponse
.newBuilder().build();
2185 public LockResponse
requestLock(RpcController controller
, final LockRequest request
)
2186 throws ServiceException
{
2188 if (request
.getDescription().isEmpty()) {
2189 throw new IllegalArgumentException("Empty description");
2191 NonceProcedureRunnable npr
;
2192 LockType type
= LockType
.valueOf(request
.getLockType().name());
2193 if (request
.getRegionInfoCount() > 0) {
2194 final RegionInfo
[] regionInfos
= new RegionInfo
[request
.getRegionInfoCount()];
2195 for (int i
= 0; i
< request
.getRegionInfoCount(); ++i
) {
2196 regionInfos
[i
] = ProtobufUtil
.toRegionInfo(request
.getRegionInfo(i
));
2198 npr
= new NonceProcedureRunnable(server
, request
.getNonceGroup(), request
.getNonce()) {
2200 protected void run() throws IOException
{
2201 setProcId(server
.getLockManager().remoteLocks().requestRegionsLock(regionInfos
,
2202 request
.getDescription(), getNonceKey()));
2206 protected String
getDescription() {
2207 return "RequestLock";
2210 } else if (request
.hasTableName()) {
2211 final TableName tableName
= ProtobufUtil
.toTableName(request
.getTableName());
2212 npr
= new NonceProcedureRunnable(server
, request
.getNonceGroup(), request
.getNonce()) {
2214 protected void run() throws IOException
{
2215 setProcId(server
.getLockManager().remoteLocks().requestTableLock(tableName
, type
,
2216 request
.getDescription(), getNonceKey()));
2220 protected String
getDescription() {
2221 return "RequestLock";
2224 } else if (request
.hasNamespace()) {
2225 npr
= new NonceProcedureRunnable(server
, request
.getNonceGroup(), request
.getNonce()) {
2227 protected void run() throws IOException
{
2228 setProcId(server
.getLockManager().remoteLocks().requestNamespaceLock(
2229 request
.getNamespace(), type
, request
.getDescription(), getNonceKey()));
2233 protected String
getDescription() {
2234 return "RequestLock";
2238 throw new IllegalArgumentException("one of table/namespace/region should be specified");
2240 long procId
= MasterProcedureUtil
.submitProcedure(npr
);
2241 return LockResponse
.newBuilder().setProcId(procId
).build();
2242 } catch (IllegalArgumentException e
) {
2243 LOG
.warn("Exception when queuing lock", e
);
2244 throw new ServiceException(new DoNotRetryIOException(e
));
2245 } catch (IOException e
) {
2246 LOG
.warn("Exception when queuing lock", e
);
2247 throw new ServiceException(e
);
2252 * @return LOCKED, if procedure is found and it has the lock; else UNLOCKED.
2253 * @throws ServiceException if given proc id is found but it is not a LockProcedure.
2256 public LockHeartbeatResponse
lockHeartbeat(RpcController controller
, LockHeartbeatRequest request
)
2257 throws ServiceException
{
2259 if (server
.getLockManager().remoteLocks().lockHeartbeat(request
.getProcId(),
2260 request
.getKeepAlive())) {
2261 return LockHeartbeatResponse
.newBuilder().setTimeoutMs(
2262 server
.getConfiguration().getInt(LockProcedure
.REMOTE_LOCKS_TIMEOUT_MS_CONF
,
2263 LockProcedure
.DEFAULT_REMOTE_LOCKS_TIMEOUT_MS
))
2264 .setLockStatus(LockHeartbeatResponse
.LockStatus
.LOCKED
).build();
2266 return LockHeartbeatResponse
.newBuilder()
2267 .setLockStatus(LockHeartbeatResponse
.LockStatus
.UNLOCKED
).build();
2269 } catch (IOException e
) {
2270 throw new ServiceException(e
);
2275 public RegionSpaceUseReportResponse
reportRegionSpaceUse(RpcController controller
,
2276 RegionSpaceUseReportRequest request
) throws ServiceException
{
2278 server
.checkInitialized();
2279 if (!QuotaUtil
.isQuotaEnabled(server
.getConfiguration())) {
2280 return RegionSpaceUseReportResponse
.newBuilder().build();
2282 MasterQuotaManager quotaManager
= this.server
.getMasterQuotaManager();
2283 if (quotaManager
!= null) {
2284 final long now
= EnvironmentEdgeManager
.currentTime();
2285 for (RegionSpaceUse report
: request
.getSpaceUseList()) {
2286 quotaManager
.addRegionSize(ProtobufUtil
.toRegionInfo(report
.getRegionInfo()),
2287 report
.getRegionSize(), now
);
2290 LOG
.debug("Received region space usage report but HMaster is not ready to process it, "
2293 return RegionSpaceUseReportResponse
.newBuilder().build();
2294 } catch (Exception e
) {
2295 throw new ServiceException(e
);
2300 public GetSpaceQuotaRegionSizesResponse
getSpaceQuotaRegionSizes(
2301 RpcController controller
, GetSpaceQuotaRegionSizesRequest request
) throws ServiceException
{
2303 server
.checkInitialized();
2304 MasterQuotaManager quotaManager
= this.server
.getMasterQuotaManager();
2305 GetSpaceQuotaRegionSizesResponse
.Builder builder
=
2306 GetSpaceQuotaRegionSizesResponse
.newBuilder();
2307 if (quotaManager
!= null) {
2308 Map
<RegionInfo
,Long
> regionSizes
= quotaManager
.snapshotRegionSizes();
2309 Map
<TableName
,Long
> regionSizesByTable
= new HashMap
<>();
2310 // Translate hregioninfo+long -> tablename+long
2311 for (Entry
<RegionInfo
,Long
> entry
: regionSizes
.entrySet()) {
2312 final TableName tableName
= entry
.getKey().getTable();
2313 Long prevSize
= regionSizesByTable
.get(tableName
);
2314 if (prevSize
== null) {
2317 regionSizesByTable
.put(tableName
, prevSize
+ entry
.getValue());
2319 // Serialize them into the protobuf
2320 for (Entry
<TableName
,Long
> tableSize
: regionSizesByTable
.entrySet()) {
2321 builder
.addSizes(RegionSizes
.newBuilder()
2322 .setTableName(ProtobufUtil
.toProtoTableName(tableSize
.getKey()))
2323 .setSize(tableSize
.getValue()).build());
2325 return builder
.build();
2327 LOG
.debug("Received space quota region size report but HMaster is not ready to process it,"
2330 return builder
.build();
2331 } catch (Exception e
) {
2332 throw new ServiceException(e
);
2337 public GetQuotaStatesResponse
getQuotaStates(
2338 RpcController controller
, GetQuotaStatesRequest request
) throws ServiceException
{
2340 server
.checkInitialized();
2341 QuotaObserverChore quotaChore
= this.server
.getQuotaObserverChore();
2342 GetQuotaStatesResponse
.Builder builder
= GetQuotaStatesResponse
.newBuilder();
2343 if (quotaChore
!= null) {
2344 // The "current" view of all tables with quotas
2345 Map
<TableName
, SpaceQuotaSnapshot
> tableSnapshots
= quotaChore
.getTableQuotaSnapshots();
2346 for (Entry
<TableName
, SpaceQuotaSnapshot
> entry
: tableSnapshots
.entrySet()) {
2347 builder
.addTableSnapshots(
2348 TableQuotaSnapshot
.newBuilder()
2349 .setTableName(ProtobufUtil
.toProtoTableName(entry
.getKey()))
2350 .setSnapshot(SpaceQuotaSnapshot
.toProtoSnapshot(entry
.getValue())).build());
2352 // The "current" view of all namespaces with quotas
2353 Map
<String
, SpaceQuotaSnapshot
> nsSnapshots
= quotaChore
.getNamespaceQuotaSnapshots();
2354 for (Entry
<String
, SpaceQuotaSnapshot
> entry
: nsSnapshots
.entrySet()) {
2355 builder
.addNsSnapshots(
2356 NamespaceQuotaSnapshot
.newBuilder()
2357 .setNamespace(entry
.getKey())
2358 .setSnapshot(SpaceQuotaSnapshot
.toProtoSnapshot(entry
.getValue())).build());
2360 return builder
.build();
2362 return builder
.build();
2363 } catch (Exception e
) {
2364 throw new ServiceException(e
);
2369 public ClearDeadServersResponse
clearDeadServers(RpcController controller
,
2370 ClearDeadServersRequest request
) throws ServiceException
{
2371 LOG
.debug(server
.getClientIdAuditPrefix() + " clear dead region servers.");
2372 ClearDeadServersResponse
.Builder response
= ClearDeadServersResponse
.newBuilder();
2374 server
.checkInitialized();
2375 if (server
.cpHost
!= null) {
2376 server
.cpHost
.preClearDeadServers();
2379 if (server
.getServerManager().areDeadServersInProgress()) {
2380 LOG
.debug("Some dead server is still under processing, won't clear the dead server list");
2381 response
.addAllServerName(request
.getServerNameList());
2383 DeadServer deadServer
= server
.getServerManager().getDeadServers();
2384 Set
<Address
> clearedServers
= new HashSet
<>();
2385 for (HBaseProtos
.ServerName pbServer
: request
.getServerNameList()) {
2386 ServerName serverName
= ProtobufUtil
.toServerName(pbServer
);
2387 final boolean deadInProcess
= server
.getProcedures().stream().anyMatch(
2388 p
-> (p
instanceof ServerCrashProcedure
)
2389 && ((ServerCrashProcedure
) p
).getServerName().equals(serverName
));
2390 if (deadInProcess
) {
2391 throw new ServiceException(
2392 String
.format("Dead server '%s' is not 'dead' in fact...", serverName
));
2395 if (!deadServer
.removeDeadServer(serverName
)) {
2396 response
.addServerName(pbServer
);
2398 clearedServers
.add(serverName
.getAddress());
2401 server
.getRSGroupInfoManager().removeServers(clearedServers
);
2402 LOG
.info("Remove decommissioned servers {} from RSGroup done", clearedServers
);
2405 if (server
.cpHost
!= null) {
2406 server
.cpHost
.postClearDeadServers(
2407 ProtobufUtil
.toServerNameList(request
.getServerNameList()),
2408 ProtobufUtil
.toServerNameList(response
.getServerNameList()));
2410 } catch (IOException io
) {
2411 throw new ServiceException(io
);
2413 return response
.build();
2417 public ReportProcedureDoneResponse
reportProcedureDone(RpcController controller
,
2418 ReportProcedureDoneRequest request
) throws ServiceException
{
2419 // Check Masters is up and ready for duty before progressing. Remote side will keep trying.
2421 this.server
.checkServiceStarted();
2422 } catch (ServerNotRunningYetException snrye
) {
2423 throw new ServiceException(snrye
);
2425 request
.getResultList().forEach(result
-> {
2426 if (result
.getStatus() == RemoteProcedureResult
.Status
.SUCCESS
) {
2427 server
.remoteProcedureCompleted(result
.getProcId());
2429 server
.remoteProcedureFailed(result
.getProcId(),
2430 RemoteProcedureException
.fromProto(result
.getError()));
2433 return ReportProcedureDoneResponse
.getDefaultInstance();
2437 public FileArchiveNotificationResponse
reportFileArchival(RpcController controller
,
2438 FileArchiveNotificationRequest request
) throws ServiceException
{
2440 server
.checkInitialized();
2441 if (!QuotaUtil
.isQuotaEnabled(server
.getConfiguration())) {
2442 return FileArchiveNotificationResponse
.newBuilder().build();
2444 server
.getMasterQuotaManager().processFileArchivals(request
, server
.getConnection(),
2445 server
.getConfiguration(), server
.getFileSystem());
2446 return FileArchiveNotificationResponse
.newBuilder().build();
2447 } catch (Exception e
) {
2448 throw new ServiceException(e
);
2455 public RunHbckChoreResponse
runHbckChore(RpcController c
, RunHbckChoreRequest req
)
2456 throws ServiceException
{
2457 rpcPreCheck("runHbckChore");
2458 LOG
.info("{} request HBCK chore to run", server
.getClientIdAuditPrefix());
2459 HbckChore hbckChore
= server
.getHbckChore();
2460 boolean ran
= hbckChore
.runChore();
2461 return RunHbckChoreResponse
.newBuilder().setRan(ran
).build();
2465 * Update state of the table in meta only. This is required by hbck in some situations to cleanup
2466 * stuck assign/ unassign regions procedures for the table.
2468 * @return previous state of the table
2471 public GetTableStateResponse
setTableStateInMeta(RpcController controller
,
2472 SetTableStateInMetaRequest request
) throws ServiceException
{
2473 rpcPreCheck("setTableStateInMeta");
2474 TableName tn
= ProtobufUtil
.toTableName(request
.getTableName());
2476 TableState prevState
= this.server
.getTableStateManager().getTableState(tn
);
2477 TableState newState
= TableState
.convert(tn
, request
.getTableState());
2478 LOG
.info("{} set table={} state from {} to {}", server
.getClientIdAuditPrefix(),
2479 tn
, prevState
.getState(), newState
.getState());
2480 this.server
.getTableStateManager().setTableState(tn
, newState
.getState());
2481 return GetTableStateResponse
.newBuilder().setTableState(prevState
.convert()).build();
2482 } catch (Exception e
) {
2483 throw new ServiceException(e
);
2488 * Update state of the region in meta only. This is required by hbck in some situations to cleanup
2489 * stuck assign/ unassign regions procedures for the table.
2491 * @return previous states of the regions
2494 public SetRegionStateInMetaResponse
setRegionStateInMeta(RpcController controller
,
2495 SetRegionStateInMetaRequest request
) throws ServiceException
{
2496 rpcPreCheck("setRegionStateInMeta");
2497 SetRegionStateInMetaResponse
.Builder builder
= SetRegionStateInMetaResponse
.newBuilder();
2499 for (RegionSpecifierAndState s
: request
.getStatesList()) {
2500 RegionSpecifier spec
= s
.getRegionSpecifier();
2502 if (spec
.getType() == RegionSpecifierType
.ENCODED_REGION_NAME
) {
2503 encodedName
= spec
.getValue().toStringUtf8();
2505 // TODO: actually, a full region name can save a lot on meta scan, improve later.
2506 encodedName
= RegionInfo
.encodeRegionName(spec
.getValue().toByteArray());
2508 RegionInfo info
= this.server
.getAssignmentManager().loadRegionFromMeta(encodedName
);
2509 LOG
.trace("region info loaded from meta table: {}", info
);
2510 RegionState prevState
=
2511 this.server
.getAssignmentManager().getRegionStates().getRegionState(info
);
2512 RegionState
.State newState
= RegionState
.State
.convert(s
.getState());
2513 LOG
.info("{} set region={} state from {} to {}", server
.getClientIdAuditPrefix(), info
,
2514 prevState
.getState(), newState
);
2515 Put metaPut
= MetaTableAccessor
.makePutFromRegionInfo(info
,
2516 EnvironmentEdgeManager
.currentTime());
2517 metaPut
.addColumn(HConstants
.CATALOG_FAMILY
, HConstants
.STATE_QUALIFIER
,
2518 Bytes
.toBytes(newState
.name()));
2519 List
<Put
> putList
= new ArrayList
<>();
2520 putList
.add(metaPut
);
2521 MetaTableAccessor
.putsToMetaTable(this.server
.getConnection(), putList
);
2522 // Loads from meta again to refresh AM cache with the new region state
2523 this.server
.getAssignmentManager().loadRegionFromMeta(encodedName
);
2524 builder
.addStates(RegionSpecifierAndState
.newBuilder().setRegionSpecifier(spec
)
2525 .setState(prevState
.getState().convert()));
2527 } catch (Exception e
) {
2528 throw new ServiceException(e
);
2530 return builder
.build();
2534 * Get RegionInfo from Master using content of RegionSpecifier as key.
2535 * @return RegionInfo found by decoding <code>rs</code> or null if none found
2537 private RegionInfo
getRegionInfo(HBaseProtos
.RegionSpecifier rs
) throws UnknownRegionException
{
2538 RegionInfo ri
= null;
2539 switch(rs
.getType()) {
2541 final byte[] regionName
= rs
.getValue().toByteArray();
2542 ri
= this.server
.getAssignmentManager().getRegionInfo(regionName
);
2544 case ENCODED_REGION_NAME
:
2545 String encodedRegionName
= Bytes
.toString(rs
.getValue().toByteArray());
2546 RegionState regionState
= this.server
.getAssignmentManager().getRegionStates().
2547 getRegionState(encodedRegionName
);
2548 ri
= regionState
== null ?
2549 this.server
.getAssignmentManager().loadRegionFromMeta(encodedRegionName
) :
2550 regionState
.getRegion();
2559 * @throws ServiceException If no MasterProcedureExecutor
2561 private void checkMasterProcedureExecutor() throws ServiceException
{
2562 if (this.server
.getMasterProcedureExecutor() == null) {
2563 throw new ServiceException("Master's ProcedureExecutor not initialized; retry later");
2568 * A 'raw' version of assign that does bulk and can skirt Master state checks if override
2569 * is set; i.e. assigns can be forced during Master startup or if RegionState is unclean.
2573 public MasterProtos
.AssignsResponse
assigns(RpcController controller
,
2574 MasterProtos
.AssignsRequest request
) throws ServiceException
{
2575 checkMasterProcedureExecutor();
2576 MasterProtos
.AssignsResponse
.Builder responseBuilder
=
2577 MasterProtos
.AssignsResponse
.newBuilder();
2579 boolean override
= request
.getOverride();
2580 LOG
.info("{} assigns, override={}", server
.getClientIdAuditPrefix(), override
);
2581 for (HBaseProtos
.RegionSpecifier rs
: request
.getRegionList()) {
2582 long pid
= Procedure
.NO_PROC_ID
;
2583 RegionInfo ri
= getRegionInfo(rs
);
2585 LOG
.info("Unknown={}", rs
);
2587 Procedure p
= this.server
.getAssignmentManager().createOneAssignProcedure(ri
, override
);
2589 pid
= this.server
.getMasterProcedureExecutor().submitProcedure(p
);
2592 responseBuilder
.addPid(pid
);
2594 return responseBuilder
.build();
2595 } catch (IOException ioe
) {
2596 throw new ServiceException(ioe
);
2601 * A 'raw' version of unassign that does bulk and can skirt Master state checks if override
2602 * is set; i.e. unassigns can be forced during Master startup or if RegionState is unclean.
2606 public MasterProtos
.UnassignsResponse
unassigns(RpcController controller
,
2607 MasterProtos
.UnassignsRequest request
) throws ServiceException
{
2608 checkMasterProcedureExecutor();
2609 MasterProtos
.UnassignsResponse
.Builder responseBuilder
=
2610 MasterProtos
.UnassignsResponse
.newBuilder();
2612 boolean override
= request
.getOverride();
2613 LOG
.info("{} unassigns, override={}", server
.getClientIdAuditPrefix(), override
);
2614 for (HBaseProtos
.RegionSpecifier rs
: request
.getRegionList()) {
2615 long pid
= Procedure
.NO_PROC_ID
;
2616 RegionInfo ri
= getRegionInfo(rs
);
2618 LOG
.info("Unknown={}", rs
);
2620 Procedure p
= this.server
.getAssignmentManager().createOneUnassignProcedure(ri
, override
);
2622 pid
= this.server
.getMasterProcedureExecutor().submitProcedure(p
);
2625 responseBuilder
.addPid(pid
);
2627 return responseBuilder
.build();
2628 } catch (IOException ioe
) {
2629 throw new ServiceException(ioe
);
2634 * Bypass specified procedure to completion. Procedure is marked completed but no actual work
2635 * is done from the current state/ step onwards. Parents of the procedure are also marked for
2638 * NOTE: this is a dangerous operation and may be used to unstuck buggy procedures. This may
2639 * leave system in inconherent state. This may need to be followed by some cleanup steps/
2640 * actions by operator.
2642 * @return BypassProcedureToCompletionResponse indicating success or failure
2645 public MasterProtos
.BypassProcedureResponse
bypassProcedure(RpcController controller
,
2646 MasterProtos
.BypassProcedureRequest request
) throws ServiceException
{
2648 LOG
.info("{} bypass procedures={}, waitTime={}, override={}, recursive={}",
2649 server
.getClientIdAuditPrefix(), request
.getProcIdList(), request
.getWaitTime(),
2650 request
.getOverride(), request
.getRecursive());
2652 server
.getMasterProcedureExecutor().bypassProcedure(request
.getProcIdList(),
2653 request
.getWaitTime(), request
.getOverride(), request
.getRecursive());
2654 return MasterProtos
.BypassProcedureResponse
.newBuilder().addAllBypassed(ret
).build();
2655 } catch (IOException e
) {
2656 throw new ServiceException(e
);
2661 public MasterProtos
.ScheduleServerCrashProcedureResponse
scheduleServerCrashProcedure(
2662 RpcController controller
, MasterProtos
.ScheduleServerCrashProcedureRequest request
)
2663 throws ServiceException
{
2664 List
<Long
> pids
= new ArrayList
<>();
2665 for (HBaseProtos
.ServerName sn
: request
.getServerNameList()) {
2666 ServerName serverName
= ProtobufUtil
.toServerName(sn
);
2667 LOG
.info("{} schedule ServerCrashProcedure for {}",
2668 this.server
.getClientIdAuditPrefix(), serverName
);
2669 if (shouldSubmitSCP(serverName
)) {
2670 pids
.add(this.server
.getServerManager().expireServer(serverName
, true));
2672 pids
.add(Procedure
.NO_PROC_ID
);
2675 return MasterProtos
.ScheduleServerCrashProcedureResponse
.newBuilder().addAllPid(pids
).build();
2679 public MasterProtos
.ScheduleSCPsForUnknownServersResponse
scheduleSCPsForUnknownServers(
2680 RpcController controller
, MasterProtos
.ScheduleSCPsForUnknownServersRequest request
)
2681 throws ServiceException
{
2682 List
<Long
> pids
= new ArrayList
<>();
2683 final Set
<ServerName
> serverNames
=
2684 server
.getAssignmentManager().getRegionStates().getRegionStates().stream()
2685 .map(RegionState
::getServerName
).collect(Collectors
.toSet());
2687 final Set
<ServerName
> unknownServerNames
= serverNames
.stream()
2688 .filter(sn
-> server
.getServerManager().isServerUnknown(sn
)).collect(Collectors
.toSet());
2690 for (ServerName sn
: unknownServerNames
) {
2691 LOG
.info("{} schedule ServerCrashProcedure for unknown {}",
2692 this.server
.getClientIdAuditPrefix(), sn
);
2693 if (shouldSubmitSCP(sn
)) {
2694 pids
.add(this.server
.getServerManager().expireServer(sn
, true));
2696 pids
.add(Procedure
.NO_PROC_ID
);
2699 return MasterProtos
.ScheduleSCPsForUnknownServersResponse
.newBuilder().addAllPid(pids
).build();
2703 public FixMetaResponse
fixMeta(RpcController controller
, FixMetaRequest request
)
2704 throws ServiceException
{
2705 rpcPreCheck("fixMeta");
2707 MetaFixer mf
= new MetaFixer(this.server
);
2709 return FixMetaResponse
.newBuilder().build();
2710 } catch (IOException ioe
) {
2711 throw new ServiceException(ioe
);
2716 public SwitchRpcThrottleResponse
switchRpcThrottle(RpcController controller
,
2717 SwitchRpcThrottleRequest request
) throws ServiceException
{
2719 server
.checkInitialized();
2720 return server
.getMasterQuotaManager().switchRpcThrottle(request
);
2721 } catch (Exception e
) {
2722 throw new ServiceException(e
);
2727 public MasterProtos
.IsRpcThrottleEnabledResponse
isRpcThrottleEnabled(RpcController controller
,
2728 MasterProtos
.IsRpcThrottleEnabledRequest request
) throws ServiceException
{
2730 server
.checkInitialized();
2731 return server
.getMasterQuotaManager().isRpcThrottleEnabled(request
);
2732 } catch (Exception e
) {
2733 throw new ServiceException(e
);
2738 public SwitchExceedThrottleQuotaResponse
switchExceedThrottleQuota(RpcController controller
,
2739 SwitchExceedThrottleQuotaRequest request
) throws ServiceException
{
2741 server
.checkInitialized();
2742 return server
.getMasterQuotaManager().switchExceedThrottleQuota(request
);
2743 } catch (Exception e
) {
2744 throw new ServiceException(e
);
2749 public GrantResponse
grant(RpcController controller
, GrantRequest request
)
2750 throws ServiceException
{
2752 server
.checkInitialized();
2753 if (server
.cpHost
!= null && hasAccessControlServiceCoprocessor(server
.cpHost
)) {
2754 final UserPermission perm
=
2755 ShadedAccessControlUtil
.toUserPermission(request
.getUserPermission());
2756 boolean mergeExistingPermissions
= request
.getMergeExistingPermissions();
2757 server
.cpHost
.preGrant(perm
, mergeExistingPermissions
);
2758 try (Table table
= server
.getConnection().getTable(PermissionStorage
.ACL_TABLE_NAME
)) {
2759 PermissionStorage
.addUserPermission(getConfiguration(), perm
, table
,
2760 mergeExistingPermissions
);
2762 server
.cpHost
.postGrant(perm
, mergeExistingPermissions
);
2763 User caller
= RpcServer
.getRequestUser().orElse(null);
2764 if (AUDITLOG
.isTraceEnabled()) {
2765 // audit log should store permission changes in addition to auth results
2766 String remoteAddress
= RpcServer
.getRemoteAddress().map(InetAddress
::toString
).orElse("");
2767 AUDITLOG
.trace("User {} (remote address: {}) granted permission {}", caller
,
2768 remoteAddress
, perm
);
2770 return GrantResponse
.getDefaultInstance();
2772 throw new DoNotRetryIOException(
2773 new UnsupportedOperationException(AccessController
.class.getName() + " is not loaded"));
2775 } catch (IOException ioe
) {
2776 throw new ServiceException(ioe
);
2781 public RevokeResponse
revoke(RpcController controller
, RevokeRequest request
)
2782 throws ServiceException
{
2784 server
.checkInitialized();
2785 if (server
.cpHost
!= null && hasAccessControlServiceCoprocessor(server
.cpHost
)) {
2786 final UserPermission userPermission
=
2787 ShadedAccessControlUtil
.toUserPermission(request
.getUserPermission());
2788 server
.cpHost
.preRevoke(userPermission
);
2789 try (Table table
= server
.getConnection().getTable(PermissionStorage
.ACL_TABLE_NAME
)) {
2790 PermissionStorage
.removeUserPermission(server
.getConfiguration(), userPermission
, table
);
2792 server
.cpHost
.postRevoke(userPermission
);
2793 User caller
= RpcServer
.getRequestUser().orElse(null);
2794 if (AUDITLOG
.isTraceEnabled()) {
2795 // audit log should record all permission changes
2796 String remoteAddress
= RpcServer
.getRemoteAddress().map(InetAddress
::toString
).orElse("");
2797 AUDITLOG
.trace("User {} (remote address: {}) revoked permission {}", caller
,
2798 remoteAddress
, userPermission
);
2800 return RevokeResponse
.getDefaultInstance();
2802 throw new DoNotRetryIOException(
2803 new UnsupportedOperationException(AccessController
.class.getName() + " is not loaded"));
2805 } catch (IOException ioe
) {
2806 throw new ServiceException(ioe
);
2811 public GetUserPermissionsResponse
getUserPermissions(RpcController controller
,
2812 GetUserPermissionsRequest request
) throws ServiceException
{
2814 server
.checkInitialized();
2815 if (server
.cpHost
!= null && hasAccessControlServiceCoprocessor(server
.cpHost
)) {
2816 final String userName
= request
.hasUserName() ? request
.getUserName().toStringUtf8() : null;
2818 request
.hasNamespaceName() ? request
.getNamespaceName().toStringUtf8() : null;
2820 request
.hasTableName() ? ProtobufUtil
.toTableName(request
.getTableName()) : null;
2821 byte[] cf
= request
.hasColumnFamily() ? request
.getColumnFamily().toByteArray() : null;
2823 request
.hasColumnQualifier() ? request
.getColumnQualifier().toByteArray() : null;
2824 Type permissionType
= request
.hasType() ? request
.getType() : null;
2825 server
.getMasterCoprocessorHost().preGetUserPermissions(userName
, namespace
, table
, cf
, cq
);
2827 List
<UserPermission
> perms
= null;
2828 if (permissionType
== Type
.Table
) {
2829 boolean filter
= (cf
!= null || userName
!= null) ?
true : false;
2830 perms
= PermissionStorage
.getUserTablePermissions(server
.getConfiguration(), table
, cf
,
2831 cq
, userName
, filter
);
2832 } else if (permissionType
== Type
.Namespace
) {
2833 perms
= PermissionStorage
.getUserNamespacePermissions(server
.getConfiguration(),
2834 namespace
, userName
, userName
!= null ?
true : false);
2836 perms
= PermissionStorage
.getUserPermissions(server
.getConfiguration(), null, null, null,
2837 userName
, userName
!= null ?
true : false);
2838 // Skip super users when filter user is specified
2839 if (userName
== null) {
2840 // Adding superusers explicitly to the result set as PermissionStorage do not store
2841 // them. Also using acl as table name to be inline with the results of global admin and
2842 // will help in avoiding any leakage of information about being superusers.
2843 for (String user
: Superusers
.getSuperUsers()) {
2844 perms
.add(new UserPermission(user
,
2845 Permission
.newBuilder().withActions(Action
.values()).build()));
2850 server
.getMasterCoprocessorHost().postGetUserPermissions(userName
, namespace
, table
, cf
,
2852 AccessControlProtos
.GetUserPermissionsResponse response
=
2853 ShadedAccessControlUtil
.buildGetUserPermissionsResponse(perms
);
2856 throw new DoNotRetryIOException(
2857 new UnsupportedOperationException(AccessController
.class.getName() + " is not loaded"));
2859 } catch (IOException ioe
) {
2860 throw new ServiceException(ioe
);
2865 public HasUserPermissionsResponse
hasUserPermissions(RpcController controller
,
2866 HasUserPermissionsRequest request
) throws ServiceException
{
2868 server
.checkInitialized();
2869 if (server
.cpHost
!= null && hasAccessControlServiceCoprocessor(server
.cpHost
)) {
2870 User caller
= RpcServer
.getRequestUser().orElse(null);
2872 request
.hasUserName() ? request
.getUserName().toStringUtf8() : caller
.getShortName();
2873 List
<Permission
> permissions
= new ArrayList
<>();
2874 for (int i
= 0; i
< request
.getPermissionCount(); i
++) {
2875 permissions
.add(ShadedAccessControlUtil
.toPermission(request
.getPermission(i
)));
2877 server
.getMasterCoprocessorHost().preHasUserPermissions(userName
, permissions
);
2878 if (!caller
.getShortName().equals(userName
)) {
2879 List
<String
> groups
= AccessChecker
.getUserGroups(userName
);
2880 caller
= new InputUser(userName
, groups
.toArray(new String
[groups
.size()]));
2882 List
<Boolean
> hasUserPermissions
= new ArrayList
<>();
2883 if (getAccessChecker() != null) {
2884 for (Permission permission
: permissions
) {
2885 boolean hasUserPermission
=
2886 getAccessChecker().hasUserPermission(caller
, "hasUserPermissions", permission
);
2887 hasUserPermissions
.add(hasUserPermission
);
2890 for (int i
= 0; i
< permissions
.size(); i
++) {
2891 hasUserPermissions
.add(true);
2894 server
.getMasterCoprocessorHost().postHasUserPermissions(userName
, permissions
);
2895 HasUserPermissionsResponse
.Builder builder
=
2896 HasUserPermissionsResponse
.newBuilder().addAllHasUserPermission(hasUserPermissions
);
2897 return builder
.build();
2899 throw new DoNotRetryIOException(
2900 new UnsupportedOperationException(AccessController
.class.getName() + " is not loaded"));
2902 } catch (IOException ioe
) {
2903 throw new ServiceException(ioe
);
2907 private boolean shouldSubmitSCP(ServerName serverName
) {
2908 // check if there is already a SCP of this server running
2909 List
<Procedure
<MasterProcedureEnv
>> procedures
=
2910 server
.getMasterProcedureExecutor().getProcedures();
2911 for (Procedure
<MasterProcedureEnv
> procedure
: procedures
) {
2912 if (procedure
instanceof ServerCrashProcedure
) {
2913 if (serverName
.compareTo(((ServerCrashProcedure
) procedure
).getServerName()) == 0
2914 && !procedure
.isFinished()) {
2915 LOG
.info("there is already a SCP of this server {} running, pid {}", serverName
,
2916 procedure
.getProcId());
2925 public GetRSGroupInfoResponse
getRSGroupInfo(RpcController controller
,
2926 GetRSGroupInfoRequest request
) throws ServiceException
{
2927 String groupName
= request
.getRSGroupName();
2929 server
.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, group=" + groupName
);
2931 if (server
.getMasterCoprocessorHost() != null) {
2932 server
.getMasterCoprocessorHost().preGetRSGroupInfo(groupName
);
2934 RSGroupInfo rsGroupInfo
= server
.getRSGroupInfoManager().getRSGroup(groupName
);
2935 GetRSGroupInfoResponse resp
;
2936 if (rsGroupInfo
!= null) {
2937 resp
= GetRSGroupInfoResponse
.newBuilder()
2938 .setRSGroupInfo(ProtobufUtil
.toProtoGroupInfo(rsGroupInfo
)).build();
2940 resp
= GetRSGroupInfoResponse
.getDefaultInstance();
2942 if (server
.getMasterCoprocessorHost() != null) {
2943 server
.getMasterCoprocessorHost().postGetRSGroupInfo(groupName
);
2946 } catch (IOException e
) {
2947 throw new ServiceException(e
);
2952 public GetRSGroupInfoOfTableResponse
getRSGroupInfoOfTable(RpcController controller
,
2953 GetRSGroupInfoOfTableRequest request
) throws ServiceException
{
2954 TableName tableName
= ProtobufUtil
.toTableName(request
.getTableName());
2956 server
.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, table=" + tableName
);
2958 if (server
.getMasterCoprocessorHost() != null) {
2959 server
.getMasterCoprocessorHost().preGetRSGroupInfoOfTable(tableName
);
2961 GetRSGroupInfoOfTableResponse resp
;
2962 TableDescriptor td
= server
.getTableDescriptors().get(tableName
);
2964 resp
= GetRSGroupInfoOfTableResponse
.getDefaultInstance();
2966 RSGroupInfo rsGroupInfo
=
2967 RSGroupUtil
.getRSGroupInfo(server
, server
.getRSGroupInfoManager(), tableName
)
2968 .orElse(server
.getRSGroupInfoManager().getRSGroup(RSGroupInfo
.DEFAULT_GROUP
));
2969 resp
= GetRSGroupInfoOfTableResponse
.newBuilder()
2970 .setRSGroupInfo(ProtobufUtil
.toProtoGroupInfo(rsGroupInfo
)).build();
2972 if (server
.getMasterCoprocessorHost() != null) {
2973 server
.getMasterCoprocessorHost().postGetRSGroupInfoOfTable(tableName
);
2976 } catch (IOException e
) {
2977 throw new ServiceException(e
);
2982 public GetRSGroupInfoOfServerResponse
getRSGroupInfoOfServer(RpcController controller
,
2983 GetRSGroupInfoOfServerRequest request
) throws ServiceException
{
2985 Address
.fromParts(request
.getServer().getHostName(), request
.getServer().getPort());
2986 LOG
.info(server
.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server=" + hp
);
2988 if (server
.getMasterCoprocessorHost() != null) {
2989 server
.getMasterCoprocessorHost().preGetRSGroupInfoOfServer(hp
);
2991 RSGroupInfo rsGroupInfo
= server
.getRSGroupInfoManager().getRSGroupOfServer(hp
);
2992 GetRSGroupInfoOfServerResponse resp
;
2993 if (rsGroupInfo
!= null) {
2994 resp
= GetRSGroupInfoOfServerResponse
.newBuilder()
2995 .setRSGroupInfo(ProtobufUtil
.toProtoGroupInfo(rsGroupInfo
)).build();
2997 resp
= GetRSGroupInfoOfServerResponse
.getDefaultInstance();
2999 if (server
.getMasterCoprocessorHost() != null) {
3000 server
.getMasterCoprocessorHost().postGetRSGroupInfoOfServer(hp
);
3003 } catch (IOException e
) {
3004 throw new ServiceException(e
);
3009 public MoveServersResponse
moveServers(RpcController controller
, MoveServersRequest request
)
3010 throws ServiceException
{
3011 Set
<Address
> hostPorts
= Sets
.newHashSet();
3012 MoveServersResponse
.Builder builder
= MoveServersResponse
.newBuilder();
3013 for (HBaseProtos
.ServerName el
: request
.getServersList()) {
3014 hostPorts
.add(Address
.fromParts(el
.getHostName(), el
.getPort()));
3016 LOG
.info(server
.getClientIdAuditPrefix() + " move servers " + hostPorts
+ " to rsgroup " +
3017 request
.getTargetGroup());
3019 if (server
.getMasterCoprocessorHost() != null) {
3020 server
.getMasterCoprocessorHost().preMoveServers(hostPorts
, request
.getTargetGroup());
3022 server
.getRSGroupInfoManager().moveServers(hostPorts
, request
.getTargetGroup());
3023 if (server
.getMasterCoprocessorHost() != null) {
3024 server
.getMasterCoprocessorHost().postMoveServers(hostPorts
, request
.getTargetGroup());
3026 } catch (IOException e
) {
3027 throw new ServiceException(e
);
3029 return builder
.build();
3033 public AddRSGroupResponse
addRSGroup(RpcController controller
, AddRSGroupRequest request
)
3034 throws ServiceException
{
3035 AddRSGroupResponse
.Builder builder
= AddRSGroupResponse
.newBuilder();
3036 LOG
.info(server
.getClientIdAuditPrefix() + " add rsgroup " + request
.getRSGroupName());
3038 if (server
.getMasterCoprocessorHost() != null) {
3039 server
.getMasterCoprocessorHost().preAddRSGroup(request
.getRSGroupName());
3041 server
.getRSGroupInfoManager().addRSGroup(new RSGroupInfo(request
.getRSGroupName()));
3042 if (server
.getMasterCoprocessorHost() != null) {
3043 server
.getMasterCoprocessorHost().postAddRSGroup(request
.getRSGroupName());
3045 } catch (IOException e
) {
3046 throw new ServiceException(e
);
3048 return builder
.build();
3052 public RemoveRSGroupResponse
removeRSGroup(RpcController controller
, RemoveRSGroupRequest request
)
3053 throws ServiceException
{
3054 RemoveRSGroupResponse
.Builder builder
= RemoveRSGroupResponse
.newBuilder();
3055 LOG
.info(server
.getClientIdAuditPrefix() + " remove rsgroup " + request
.getRSGroupName());
3057 if (server
.getMasterCoprocessorHost() != null) {
3058 server
.getMasterCoprocessorHost().preRemoveRSGroup(request
.getRSGroupName());
3060 server
.getRSGroupInfoManager().removeRSGroup(request
.getRSGroupName());
3061 if (server
.getMasterCoprocessorHost() != null) {
3062 server
.getMasterCoprocessorHost().postRemoveRSGroup(request
.getRSGroupName());
3064 } catch (IOException e
) {
3065 throw new ServiceException(e
);
3067 return builder
.build();
3071 public BalanceRSGroupResponse
balanceRSGroup(RpcController controller
,
3072 BalanceRSGroupRequest request
) throws ServiceException
{
3073 BalanceRequest balanceRequest
= ProtobufUtil
.toBalanceRequest(request
);
3075 BalanceRSGroupResponse
.Builder builder
= BalanceRSGroupResponse
.newBuilder()
3076 .setBalanceRan(false);
3079 server
.getClientIdAuditPrefix() + " balance rsgroup, group=" + request
.getRSGroupName());
3081 if (server
.getMasterCoprocessorHost() != null) {
3082 server
.getMasterCoprocessorHost()
3083 .preBalanceRSGroup(request
.getRSGroupName(), balanceRequest
);
3085 BalanceResponse response
=
3086 server
.getRSGroupInfoManager().balanceRSGroup(request
.getRSGroupName(), balanceRequest
);
3087 ProtobufUtil
.populateBalanceRSGroupResponse(builder
, response
);
3088 if (server
.getMasterCoprocessorHost() != null) {
3089 server
.getMasterCoprocessorHost()
3090 .postBalanceRSGroup(request
.getRSGroupName(), balanceRequest
, response
);
3092 } catch (IOException e
) {
3093 throw new ServiceException(e
);
3095 return builder
.build();
3099 public ListRSGroupInfosResponse
listRSGroupInfos(RpcController controller
,
3100 ListRSGroupInfosRequest request
) throws ServiceException
{
3101 ListRSGroupInfosResponse
.Builder builder
= ListRSGroupInfosResponse
.newBuilder();
3102 LOG
.info(server
.getClientIdAuditPrefix() + " list rsgroup");
3104 if (server
.getMasterCoprocessorHost() != null) {
3105 server
.getMasterCoprocessorHost().preListRSGroups();
3107 List
<RSGroupInfo
> rsGroupInfos
= server
.getRSGroupInfoManager().listRSGroups().stream()
3108 .map(RSGroupInfo
::new).collect(Collectors
.toList());
3109 Map
<String
, RSGroupInfo
> name2Info
= new HashMap
<>();
3110 List
<TableDescriptor
> needToFill
=
3111 new ArrayList
<>(server
.getTableDescriptors().getAll().values());
3112 for (RSGroupInfo rsGroupInfo
: rsGroupInfos
) {
3113 name2Info
.put(rsGroupInfo
.getName(), rsGroupInfo
);
3114 for (TableDescriptor td
: server
.getTableDescriptors().getAll().values()) {
3115 if (rsGroupInfo
.containsTable(td
.getTableName())){
3116 needToFill
.remove(td
);
3120 for (TableDescriptor td
: needToFill
) {
3121 String groupName
= td
.getRegionServerGroup().orElse(RSGroupInfo
.DEFAULT_GROUP
);
3122 RSGroupInfo rsGroupInfo
= name2Info
.get(groupName
);
3123 if (rsGroupInfo
!= null) {
3124 rsGroupInfo
.addTable(td
.getTableName());
3127 for (RSGroupInfo rsGroupInfo
: rsGroupInfos
) {
3128 // TODO: this can be done at once outside this loop, do not need to scan all every time.
3129 builder
.addRSGroupInfo(ProtobufUtil
.toProtoGroupInfo(rsGroupInfo
));
3131 if (server
.getMasterCoprocessorHost() != null) {
3132 server
.getMasterCoprocessorHost().postListRSGroups();
3134 } catch (IOException e
) {
3135 throw new ServiceException(e
);
3137 return builder
.build();
3141 public RemoveServersResponse
removeServers(RpcController controller
,
3142 RemoveServersRequest request
) throws ServiceException
{
3143 RemoveServersResponse
.Builder builder
= RemoveServersResponse
.newBuilder();
3144 Set
<Address
> servers
= Sets
.newHashSet();
3145 for (HBaseProtos
.ServerName el
: request
.getServersList()) {
3146 servers
.add(Address
.fromParts(el
.getHostName(), el
.getPort()));
3148 LOG
.info(server
.getClientIdAuditPrefix() + " remove decommissioned servers from rsgroup: " +
3151 if (server
.getMasterCoprocessorHost() != null) {
3152 server
.getMasterCoprocessorHost().preRemoveServers(servers
);
3154 server
.getRSGroupInfoManager().removeServers(servers
);
3155 if (server
.getMasterCoprocessorHost() != null) {
3156 server
.getMasterCoprocessorHost().postRemoveServers(servers
);
3158 } catch (IOException e
) {
3159 throw new ServiceException(e
);
3161 return builder
.build();
3165 public ListTablesInRSGroupResponse
listTablesInRSGroup(RpcController controller
,
3166 ListTablesInRSGroupRequest request
) throws ServiceException
{
3167 ListTablesInRSGroupResponse
.Builder builder
= ListTablesInRSGroupResponse
.newBuilder();
3168 String groupName
= request
.getGroupName();
3169 LOG
.info(server
.getClientIdAuditPrefix() + " list tables in rsgroup " + groupName
);
3171 if (server
.getMasterCoprocessorHost() != null) {
3172 server
.getMasterCoprocessorHost().preListTablesInRSGroup(groupName
);
3174 RSGroupUtil
.listTablesInRSGroup(server
, groupName
).stream()
3175 .map(ProtobufUtil
::toProtoTableName
).forEach(builder
::addTableName
);
3176 if (server
.getMasterCoprocessorHost() != null) {
3177 server
.getMasterCoprocessorHost().postListTablesInRSGroup(groupName
);
3179 } catch (IOException e
) {
3180 throw new ServiceException(e
);
3182 return builder
.build();
3186 public GetConfiguredNamespacesAndTablesInRSGroupResponse
3187 getConfiguredNamespacesAndTablesInRSGroup(RpcController controller
,
3188 GetConfiguredNamespacesAndTablesInRSGroupRequest request
) throws ServiceException
{
3189 GetConfiguredNamespacesAndTablesInRSGroupResponse
.Builder builder
=
3190 GetConfiguredNamespacesAndTablesInRSGroupResponse
.newBuilder();
3191 String groupName
= request
.getGroupName();
3192 LOG
.info(server
.getClientIdAuditPrefix() + " get configured namespaces and tables in rsgroup " +
3195 if (server
.getMasterCoprocessorHost() != null) {
3196 server
.getMasterCoprocessorHost().preGetConfiguredNamespacesAndTablesInRSGroup(groupName
);
3198 for (NamespaceDescriptor nd
: server
.getClusterSchema().getNamespaces()) {
3199 if (groupName
.equals(nd
.getConfigurationValue(RSGroupInfo
.NAMESPACE_DESC_PROP_GROUP
))) {
3200 builder
.addNamespace(nd
.getName());
3203 for (TableDescriptor td
: server
.getTableDescriptors().getAll().values()) {
3204 if (td
.getRegionServerGroup().map(g
-> g
.equals(groupName
)).orElse(false)) {
3205 builder
.addTableName(ProtobufUtil
.toProtoTableName(td
.getTableName()));
3208 if (server
.getMasterCoprocessorHost() != null) {
3209 server
.getMasterCoprocessorHost().postGetConfiguredNamespacesAndTablesInRSGroup(groupName
);
3211 } catch (IOException e
) {
3212 throw new ServiceException(e
);
3214 return builder
.build();
3218 public RenameRSGroupResponse
renameRSGroup(RpcController controller
,
3219 RenameRSGroupRequest request
) throws ServiceException
{
3220 RenameRSGroupResponse
.Builder builder
= RenameRSGroupResponse
.newBuilder();
3221 String oldRSGroup
= request
.getOldRsgroupName();
3222 String newRSGroup
= request
.getNewRsgroupName();
3223 LOG
.info("{} rename rsgroup from {} to {} ",
3224 server
.getClientIdAuditPrefix(), oldRSGroup
, newRSGroup
);
3226 if (server
.getMasterCoprocessorHost() != null) {
3227 server
.getMasterCoprocessorHost().preRenameRSGroup(oldRSGroup
, newRSGroup
);
3229 server
.getRSGroupInfoManager().renameRSGroup(oldRSGroup
, newRSGroup
);
3230 if (server
.getMasterCoprocessorHost() != null) {
3231 server
.getMasterCoprocessorHost().postRenameRSGroup(oldRSGroup
, newRSGroup
);
3233 } catch (IOException e
) {
3234 throw new ServiceException(e
);
3236 return builder
.build();
3240 public UpdateRSGroupConfigResponse
updateRSGroupConfig(RpcController controller
,
3241 UpdateRSGroupConfigRequest request
)
3242 throws ServiceException
{
3243 UpdateRSGroupConfigResponse
.Builder builder
= UpdateRSGroupConfigResponse
.newBuilder();
3244 String groupName
= request
.getGroupName();
3245 Map
<String
, String
> configuration
= new HashMap
<>();
3246 request
.getConfigurationList().forEach(p
-> configuration
.put(p
.getName(), p
.getValue()));
3247 LOG
.info("{} update rsgroup {} configuration {}", server
.getClientIdAuditPrefix(), groupName
,
3250 if (server
.getMasterCoprocessorHost() != null) {
3251 server
.getMasterCoprocessorHost().preUpdateRSGroupConfig(groupName
, configuration
);
3253 server
.getRSGroupInfoManager().updateRSGroupConfig(groupName
, configuration
);
3254 if (server
.getMasterCoprocessorHost() != null) {
3255 server
.getMasterCoprocessorHost().postUpdateRSGroupConfig(groupName
, configuration
);
3257 } catch (IOException e
) {
3258 throw new ServiceException(e
);
3260 return builder
.build();
3264 public HBaseProtos
.LogEntry
getLogEntries(RpcController controller
,
3265 HBaseProtos
.LogRequest request
) throws ServiceException
{
3267 final String logClassName
= request
.getLogClassName();
3268 Class
<?
> logClass
= Class
.forName(logClassName
)
3269 .asSubclass(Message
.class);
3270 Method method
= logClass
.getMethod("parseFrom", ByteString
.class);
3271 if (logClassName
.contains("BalancerDecisionsRequest")) {
3272 MasterProtos
.BalancerDecisionsRequest balancerDecisionsRequest
=
3273 (MasterProtos
.BalancerDecisionsRequest
) method
3274 .invoke(null, request
.getLogMessage());
3275 MasterProtos
.BalancerDecisionsResponse balancerDecisionsResponse
=
3276 getBalancerDecisions(balancerDecisionsRequest
);
3277 return HBaseProtos
.LogEntry
.newBuilder()
3278 .setLogClassName(balancerDecisionsResponse
.getClass().getName())
3279 .setLogMessage(balancerDecisionsResponse
.toByteString())
3281 }else if (logClassName
.contains("BalancerRejectionsRequest")){
3282 MasterProtos
.BalancerRejectionsRequest balancerRejectionsRequest
=
3283 (MasterProtos
.BalancerRejectionsRequest
) method
3284 .invoke(null, request
.getLogMessage());
3285 MasterProtos
.BalancerRejectionsResponse balancerRejectionsResponse
=
3286 getBalancerRejections(balancerRejectionsRequest
);
3287 return HBaseProtos
.LogEntry
.newBuilder()
3288 .setLogClassName(balancerRejectionsResponse
.getClass().getName())
3289 .setLogMessage(balancerRejectionsResponse
.toByteString())
3292 } catch (ClassNotFoundException
| NoSuchMethodException
| IllegalAccessException
3293 | InvocationTargetException e
) {
3294 LOG
.error("Error while retrieving log entries.", e
);
3295 throw new ServiceException(e
);
3297 throw new ServiceException("Invalid request params");
3300 private MasterProtos
.BalancerDecisionsResponse
3301 getBalancerDecisions(MasterProtos
.BalancerDecisionsRequest request
) {
3302 final NamedQueueRecorder namedQueueRecorder
= this.server
.getNamedQueueRecorder();
3303 if (namedQueueRecorder
== null) {
3304 return MasterProtos
.BalancerDecisionsResponse
.newBuilder()
3305 .addAllBalancerDecision(Collections
.emptyList()).build();
3307 final NamedQueueGetRequest namedQueueGetRequest
= new NamedQueueGetRequest();
3308 namedQueueGetRequest
.setNamedQueueEvent(BalancerDecisionDetails
.BALANCER_DECISION_EVENT
);
3309 namedQueueGetRequest
.setBalancerDecisionsRequest(request
);
3310 NamedQueueGetResponse namedQueueGetResponse
=
3311 namedQueueRecorder
.getNamedQueueRecords(namedQueueGetRequest
);
3312 List
<RecentLogs
.BalancerDecision
> balancerDecisions
= namedQueueGetResponse
!= null ?
3313 namedQueueGetResponse
.getBalancerDecisions() :
3314 Collections
.emptyList();
3315 return MasterProtos
.BalancerDecisionsResponse
.newBuilder()
3316 .addAllBalancerDecision(balancerDecisions
).build();
3319 private MasterProtos
.BalancerRejectionsResponse
getBalancerRejections(
3320 MasterProtos
.BalancerRejectionsRequest request
) {
3321 final NamedQueueRecorder namedQueueRecorder
= this.server
.getNamedQueueRecorder();
3322 if (namedQueueRecorder
== null) {
3323 return MasterProtos
.BalancerRejectionsResponse
.newBuilder()
3324 .addAllBalancerRejection(Collections
.emptyList()).build();
3326 final NamedQueueGetRequest namedQueueGetRequest
= new NamedQueueGetRequest();
3327 namedQueueGetRequest
.setNamedQueueEvent(BalancerRejectionDetails
.BALANCER_REJECTION_EVENT
);
3328 namedQueueGetRequest
.setBalancerRejectionsRequest(request
);
3329 NamedQueueGetResponse namedQueueGetResponse
=
3330 namedQueueRecorder
.getNamedQueueRecords(namedQueueGetRequest
);
3331 List
<RecentLogs
.BalancerRejection
> balancerRejections
= namedQueueGetResponse
!= null ?
3332 namedQueueGetResponse
.getBalancerRejections() :
3333 Collections
.emptyList();
3334 return MasterProtos
.BalancerRejectionsResponse
.newBuilder()
3335 .addAllBalancerRejection(balancerRejections
).build();
3339 @QosPriority(priority
=HConstants
.ADMIN_QOS
)
3340 public GetRegionInfoResponse
getRegionInfo(final RpcController controller
,
3341 final GetRegionInfoRequest request
) throws ServiceException
{
3342 RegionInfo ri
= null;
3344 ri
= getRegionInfo(request
.getRegion());
3345 } catch(UnknownRegionException ure
) {
3346 throw new ServiceException(ure
);
3348 GetRegionInfoResponse
.Builder builder
= GetRegionInfoResponse
.newBuilder();
3350 builder
.setRegionInfo(ProtobufUtil
.toRegionInfo(ri
));
3352 // Is it a MOB name? These work differently.
3353 byte [] regionName
= request
.getRegion().getValue().toByteArray();
3354 TableName tableName
= RegionInfo
.getTable(regionName
);
3355 if (MobUtils
.isMobRegionName(tableName
, regionName
)) {
3356 // a dummy region info contains the compaction state.
3357 RegionInfo mobRegionInfo
= MobUtils
.getMobRegionInfo(tableName
);
3358 builder
.setRegionInfo(ProtobufUtil
.toRegionInfo(mobRegionInfo
));
3359 if (request
.hasCompactionState() && request
.getCompactionState()) {
3360 builder
.setCompactionState(server
.getMobCompactionState(tableName
));
3363 // If unknown RegionInfo and not a MOB region, it is unknown.
3364 throw new ServiceException(new UnknownRegionException(Bytes
.toString(regionName
)));
3367 return builder
.build();
3371 public GetStoreFileResponse
getStoreFile(RpcController controller
, GetStoreFileRequest request
)
3372 throws ServiceException
{
3373 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3377 public GetOnlineRegionResponse
getOnlineRegion(RpcController controller
,
3378 GetOnlineRegionRequest request
) throws ServiceException
{
3379 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3383 public OpenRegionResponse
openRegion(RpcController controller
, OpenRegionRequest request
)
3384 throws ServiceException
{
3385 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3389 public WarmupRegionResponse
warmupRegion(RpcController controller
, WarmupRegionRequest request
)
3390 throws ServiceException
{
3391 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3395 public CloseRegionResponse
closeRegion(RpcController controller
, CloseRegionRequest request
)
3396 throws ServiceException
{
3397 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3401 public FlushRegionResponse
flushRegion(RpcController controller
, FlushRegionRequest request
)
3402 throws ServiceException
{
3403 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3407 public CompactionSwitchResponse
compactionSwitch(RpcController controller
,
3408 CompactionSwitchRequest request
) throws ServiceException
{
3409 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3413 public CompactRegionResponse
compactRegion(RpcController controller
, CompactRegionRequest request
)
3414 throws ServiceException
{
3415 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3419 public ReplicateWALEntryResponse
replicateWALEntry(RpcController controller
,
3420 ReplicateWALEntryRequest request
) throws ServiceException
{
3421 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3425 public ReplicateWALEntryResponse
replay(RpcController controller
,
3426 ReplicateWALEntryRequest request
) throws ServiceException
{
3427 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3431 public RollWALWriterResponse
rollWALWriter(RpcController controller
, RollWALWriterRequest request
)
3432 throws ServiceException
{
3433 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3437 public GetServerInfoResponse
getServerInfo(RpcController controller
, GetServerInfoRequest request
)
3438 throws ServiceException
{
3439 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3443 public StopServerResponse
stopServer(RpcController controller
, StopServerRequest request
)
3444 throws ServiceException
{
3445 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3449 public UpdateFavoredNodesResponse
updateFavoredNodes(RpcController controller
,
3450 UpdateFavoredNodesRequest request
) throws ServiceException
{
3451 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3455 public GetRegionLoadResponse
getRegionLoad(RpcController controller
, GetRegionLoadRequest request
)
3456 throws ServiceException
{
3457 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3461 public ClearCompactionQueuesResponse
clearCompactionQueues(RpcController controller
,
3462 ClearCompactionQueuesRequest request
) throws ServiceException
{
3463 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3467 public ClearRegionBlockCacheResponse
clearRegionBlockCache(RpcController controller
,
3468 ClearRegionBlockCacheRequest request
) throws ServiceException
{
3469 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3473 public GetSpaceQuotaSnapshotsResponse
getSpaceQuotaSnapshots(RpcController controller
,
3474 GetSpaceQuotaSnapshotsRequest request
) throws ServiceException
{
3475 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3479 public ExecuteProceduresResponse
executeProcedures(RpcController controller
,
3480 ExecuteProceduresRequest request
) throws ServiceException
{
3481 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
3485 public GetLiveRegionServersResponse
getLiveRegionServers(RpcController controller
,
3486 GetLiveRegionServersRequest request
) throws ServiceException
{
3487 List
<ServerName
> regionServers
= new ArrayList
<>(server
.getLiveRegionServers());
3488 Collections
.shuffle(regionServers
, ThreadLocalRandom
.current());
3489 GetLiveRegionServersResponse
.Builder builder
=
3490 GetLiveRegionServersResponse
.newBuilder().setTotal(regionServers
.size());
3491 regionServers
.stream().limit(request
.getCount()).map(ProtobufUtil
::toServerName
)
3492 .forEach(builder
::addServer
);
3493 return builder
.build();
3497 public ReplicateWALEntryResponse
replicateToReplica(RpcController controller
,
3498 ReplicateWALEntryRequest request
) throws ServiceException
{
3499 throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));