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 static org
.apache
.hadoop
.hbase
.HConstants
.DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK
;
21 import static org
.apache
.hadoop
.hbase
.HConstants
.HBASE_MASTER_LOGCLEANER_PLUGINS
;
22 import static org
.apache
.hadoop
.hbase
.HConstants
.HBASE_SPLIT_WAL_COORDINATED_BY_ZK
;
23 import static org
.apache
.hadoop
.hbase
.util
.DNS
.MASTER_HOSTNAME_KEY
;
25 import com
.google
.errorprone
.annotations
.RestrictedApi
;
26 import java
.io
.IOException
;
27 import java
.io
.InterruptedIOException
;
28 import java
.lang
.reflect
.Constructor
;
29 import java
.lang
.reflect
.InvocationTargetException
;
30 import java
.net
.InetAddress
;
31 import java
.net
.InetSocketAddress
;
32 import java
.net
.UnknownHostException
;
33 import java
.util
.ArrayList
;
34 import java
.util
.Arrays
;
35 import java
.util
.Collection
;
36 import java
.util
.Collections
;
37 import java
.util
.Comparator
;
38 import java
.util
.EnumSet
;
39 import java
.util
.HashMap
;
40 import java
.util
.Iterator
;
41 import java
.util
.LinkedList
;
42 import java
.util
.List
;
44 import java
.util
.Objects
;
45 import java
.util
.Optional
;
47 import java
.util
.concurrent
.ExecutionException
;
48 import java
.util
.concurrent
.Future
;
49 import java
.util
.concurrent
.TimeUnit
;
50 import java
.util
.concurrent
.TimeoutException
;
51 import java
.util
.concurrent
.atomic
.AtomicInteger
;
52 import java
.util
.regex
.Pattern
;
53 import java
.util
.stream
.Collectors
;
54 import javax
.servlet
.http
.HttpServlet
;
55 import org
.apache
.commons
.lang3
.StringUtils
;
56 import org
.apache
.hadoop
.conf
.Configuration
;
57 import org
.apache
.hadoop
.fs
.FSDataOutputStream
;
58 import org
.apache
.hadoop
.fs
.Path
;
59 import org
.apache
.hadoop
.hbase
.CatalogFamilyFormat
;
60 import org
.apache
.hadoop
.hbase
.Cell
;
61 import org
.apache
.hadoop
.hbase
.CellBuilderFactory
;
62 import org
.apache
.hadoop
.hbase
.CellBuilderType
;
63 import org
.apache
.hadoop
.hbase
.ClusterId
;
64 import org
.apache
.hadoop
.hbase
.ClusterMetrics
;
65 import org
.apache
.hadoop
.hbase
.ClusterMetrics
.Option
;
66 import org
.apache
.hadoop
.hbase
.ClusterMetricsBuilder
;
67 import org
.apache
.hadoop
.hbase
.DoNotRetryIOException
;
68 import org
.apache
.hadoop
.hbase
.HBaseIOException
;
69 import org
.apache
.hadoop
.hbase
.HBaseInterfaceAudience
;
70 import org
.apache
.hadoop
.hbase
.HBaseServerBase
;
71 import org
.apache
.hadoop
.hbase
.HConstants
;
72 import org
.apache
.hadoop
.hbase
.HRegionLocation
;
73 import org
.apache
.hadoop
.hbase
.InvalidFamilyOperationException
;
74 import org
.apache
.hadoop
.hbase
.MasterNotRunningException
;
75 import org
.apache
.hadoop
.hbase
.MetaTableAccessor
;
76 import org
.apache
.hadoop
.hbase
.NamespaceDescriptor
;
77 import org
.apache
.hadoop
.hbase
.PleaseHoldException
;
78 import org
.apache
.hadoop
.hbase
.PleaseRestartMasterException
;
79 import org
.apache
.hadoop
.hbase
.RegionMetrics
;
80 import org
.apache
.hadoop
.hbase
.ReplicationPeerNotFoundException
;
81 import org
.apache
.hadoop
.hbase
.ServerMetrics
;
82 import org
.apache
.hadoop
.hbase
.ServerName
;
83 import org
.apache
.hadoop
.hbase
.ServerTask
;
84 import org
.apache
.hadoop
.hbase
.ServerTaskBuilder
;
85 import org
.apache
.hadoop
.hbase
.TableName
;
86 import org
.apache
.hadoop
.hbase
.TableNotDisabledException
;
87 import org
.apache
.hadoop
.hbase
.TableNotFoundException
;
88 import org
.apache
.hadoop
.hbase
.UnknownRegionException
;
89 import org
.apache
.hadoop
.hbase
.client
.BalanceRequest
;
90 import org
.apache
.hadoop
.hbase
.client
.BalanceResponse
;
91 import org
.apache
.hadoop
.hbase
.client
.ColumnFamilyDescriptor
;
92 import org
.apache
.hadoop
.hbase
.client
.CompactionState
;
93 import org
.apache
.hadoop
.hbase
.client
.MasterSwitchType
;
94 import org
.apache
.hadoop
.hbase
.client
.NormalizeTableFilterParams
;
95 import org
.apache
.hadoop
.hbase
.client
.Put
;
96 import org
.apache
.hadoop
.hbase
.client
.RegionInfo
;
97 import org
.apache
.hadoop
.hbase
.client
.RegionInfoBuilder
;
98 import org
.apache
.hadoop
.hbase
.client
.RegionStatesCount
;
99 import org
.apache
.hadoop
.hbase
.client
.ResultScanner
;
100 import org
.apache
.hadoop
.hbase
.client
.Scan
;
101 import org
.apache
.hadoop
.hbase
.client
.TableDescriptor
;
102 import org
.apache
.hadoop
.hbase
.client
.TableDescriptorBuilder
;
103 import org
.apache
.hadoop
.hbase
.client
.TableState
;
104 import org
.apache
.hadoop
.hbase
.conf
.ConfigurationManager
;
105 import org
.apache
.hadoop
.hbase
.coprocessor
.CoprocessorHost
;
106 import org
.apache
.hadoop
.hbase
.exceptions
.MasterStoppedException
;
107 import org
.apache
.hadoop
.hbase
.executor
.ExecutorType
;
108 import org
.apache
.hadoop
.hbase
.favored
.FavoredNodesManager
;
109 import org
.apache
.hadoop
.hbase
.http
.HttpServer
;
110 import org
.apache
.hadoop
.hbase
.http
.InfoServer
;
111 import org
.apache
.hadoop
.hbase
.ipc
.CoprocessorRpcUtils
;
112 import org
.apache
.hadoop
.hbase
.ipc
.RpcServer
;
113 import org
.apache
.hadoop
.hbase
.ipc
.ServerNotRunningYetException
;
114 import org
.apache
.hadoop
.hbase
.log
.HBaseMarkers
;
115 import org
.apache
.hadoop
.hbase
.master
.MasterRpcServices
.BalanceSwitchMode
;
116 import org
.apache
.hadoop
.hbase
.master
.assignment
.AssignmentManager
;
117 import org
.apache
.hadoop
.hbase
.master
.assignment
.MergeTableRegionsProcedure
;
118 import org
.apache
.hadoop
.hbase
.master
.assignment
.RegionStateNode
;
119 import org
.apache
.hadoop
.hbase
.master
.assignment
.RegionStateStore
;
120 import org
.apache
.hadoop
.hbase
.master
.assignment
.RegionStates
;
121 import org
.apache
.hadoop
.hbase
.master
.assignment
.TransitRegionStateProcedure
;
122 import org
.apache
.hadoop
.hbase
.master
.balancer
.BalancerChore
;
123 import org
.apache
.hadoop
.hbase
.master
.balancer
.BaseLoadBalancer
;
124 import org
.apache
.hadoop
.hbase
.master
.balancer
.ClusterStatusChore
;
125 import org
.apache
.hadoop
.hbase
.master
.balancer
.LoadBalancerFactory
;
126 import org
.apache
.hadoop
.hbase
.master
.balancer
.MaintenanceLoadBalancer
;
127 import org
.apache
.hadoop
.hbase
.master
.cleaner
.DirScanPool
;
128 import org
.apache
.hadoop
.hbase
.master
.cleaner
.HFileCleaner
;
129 import org
.apache
.hadoop
.hbase
.master
.cleaner
.LogCleaner
;
130 import org
.apache
.hadoop
.hbase
.master
.cleaner
.ReplicationBarrierCleaner
;
131 import org
.apache
.hadoop
.hbase
.master
.cleaner
.SnapshotCleanerChore
;
132 import org
.apache
.hadoop
.hbase
.master
.http
.MasterDumpServlet
;
133 import org
.apache
.hadoop
.hbase
.master
.http
.MasterRedirectServlet
;
134 import org
.apache
.hadoop
.hbase
.master
.http
.MasterStatusServlet
;
135 import org
.apache
.hadoop
.hbase
.master
.http
.api_v1
.ResourceConfigFactory
;
136 import org
.apache
.hadoop
.hbase
.master
.janitor
.CatalogJanitor
;
137 import org
.apache
.hadoop
.hbase
.master
.locking
.LockManager
;
138 import org
.apache
.hadoop
.hbase
.master
.migrate
.RollingUpgradeChore
;
139 import org
.apache
.hadoop
.hbase
.master
.normalizer
.RegionNormalizerFactory
;
140 import org
.apache
.hadoop
.hbase
.master
.normalizer
.RegionNormalizerManager
;
141 import org
.apache
.hadoop
.hbase
.master
.procedure
.CreateTableProcedure
;
142 import org
.apache
.hadoop
.hbase
.master
.procedure
.DeleteNamespaceProcedure
;
143 import org
.apache
.hadoop
.hbase
.master
.procedure
.DeleteTableProcedure
;
144 import org
.apache
.hadoop
.hbase
.master
.procedure
.DisableTableProcedure
;
145 import org
.apache
.hadoop
.hbase
.master
.procedure
.EnableTableProcedure
;
146 import org
.apache
.hadoop
.hbase
.master
.procedure
.InitMetaProcedure
;
147 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureConstants
;
148 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureEnv
;
149 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureScheduler
;
150 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureUtil
;
151 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureUtil
.NonceProcedureRunnable
;
152 import org
.apache
.hadoop
.hbase
.master
.procedure
.ModifyTableProcedure
;
153 import org
.apache
.hadoop
.hbase
.master
.procedure
.ProcedurePrepareLatch
;
154 import org
.apache
.hadoop
.hbase
.master
.procedure
.ProcedureSyncWait
;
155 import org
.apache
.hadoop
.hbase
.master
.procedure
.ReopenTableRegionsProcedure
;
156 import org
.apache
.hadoop
.hbase
.master
.procedure
.ServerCrashProcedure
;
157 import org
.apache
.hadoop
.hbase
.master
.procedure
.TruncateTableProcedure
;
158 import org
.apache
.hadoop
.hbase
.master
.region
.MasterRegion
;
159 import org
.apache
.hadoop
.hbase
.master
.region
.MasterRegionFactory
;
160 import org
.apache
.hadoop
.hbase
.master
.replication
.AbstractPeerProcedure
;
161 import org
.apache
.hadoop
.hbase
.master
.replication
.AddPeerProcedure
;
162 import org
.apache
.hadoop
.hbase
.master
.replication
.DisablePeerProcedure
;
163 import org
.apache
.hadoop
.hbase
.master
.replication
.EnablePeerProcedure
;
164 import org
.apache
.hadoop
.hbase
.master
.replication
.RemovePeerProcedure
;
165 import org
.apache
.hadoop
.hbase
.master
.replication
.ReplicationPeerManager
;
166 import org
.apache
.hadoop
.hbase
.master
.replication
.SyncReplicationReplayWALManager
;
167 import org
.apache
.hadoop
.hbase
.master
.replication
.TransitPeerSyncReplicationStateProcedure
;
168 import org
.apache
.hadoop
.hbase
.master
.replication
.UpdatePeerConfigProcedure
;
169 import org
.apache
.hadoop
.hbase
.master
.slowlog
.SlowLogMasterService
;
170 import org
.apache
.hadoop
.hbase
.master
.snapshot
.SnapshotManager
;
171 import org
.apache
.hadoop
.hbase
.master
.zksyncer
.MasterAddressSyncer
;
172 import org
.apache
.hadoop
.hbase
.master
.zksyncer
.MetaLocationSyncer
;
173 import org
.apache
.hadoop
.hbase
.mob
.MobFileCleanerChore
;
174 import org
.apache
.hadoop
.hbase
.mob
.MobFileCompactionChore
;
175 import org
.apache
.hadoop
.hbase
.monitoring
.MemoryBoundedLogMessageBuffer
;
176 import org
.apache
.hadoop
.hbase
.monitoring
.MonitoredTask
;
177 import org
.apache
.hadoop
.hbase
.monitoring
.TaskMonitor
;
178 import org
.apache
.hadoop
.hbase
.namequeues
.NamedQueueRecorder
;
179 import org
.apache
.hadoop
.hbase
.procedure
.MasterProcedureManagerHost
;
180 import org
.apache
.hadoop
.hbase
.procedure
.flush
.MasterFlushTableProcedureManager
;
181 import org
.apache
.hadoop
.hbase
.procedure2
.LockedResource
;
182 import org
.apache
.hadoop
.hbase
.procedure2
.Procedure
;
183 import org
.apache
.hadoop
.hbase
.procedure2
.ProcedureEvent
;
184 import org
.apache
.hadoop
.hbase
.procedure2
.ProcedureExecutor
;
185 import org
.apache
.hadoop
.hbase
.procedure2
.RemoteProcedureDispatcher
.RemoteProcedure
;
186 import org
.apache
.hadoop
.hbase
.procedure2
.RemoteProcedureException
;
187 import org
.apache
.hadoop
.hbase
.procedure2
.store
.ProcedureStore
;
188 import org
.apache
.hadoop
.hbase
.procedure2
.store
.ProcedureStore
.ProcedureStoreListener
;
189 import org
.apache
.hadoop
.hbase
.procedure2
.store
.region
.RegionProcedureStore
;
190 import org
.apache
.hadoop
.hbase
.quotas
.MasterQuotaManager
;
191 import org
.apache
.hadoop
.hbase
.quotas
.MasterQuotasObserver
;
192 import org
.apache
.hadoop
.hbase
.quotas
.QuotaObserverChore
;
193 import org
.apache
.hadoop
.hbase
.quotas
.QuotaTableUtil
;
194 import org
.apache
.hadoop
.hbase
.quotas
.QuotaUtil
;
195 import org
.apache
.hadoop
.hbase
.quotas
.SnapshotQuotaObserverChore
;
196 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshot
;
197 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshot
.SpaceQuotaStatus
;
198 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshotNotifier
;
199 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshotNotifierFactory
;
200 import org
.apache
.hadoop
.hbase
.quotas
.SpaceViolationPolicy
;
201 import org
.apache
.hadoop
.hbase
.regionserver
.HRegionServer
;
202 import org
.apache
.hadoop
.hbase
.regionserver
.NoSuchColumnFamilyException
;
203 import org
.apache
.hadoop
.hbase
.regionserver
.storefiletracker
.ModifyColumnFamilyStoreFileTrackerProcedure
;
204 import org
.apache
.hadoop
.hbase
.regionserver
.storefiletracker
.ModifyTableStoreFileTrackerProcedure
;
205 import org
.apache
.hadoop
.hbase
.replication
.ReplicationException
;
206 import org
.apache
.hadoop
.hbase
.replication
.ReplicationLoadSource
;
207 import org
.apache
.hadoop
.hbase
.replication
.ReplicationPeerConfig
;
208 import org
.apache
.hadoop
.hbase
.replication
.ReplicationPeerDescription
;
209 import org
.apache
.hadoop
.hbase
.replication
.ReplicationUtils
;
210 import org
.apache
.hadoop
.hbase
.replication
.SyncReplicationState
;
211 import org
.apache
.hadoop
.hbase
.replication
.master
.ReplicationHFileCleaner
;
212 import org
.apache
.hadoop
.hbase
.replication
.master
.ReplicationLogCleaner
;
213 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupAdminEndpoint
;
214 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupBasedLoadBalancer
;
215 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupInfoManager
;
216 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupUtil
;
217 import org
.apache
.hadoop
.hbase
.security
.AccessDeniedException
;
218 import org
.apache
.hadoop
.hbase
.security
.SecurityConstants
;
219 import org
.apache
.hadoop
.hbase
.security
.Superusers
;
220 import org
.apache
.hadoop
.hbase
.security
.UserProvider
;
221 import org
.apache
.hadoop
.hbase
.util
.Addressing
;
222 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
223 import org
.apache
.hadoop
.hbase
.util
.CommonFSUtils
;
224 import org
.apache
.hadoop
.hbase
.util
.CoprocessorConfigurationUtil
;
225 import org
.apache
.hadoop
.hbase
.util
.EnvironmentEdgeManager
;
226 import org
.apache
.hadoop
.hbase
.util
.FSTableDescriptors
;
227 import org
.apache
.hadoop
.hbase
.util
.FutureUtils
;
228 import org
.apache
.hadoop
.hbase
.util
.HBaseFsck
;
229 import org
.apache
.hadoop
.hbase
.util
.HFileArchiveUtil
;
230 import org
.apache
.hadoop
.hbase
.util
.IdLock
;
231 import org
.apache
.hadoop
.hbase
.util
.JVMClusterUtil
;
232 import org
.apache
.hadoop
.hbase
.util
.ModifyRegionUtils
;
233 import org
.apache
.hadoop
.hbase
.util
.Pair
;
234 import org
.apache
.hadoop
.hbase
.util
.RetryCounter
;
235 import org
.apache
.hadoop
.hbase
.util
.RetryCounterFactory
;
236 import org
.apache
.hadoop
.hbase
.util
.TableDescriptorChecker
;
237 import org
.apache
.hadoop
.hbase
.util
.Threads
;
238 import org
.apache
.hadoop
.hbase
.util
.VersionInfo
;
239 import org
.apache
.hadoop
.hbase
.zookeeper
.LoadBalancerTracker
;
240 import org
.apache
.hadoop
.hbase
.zookeeper
.MasterAddressTracker
;
241 import org
.apache
.hadoop
.hbase
.zookeeper
.MetaTableLocator
;
242 import org
.apache
.hadoop
.hbase
.zookeeper
.RegionNormalizerTracker
;
243 import org
.apache
.hadoop
.hbase
.zookeeper
.SnapshotCleanupTracker
;
244 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKClusterId
;
245 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
;
246 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKWatcher
;
247 import org
.apache
.hadoop
.hbase
.zookeeper
.ZNodePaths
;
248 import org
.apache
.yetus
.audience
.InterfaceAudience
;
249 import org
.apache
.zookeeper
.KeeperException
;
250 import org
.slf4j
.Logger
;
251 import org
.slf4j
.LoggerFactory
;
253 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Lists
;
254 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Maps
;
255 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Sets
;
256 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.io
.Closeables
;
257 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Descriptors
;
258 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Service
;
259 import org
.apache
.hbase
.thirdparty
.org
.eclipse
.jetty
.server
.Server
;
260 import org
.apache
.hbase
.thirdparty
.org
.eclipse
.jetty
.server
.ServerConnector
;
261 import org
.apache
.hbase
.thirdparty
.org
.eclipse
.jetty
.servlet
.ServletHolder
;
262 import org
.apache
.hbase
.thirdparty
.org
.eclipse
.jetty
.webapp
.WebAppContext
;
263 import org
.apache
.hbase
.thirdparty
.org
.glassfish
.jersey
.server
.ResourceConfig
;
264 import org
.apache
.hbase
.thirdparty
.org
.glassfish
.jersey
.servlet
.ServletContainer
;
265 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.RequestConverter
;
266 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetRegionInfoResponse
;
267 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.SnapshotProtos
.SnapshotDescription
;
270 * HMaster is the "master server" for HBase. An HBase cluster has one active master. If many masters
271 * are started, all compete. Whichever wins goes on to run the cluster. All others park themselves
272 * in their constructor until master or cluster shutdown or until the active master loses its lease
273 * in zookeeper. Thereafter, all running master jostle to take over master role.
275 * The Master can be asked shutdown the cluster. See {@link #shutdown()}. In this case it will tell
276 * all regionservers to go down and then wait on them all reporting in that they are down. This
277 * master will then shut itself down.
279 * You can also shutdown just this master. Call {@link #stopMaster()}.
280 * @see org.apache.zookeeper.Watcher
282 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience
.TOOLS
)
283 public class HMaster
extends HBaseServerBase
<MasterRpcServices
> implements MasterServices
{
285 private static final Logger LOG
= LoggerFactory
.getLogger(HMaster
.class);
287 // MASTER is name of the webapp and the attribute name used stuffing this
288 // instance into a web context !! AND OTHER PLACES !!
289 public static final String MASTER
= "master";
291 // Manager and zk listener for master election
292 private final ActiveMasterManager activeMasterManager
;
293 // Region server tracker
294 private final RegionServerTracker regionServerTracker
;
295 // Draining region server tracker
296 private DrainingServerTracker drainingServerTracker
;
297 // Tracker for load balancer state
298 LoadBalancerTracker loadBalancerTracker
;
299 // Tracker for meta location, if any client ZK quorum specified
300 private MetaLocationSyncer metaLocationSyncer
;
301 // Tracker for active master location, if any client ZK quorum specified
302 @InterfaceAudience.Private
303 MasterAddressSyncer masterAddressSyncer
;
304 // Tracker for auto snapshot cleanup state
305 SnapshotCleanupTracker snapshotCleanupTracker
;
307 // Tracker for split and merge state
308 private SplitOrMergeTracker splitOrMergeTracker
;
310 private ClusterSchemaService clusterSchemaService
;
312 public static final String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
=
313 "hbase.master.wait.on.service.seconds";
314 public static final int DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
= 5 * 60;
316 public static final String HBASE_MASTER_CLEANER_INTERVAL
= "hbase.master.cleaner.interval";
318 public static final int DEFAULT_HBASE_MASTER_CLEANER_INTERVAL
= 600 * 1000;
320 private String clusterId
;
322 // Metrics for the HMaster
323 final MetricsMaster metricsMaster
;
324 // file system manager for the master FS operations
325 private MasterFileSystem fileSystemManager
;
326 private MasterWalManager walManager
;
328 // manager to manage procedure-based WAL splitting, can be null if current
329 // is zk-based WAL splitting. SplitWALManager will replace SplitLogManager
330 // and MasterWalManager, which means zk-based WAL splitting code will be
331 // useless after we switch to the procedure-based one. our eventual goal
332 // is to remove all the zk-based WAL splitting code.
333 private SplitWALManager splitWALManager
;
335 // server manager to deal with region server info
336 private volatile ServerManager serverManager
;
338 // manager of assignment nodes in zookeeper
339 private AssignmentManager assignmentManager
;
341 private RSGroupInfoManager rsGroupInfoManager
;
343 // manager of replication
344 private ReplicationPeerManager replicationPeerManager
;
346 private SyncReplicationReplayWALManager syncReplicationReplayWALManager
;
348 // buffer for "fatal error" notices from region servers
349 // in the cluster. This is only used for assisting
350 // operations/debugging.
351 MemoryBoundedLogMessageBuffer rsFatals
;
353 // flag set after we become the active master (used for testing)
354 private volatile boolean activeMaster
= false;
356 // flag set after we complete initialization once active
357 private final ProcedureEvent
<?
> initialized
= new ProcedureEvent
<>("master initialized");
359 // flag set after master services are started,
360 // initialization may have not completed yet.
361 volatile boolean serviceStarted
= false;
363 // Maximum time we should run balancer for
364 private final int maxBalancingTime
;
365 // Maximum percent of regions in transition when balancing
366 private final double maxRitPercent
;
368 private final LockManager lockManager
= new LockManager(this);
370 private RSGroupBasedLoadBalancer balancer
;
371 private BalancerChore balancerChore
;
372 private static boolean disableBalancerChoreForTest
= false;
373 private RegionNormalizerManager regionNormalizerManager
;
374 private ClusterStatusChore clusterStatusChore
;
375 private ClusterStatusPublisher clusterStatusPublisherChore
= null;
376 private SnapshotCleanerChore snapshotCleanerChore
= null;
378 private HbckChore hbckChore
;
379 CatalogJanitor catalogJanitorChore
;
380 // Threadpool for scanning the archive directory, used by the HFileCleaner
381 private DirScanPool hfileCleanerPool
;
382 // Threadpool for scanning the Old logs directory, used by the LogCleaner
383 private DirScanPool logCleanerPool
;
384 private LogCleaner logCleaner
;
385 private HFileCleaner hfileCleaner
;
386 private ReplicationBarrierCleaner replicationBarrierCleaner
;
387 private MobFileCleanerChore mobFileCleanerChore
;
388 private MobFileCompactionChore mobFileCompactionChore
;
389 private RollingUpgradeChore rollingUpgradeChore
;
390 // used to synchronize the mobCompactionStates
391 private final IdLock mobCompactionLock
= new IdLock();
392 // save the information of mob compactions in tables.
393 // the key is table name, the value is the number of compactions in that table.
394 private Map
<TableName
, AtomicInteger
> mobCompactionStates
= Maps
.newConcurrentMap();
396 volatile MasterCoprocessorHost cpHost
;
398 private final boolean preLoadTableDescriptors
;
400 // Time stamps for when a hmaster became active
401 private long masterActiveTime
;
403 // Time stamp for when HMaster finishes becoming Active Master
404 private long masterFinishedInitializationTime
;
406 Map
<String
, Service
> coprocessorServiceHandlers
= Maps
.newHashMap();
408 // monitor for snapshot of hbase tables
409 SnapshotManager snapshotManager
;
410 // monitor for distributed procedures
411 private MasterProcedureManagerHost mpmHost
;
413 private RegionsRecoveryChore regionsRecoveryChore
= null;
415 private RegionsRecoveryConfigManager regionsRecoveryConfigManager
= null;
416 // it is assigned after 'initialized' guard set to true, so should be volatile
417 private volatile MasterQuotaManager quotaManager
;
418 private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier
;
419 private QuotaObserverChore quotaObserverChore
;
420 private SnapshotQuotaObserverChore snapshotQuotaChore
;
422 private ProcedureExecutor
<MasterProcedureEnv
> procedureExecutor
;
423 private ProcedureStore procedureStore
;
425 // the master local storage to store procedure data, meta region locations, etc.
426 private MasterRegion masterRegion
;
428 private RegionServerList rsListStorage
;
430 // handle table states
431 private TableStateManager tableStateManager
;
433 /** jetty server for master to redirect requests to regionserver infoServer */
434 private Server masterJettyServer
;
436 // Determine if we should do normal startup or minimal "single-user" mode with no region
437 // servers and no user tables. Useful for repair and recovery of hbase:meta
438 private final boolean maintenanceMode
;
439 static final String MAINTENANCE_MODE
= "hbase.master.maintenance_mode";
441 // the in process region server for carry system regions in maintenanceMode
442 private JVMClusterUtil
.RegionServerThread maintenanceRegionServer
;
444 // Cached clusterId on stand by masters to serve clusterID requests from clients.
445 private final CachedClusterId cachedClusterId
;
447 public static final String WARMUP_BEFORE_MOVE
= "hbase.master.warmup.before.move";
448 private static final boolean DEFAULT_WARMUP_BEFORE_MOVE
= true;
451 * Initializes the HMaster. The steps are as follows:
454 * <li>Initialize the local HRegionServer
455 * <li>Start the ActiveMasterManager.
458 * Remaining steps of initialization occur in
459 * {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the
462 public HMaster(final Configuration conf
) throws IOException
{
463 super(conf
, "Master");
465 if (conf
.getBoolean(MAINTENANCE_MODE
, false)) {
466 LOG
.info("Detected {}=true via configuration.", MAINTENANCE_MODE
);
467 maintenanceMode
= true;
468 } else if (Boolean
.getBoolean(MAINTENANCE_MODE
)) {
469 LOG
.info("Detected {}=true via environment variables.", MAINTENANCE_MODE
);
470 maintenanceMode
= true;
472 maintenanceMode
= false;
474 this.rsFatals
= new MemoryBoundedLogMessageBuffer(
475 conf
.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));
476 LOG
.info("hbase.rootdir={}, hbase.cluster.distributed={}",
477 CommonFSUtils
.getRootDir(this.conf
),
478 this.conf
.getBoolean(HConstants
.CLUSTER_DISTRIBUTED
, false));
480 // Disable usage of meta replicas in the master
481 this.conf
.setBoolean(HConstants
.USE_META_REPLICAS
, false);
483 decorateMasterConfiguration(this.conf
);
485 // Hack! Maps DFSClient => Master for logs. HDFS made this
486 // config param for task trackers, but we can piggyback off of it.
487 if (this.conf
.get("mapreduce.task.attempt.id") == null) {
488 this.conf
.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName
.toString());
491 this.metricsMaster
= new MetricsMaster(new MetricsMasterWrapperImpl(this));
493 // preload table descriptor at startup
494 this.preLoadTableDescriptors
= conf
.getBoolean("hbase.master.preload.tabledescriptors", true);
496 this.maxBalancingTime
= getMaxBalancingTime();
497 this.maxRitPercent
= conf
.getDouble(HConstants
.HBASE_MASTER_BALANCER_MAX_RIT_PERCENT
,
498 HConstants
.DEFAULT_HBASE_MASTER_BALANCER_MAX_RIT_PERCENT
);
500 // Do we publish the status?
502 boolean shouldPublish
= conf
.getBoolean(HConstants
.STATUS_PUBLISHED
,
503 HConstants
.STATUS_PUBLISHED_DEFAULT
);
504 Class
<?
extends ClusterStatusPublisher
.Publisher
> publisherClass
=
505 conf
.getClass(ClusterStatusPublisher
.STATUS_PUBLISHER_CLASS
,
506 ClusterStatusPublisher
.DEFAULT_STATUS_PUBLISHER_CLASS
,
507 ClusterStatusPublisher
.Publisher
.class);
510 if (publisherClass
== null) {
511 LOG
.warn(HConstants
.STATUS_PUBLISHED
+ " is true, but " +
512 ClusterStatusPublisher
.DEFAULT_STATUS_PUBLISHER_CLASS
+
513 " is not set - not publishing status");
515 clusterStatusPublisherChore
= new ClusterStatusPublisher(this, conf
, publisherClass
);
516 LOG
.debug("Created {}", this.clusterStatusPublisherChore
);
517 getChoreService().scheduleChore(clusterStatusPublisherChore
);
520 this.activeMasterManager
= createActiveMasterManager(zooKeeper
, serverName
, this);
521 cachedClusterId
= new CachedClusterId(this, conf
);
522 this.regionServerTracker
= new RegionServerTracker(zooKeeper
, this);
523 this.rpcServices
.start(zooKeeper
);
524 } catch (Throwable t
) {
525 // Make sure we log the exception. HMaster is often started via reflection and the
526 // cause of failed startup is lost.
527 LOG
.error("Failed construction of Master", t
);
533 * Protected to have custom implementations in tests override the default ActiveMaster
536 protected ActiveMasterManager
createActiveMasterManager(ZKWatcher zk
, ServerName sn
,
537 org
.apache
.hadoop
.hbase
.Server server
) throws InterruptedIOException
{
538 return new ActiveMasterManager(zk
, sn
, server
);
542 protected String
getUseThisHostnameInstead(Configuration conf
) {
543 return conf
.get(MASTER_HOSTNAME_KEY
);
546 private void registerConfigurationObservers() {
547 configurationManager
.registerObserver(this.rpcServices
);
548 configurationManager
.registerObserver(this);
551 // Main run loop. Calls through to the regionserver run loop AFTER becoming active Master; will
552 // block in here until then.
556 registerConfigurationObservers();
557 Threads
.setDaemonThreadRunning(new Thread(() -> {
559 int infoPort
= putUpJettyServer();
560 startActiveMasterManager(infoPort
);
561 } catch (Throwable t
) {
562 // Make sure we log the exception.
563 String error
= "Failed to become Active Master";
565 // Abort should have been called already.
570 }), getName() + ":becomeActiveMaster");
571 while (!isStopped() && !isAborted()) {
575 closeClusterConnection();
576 stopServiceThreads();
577 if (this.rpcServices
!= null) {
578 this.rpcServices
.stop();
582 if (this.clusterSchemaService
!= null) {
583 // If on way out, then we are no longer active master.
584 this.clusterSchemaService
.stopAsync();
586 this.clusterSchemaService
.awaitTerminated(
587 getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
,
588 DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
), TimeUnit
.SECONDS
);
589 } catch (TimeoutException te
) {
590 LOG
.warn("Failed shutdown of clusterSchemaService", te
);
593 this.activeMaster
= false;
597 // return the actual infoPort, -1 means disable info server.
598 private int putUpJettyServer() throws IOException
{
599 if (!conf
.getBoolean("hbase.master.infoserver.redirect", true)) {
602 final int infoPort
= conf
.getInt("hbase.master.info.port.orig",
603 HConstants
.DEFAULT_MASTER_INFOPORT
);
604 // -1 is for disabling info server, so no redirecting
605 if (infoPort
< 0 || infoServer
== null) {
608 if (infoPort
== infoServer
.getPort()) {
609 // server is already running
612 final String addr
= conf
.get("hbase.master.info.bindAddress", "0.0.0.0");
613 if (!Addressing
.isLocalAddress(InetAddress
.getByName(addr
))) {
615 "Failed to start redirecting jetty server. Address " + addr
616 + " does not belong to this host. Correct configuration parameter: "
617 + "hbase.master.info.bindAddress";
619 throw new IOException(msg
);
622 // TODO I'm pretty sure we could just add another binding to the InfoServer run by
623 // the RegionServer and have it run the RedirectServlet instead of standing up
624 // a second entire stack here.
625 masterJettyServer
= new Server();
626 final ServerConnector connector
= new ServerConnector(masterJettyServer
);
627 connector
.setHost(addr
);
628 connector
.setPort(infoPort
);
629 masterJettyServer
.addConnector(connector
);
630 masterJettyServer
.setStopAtShutdown(true);
631 masterJettyServer
.setHandler(HttpServer
.buildGzipHandler(masterJettyServer
.getHandler()));
633 final String redirectHostname
=
634 StringUtils
.isBlank(useThisHostnameInstead
) ?
null : useThisHostnameInstead
;
636 final MasterRedirectServlet redirect
= new MasterRedirectServlet(infoServer
, redirectHostname
);
637 final WebAppContext context
= new WebAppContext(null, "/", null, null, null, null, WebAppContext
.NO_SESSIONS
);
638 context
.addServlet(new ServletHolder(redirect
), "/*");
639 context
.setServer(masterJettyServer
);
642 masterJettyServer
.start();
643 } catch (Exception e
) {
644 throw new IOException("Failed to start redirecting jetty server", e
);
646 return connector
.getLocalPort();
650 * For compatibility, if failed with regionserver credentials, try the master one
653 protected void login(UserProvider user
, String host
) throws IOException
{
655 user
.login(SecurityConstants
.REGIONSERVER_KRB_KEYTAB_FILE
,
656 SecurityConstants
.REGIONSERVER_KRB_PRINCIPAL
, host
);
657 } catch (IOException ie
) {
658 user
.login(SecurityConstants
.MASTER_KRB_KEYTAB_FILE
, SecurityConstants
.MASTER_KRB_PRINCIPAL
,
663 public MasterRpcServices
getMasterRpcServices() {
667 public boolean balanceSwitch(final boolean b
) throws IOException
{
668 return getMasterRpcServices().switchBalancer(b
, BalanceSwitchMode
.ASYNC
);
672 protected String
getProcessName() {
677 protected boolean canCreateBaseZNode() {
682 protected boolean canUpdateTableDescriptor() {
687 protected boolean cacheTableDescriptor() {
691 protected MasterRpcServices
createRpcServices() throws IOException
{
692 return new MasterRpcServices(this);
696 protected void configureInfoServer(InfoServer infoServer
) {
697 infoServer
.addUnprivilegedServlet("master-status", "/master-status", MasterStatusServlet
.class);
698 infoServer
.addUnprivilegedServlet("api_v1", "/api/v1/*", buildApiV1Servlet());
700 infoServer
.setAttribute(MASTER
, this);
703 private ServletHolder
buildApiV1Servlet() {
704 final ResourceConfig config
= ResourceConfigFactory
.createResourceConfig(conf
, this);
705 return new ServletHolder(new ServletContainer(config
));
709 protected Class
<?
extends HttpServlet
> getDumpServlet() {
710 return MasterDumpServlet
.class;
714 public MetricsMaster
getMasterMetrics() {
715 return metricsMaster
;
719 * Initialize all ZK based system trackers. But do not include {@link RegionServerTracker}, it
720 * should have already been initialized along with {@link ServerManager}.
722 private void initializeZKBasedSystemTrackers()
723 throws IOException
, KeeperException
, ReplicationException
{
724 if (maintenanceMode
) {
725 // in maintenance mode, always use MaintenanceLoadBalancer.
726 conf
.unset(LoadBalancer
.HBASE_RSGROUP_LOADBALANCER_CLASS
);
727 conf
.setClass(HConstants
.HBASE_MASTER_LOADBALANCER_CLASS
, MaintenanceLoadBalancer
.class,
730 this.balancer
= new RSGroupBasedLoadBalancer();
731 this.loadBalancerTracker
= new LoadBalancerTracker(zooKeeper
, this);
732 this.loadBalancerTracker
.start();
734 this.regionNormalizerManager
=
735 RegionNormalizerFactory
.createNormalizerManager(conf
, zooKeeper
, this);
736 this.configurationManager
.registerObserver(regionNormalizerManager
);
737 this.regionNormalizerManager
.start();
739 this.splitOrMergeTracker
= new SplitOrMergeTracker(zooKeeper
, conf
, this);
740 this.splitOrMergeTracker
.start();
742 // This is for backwards compatible. We do not need the CP for rs group now but if user want to
743 // load it, we need to enable rs group.
744 String
[] cpClasses
= conf
.getStrings(MasterCoprocessorHost
.MASTER_COPROCESSOR_CONF_KEY
);
745 if (cpClasses
!= null) {
746 for (String cpClass
: cpClasses
) {
747 if (RSGroupAdminEndpoint
.class.getName().equals(cpClass
)) {
748 RSGroupUtil
.enableRSGroup(conf
);
753 this.rsGroupInfoManager
= RSGroupInfoManager
.create(this);
755 this.replicationPeerManager
= ReplicationPeerManager
.create(zooKeeper
, conf
, clusterId
);
757 this.drainingServerTracker
= new DrainingServerTracker(zooKeeper
, this, this.serverManager
);
758 this.drainingServerTracker
.start();
760 this.snapshotCleanupTracker
= new SnapshotCleanupTracker(zooKeeper
, this);
761 this.snapshotCleanupTracker
.start();
763 String clientQuorumServers
= conf
.get(HConstants
.CLIENT_ZOOKEEPER_QUORUM
);
764 boolean clientZkObserverMode
= conf
.getBoolean(HConstants
.CLIENT_ZOOKEEPER_OBSERVER_MODE
,
765 HConstants
.DEFAULT_CLIENT_ZOOKEEPER_OBSERVER_MODE
);
766 if (clientQuorumServers
!= null && !clientZkObserverMode
) {
767 // we need to take care of the ZK information synchronization
768 // if given client ZK are not observer nodes
769 ZKWatcher clientZkWatcher
= new ZKWatcher(conf
,
770 getProcessName() + ":" + rpcServices
.getSocketAddress().getPort() + "-clientZK", this,
772 this.metaLocationSyncer
= new MetaLocationSyncer(zooKeeper
, clientZkWatcher
, this);
773 this.metaLocationSyncer
.start();
774 this.masterAddressSyncer
= new MasterAddressSyncer(zooKeeper
, clientZkWatcher
, this);
775 this.masterAddressSyncer
.start();
776 // set cluster id is a one-go effort
777 ZKClusterId
.setClusterId(clientZkWatcher
, fileSystemManager
.getClusterId());
780 // Set the cluster as up. If new RSs, they'll be waiting on this before
781 // going ahead with their startup.
782 boolean wasUp
= this.clusterStatusTracker
.isClusterUp();
783 if (!wasUp
) this.clusterStatusTracker
.setClusterUp();
785 LOG
.info("Active/primary master=" + this.serverName
+
787 Long
.toHexString(this.zooKeeper
.getRecoverableZooKeeper().getSessionId()) +
788 ", setting cluster-up flag (Was=" + wasUp
+ ")");
790 // create/initialize the snapshot manager and other procedure managers
791 this.snapshotManager
= new SnapshotManager();
792 this.mpmHost
= new MasterProcedureManagerHost();
793 this.mpmHost
.register(this.snapshotManager
);
794 this.mpmHost
.register(new MasterFlushTableProcedureManager());
795 this.mpmHost
.loadProcedures(conf
);
796 this.mpmHost
.initialize(this, this.metricsMaster
);
799 // Will be overriden in test to inject customized AssignmentManager
800 @InterfaceAudience.Private
801 protected AssignmentManager
createAssignmentManager(MasterServices master
,
802 MasterRegion masterRegion
) {
803 return new AssignmentManager(master
, masterRegion
);
806 private void tryMigrateMetaLocationsFromZooKeeper() throws IOException
, KeeperException
{
807 // try migrate data from zookeeper
808 try (ResultScanner scanner
=
809 masterRegion
.getScanner(new Scan().addFamily(HConstants
.CATALOG_FAMILY
))) {
810 if (scanner
.next() != null) {
811 // notice that all replicas for a region are in the same row, so the migration can be
812 // done with in a one row put, which means if we have data in catalog family then we can
813 // make sure that the migration is done.
814 LOG
.info("The {} family in master local region already has data in it, skip migrating...",
815 HConstants
.CATALOG_FAMILY_STR
);
820 byte[] row
= CatalogFamilyFormat
.getMetaKeyForRegion(RegionInfoBuilder
.FIRST_META_REGIONINFO
);
821 Put put
= new Put(row
);
822 List
<String
> metaReplicaNodes
= zooKeeper
.getMetaReplicaNodes();
823 StringBuilder info
= new StringBuilder("Migrating meta locations:");
824 for (String metaReplicaNode
: metaReplicaNodes
) {
825 int replicaId
= zooKeeper
.getZNodePaths().getMetaReplicaIdFromZNode(metaReplicaNode
);
826 RegionState state
= MetaTableLocator
.getMetaRegionState(zooKeeper
, replicaId
);
827 info
.append(" ").append(state
);
828 put
.setTimestamp(state
.getStamp());
829 MetaTableAccessor
.addRegionInfo(put
, state
.getRegion());
830 if (state
.getServerName() != null) {
831 MetaTableAccessor
.addLocation(put
, state
.getServerName(), HConstants
.NO_SEQNUM
, replicaId
);
833 put
.add(CellBuilderFactory
.create(CellBuilderType
.SHALLOW_COPY
).setRow(put
.getRow())
834 .setFamily(HConstants
.CATALOG_FAMILY
)
835 .setQualifier(RegionStateStore
.getStateColumn(replicaId
)).setTimestamp(put
.getTimestamp())
836 .setType(Cell
.Type
.Put
).setValue(Bytes
.toBytes(state
.getState().name())).build());
838 if (!put
.isEmpty()) {
839 LOG
.info(info
.toString());
840 masterRegion
.update(r
-> r
.put(put
));
842 LOG
.info("No meta location available on zookeeper, skip migrating...");
847 * Finish initialization of HMaster after becoming the primary master.
849 * The startup order is a bit complicated but very important, do not change it unless you know
850 * what you are doing.
852 * <li>Initialize file system based components - file system manager, wal manager, table
853 * descriptors, etc</li>
854 * <li>Publish cluster id</li>
855 * <li>Here comes the most complicated part - initialize server manager, assignment manager and
856 * region server tracker
858 * <li>Create server manager</li>
859 * <li>Create master local region</li>
860 * <li>Create procedure executor, load the procedures, but do not start workers. We will start it
861 * later after we finish scheduling SCPs to avoid scheduling duplicated SCPs for the same
863 * <li>Create assignment manager and start it, load the meta region state, but do not load data
864 * from meta region</li>
865 * <li>Start region server tracker, construct the online servers set and find out dead servers and
866 * schedule SCP for them. The online servers will be constructed by scanning zk, and we will also
867 * scan the wal directory to find out possible live region servers, and the differences between
868 * these two sets are the dead servers</li>
871 * <li>If this is a new deploy, schedule a InitMetaProcedure to initialize meta</li>
872 * <li>Start necessary service threads - balancer, catalog janitor, executor services, and also
873 * the procedure executor, etc. Notice that the balancer must be created first as assignment
874 * manager may use it when assigning regions.</li>
875 * <li>Wait for meta to be initialized if necessary, start table state manager.</li>
876 * <li>Wait for enough region servers to check-in</li>
877 * <li>Let assignment manager load data from meta and construct region states</li>
878 * <li>Start all other things such as chore services, etc</li>
881 * Notice that now we will not schedule a special procedure to make meta online(unless the first
882 * time where meta has not been created yet), we will rely on SCP to bring meta online.
884 private void finishActiveMasterInitialization(MonitoredTask status
) throws IOException
,
885 InterruptedException
, KeeperException
, ReplicationException
{
887 * We are active master now... go initialize components we need to run.
889 status
.setStatus("Initializing Master file system");
891 this.masterActiveTime
= EnvironmentEdgeManager
.currentTime();
892 // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
894 // always initialize the MemStoreLAB as we use a region to store data in master now, see
896 initializeMemStoreChunkCreator(null);
897 this.fileSystemManager
= new MasterFileSystem(conf
);
898 this.walManager
= new MasterWalManager(this);
900 // warm-up HTDs cache on master initialization
901 if (preLoadTableDescriptors
) {
902 status
.setStatus("Pre-loading table descriptors");
903 this.tableDescriptors
.getAll();
906 // Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
907 // only after it has checked in with the Master. At least a few tests ask Master for clusterId
908 // before it has called its run method and before RegionServer has done the reportForDuty.
909 ClusterId clusterId
= fileSystemManager
.getClusterId();
910 status
.setStatus("Publishing Cluster ID " + clusterId
+ " in ZooKeeper");
911 ZKClusterId
.setClusterId(this.zooKeeper
, fileSystemManager
.getClusterId());
912 this.clusterId
= clusterId
.toString();
914 // Precaution. Put in place the old hbck1 lock file to fence out old hbase1s running their
915 // hbck1s against an hbase2 cluster; it could do damage. To skip this behavior, set
916 // hbase.write.hbck1.lock.file to false.
917 if (this.conf
.getBoolean("hbase.write.hbck1.lock.file", true)) {
918 Pair
<Path
, FSDataOutputStream
> result
= null;
920 result
= HBaseFsck
.checkAndMarkRunningHbck(this.conf
,
921 HBaseFsck
.createLockRetryCounterFactory(this.conf
).create());
923 if (result
!= null) {
924 Closeables
.close(result
.getSecond(), true);
929 status
.setStatus("Initialize ServerManager and schedule SCP for crash servers");
930 // The below two managers must be created before loading procedures, as they will be used during
932 // initialize master local region
933 masterRegion
= MasterRegionFactory
.create(this);
934 rsListStorage
= new MasterRegionServerList(masterRegion
, this);
936 this.serverManager
= createServerManager(this, rsListStorage
);
937 this.syncReplicationReplayWALManager
= new SyncReplicationReplayWALManager(this);
938 if (!conf
.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK
,
939 DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK
)) {
940 this.splitWALManager
= new SplitWALManager(this);
945 tryMigrateMetaLocationsFromZooKeeper();
947 createProcedureExecutor();
948 Map
<Class
<?
>, List
<Procedure
<MasterProcedureEnv
>>> procsByType
=
949 procedureExecutor
.getActiveProceduresNoCopy().stream()
950 .collect(Collectors
.groupingBy(p
-> p
.getClass()));
952 // Create Assignment Manager
953 this.assignmentManager
= createAssignmentManager(this, masterRegion
);
954 this.assignmentManager
.start();
955 // TODO: TRSP can perform as the sub procedure for other procedures, so even if it is marked as
956 // completed, it could still be in the procedure list. This is a bit strange but is another
957 // story, need to verify the implementation for ProcedureExecutor and ProcedureStore.
958 List
<TransitRegionStateProcedure
> ritList
=
959 procsByType
.getOrDefault(TransitRegionStateProcedure
.class, Collections
.emptyList()).stream()
960 .filter(p
-> !p
.isFinished()).map(p
-> (TransitRegionStateProcedure
) p
)
961 .collect(Collectors
.toList());
962 this.assignmentManager
.setupRIT(ritList
);
964 // Start RegionServerTracker with listing of servers found with exiting SCPs -- these should
965 // be registered in the deadServers set -- and with the list of servernames out on the
966 // filesystem that COULD BE 'alive' (we'll schedule SCPs for each and let SCP figure it out).
967 // We also pass dirs that are already 'splitting'... so we can do some checks down in tracker.
968 // TODO: Generate the splitting and live Set in one pass instead of two as we currently do.
969 this.regionServerTracker
.upgrade(
970 procsByType
.getOrDefault(ServerCrashProcedure
.class, Collections
.emptyList()).stream()
971 .map(p
-> (ServerCrashProcedure
) p
).map(p
-> p
.getServerName()).collect(Collectors
.toSet()),
972 Sets
.union(rsListStorage
.getAll(), walManager
.getLiveServersFromWALDir()),
973 walManager
.getSplittingServersFromWALDir());
974 // This manager must be accessed AFTER hbase:meta is confirmed on line..
975 this.tableStateManager
= new TableStateManager(this);
977 status
.setStatus("Initializing ZK system trackers");
978 initializeZKBasedSystemTrackers();
979 status
.setStatus("Loading last flushed sequence id of regions");
981 this.serverManager
.loadLastFlushedSequenceIds();
982 } catch (IOException e
) {
983 LOG
.info("Failed to load last flushed sequence id of regions"
984 + " from file system", e
);
986 // Set ourselves as active Master now our claim has succeeded up in zk.
987 this.activeMaster
= true;
989 // Start the Zombie master detector after setting master as active, see HBASE-21535
990 Thread zombieDetector
= new Thread(new MasterInitializationMonitor(this),
991 "ActiveMasterInitializationMonitor-" + EnvironmentEdgeManager
.currentTime());
992 zombieDetector
.setDaemon(true);
993 zombieDetector
.start();
995 if (!maintenanceMode
) {
996 status
.setStatus("Initializing master coprocessors");
997 setQuotasObserver(conf
);
998 initializeCoprocessorHost(conf
);
1000 // start an in process region server for carrying system regions
1001 maintenanceRegionServer
=
1002 JVMClusterUtil
.createRegionServerThread(getConfiguration(), HRegionServer
.class, 0);
1003 maintenanceRegionServer
.start();
1006 // Checking if meta needs initializing.
1007 status
.setStatus("Initializing meta table if this is a new deploy");
1008 InitMetaProcedure initMetaProc
= null;
1009 // Print out state of hbase:meta on startup; helps debugging.
1010 if (!this.assignmentManager
.getRegionStates().hasTableRegionStates(TableName
.META_TABLE_NAME
)) {
1011 Optional
<InitMetaProcedure
> optProc
= procedureExecutor
.getProcedures().stream()
1012 .filter(p
-> p
instanceof InitMetaProcedure
).map(o
-> (InitMetaProcedure
) o
).findAny();
1013 initMetaProc
= optProc
.orElseGet(() -> {
1014 // schedule an init meta procedure if meta has not been deployed yet
1015 InitMetaProcedure temp
= new InitMetaProcedure();
1016 procedureExecutor
.submitProcedure(temp
);
1021 // initialize load balancer
1022 this.balancer
.setMasterServices(this);
1023 this.balancer
.initialize();
1024 this.balancer
.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1026 // start up all service threads.
1027 status
.setStatus("Initializing master service threads");
1028 startServiceThreads();
1029 // wait meta to be initialized after we start procedure executor
1030 if (initMetaProc
!= null) {
1031 initMetaProc
.await();
1033 // Wake up this server to check in
1034 sleeper
.skipSleepCycle();
1036 // Wait for region servers to report in.
1037 // With this as part of master initialization, it precludes our being able to start a single
1038 // server that is both Master and RegionServer. Needs more thought. TODO.
1039 String statusStr
= "Wait for region servers to report in";
1040 status
.setStatus(statusStr
);
1041 LOG
.info(Objects
.toString(status
));
1042 waitForRegionServers(status
);
1044 // Check if master is shutting down because issue initializing regionservers or balancer.
1049 status
.setStatus("Starting assignment manager");
1050 // FIRST HBASE:META READ!!!!
1051 // The below cannot make progress w/o hbase:meta being online.
1052 // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
1053 // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
1054 // if it is down. It may take a while to come online. So, wait here until meta if for sure
1055 // available. That's what waitForMetaOnline does.
1056 if (!waitForMetaOnline()) {
1060 TableDescriptor metaDescriptor
=
1061 tableDescriptors
.get(TableName
.META_TABLE_NAME
);
1062 final ColumnFamilyDescriptor tableFamilyDesc
=
1063 metaDescriptor
.getColumnFamily(HConstants
.TABLE_FAMILY
);
1064 final ColumnFamilyDescriptor replBarrierFamilyDesc
=
1065 metaDescriptor
.getColumnFamily(HConstants
.REPLICATION_BARRIER_FAMILY
);
1067 this.assignmentManager
.joinCluster();
1068 // The below depends on hbase:meta being online.
1069 this.assignmentManager
.processOfflineRegions();
1070 // this must be called after the above processOfflineRegions to prevent race
1071 this.assignmentManager
.wakeMetaLoadedEvent();
1073 // for migrating from a version without HBASE-25099, and also for honoring the configuration
1075 if (conf
.get(HConstants
.META_REPLICAS_NUM
) != null) {
1076 int replicasNumInConf
=
1077 conf
.getInt(HConstants
.META_REPLICAS_NUM
, HConstants
.DEFAULT_META_REPLICA_NUM
);
1078 TableDescriptor metaDesc
= tableDescriptors
.get(TableName
.META_TABLE_NAME
);
1079 if (metaDesc
.getRegionReplication() != replicasNumInConf
) {
1080 // it is possible that we already have some replicas before upgrading, so we must set the
1081 // region replication number in meta TableDescriptor directly first, without creating a
1082 // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas.
1083 int existingReplicasCount
=
1084 assignmentManager
.getRegionStates().getRegionsOfTable(TableName
.META_TABLE_NAME
).size();
1085 if (existingReplicasCount
> metaDesc
.getRegionReplication()) {
1086 LOG
.info("Update replica count of hbase:meta from {}(in TableDescriptor)" +
1087 " to {}(existing ZNodes)", metaDesc
.getRegionReplication(), existingReplicasCount
);
1088 metaDesc
= TableDescriptorBuilder
.newBuilder(metaDesc
)
1089 .setRegionReplication(existingReplicasCount
).build();
1090 tableDescriptors
.update(metaDesc
);
1092 // check again, and issue a ModifyTableProcedure if needed
1093 if (metaDesc
.getRegionReplication() != replicasNumInConf
) {
1095 "The {} config is {} while the replica count in TableDescriptor is {}" +
1096 " for hbase:meta, altering...",
1097 HConstants
.META_REPLICAS_NUM
, replicasNumInConf
, metaDesc
.getRegionReplication());
1098 procedureExecutor
.submitProcedure(new ModifyTableProcedure(
1099 procedureExecutor
.getEnvironment(), TableDescriptorBuilder
.newBuilder(metaDesc
)
1100 .setRegionReplication(replicasNumInConf
).build(),
1101 null, metaDesc
, false));
1105 // Initialize after meta is up as below scans meta
1106 FavoredNodesManager fnm
= getFavoredNodesManager();
1108 fnm
.initializeFromMeta();
1111 // set cluster status again after user regions are assigned
1112 this.balancer
.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1114 // Start balancer and meta catalog janitor after meta and regions have been assigned.
1115 status
.setStatus("Starting balancer and catalog janitor");
1116 this.clusterStatusChore
= new ClusterStatusChore(this, balancer
);
1117 getChoreService().scheduleChore(clusterStatusChore
);
1118 this.balancerChore
= new BalancerChore(this);
1119 if (!disableBalancerChoreForTest
) {
1120 getChoreService().scheduleChore(balancerChore
);
1122 if (regionNormalizerManager
!= null) {
1123 getChoreService().scheduleChore(regionNormalizerManager
.getRegionNormalizerChore());
1125 this.catalogJanitorChore
= new CatalogJanitor(this);
1126 getChoreService().scheduleChore(catalogJanitorChore
);
1127 this.hbckChore
= new HbckChore(this);
1128 getChoreService().scheduleChore(hbckChore
);
1129 this.serverManager
.startChore();
1131 // Only for rolling upgrade, where we need to migrate the data in namespace table to meta table.
1132 if (!waitForNamespaceOnline()) {
1135 status
.setStatus("Starting cluster schema service");
1137 initClusterSchemaService();
1138 } catch (IllegalStateException e
) {
1139 if (e
.getCause() != null && e
.getCause() instanceof NoSuchColumnFamilyException
1140 && tableFamilyDesc
== null && replBarrierFamilyDesc
== null) {
1141 LOG
.info("ClusterSchema service could not be initialized. This is "
1142 + "expected during HBase 1 to 2 upgrade", e
);
1148 if (this.cpHost
!= null) {
1150 this.cpHost
.preMasterInitialization();
1151 } catch (IOException e
) {
1152 LOG
.error("Coprocessor preMasterInitialization() hook failed", e
);
1156 status
.markComplete("Initialization successful");
1157 LOG
.info(String
.format("Master has completed initialization %.3fsec",
1158 (EnvironmentEdgeManager
.currentTime() - masterActiveTime
) / 1000.0f
));
1159 this.masterFinishedInitializationTime
= EnvironmentEdgeManager
.currentTime();
1160 configurationManager
.registerObserver(this.balancer
);
1161 configurationManager
.registerObserver(this.hfileCleanerPool
);
1162 configurationManager
.registerObserver(this.logCleanerPool
);
1163 configurationManager
.registerObserver(this.hfileCleaner
);
1164 configurationManager
.registerObserver(this.logCleaner
);
1165 configurationManager
.registerObserver(this.regionsRecoveryConfigManager
);
1166 // Set master as 'initialized'.
1167 setInitialized(true);
1169 if (tableFamilyDesc
== null && replBarrierFamilyDesc
== null) {
1170 // create missing CFs in meta table after master is set to 'initialized'.
1171 createMissingCFsInMetaDuringUpgrade(metaDescriptor
);
1173 // Throwing this Exception to abort active master is painful but this
1174 // seems the only way to add missing CFs in meta while upgrading from
1175 // HBase 1 to 2 (where HBase 2 has HBASE-23055 & HBASE-23782 checked-in).
1176 // So, why do we abort active master after adding missing CFs in meta?
1177 // When we reach here, we would have already bypassed NoSuchColumnFamilyException
1178 // in initClusterSchemaService(), meaning ClusterSchemaService is not
1179 // correctly initialized but we bypassed it. Similarly, we bypassed
1180 // tableStateManager.start() as well. Hence, we should better abort
1181 // current active master because our main task - adding missing CFs
1182 // in meta table is done (possible only after master state is set as
1183 // initialized) at the expense of bypassing few important tasks as part
1184 // of active master init routine. So now we abort active master so that
1185 // next active master init will not face any issues and all mandatory
1186 // services will be started during master init phase.
1187 throw new PleaseRestartMasterException("Aborting active master after missing"
1188 + " CFs are successfully added in meta. Subsequent active master "
1189 + "initialization should be uninterrupted");
1192 if (maintenanceMode
) {
1193 LOG
.info("Detected repair mode, skipping final initialization steps.");
1197 assignmentManager
.checkIfShouldMoveSystemRegionAsync();
1198 status
.setStatus("Starting quota manager");
1200 if (QuotaUtil
.isQuotaEnabled(conf
)) {
1201 // Create the quota snapshot notifier
1202 spaceQuotaSnapshotNotifier
= createQuotaSnapshotNotifier();
1203 spaceQuotaSnapshotNotifier
.initialize(getConnection());
1204 this.quotaObserverChore
= new QuotaObserverChore(this, getMasterMetrics());
1205 // Start the chore to read the region FS space reports and act on them
1206 getChoreService().scheduleChore(quotaObserverChore
);
1208 this.snapshotQuotaChore
= new SnapshotQuotaObserverChore(this, getMasterMetrics());
1209 // Start the chore to read snapshots and add their usage to table/NS quotas
1210 getChoreService().scheduleChore(snapshotQuotaChore
);
1212 final SlowLogMasterService slowLogMasterService
= new SlowLogMasterService(conf
, this);
1213 slowLogMasterService
.init();
1215 // clear the dead servers with same host name and port of online server because we are not
1216 // removing dead server with same hostname and port of rs which is trying to check in before
1217 // master initialization. See HBASE-5916.
1218 this.serverManager
.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
1220 // Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
1221 status
.setStatus("Checking ZNode ACLs");
1222 zooKeeper
.checkAndSetZNodeAcls();
1224 status
.setStatus("Initializing MOB Cleaner");
1227 status
.setStatus("Calling postStartMaster coprocessors");
1228 if (this.cpHost
!= null) {
1229 // don't let cp initialization errors kill the master
1231 this.cpHost
.postStartMaster();
1232 } catch (IOException ioe
) {
1233 LOG
.error("Coprocessor postStartMaster() hook failed", ioe
);
1237 zombieDetector
.interrupt();
1240 * After master has started up, lets do balancer post startup initialization. Since this runs
1241 * in activeMasterManager thread, it should be fine.
1243 long start
= EnvironmentEdgeManager
.currentTime();
1244 this.balancer
.postMasterStartupInitialize();
1245 if (LOG
.isDebugEnabled()) {
1246 LOG
.debug("Balancer post startup initialization complete, took " + (
1247 (EnvironmentEdgeManager
.currentTime() - start
) / 1000) + " seconds");
1250 this.rollingUpgradeChore
= new RollingUpgradeChore(this);
1251 getChoreService().scheduleChore(rollingUpgradeChore
);
1254 private void createMissingCFsInMetaDuringUpgrade(
1255 TableDescriptor metaDescriptor
) throws IOException
{
1256 TableDescriptor newMetaDesc
=
1257 TableDescriptorBuilder
.newBuilder(metaDescriptor
)
1258 .setColumnFamily(FSTableDescriptors
.getTableFamilyDescForMeta(conf
))
1259 .setColumnFamily(FSTableDescriptors
.getReplBarrierFamilyDescForMeta())
1261 long pid
= this.modifyTable(TableName
.META_TABLE_NAME
, () -> newMetaDesc
,
1264 while (!(getMasterProcedureExecutor().isFinished(pid
))
1265 && getMasterProcedureExecutor().isRunning() && tries
> 0) {
1268 } catch (InterruptedException e
) {
1269 throw new IOException("Wait interrupted", e
);
1274 throw new HBaseIOException(
1275 "Failed to add table and rep_barrier CFs to meta in a given time.");
1277 Procedure
<?
> result
= getMasterProcedureExecutor().getResult(pid
);
1278 if (result
!= null && result
.isFailed()) {
1279 throw new IOException(
1280 "Failed to add table and rep_barrier CFs to meta. "
1281 + MasterProcedureUtil
.unwrapRemoteIOException(result
));
1287 * Check hbase:meta is up and ready for reading. For use during Master startup only.
1288 * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
1289 * and we will hold here until operator intervention.
1291 @InterfaceAudience.Private
1292 public boolean waitForMetaOnline() {
1293 return isRegionOnline(RegionInfoBuilder
.FIRST_META_REGIONINFO
);
1297 * @return True if region is online and scannable else false if an error or shutdown (Otherwise
1298 * we just block in here holding up all forward-progess).
1300 private boolean isRegionOnline(RegionInfo ri
) {
1301 RetryCounter rc
= null;
1302 while (!isStopped()) {
1303 RegionState rs
= this.assignmentManager
.getRegionStates().getRegionState(ri
);
1304 if (rs
.isOpened()) {
1305 if (this.getServerManager().isServerOnline(rs
.getServerName())) {
1309 // Region is not OPEN.
1310 Optional
<Procedure
<MasterProcedureEnv
>> optProc
= this.procedureExecutor
.getProcedures().
1311 stream().filter(p
-> p
instanceof ServerCrashProcedure
).findAny();
1312 // TODO: Add a page to refguide on how to do repair. Have this log message point to it.
1313 // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
1314 // then how to assign including how to break region lock if one held.
1315 LOG
.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " +
1316 "progress, in holding-pattern until region onlined.",
1317 ri
.getRegionNameAsString(), rs
, optProc
.isPresent());
1318 // Check once-a-minute.
1320 rc
= new RetryCounterFactory(Integer
.MAX_VALUE
, 1000, 60_000
).create();
1322 Threads
.sleep(rc
.getBackoffTimeAndIncrementAttempts());
1328 * Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table
1330 * This is for rolling upgrading, later we will migrate the data in ns table to the ns family of
1331 * meta table. And if this is a new cluster, this method will return immediately as there will be
1332 * no namespace table/region.
1333 * @return True if namespace table is up/online.
1335 private boolean waitForNamespaceOnline() throws IOException
{
1336 TableState nsTableState
=
1337 MetaTableAccessor
.getTableState(getConnection(), TableName
.NAMESPACE_TABLE_NAME
);
1338 if (nsTableState
== null || nsTableState
.isDisabled()) {
1339 // this means we have already migrated the data and disabled or deleted the namespace table,
1340 // or this is a new deploy which does not have a namespace table from the beginning.
1343 List
<RegionInfo
> ris
=
1344 this.assignmentManager
.getRegionStates().getRegionsOfTable(TableName
.NAMESPACE_TABLE_NAME
);
1345 if (ris
.isEmpty()) {
1346 // maybe this will not happen any more, but anyway, no harm to add a check here...
1349 // Else there are namespace regions up in meta. Ensure they are assigned before we go on.
1350 for (RegionInfo ri
: ris
) {
1351 if (!isRegionOnline(ri
)) {
1359 * Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
1360 * automatically remove quotas for a table when that table is deleted.
1362 @InterfaceAudience.Private
1363 public void updateConfigurationForQuotasObserver(Configuration conf
) {
1364 // We're configured to not delete quotas on table deletion, so we don't need to add the obs.
1365 if (!conf
.getBoolean(
1366 MasterQuotasObserver
.REMOVE_QUOTA_ON_TABLE_DELETE
,
1367 MasterQuotasObserver
.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT
)) {
1370 String
[] masterCoprocs
= conf
.getStrings(CoprocessorHost
.MASTER_COPROCESSOR_CONF_KEY
);
1371 final int length
= null == masterCoprocs ?
0 : masterCoprocs
.length
;
1372 String
[] updatedCoprocs
= new String
[length
+ 1];
1374 System
.arraycopy(masterCoprocs
, 0, updatedCoprocs
, 0, masterCoprocs
.length
);
1376 updatedCoprocs
[length
] = MasterQuotasObserver
.class.getName();
1377 conf
.setStrings(CoprocessorHost
.MASTER_COPROCESSOR_CONF_KEY
, updatedCoprocs
);
1380 private void initMobCleaner() {
1381 this.mobFileCleanerChore
= new MobFileCleanerChore(this);
1382 getChoreService().scheduleChore(mobFileCleanerChore
);
1383 this.mobFileCompactionChore
= new MobFileCompactionChore(this);
1384 getChoreService().scheduleChore(mobFileCompactionChore
);
1389 * Create a {@link ServerManager} instance.
1392 * Will be overridden in tests.
1395 @InterfaceAudience.Private
1396 protected ServerManager
createServerManager(MasterServices master
,
1397 RegionServerList storage
) throws IOException
{
1398 // We put this out here in a method so can do a Mockito.spy and stub it out
1399 // w/ a mocked up ServerManager.
1400 setupClusterConnection();
1401 return new ServerManager(master
, storage
);
1404 private void waitForRegionServers(final MonitoredTask status
)
1405 throws IOException
, InterruptedException
{
1406 this.serverManager
.waitForRegionServers(status
);
1409 // Will be overridden in tests
1410 @InterfaceAudience.Private
1411 protected void initClusterSchemaService() throws IOException
, InterruptedException
{
1412 this.clusterSchemaService
= new ClusterSchemaServiceImpl(this);
1413 this.clusterSchemaService
.startAsync();
1415 this.clusterSchemaService
.awaitRunning(getConfiguration().getInt(
1416 HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
,
1417 DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
), TimeUnit
.SECONDS
);
1418 } catch (TimeoutException toe
) {
1419 throw new IOException("Timedout starting ClusterSchemaService", toe
);
1423 private void initQuotaManager() throws IOException
{
1424 MasterQuotaManager quotaManager
= new MasterQuotaManager(this);
1425 quotaManager
.start();
1426 this.quotaManager
= quotaManager
;
1429 private SpaceQuotaSnapshotNotifier
createQuotaSnapshotNotifier() {
1430 SpaceQuotaSnapshotNotifier notifier
=
1431 SpaceQuotaSnapshotNotifierFactory
.getInstance().create(getConfiguration());
1435 public boolean isCatalogJanitorEnabled() {
1436 return catalogJanitorChore
!= null ? catalogJanitorChore
.getEnabled() : false;
1439 boolean isCleanerChoreEnabled() {
1440 boolean hfileCleanerFlag
= true, logCleanerFlag
= true;
1442 if (hfileCleaner
!= null) {
1443 hfileCleanerFlag
= hfileCleaner
.getEnabled();
1446 if (logCleaner
!= null) {
1447 logCleanerFlag
= logCleaner
.getEnabled();
1450 return (hfileCleanerFlag
&& logCleanerFlag
);
1454 public ServerManager
getServerManager() {
1455 return this.serverManager
;
1459 public MasterFileSystem
getMasterFileSystem() {
1460 return this.fileSystemManager
;
1464 public MasterWalManager
getMasterWalManager() {
1465 return this.walManager
;
1469 public SplitWALManager
getSplitWALManager() {
1470 return splitWALManager
;
1474 public TableStateManager
getTableStateManager() {
1475 return tableStateManager
;
1479 * Start up all services. If any of these threads gets an unhandled exception
1480 * then they just die with a logged message. This should be fine because
1481 * in general, we do not expect the master to get such unhandled exceptions
1482 * as OOMEs; it should be lightly loaded. See what HRegionServer does if
1483 * need to install an unexpected exception handler.
1485 private void startServiceThreads() throws IOException
{
1486 // Start the executor service pools
1487 final int masterOpenRegionPoolSize
= conf
.getInt(
1488 HConstants
.MASTER_OPEN_REGION_THREADS
, HConstants
.MASTER_OPEN_REGION_THREADS_DEFAULT
);
1489 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1490 ExecutorType
.MASTER_OPEN_REGION
).setCorePoolSize(masterOpenRegionPoolSize
));
1491 final int masterCloseRegionPoolSize
= conf
.getInt(
1492 HConstants
.MASTER_CLOSE_REGION_THREADS
, HConstants
.MASTER_CLOSE_REGION_THREADS_DEFAULT
);
1493 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1494 ExecutorType
.MASTER_CLOSE_REGION
).setCorePoolSize(masterCloseRegionPoolSize
));
1495 final int masterServerOpThreads
= conf
.getInt(HConstants
.MASTER_SERVER_OPERATIONS_THREADS
,
1496 HConstants
.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT
);
1497 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1498 ExecutorType
.MASTER_SERVER_OPERATIONS
).setCorePoolSize(masterServerOpThreads
));
1499 final int masterServerMetaOpsThreads
= conf
.getInt(
1500 HConstants
.MASTER_META_SERVER_OPERATIONS_THREADS
,
1501 HConstants
.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT
);
1502 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1503 ExecutorType
.MASTER_META_SERVER_OPERATIONS
).setCorePoolSize(masterServerMetaOpsThreads
));
1504 final int masterLogReplayThreads
= conf
.getInt(
1505 HConstants
.MASTER_LOG_REPLAY_OPS_THREADS
, HConstants
.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT
);
1506 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1507 ExecutorType
.M_LOG_REPLAY_OPS
).setCorePoolSize(masterLogReplayThreads
));
1508 final int masterSnapshotThreads
= conf
.getInt(
1509 SnapshotManager
.SNAPSHOT_POOL_THREADS_KEY
, SnapshotManager
.SNAPSHOT_POOL_THREADS_DEFAULT
);
1510 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1511 ExecutorType
.MASTER_SNAPSHOT_OPERATIONS
).setCorePoolSize(masterSnapshotThreads
)
1512 .setAllowCoreThreadTimeout(true));
1513 final int masterMergeDispatchThreads
= conf
.getInt(HConstants
.MASTER_MERGE_DISPATCH_THREADS
,
1514 HConstants
.MASTER_MERGE_DISPATCH_THREADS_DEFAULT
);
1515 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1516 ExecutorType
.MASTER_MERGE_OPERATIONS
).setCorePoolSize(masterMergeDispatchThreads
)
1517 .setAllowCoreThreadTimeout(true));
1519 // We depend on there being only one instance of this executor running
1520 // at a time. To do concurrency, would need fencing of enable/disable of
1522 // Any time changing this maxThreads to > 1, pls see the comment at
1523 // AccessController#postCompletedCreateTableAction
1524 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1525 ExecutorType
.MASTER_TABLE_OPERATIONS
).setCorePoolSize(1));
1526 startProcedureExecutor();
1528 // Create log cleaner thread pool
1529 logCleanerPool
= DirScanPool
.getLogCleanerScanPool(conf
);
1530 Map
<String
, Object
> params
= new HashMap
<>();
1531 params
.put(MASTER
, this);
1532 // Start log cleaner thread
1533 int cleanerInterval
=
1534 conf
.getInt(HBASE_MASTER_CLEANER_INTERVAL
, DEFAULT_HBASE_MASTER_CLEANER_INTERVAL
);
1535 this.logCleaner
= new LogCleaner(cleanerInterval
, this, conf
,
1536 getMasterWalManager().getFileSystem(), getMasterWalManager().getOldLogDir(),
1537 logCleanerPool
, params
);
1538 getChoreService().scheduleChore(logCleaner
);
1540 // start the hfile archive cleaner thread
1541 Path archiveDir
= HFileArchiveUtil
.getArchivePath(conf
);
1542 // Create archive cleaner thread pool
1543 hfileCleanerPool
= DirScanPool
.getHFileCleanerScanPool(conf
);
1544 this.hfileCleaner
= new HFileCleaner(cleanerInterval
, this, conf
,
1545 getMasterFileSystem().getFileSystem(), archiveDir
, hfileCleanerPool
, params
);
1546 getChoreService().scheduleChore(hfileCleaner
);
1548 // Regions Reopen based on very high storeFileRefCount is considered enabled
1549 // only if hbase.regions.recovery.store.file.ref.count has value > 0
1550 final int maxStoreFileRefCount
= conf
.getInt(
1551 HConstants
.STORE_FILE_REF_COUNT_THRESHOLD
,
1552 HConstants
.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD
);
1553 if (maxStoreFileRefCount
> 0) {
1554 this.regionsRecoveryChore
= new RegionsRecoveryChore(this, conf
, this);
1555 getChoreService().scheduleChore(this.regionsRecoveryChore
);
1557 LOG
.info("Reopening regions with very high storeFileRefCount is disabled. " +
1558 "Provide threshold value > 0 for {} to enable it.",
1559 HConstants
.STORE_FILE_REF_COUNT_THRESHOLD
);
1562 this.regionsRecoveryConfigManager
= new RegionsRecoveryConfigManager(this);
1564 replicationBarrierCleaner
= new ReplicationBarrierCleaner(conf
, this, getConnection(),
1565 replicationPeerManager
);
1566 getChoreService().scheduleChore(replicationBarrierCleaner
);
1568 final boolean isSnapshotChoreEnabled
= this.snapshotCleanupTracker
1569 .isSnapshotCleanupEnabled();
1570 this.snapshotCleanerChore
= new SnapshotCleanerChore(this, conf
, getSnapshotManager());
1571 if (isSnapshotChoreEnabled
) {
1572 getChoreService().scheduleChore(this.snapshotCleanerChore
);
1574 if (LOG
.isTraceEnabled()) {
1575 LOG
.trace("Snapshot Cleaner Chore is disabled. Not starting up the chore..");
1578 serviceStarted
= true;
1579 if (LOG
.isTraceEnabled()) {
1580 LOG
.trace("Started service threads");
1584 protected void stopServiceThreads() {
1585 if (masterJettyServer
!= null) {
1586 LOG
.info("Stopping master jetty server");
1588 masterJettyServer
.stop();
1589 } catch (Exception e
) {
1590 LOG
.error("Failed to stop master jetty server", e
);
1594 stopExecutorService();
1595 if (hfileCleanerPool
!= null) {
1596 hfileCleanerPool
.shutdownNow();
1597 hfileCleanerPool
= null;
1599 if (logCleanerPool
!= null) {
1600 logCleanerPool
.shutdownNow();
1601 logCleanerPool
= null;
1603 if (maintenanceRegionServer
!= null) {
1604 maintenanceRegionServer
.getRegionServer().stop(HBASE_MASTER_CLEANER_INTERVAL
);
1607 LOG
.debug("Stopping service threads");
1608 // stop procedure executor prior to other services such as server manager and assignment
1609 // manager, as these services are important for some running procedures. See HBASE-24117 for
1611 stopProcedureExecutor();
1613 if (regionNormalizerManager
!= null) {
1614 regionNormalizerManager
.stop();
1616 if (this.quotaManager
!= null) {
1617 this.quotaManager
.stop();
1620 if (this.activeMasterManager
!= null) {
1621 this.activeMasterManager
.stop();
1623 if (this.serverManager
!= null) {
1624 this.serverManager
.stop();
1626 if (this.assignmentManager
!= null) {
1627 this.assignmentManager
.stop();
1630 if (masterRegion
!= null) {
1631 masterRegion
.close(isAborted());
1633 if (this.walManager
!= null) {
1634 this.walManager
.stop();
1636 if (this.fileSystemManager
!= null) {
1637 this.fileSystemManager
.stop();
1639 if (this.mpmHost
!= null) {
1640 this.mpmHost
.stop("server shutting down.");
1642 if (this.regionServerTracker
!= null) {
1643 this.regionServerTracker
.stop();
1647 private void createProcedureExecutor() throws IOException
{
1648 MasterProcedureEnv procEnv
= new MasterProcedureEnv(this);
1650 new RegionProcedureStore(this, masterRegion
, new MasterProcedureEnv
.FsUtilsLeaseRecovery(this));
1651 procedureStore
.registerListener(new ProcedureStoreListener() {
1654 public void abortProcess() {
1655 abort("The Procedure Store lost the lease", null);
1658 MasterProcedureScheduler procedureScheduler
= procEnv
.getProcedureScheduler();
1659 procedureExecutor
= new ProcedureExecutor
<>(conf
, procEnv
, procedureStore
, procedureScheduler
);
1660 configurationManager
.registerObserver(procEnv
);
1662 int cpus
= Runtime
.getRuntime().availableProcessors();
1663 final int numThreads
= conf
.getInt(MasterProcedureConstants
.MASTER_PROCEDURE_THREADS
, Math
.max(
1664 (cpus
> 0 ? cpus
/ 4 : 0), MasterProcedureConstants
.DEFAULT_MIN_MASTER_PROCEDURE_THREADS
));
1665 final boolean abortOnCorruption
=
1666 conf
.getBoolean(MasterProcedureConstants
.EXECUTOR_ABORT_ON_CORRUPTION
,
1667 MasterProcedureConstants
.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION
);
1668 procedureStore
.start(numThreads
);
1669 // Just initialize it but do not start the workers, we will start the workers later by calling
1670 // startProcedureExecutor. See the javadoc for finishActiveMasterInitialization for more
1672 procedureExecutor
.init(numThreads
, abortOnCorruption
);
1673 if (!procEnv
.getRemoteDispatcher().start()) {
1674 throw new HBaseIOException("Failed start of remote dispatcher");
1678 // will be override in UT
1679 protected void startProcedureExecutor() throws IOException
{
1680 procedureExecutor
.startWorkers();
1684 * Turn on/off Snapshot Cleanup Chore
1686 * @param on indicates whether Snapshot Cleanup Chore is to be run
1688 void switchSnapshotCleanup(final boolean on
, final boolean synchronous
) {
1690 synchronized (this.snapshotCleanerChore
) {
1691 switchSnapshotCleanup(on
);
1694 switchSnapshotCleanup(on
);
1698 private void switchSnapshotCleanup(final boolean on
) {
1700 snapshotCleanupTracker
.setSnapshotCleanupEnabled(on
);
1702 getChoreService().scheduleChore(this.snapshotCleanerChore
);
1704 this.snapshotCleanerChore
.cancel();
1706 } catch (KeeperException e
) {
1707 LOG
.error("Error updating snapshot cleanup mode to {}", on
, e
);
1712 private void stopProcedureExecutor() {
1713 if (procedureExecutor
!= null) {
1714 configurationManager
.deregisterObserver(procedureExecutor
.getEnvironment());
1715 procedureExecutor
.getEnvironment().getRemoteDispatcher().stop();
1716 procedureExecutor
.stop();
1717 procedureExecutor
.join();
1718 procedureExecutor
= null;
1721 if (procedureStore
!= null) {
1722 procedureStore
.stop(isAborted());
1723 procedureStore
= null;
1727 protected void stopChores() {
1728 shutdownChore(mobFileCleanerChore
);
1729 shutdownChore(mobFileCompactionChore
);
1730 shutdownChore(balancerChore
);
1731 if (regionNormalizerManager
!= null) {
1732 shutdownChore(regionNormalizerManager
.getRegionNormalizerChore());
1734 shutdownChore(clusterStatusChore
);
1735 shutdownChore(catalogJanitorChore
);
1736 shutdownChore(clusterStatusPublisherChore
);
1737 shutdownChore(snapshotQuotaChore
);
1738 shutdownChore(logCleaner
);
1739 shutdownChore(hfileCleaner
);
1740 shutdownChore(replicationBarrierCleaner
);
1741 shutdownChore(snapshotCleanerChore
);
1742 shutdownChore(hbckChore
);
1743 shutdownChore(regionsRecoveryChore
);
1744 shutdownChore(rollingUpgradeChore
);
1748 * @return Get remote side's InetAddress
1750 InetAddress
getRemoteInetAddress(final int port
,
1751 final long serverStartCode
) throws UnknownHostException
{
1752 // Do it out here in its own little method so can fake an address when
1753 // mocking up in tests.
1754 InetAddress ia
= RpcServer
.getRemoteIp();
1756 // The call could be from the local regionserver,
1757 // in which case, there is no remote address.
1758 if (ia
== null && serverStartCode
== startcode
) {
1759 InetSocketAddress isa
= rpcServices
.getSocketAddress();
1760 if (isa
!= null && isa
.getPort() == port
) {
1761 ia
= isa
.getAddress();
1768 * @return Maximum time we should run balancer for
1770 private int getMaxBalancingTime() {
1771 // if max balancing time isn't set, defaulting it to period time
1772 int maxBalancingTime
= getConfiguration().getInt(HConstants
.HBASE_BALANCER_MAX_BALANCING
,
1774 .getInt(HConstants
.HBASE_BALANCER_PERIOD
, HConstants
.DEFAULT_HBASE_BALANCER_PERIOD
));
1775 return maxBalancingTime
;
1779 * @return Maximum number of regions in transition
1781 private int getMaxRegionsInTransition() {
1782 int numRegions
= this.assignmentManager
.getRegionStates().getRegionAssignments().size();
1783 return Math
.max((int) Math
.floor(numRegions
* this.maxRitPercent
), 1);
1787 * It first sleep to the next balance plan start time. Meanwhile, throttling by the max
1788 * number regions in transition to protect availability.
1789 * @param nextBalanceStartTime The next balance plan start time
1790 * @param maxRegionsInTransition max number of regions in transition
1791 * @param cutoffTime when to exit balancer
1793 private void balanceThrottling(long nextBalanceStartTime
, int maxRegionsInTransition
,
1795 boolean interrupted
= false;
1797 // Sleep to next balance plan start time
1798 // But if there are zero regions in transition, it can skip sleep to speed up.
1799 while (!interrupted
&& EnvironmentEdgeManager
.currentTime() < nextBalanceStartTime
1800 && this.assignmentManager
.getRegionStates().hasRegionsInTransition()) {
1803 } catch (InterruptedException ie
) {
1808 // Throttling by max number regions in transition
1810 && maxRegionsInTransition
> 0
1811 && this.assignmentManager
.getRegionStates().getRegionsInTransitionCount()
1812 >= maxRegionsInTransition
&& EnvironmentEdgeManager
.currentTime() <= cutoffTime
) {
1814 // sleep if the number of regions in transition exceeds the limit
1816 } catch (InterruptedException ie
) {
1821 if (interrupted
) Thread
.currentThread().interrupt();
1824 public BalanceResponse
balance() throws IOException
{
1825 return balance(BalanceRequest
.defaultInstance());
1829 * Trigger a normal balance, see {@link HMaster#balance()} . If the balance is not executed
1830 * this time, the metrics related to the balance will be updated.
1832 * When balance is running, related metrics will be updated at the same time. But if some
1833 * checking logic failed and cause the balancer exit early, we lost the chance to update
1834 * balancer metrics. This will lead to user missing the latest balancer info.
1836 public BalanceResponse
balanceOrUpdateMetrics() throws IOException
{
1837 synchronized (this.balancer
) {
1838 BalanceResponse response
= balance();
1839 if (!response
.isBalancerRan()) {
1840 Map
<TableName
, Map
<ServerName
, List
<RegionInfo
>>> assignments
=
1841 this.assignmentManager
.getRegionStates().getAssignmentsForBalancer(this.tableStateManager
,
1842 this.serverManager
.getOnlineServersList());
1843 for (Map
<ServerName
, List
<RegionInfo
>> serverMap
: assignments
.values()) {
1844 serverMap
.keySet().removeAll(this.serverManager
.getDrainingServersList());
1846 this.balancer
.updateBalancerLoadInfo(assignments
);
1853 * Checks master state before initiating action over region topology.
1854 * @param action the name of the action under consideration, for logging.
1855 * @return {@code true} when the caller should exit early, {@code false} otherwise.
1858 public boolean skipRegionManagementAction(final String action
) {
1859 // Note: this method could be `default` on MasterServices if but for logging.
1860 if (!isInitialized()) {
1861 LOG
.debug("Master has not been initialized, don't run {}.", action
);
1864 if (this.getServerManager().isClusterShutdown()) {
1865 LOG
.info("Cluster is shutting down, don't run {}.", action
);
1868 if (isInMaintenanceMode()) {
1869 LOG
.info("Master is in maintenance mode, don't run {}.", action
);
1875 public BalanceResponse
balance(BalanceRequest request
) throws IOException
{
1878 BalanceResponse
.Builder responseBuilder
= BalanceResponse
.newBuilder();
1880 if (loadBalancerTracker
== null
1881 || !(loadBalancerTracker
.isBalancerOn() || request
.isDryRun())) {
1882 return responseBuilder
.build();
1885 if (skipRegionManagementAction("balancer")) {
1886 return responseBuilder
.build();
1889 synchronized (this.balancer
) {
1890 // Only allow one balance run at at time.
1891 if (this.assignmentManager
.hasRegionsInTransition()) {
1892 List
<RegionStateNode
> regionsInTransition
= assignmentManager
.getRegionsInTransition();
1893 // if hbase:meta region is in transition, result of assignment cannot be recorded
1894 // ignore the force flag in that case
1895 boolean metaInTransition
= assignmentManager
.isMetaRegionInTransition();
1896 List
<RegionStateNode
> toPrint
= regionsInTransition
;
1898 boolean truncated
= false;
1899 if (regionsInTransition
.size() > max
) {
1900 toPrint
= regionsInTransition
.subList(0, max
);
1904 if (!request
.isIgnoreRegionsInTransition() || metaInTransition
) {
1905 LOG
.info("Not running balancer (ignoreRIT=false" + ", metaRIT=" + metaInTransition
+
1906 ") because " + regionsInTransition
.size() + " region(s) in transition: " + toPrint
1907 + (truncated?
"(truncated list)": ""));
1908 return responseBuilder
.build();
1911 if (this.serverManager
.areDeadServersInProgress()) {
1912 LOG
.info("Not running balancer because processing dead regionserver(s): " +
1913 this.serverManager
.getDeadServers());
1914 return responseBuilder
.build();
1917 if (this.cpHost
!= null) {
1919 if (this.cpHost
.preBalance(request
)) {
1920 LOG
.debug("Coprocessor bypassing balancer request");
1921 return responseBuilder
.build();
1923 } catch (IOException ioe
) {
1924 LOG
.error("Error invoking master coprocessor preBalance()", ioe
);
1925 return responseBuilder
.build();
1929 Map
<TableName
, Map
<ServerName
, List
<RegionInfo
>>> assignments
=
1930 this.assignmentManager
.getRegionStates()
1931 .getAssignmentsForBalancer(tableStateManager
, this.serverManager
.getOnlineServersList());
1932 for (Map
<ServerName
, List
<RegionInfo
>> serverMap
: assignments
.values()) {
1933 serverMap
.keySet().removeAll(this.serverManager
.getDrainingServersList());
1936 //Give the balancer the current cluster state.
1937 this.balancer
.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1939 List
<RegionPlan
> plans
= this.balancer
.balanceCluster(assignments
);
1941 responseBuilder
.setBalancerRan(true).setMovesCalculated(plans
== null ?
0 : plans
.size());
1943 if (skipRegionManagementAction("balancer")) {
1944 // make one last check that the cluster isn't shutting down before proceeding.
1945 return responseBuilder
.build();
1948 // For dry run we don't actually want to execute the moves, but we do want
1949 // to execute the coprocessor below
1950 List
<RegionPlan
> sucRPs
= request
.isDryRun()
1951 ? Collections
.emptyList()
1952 : executeRegionPlansWithThrottling(plans
);
1954 if (this.cpHost
!= null) {
1956 this.cpHost
.postBalance(request
, sucRPs
);
1957 } catch (IOException ioe
) {
1958 // balancing already succeeded so don't change the result
1959 LOG
.error("Error invoking master coprocessor postBalance()", ioe
);
1963 responseBuilder
.setMovesExecuted(sucRPs
.size());
1966 // If LoadBalancer did not generate any plans, it means the cluster is already balanced.
1967 // Return true indicating a success.
1968 return responseBuilder
.build();
1972 * Execute region plans with throttling
1973 * @param plans to execute
1974 * @return succeeded plans
1976 public List
<RegionPlan
> executeRegionPlansWithThrottling(List
<RegionPlan
> plans
) {
1977 List
<RegionPlan
> successRegionPlans
= new ArrayList
<>();
1978 int maxRegionsInTransition
= getMaxRegionsInTransition();
1979 long balanceStartTime
= EnvironmentEdgeManager
.currentTime();
1980 long cutoffTime
= balanceStartTime
+ this.maxBalancingTime
;
1981 int rpCount
= 0; // number of RegionPlans balanced so far
1982 if (plans
!= null && !plans
.isEmpty()) {
1983 int balanceInterval
= this.maxBalancingTime
/ plans
.size();
1984 LOG
.info("Balancer plans size is " + plans
.size() + ", the balance interval is "
1985 + balanceInterval
+ " ms, and the max number regions in transition is "
1986 + maxRegionsInTransition
);
1988 for (RegionPlan plan
: plans
) {
1989 LOG
.info("balance " + plan
);
1992 this.assignmentManager
.balance(plan
);
1993 } catch (HBaseIOException hioe
) {
1994 //should ignore failed plans here, avoiding the whole balance plans be aborted
1995 //later calls of balance() can fetch up the failed and skipped plans
1996 LOG
.warn("Failed balance plan {}, skipping...", plan
, hioe
);
1998 //rpCount records balance plans processed, does not care if a plan succeeds
2000 successRegionPlans
.add(plan
);
2002 if (this.maxBalancingTime
> 0) {
2003 balanceThrottling(balanceStartTime
+ rpCount
* balanceInterval
, maxRegionsInTransition
,
2007 // if performing next balance exceeds cutoff time, exit the loop
2008 if (this.maxBalancingTime
> 0 && rpCount
< plans
.size()
2009 && EnvironmentEdgeManager
.currentTime() > cutoffTime
) {
2010 // TODO: After balance, there should not be a cutoff time (keeping it as
2011 // a security net for now)
2012 LOG
.debug("No more balancing till next balance run; maxBalanceTime="
2013 + this.maxBalancingTime
);
2018 LOG
.debug("Balancer is going into sleep until next period in {}ms", getConfiguration()
2019 .getInt(HConstants
.HBASE_BALANCER_PERIOD
, HConstants
.DEFAULT_HBASE_BALANCER_PERIOD
));
2020 return successRegionPlans
;
2024 public RegionNormalizerManager
getRegionNormalizerManager() {
2025 return regionNormalizerManager
;
2029 public boolean normalizeRegions(
2030 final NormalizeTableFilterParams ntfp
,
2031 final boolean isHighPriority
2032 ) throws IOException
{
2033 if (regionNormalizerManager
== null || !regionNormalizerManager
.isNormalizerOn()) {
2034 LOG
.debug("Region normalization is disabled, don't run region normalizer.");
2037 if (skipRegionManagementAction("region normalizer")) {
2040 if (assignmentManager
.hasRegionsInTransition()) {
2044 final Set
<TableName
> matchingTables
= getTableDescriptors(new LinkedList
<>(),
2045 ntfp
.getNamespace(), ntfp
.getRegex(), ntfp
.getTableNames(), false)
2047 .map(TableDescriptor
::getTableName
)
2048 .collect(Collectors
.toSet());
2049 final Set
<TableName
> allEnabledTables
=
2050 tableStateManager
.getTablesInStates(TableState
.State
.ENABLED
);
2051 final List
<TableName
> targetTables
=
2052 new ArrayList
<>(Sets
.intersection(matchingTables
, allEnabledTables
));
2053 Collections
.shuffle(targetTables
);
2054 return regionNormalizerManager
.normalizeRegions(targetTables
, isHighPriority
);
2058 * @return Client info for use as prefix on an audit log string; who did an action
2061 public String
getClientIdAuditPrefix() {
2062 return "Client=" + RpcServer
.getRequestUserName().orElse(null)
2063 + "/" + RpcServer
.getRemoteAddress().orElse(null);
2067 * Switch for the background CatalogJanitor thread.
2068 * Used for testing. The thread will continue to run. It will just be a noop
2070 * @param b If false, the catalog janitor won't do anything.
2072 public void setCatalogJanitorEnabled(final boolean b
) {
2073 this.catalogJanitorChore
.setEnabled(b
);
2077 public long mergeRegions(
2078 final RegionInfo
[] regionsToMerge
,
2079 final boolean forcible
,
2081 final long nonce
) throws IOException
{
2084 if (!isSplitOrMergeEnabled(MasterSwitchType
.MERGE
)) {
2085 String regionsStr
= Arrays
.deepToString(regionsToMerge
);
2086 LOG
.warn("Merge switch is off! skip merge of " + regionsStr
);
2087 throw new DoNotRetryIOException("Merge of " + regionsStr
+
2088 " failed because merge switch is off");
2091 final String mergeRegionsStr
= Arrays
.stream(regionsToMerge
).map(RegionInfo
::getEncodedName
)
2092 .collect(Collectors
.joining(", "));
2093 return MasterProcedureUtil
.submitProcedure(new NonceProcedureRunnable(this, ng
, nonce
) {
2095 protected void run() throws IOException
{
2096 getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge
);
2097 String aid
= getClientIdAuditPrefix();
2098 LOG
.info("{} merge regions {}", aid
, mergeRegionsStr
);
2099 submitProcedure(new MergeTableRegionsProcedure(procedureExecutor
.getEnvironment(),
2100 regionsToMerge
, forcible
));
2101 getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge
);
2105 protected String
getDescription() {
2106 return "MergeTableProcedure";
2112 public long splitRegion(final RegionInfo regionInfo
, final byte[] splitRow
,
2113 final long nonceGroup
, final long nonce
)
2114 throws IOException
{
2117 if (!isSplitOrMergeEnabled(MasterSwitchType
.SPLIT
)) {
2118 LOG
.warn("Split switch is off! skip split of " + regionInfo
);
2119 throw new DoNotRetryIOException("Split region " + regionInfo
.getRegionNameAsString() +
2120 " failed due to split switch off");
2123 return MasterProcedureUtil
.submitProcedure(
2124 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2126 protected void run() throws IOException
{
2127 getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo
.getTable(), splitRow
);
2128 LOG
.info(getClientIdAuditPrefix() + " split " + regionInfo
.getRegionNameAsString());
2130 // Execute the operation asynchronously
2131 submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo
, splitRow
));
2135 protected String
getDescription() {
2136 return "SplitTableProcedure";
2141 private void warmUpRegion(ServerName server
, RegionInfo region
) {
2142 FutureUtils
.addListener(asyncClusterConnection
.getRegionServerAdmin(server
)
2143 .warmupRegion(RequestConverter
.buildWarmupRegionRequest(region
)), (r
, e
) -> {
2145 LOG
.warn("Failed to warm up region {} on server {}", region
, server
, e
);
2150 // Public so can be accessed by tests. Blocks until move is done.
2151 // Replace with an async implementation from which you can get
2152 // a success/failure result.
2153 @InterfaceAudience.Private
2154 public void move(final byte[] encodedRegionName
, byte[] destServerName
) throws IOException
{
2155 RegionState regionState
= assignmentManager
.getRegionStates().
2156 getRegionState(Bytes
.toString(encodedRegionName
));
2159 if (regionState
!= null) {
2160 hri
= regionState
.getRegion();
2162 throw new UnknownRegionException(Bytes
.toStringBinary(encodedRegionName
));
2166 List
<ServerName
> exclude
= hri
.getTable().isSystemTable() ? assignmentManager
.getExcludedServersForSystemTable()
2167 : new ArrayList
<>(1);
2168 if (destServerName
!= null && exclude
.contains(ServerName
.valueOf(Bytes
.toString(destServerName
)))) {
2170 Bytes
.toString(encodedRegionName
) + " can not move to " + Bytes
.toString(destServerName
)
2171 + " because the server is in exclude list");
2172 destServerName
= null;
2174 if (destServerName
== null || destServerName
.length
== 0) {
2175 LOG
.info("Passed destination servername is null/empty so " +
2176 "choosing a server at random");
2177 exclude
.add(regionState
.getServerName());
2178 final List
<ServerName
> destServers
= this.serverManager
.createDestinationServersList(exclude
);
2179 dest
= balancer
.randomAssignment(hri
, destServers
);
2181 LOG
.debug("Unable to determine a plan to assign " + hri
);
2185 ServerName candidate
= ServerName
.valueOf(Bytes
.toString(destServerName
));
2186 dest
= balancer
.randomAssignment(hri
, Lists
.newArrayList(candidate
));
2188 LOG
.debug("Unable to determine a plan to assign " + hri
);
2191 // TODO: deal with table on master for rs group.
2192 if (dest
.equals(serverName
)) {
2193 // To avoid unnecessary region moving later by balancer. Don't put user
2194 // regions on master.
2195 LOG
.debug("Skipping move of region " + hri
.getRegionNameAsString() +
2196 " to avoid unnecessary region moving later by load balancer," +
2197 " because it should not be on master");
2202 if (dest
.equals(regionState
.getServerName())) {
2203 LOG
.debug("Skipping move of region " + hri
.getRegionNameAsString()
2204 + " because region already assigned to the same server " + dest
+ ".");
2208 // Now we can do the move
2209 RegionPlan rp
= new RegionPlan(hri
, regionState
.getServerName(), dest
);
2210 assert rp
.getDestination() != null: rp
.toString() + " " + dest
;
2214 if (this.cpHost
!= null) {
2215 this.cpHost
.preMove(hri
, rp
.getSource(), rp
.getDestination());
2218 TransitRegionStateProcedure proc
=
2219 this.assignmentManager
.createMoveRegionProcedure(rp
.getRegionInfo(), rp
.getDestination());
2220 if (conf
.getBoolean(WARMUP_BEFORE_MOVE
, DEFAULT_WARMUP_BEFORE_MOVE
)) {
2221 // Warmup the region on the destination before initiating the move.
2222 // A region server could reject the close request because it either does not
2223 // have the specified region or the region is being split.
2224 LOG
.info(getClientIdAuditPrefix() + " move " + rp
+ ", warming up region on " +
2225 rp
.getDestination());
2226 warmUpRegion(rp
.getDestination(), hri
);
2228 LOG
.info(getClientIdAuditPrefix() + " move " + rp
+ ", running balancer");
2229 Future
<byte[]> future
= ProcedureSyncWait
.submitProcedure(this.procedureExecutor
, proc
);
2231 // Is this going to work? Will we throw exception on error?
2232 // TODO: CompletableFuture rather than this stunted Future.
2234 } catch (InterruptedException
| ExecutionException e
) {
2235 throw new HBaseIOException(e
);
2237 if (this.cpHost
!= null) {
2238 this.cpHost
.postMove(hri
, rp
.getSource(), rp
.getDestination());
2240 } catch (IOException ioe
) {
2241 if (ioe
instanceof HBaseIOException
) {
2242 throw (HBaseIOException
)ioe
;
2244 throw new HBaseIOException(ioe
);
2249 public long createTable(final TableDescriptor tableDescriptor
, final byte[][] splitKeys
,
2250 final long nonceGroup
, final long nonce
) throws IOException
{
2252 TableDescriptor desc
= getMasterCoprocessorHost().preCreateTableRegionsInfos(tableDescriptor
);
2254 throw new IOException("Creation for " + tableDescriptor
+ " is canceled by CP");
2256 String namespace
= desc
.getTableName().getNamespaceAsString();
2257 this.clusterSchemaService
.getNamespace(namespace
);
2259 RegionInfo
[] newRegions
= ModifyRegionUtils
.createRegionInfos(desc
, splitKeys
);
2260 TableDescriptorChecker
.sanityCheck(conf
, desc
);
2262 return MasterProcedureUtil
2263 .submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2265 protected void run() throws IOException
{
2266 getMaster().getMasterCoprocessorHost().preCreateTable(desc
, newRegions
);
2268 LOG
.info(getClientIdAuditPrefix() + " create " + desc
);
2270 // TODO: We can handle/merge duplicate requests, and differentiate the case of
2271 // TableExistsException by saying if the schema is the same or not.
2273 // We need to wait for the procedure to potentially fail due to "prepare" sanity
2274 // checks. This will block only the beginning of the procedure. See HBASE-19953.
2275 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
2277 new CreateTableProcedure(procedureExecutor
.getEnvironment(), desc
, newRegions
, latch
));
2280 getMaster().getMasterCoprocessorHost().postCreateTable(desc
, newRegions
);
2284 protected String
getDescription() {
2285 return "CreateTableProcedure";
2291 public long createSystemTable(final TableDescriptor tableDescriptor
) throws IOException
{
2293 throw new MasterNotRunningException();
2296 TableName tableName
= tableDescriptor
.getTableName();
2297 if (!(tableName
.isSystemTable())) {
2298 throw new IllegalArgumentException(
2299 "Only system table creation can use this createSystemTable API");
2302 RegionInfo
[] newRegions
= ModifyRegionUtils
.createRegionInfos(tableDescriptor
, null);
2304 LOG
.info(getClientIdAuditPrefix() + " create " + tableDescriptor
);
2306 // This special create table is called locally to master. Therefore, no RPC means no need
2307 // to use nonce to detect duplicated RPC call.
2308 long procId
= this.procedureExecutor
.submitProcedure(
2309 new CreateTableProcedure(procedureExecutor
.getEnvironment(), tableDescriptor
, newRegions
));
2314 private void startActiveMasterManager(int infoPort
) throws KeeperException
{
2315 String backupZNode
= ZNodePaths
.joinZNode(
2316 zooKeeper
.getZNodePaths().backupMasterAddressesZNode
, serverName
.toString());
2318 * Add a ZNode for ourselves in the backup master directory since we
2319 * may not become the active master. If so, we want the actual active
2320 * master to know we are backup masters, so that it won't assign
2321 * regions to us if so configured.
2323 * If we become the active master later, ActiveMasterManager will delete
2324 * this node explicitly. If we crash before then, ZooKeeper will delete
2325 * this node for us since it is ephemeral.
2327 LOG
.info("Adding backup master ZNode " + backupZNode
);
2328 if (!MasterAddressTracker
.setMasterAddress(zooKeeper
, backupZNode
, serverName
, infoPort
)) {
2329 LOG
.warn("Failed create of " + backupZNode
+ " by " + serverName
);
2331 this.activeMasterManager
.setInfoPort(infoPort
);
2332 int timeout
= conf
.getInt(HConstants
.ZK_SESSION_TIMEOUT
, HConstants
.DEFAULT_ZK_SESSION_TIMEOUT
);
2333 // If we're a backup master, stall until a primary to write this address
2334 if (conf
.getBoolean(HConstants
.MASTER_TYPE_BACKUP
, HConstants
.DEFAULT_MASTER_TYPE_BACKUP
)) {
2335 LOG
.debug("HMaster started in backup mode. Stalling until master znode is written.");
2336 // This will only be a minute or so while the cluster starts up,
2337 // so don't worry about setting watches on the parent znode
2338 while (!activeMasterManager
.hasActiveMaster()) {
2339 LOG
.debug("Waiting for master address and cluster state znode to be written.");
2340 Threads
.sleep(timeout
);
2343 MonitoredTask status
= TaskMonitor
.get().createStatus("Master startup");
2344 status
.setDescription("Master startup");
2346 if (activeMasterManager
.blockUntilBecomingActiveMaster(timeout
, status
)) {
2347 finishActiveMasterInitialization(status
);
2349 } catch (Throwable t
) {
2350 status
.setStatus("Failed to become active: " + t
.getMessage());
2351 LOG
.error(HBaseMarkers
.FATAL
, "Failed to become active master", t
);
2352 // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
2353 if (t
instanceof NoClassDefFoundError
&& t
.getMessage().
2354 contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")) {
2355 // improved error message for this special case
2356 abort("HBase is having a problem with its Hadoop jars. You may need to recompile " +
2357 "HBase against Hadoop version " + org
.apache
.hadoop
.util
.VersionInfo
.getVersion() +
2358 " or change your hadoop jars to start properly", t
);
2360 abort("Unhandled exception. Starting shutdown.", t
);
2367 private static boolean isCatalogTable(final TableName tableName
) {
2368 return tableName
.equals(TableName
.META_TABLE_NAME
);
2372 public long deleteTable(
2373 final TableName tableName
,
2374 final long nonceGroup
,
2375 final long nonce
) throws IOException
{
2378 return MasterProcedureUtil
.submitProcedure(
2379 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2381 protected void run() throws IOException
{
2382 getMaster().getMasterCoprocessorHost().preDeleteTable(tableName
);
2384 LOG
.info(getClientIdAuditPrefix() + " delete " + tableName
);
2386 // TODO: We can handle/merge duplicate request
2388 // We need to wait for the procedure to potentially fail due to "prepare" sanity
2389 // checks. This will block only the beginning of the procedure. See HBASE-19953.
2390 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
2391 submitProcedure(new DeleteTableProcedure(procedureExecutor
.getEnvironment(),
2395 getMaster().getMasterCoprocessorHost().postDeleteTable(tableName
);
2399 protected String
getDescription() {
2400 return "DeleteTableProcedure";
2406 public long truncateTable(
2407 final TableName tableName
,
2408 final boolean preserveSplits
,
2409 final long nonceGroup
,
2410 final long nonce
) throws IOException
{
2413 return MasterProcedureUtil
.submitProcedure(
2414 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2416 protected void run() throws IOException
{
2417 getMaster().getMasterCoprocessorHost().preTruncateTable(tableName
);
2419 LOG
.info(getClientIdAuditPrefix() + " truncate " + tableName
);
2420 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createLatch(2, 0);
2421 submitProcedure(new TruncateTableProcedure(procedureExecutor
.getEnvironment(),
2422 tableName
, preserveSplits
, latch
));
2425 getMaster().getMasterCoprocessorHost().postTruncateTable(tableName
);
2429 protected String
getDescription() {
2430 return "TruncateTableProcedure";
2436 public long addColumn(final TableName tableName
, final ColumnFamilyDescriptor column
,
2437 final long nonceGroup
, final long nonce
) throws IOException
{
2439 checkTableExists(tableName
);
2441 return modifyTable(tableName
, new TableDescriptorGetter() {
2444 public TableDescriptor
get() throws IOException
{
2445 TableDescriptor old
= getTableDescriptors().get(tableName
);
2446 if (old
.hasColumnFamily(column
.getName())) {
2447 throw new InvalidFamilyOperationException("Column family '" + column
.getNameAsString()
2448 + "' in table '" + tableName
+ "' already exists so cannot be added");
2451 return TableDescriptorBuilder
.newBuilder(old
).setColumnFamily(column
).build();
2453 }, nonceGroup
, nonce
, true);
2457 * Implement to return TableDescriptor after pre-checks
2459 protected interface TableDescriptorGetter
{
2460 TableDescriptor
get() throws IOException
;
2464 public long modifyColumn(final TableName tableName
, final ColumnFamilyDescriptor descriptor
,
2465 final long nonceGroup
, final long nonce
) throws IOException
{
2467 checkTableExists(tableName
);
2468 return modifyTable(tableName
, new TableDescriptorGetter() {
2471 public TableDescriptor
get() throws IOException
{
2472 TableDescriptor old
= getTableDescriptors().get(tableName
);
2473 if (!old
.hasColumnFamily(descriptor
.getName())) {
2474 throw new InvalidFamilyOperationException("Family '" + descriptor
.getNameAsString()
2475 + "' does not exist, so it cannot be modified");
2478 return TableDescriptorBuilder
.newBuilder(old
).modifyColumnFamily(descriptor
).build();
2480 }, nonceGroup
, nonce
, true);
2485 public long modifyColumnStoreFileTracker(TableName tableName
, byte[] family
, String dstSFT
,
2486 long nonceGroup
, long nonce
) throws IOException
{
2488 return MasterProcedureUtil
2489 .submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2492 protected void run() throws IOException
{
2493 String sft
= getMaster().getMasterCoprocessorHost()
2494 .preModifyColumnFamilyStoreFileTracker(tableName
, family
, dstSFT
);
2495 LOG
.info("{} modify column {} store file tracker of table {} to {}",
2496 getClientIdAuditPrefix(), Bytes
.toStringBinary(family
), tableName
, sft
);
2497 submitProcedure(new ModifyColumnFamilyStoreFileTrackerProcedure(
2498 procedureExecutor
.getEnvironment(), tableName
, family
, sft
));
2499 getMaster().getMasterCoprocessorHost().postModifyColumnFamilyStoreFileTracker(tableName
,
2504 protected String
getDescription() {
2505 return "ModifyColumnFamilyStoreFileTrackerProcedure";
2511 public long deleteColumn(final TableName tableName
, final byte[] columnName
,
2512 final long nonceGroup
, final long nonce
) throws IOException
{
2514 checkTableExists(tableName
);
2516 return modifyTable(tableName
, new TableDescriptorGetter() {
2519 public TableDescriptor
get() throws IOException
{
2520 TableDescriptor old
= getTableDescriptors().get(tableName
);
2522 if (!old
.hasColumnFamily(columnName
)) {
2523 throw new InvalidFamilyOperationException("Family '" + Bytes
.toString(columnName
)
2524 + "' does not exist, so it cannot be deleted");
2526 if (old
.getColumnFamilyCount() == 1) {
2527 throw new InvalidFamilyOperationException("Family '" + Bytes
.toString(columnName
)
2528 + "' is the only column family in the table, so it cannot be deleted");
2530 return TableDescriptorBuilder
.newBuilder(old
).removeColumnFamily(columnName
).build();
2532 }, nonceGroup
, nonce
, true);
2536 public long enableTable(final TableName tableName
, final long nonceGroup
, final long nonce
)
2537 throws IOException
{
2540 return MasterProcedureUtil
.submitProcedure(
2541 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2543 protected void run() throws IOException
{
2544 getMaster().getMasterCoprocessorHost().preEnableTable(tableName
);
2546 // Normally, it would make sense for this authorization check to exist inside
2547 // AccessController, but because the authorization check is done based on internal state
2548 // (rather than explicit permissions) we'll do the check here instead of in the
2550 MasterQuotaManager quotaManager
= getMasterQuotaManager();
2551 if (quotaManager
!= null) {
2552 if (quotaManager
.isQuotaInitialized()) {
2553 SpaceQuotaSnapshot currSnapshotOfTable
=
2554 QuotaTableUtil
.getCurrentSnapshotFromQuotaTable(getConnection(), tableName
);
2555 if (currSnapshotOfTable
!= null) {
2556 SpaceQuotaStatus quotaStatus
= currSnapshotOfTable
.getQuotaStatus();
2557 if (quotaStatus
.isInViolation()
2558 && SpaceViolationPolicy
.DISABLE
== quotaStatus
.getPolicy().orElse(null)) {
2559 throw new AccessDeniedException("Enabling the table '" + tableName
2560 + "' is disallowed due to a violated space quota.");
2563 } else if (LOG
.isTraceEnabled()) {
2564 LOG
.trace("Unable to check for space quotas as the MasterQuotaManager is not enabled");
2568 LOG
.info(getClientIdAuditPrefix() + " enable " + tableName
);
2570 // Execute the operation asynchronously - client will check the progress of the operation
2571 // In case the request is from a <1.1 client before returning,
2572 // we want to make sure that the table is prepared to be
2573 // enabled (the table is locked and the table state is set).
2574 // Note: if the procedure throws exception, we will catch it and rethrow.
2575 final ProcedurePrepareLatch prepareLatch
= ProcedurePrepareLatch
.createLatch();
2576 submitProcedure(new EnableTableProcedure(procedureExecutor
.getEnvironment(),
2577 tableName
, prepareLatch
));
2578 prepareLatch
.await();
2580 getMaster().getMasterCoprocessorHost().postEnableTable(tableName
);
2584 protected String
getDescription() {
2585 return "EnableTableProcedure";
2591 public long disableTable(final TableName tableName
, final long nonceGroup
, final long nonce
)
2592 throws IOException
{
2595 return MasterProcedureUtil
.submitProcedure(
2596 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2598 protected void run() throws IOException
{
2599 getMaster().getMasterCoprocessorHost().preDisableTable(tableName
);
2601 LOG
.info(getClientIdAuditPrefix() + " disable " + tableName
);
2603 // Execute the operation asynchronously - client will check the progress of the operation
2604 // In case the request is from a <1.1 client before returning,
2605 // we want to make sure that the table is prepared to be
2606 // enabled (the table is locked and the table state is set).
2607 // Note: if the procedure throws exception, we will catch it and rethrow.
2609 // We need to wait for the procedure to potentially fail due to "prepare" sanity
2610 // checks. This will block only the beginning of the procedure. See HBASE-19953.
2611 final ProcedurePrepareLatch prepareLatch
= ProcedurePrepareLatch
.createBlockingLatch();
2612 submitProcedure(new DisableTableProcedure(procedureExecutor
.getEnvironment(),
2613 tableName
, false, prepareLatch
));
2614 prepareLatch
.await();
2616 getMaster().getMasterCoprocessorHost().postDisableTable(tableName
);
2620 protected String
getDescription() {
2621 return "DisableTableProcedure";
2626 private long modifyTable(final TableName tableName
,
2627 final TableDescriptorGetter newDescriptorGetter
, final long nonceGroup
, final long nonce
,
2628 final boolean shouldCheckDescriptor
) throws IOException
{
2629 return MasterProcedureUtil
2630 .submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2632 protected void run() throws IOException
{
2633 TableDescriptor oldDescriptor
= getMaster().getTableDescriptors().get(tableName
);
2634 TableDescriptor newDescriptor
= getMaster().getMasterCoprocessorHost()
2635 .preModifyTable(tableName
, oldDescriptor
, newDescriptorGetter
.get());
2636 TableDescriptorChecker
.sanityCheck(conf
, newDescriptor
);
2637 LOG
.info("{} modify table {} from {} to {}", getClientIdAuditPrefix(), tableName
,
2638 oldDescriptor
, newDescriptor
);
2640 // Execute the operation synchronously - wait for the operation completes before
2643 // We need to wait for the procedure to potentially fail due to "prepare" sanity
2644 // checks. This will block only the beginning of the procedure. See HBASE-19953.
2645 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
2646 submitProcedure(new ModifyTableProcedure(procedureExecutor
.getEnvironment(),
2647 newDescriptor
, latch
, oldDescriptor
, shouldCheckDescriptor
));
2650 getMaster().getMasterCoprocessorHost().postModifyTable(tableName
, oldDescriptor
,
2655 protected String
getDescription() {
2656 return "ModifyTableProcedure";
2663 public long modifyTable(final TableName tableName
, final TableDescriptor newDescriptor
,
2664 final long nonceGroup
, final long nonce
) throws IOException
{
2666 return modifyTable(tableName
, new TableDescriptorGetter() {
2668 public TableDescriptor
get() throws IOException
{
2669 return newDescriptor
;
2671 }, nonceGroup
, nonce
, false);
2676 public long modifyTableStoreFileTracker(TableName tableName
, String dstSFT
, long nonceGroup
,
2677 long nonce
) throws IOException
{
2679 return MasterProcedureUtil
2680 .submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2683 protected void run() throws IOException
{
2684 String sft
= getMaster().getMasterCoprocessorHost()
2685 .preModifyTableStoreFileTracker(tableName
, dstSFT
);
2686 LOG
.info("{} modify table store file tracker of table {} to {}", getClientIdAuditPrefix(),
2688 submitProcedure(new ModifyTableStoreFileTrackerProcedure(
2689 procedureExecutor
.getEnvironment(), tableName
, sft
));
2690 getMaster().getMasterCoprocessorHost().postModifyTableStoreFileTracker(tableName
, sft
);
2694 protected String
getDescription() {
2695 return "ModifyTableStoreFileTrackerProcedure";
2700 public long restoreSnapshot(final SnapshotDescription snapshotDesc
, final long nonceGroup
,
2701 final long nonce
, final boolean restoreAcl
, final String customSFT
) throws IOException
{
2703 getSnapshotManager().checkSnapshotSupport();
2705 // Ensure namespace exists. Will throw exception if non-known NS.
2706 final TableName dstTable
= TableName
.valueOf(snapshotDesc
.getTable());
2707 getClusterSchema().getNamespace(dstTable
.getNamespaceAsString());
2709 return MasterProcedureUtil
.submitProcedure(
2710 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2712 protected void run() throws IOException
{
2714 getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc
, getNonceKey(), restoreAcl
,
2719 protected String
getDescription() {
2720 return "RestoreSnapshotProcedure";
2725 private void checkTableExists(final TableName tableName
)
2726 throws IOException
, TableNotFoundException
{
2727 if (!tableDescriptors
.exists(tableName
)) {
2728 throw new TableNotFoundException(tableName
);
2733 public void checkTableModifiable(final TableName tableName
)
2734 throws IOException
, TableNotFoundException
, TableNotDisabledException
{
2735 if (isCatalogTable(tableName
)) {
2736 throw new IOException("Can't modify catalog tables");
2738 checkTableExists(tableName
);
2739 TableState ts
= getTableStateManager().getTableState(tableName
);
2740 if (!ts
.isDisabled()) {
2741 throw new TableNotDisabledException("Not DISABLED; " + ts
);
2745 public ClusterMetrics
getClusterMetricsWithoutCoprocessor() throws InterruptedIOException
{
2746 return getClusterMetricsWithoutCoprocessor(EnumSet
.allOf(Option
.class));
2749 public ClusterMetrics
getClusterMetricsWithoutCoprocessor(EnumSet
<Option
> options
)
2750 throws InterruptedIOException
{
2751 ClusterMetricsBuilder builder
= ClusterMetricsBuilder
.newBuilder();
2752 // given that hbase1 can't submit the request with Option,
2753 // we return all information to client if the list of Option is empty.
2754 if (options
.isEmpty()) {
2755 options
= EnumSet
.allOf(Option
.class);
2758 // TASKS and/or LIVE_SERVERS will populate this map, which will be given to the builder if
2759 // not null after option processing completes.
2760 Map
<ServerName
, ServerMetrics
> serverMetricsMap
= null;
2762 for (Option opt
: options
) {
2764 case HBASE_VERSION
: builder
.setHBaseVersion(VersionInfo
.getVersion()); break;
2765 case CLUSTER_ID
: builder
.setClusterId(getClusterId()); break;
2766 case MASTER
: builder
.setMasterName(getServerName()); break;
2767 case BACKUP_MASTERS
: builder
.setBackerMasterNames(getBackupMasters()); break;
2770 builder
.setMasterTasks(TaskMonitor
.get().getTasks().stream()
2771 .map(task
-> ServerTaskBuilder
.newBuilder()
2772 .setDescription(task
.getDescription())
2773 .setStatus(task
.getStatus())
2774 .setState(ServerTask
.State
.valueOf(task
.getState().name()))
2775 .setStartTime(task
.getStartTime())
2776 .setCompletionTime(task
.getCompletionTimestamp())
2778 .collect(Collectors
.toList()));
2779 // TASKS is also synonymous with LIVE_SERVERS for now because task information for
2780 // regionservers is carried in ServerLoad.
2781 // Add entries to serverMetricsMap for all live servers, if we haven't already done so
2782 if (serverMetricsMap
== null) {
2783 serverMetricsMap
= getOnlineServers();
2787 case LIVE_SERVERS
: {
2788 // Add entries to serverMetricsMap for all live servers, if we haven't already done so
2789 if (serverMetricsMap
== null) {
2790 serverMetricsMap
= getOnlineServers();
2794 case DEAD_SERVERS
: {
2795 if (serverManager
!= null) {
2796 builder
.setDeadServerNames(new ArrayList
<>(
2797 serverManager
.getDeadServers().copyServerNames()));
2801 case MASTER_COPROCESSORS
: {
2802 if (cpHost
!= null) {
2803 builder
.setMasterCoprocessorNames(Arrays
.asList(getMasterCoprocessors()));
2807 case REGIONS_IN_TRANSITION
: {
2808 if (assignmentManager
!= null) {
2809 builder
.setRegionsInTransition(assignmentManager
.getRegionStates()
2810 .getRegionsStateInTransition());
2815 if (loadBalancerTracker
!= null) {
2816 builder
.setBalancerOn(loadBalancerTracker
.isBalancerOn());
2820 case MASTER_INFO_PORT
: {
2821 if (infoServer
!= null) {
2822 builder
.setMasterInfoPort(infoServer
.getPort());
2826 case SERVERS_NAME
: {
2827 if (serverManager
!= null) {
2828 builder
.setServerNames(serverManager
.getOnlineServersList());
2832 case TABLE_TO_REGIONS_COUNT
: {
2833 if (isActiveMaster() && isInitialized() && assignmentManager
!= null) {
2835 Map
<TableName
, RegionStatesCount
> tableRegionStatesCountMap
= new HashMap
<>();
2836 Map
<String
, TableDescriptor
> tableDescriptorMap
= getTableDescriptors().getAll();
2837 for (TableDescriptor tableDescriptor
: tableDescriptorMap
.values()) {
2838 TableName tableName
= tableDescriptor
.getTableName();
2839 RegionStatesCount regionStatesCount
= assignmentManager
2840 .getRegionStatesCount(tableName
);
2841 tableRegionStatesCountMap
.put(tableName
, regionStatesCount
);
2843 builder
.setTableRegionStatesCount(tableRegionStatesCountMap
);
2844 } catch (IOException e
) {
2845 LOG
.error("Error while populating TABLE_TO_REGIONS_COUNT for Cluster Metrics..", e
);
2853 if (serverMetricsMap
!= null) {
2854 builder
.setLiveServerMetrics(serverMetricsMap
);
2857 return builder
.build();
2860 private Map
<ServerName
, ServerMetrics
> getOnlineServers() {
2861 if (serverManager
!= null) {
2862 final Map
<ServerName
, ServerMetrics
> map
= new HashMap
<>();
2863 serverManager
.getOnlineServers().entrySet()
2864 .forEach(e
-> map
.put(e
.getKey(), e
.getValue()));
2871 * @return cluster status
2873 public ClusterMetrics
getClusterMetrics() throws IOException
{
2874 return getClusterMetrics(EnumSet
.allOf(Option
.class));
2877 public ClusterMetrics
getClusterMetrics(EnumSet
<Option
> options
) throws IOException
{
2878 if (cpHost
!= null) {
2879 cpHost
.preGetClusterMetrics();
2881 ClusterMetrics status
= getClusterMetricsWithoutCoprocessor(options
);
2882 if (cpHost
!= null) {
2883 cpHost
.postGetClusterMetrics(status
);
2889 * @return info port of active master or 0 if any exception occurs.
2891 public int getActiveMasterInfoPort() {
2892 return activeMasterManager
.getActiveMasterInfoPort();
2896 * @param sn is ServerName of the backup master
2897 * @return info port of backup master or 0 if any exception occurs.
2899 public int getBackupMasterInfoPort(final ServerName sn
) {
2900 return activeMasterManager
.getBackupMasterInfoPort(sn
);
2904 * The set of loaded coprocessors is stored in a static set. Since it's
2905 * statically allocated, it does not require that HMaster's cpHost be
2906 * initialized prior to accessing it.
2907 * @return a String representation of the set of names of the loaded coprocessors.
2909 public static String
getLoadedCoprocessors() {
2910 return CoprocessorHost
.getLoadedCoprocessors().toString();
2914 * @return timestamp in millis when HMaster was started.
2916 public long getMasterStartTime() {
2921 * @return timestamp in millis when HMaster became the active master.
2923 public long getMasterActiveTime() {
2924 return masterActiveTime
;
2928 * @return timestamp in millis when HMaster finished becoming the active master
2930 public long getMasterFinishedInitializationTime() {
2931 return masterFinishedInitializationTime
;
2934 public int getNumWALFiles() {
2938 public ProcedureStore
getProcedureStore() {
2939 return procedureStore
;
2942 public int getRegionServerInfoPort(final ServerName sn
) {
2943 int port
= this.serverManager
.getInfoPort(sn
);
2944 return port
== 0 ? conf
.getInt(HConstants
.REGIONSERVER_INFO_PORT
,
2945 HConstants
.DEFAULT_REGIONSERVER_INFOPORT
) : port
;
2949 public String
getRegionServerVersion(ServerName sn
) {
2950 // Will return "0.0.0" if the server is not online to prevent move system region to unknown
2952 return this.serverManager
.getVersion(sn
);
2956 public void checkIfShouldMoveSystemRegionAsync() {
2957 assignmentManager
.checkIfShouldMoveSystemRegionAsync();
2961 * @return array of coprocessor SimpleNames.
2963 public String
[] getMasterCoprocessors() {
2964 Set
<String
> masterCoprocessors
= getMasterCoprocessorHost().getCoprocessors();
2965 return masterCoprocessors
.toArray(new String
[masterCoprocessors
.size()]);
2969 public void abort(String reason
, Throwable cause
) {
2970 if (!setAbortRequested() || isStopped()) {
2971 LOG
.debug("Abort called but aborted={}, stopped={}", isAborted(), isStopped());
2974 if (cpHost
!= null) {
2975 // HBASE-4014: dump a list of loaded coprocessors.
2976 LOG
.error(HBaseMarkers
.FATAL
, "Master server abort: loaded coprocessors are: " +
2977 getLoadedCoprocessors());
2979 String msg
= "***** ABORTING master " + this + ": " + reason
+ " *****";
2980 if (cause
!= null) {
2981 LOG
.error(HBaseMarkers
.FATAL
, msg
, cause
);
2983 LOG
.error(HBaseMarkers
.FATAL
, msg
);
2988 } catch (IOException e
) {
2989 LOG
.error("Exception occurred while stopping master", e
);
2994 public MasterCoprocessorHost
getMasterCoprocessorHost() {
2999 public MasterQuotaManager
getMasterQuotaManager() {
3000 return quotaManager
;
3004 public ProcedureExecutor
<MasterProcedureEnv
> getMasterProcedureExecutor() {
3005 return procedureExecutor
;
3009 public ServerName
getServerName() {
3010 return this.serverName
;
3014 public AssignmentManager
getAssignmentManager() {
3015 return this.assignmentManager
;
3019 public CatalogJanitor
getCatalogJanitor() {
3020 return this.catalogJanitorChore
;
3023 public MemoryBoundedLogMessageBuffer
getRegionServerFatalLogBuffer() {
3028 * Shutdown the cluster.
3029 * Master runs a coordinated stop of all RegionServers and then itself.
3031 public void shutdown() throws IOException
{
3032 if (cpHost
!= null) {
3033 cpHost
.preShutdown();
3036 // Tell the servermanager cluster shutdown has been called. This makes it so when Master is
3037 // last running server, it'll stop itself. Next, we broadcast the cluster shutdown by setting
3038 // the cluster status as down. RegionServers will notice this change in state and will start
3039 // shutting themselves down. When last has exited, Master can go down.
3040 if (this.serverManager
!= null) {
3041 this.serverManager
.shutdownCluster();
3043 if (this.clusterStatusTracker
!= null) {
3045 this.clusterStatusTracker
.setClusterDown();
3046 } catch (KeeperException e
) {
3047 LOG
.error("ZooKeeper exception trying to set cluster as down in ZK", e
);
3050 // Stop the procedure executor. Will stop any ongoing assign, unassign, server crash etc.,
3051 // processing so we can go down.
3052 if (this.procedureExecutor
!= null) {
3053 this.procedureExecutor
.stop();
3055 // Shutdown our cluster connection. This will kill any hosted RPCs that might be going on;
3056 // this is what we want especially if the Master is in startup phase doing call outs to
3057 // hbase:meta, etc. when cluster is down. Without ths connection close, we'd have to wait on
3058 // the rpc to timeout.
3059 if (this.asyncClusterConnection
!= null) {
3060 this.asyncClusterConnection
.close();
3064 public void stopMaster() throws IOException
{
3065 if (cpHost
!= null) {
3066 cpHost
.preStopMaster();
3068 stop("Stopped by " + Thread
.currentThread().getName());
3072 public void stop(String msg
) {
3073 if (!this.stopped
) {
3074 LOG
.info("***** STOPPING master '" + this + "' *****");
3075 this.stopped
= true;
3076 LOG
.info("STOPPED: " + msg
);
3077 // Wakes run() if it is sleeping
3078 sleeper
.skipSleepCycle();
3079 if (this.activeMasterManager
!= null) {
3080 this.activeMasterManager
.stop();
3085 protected void checkServiceStarted() throws ServerNotRunningYetException
{
3086 if (!serviceStarted
) {
3087 throw new ServerNotRunningYetException("Server is not running yet");
3091 void checkInitialized() throws PleaseHoldException
, ServerNotRunningYetException
,
3092 MasterNotRunningException
, MasterStoppedException
{
3093 checkServiceStarted();
3094 if (!isInitialized()) {
3095 throw new PleaseHoldException("Master is initializing");
3098 throw new MasterStoppedException();
3103 * Report whether this master is currently the active master or not.
3104 * If not active master, we are parked on ZK waiting to become active.
3106 * This method is used for testing.
3108 * @return true if active master, false if not.
3111 public boolean isActiveMaster() {
3112 return activeMaster
;
3116 * Report whether this master has completed with its initialization and is
3117 * ready. If ready, the master is also the active master. A standby master
3120 * This method is used for testing.
3122 * @return true if master is ready to go, false if not.
3125 public boolean isInitialized() {
3126 return initialized
.isReady();
3130 * Report whether this master is started
3132 * This method is used for testing.
3134 * @return true if master is ready to go, false if not.
3136 public boolean isOnline() {
3137 return serviceStarted
;
3141 * Report whether this master is in maintenance mode.
3143 * @return true if master is in maintenanceMode
3146 public boolean isInMaintenanceMode() {
3147 return maintenanceMode
;
3150 public void setInitialized(boolean isInitialized
) {
3151 procedureExecutor
.getEnvironment().setEventReady(initialized
, isInitialized
);
3155 public ProcedureEvent
<?
> getInitializedEvent() {
3160 * Compute the average load across all region servers.
3161 * Currently, this uses a very naive computation - just uses the number of
3162 * regions being served, ignoring stats about number of requests.
3163 * @return the average load
3165 public double getAverageLoad() {
3166 if (this.assignmentManager
== null) {
3170 RegionStates regionStates
= this.assignmentManager
.getRegionStates();
3171 if (regionStates
== null) {
3174 return regionStates
.getAverageLoad();
3178 public boolean registerService(Service instance
) {
3180 * No stacking of instances is allowed for a single service name
3182 Descriptors
.ServiceDescriptor serviceDesc
= instance
.getDescriptorForType();
3183 String serviceName
= CoprocessorRpcUtils
.getServiceName(serviceDesc
);
3184 if (coprocessorServiceHandlers
.containsKey(serviceName
)) {
3185 LOG
.error("Coprocessor service "+serviceName
+
3186 " already registered, rejecting request from "+instance
3191 coprocessorServiceHandlers
.put(serviceName
, instance
);
3192 if (LOG
.isDebugEnabled()) {
3193 LOG
.debug("Registered master coprocessor service: service="+serviceName
);
3199 * Utility for constructing an instance of the passed HMaster class.
3200 * @param masterClass
3201 * @return HMaster instance.
3203 public static HMaster
constructMaster(Class
<?
extends HMaster
> masterClass
,
3204 final Configuration conf
) {
3206 Constructor
<?
extends HMaster
> c
= masterClass
.getConstructor(Configuration
.class);
3207 return c
.newInstance(conf
);
3208 } catch(Exception e
) {
3209 Throwable error
= e
;
3210 if (e
instanceof InvocationTargetException
&&
3211 ((InvocationTargetException
)e
).getTargetException() != null) {
3212 error
= ((InvocationTargetException
)e
).getTargetException();
3214 throw new RuntimeException("Failed construction of Master: " + masterClass
.toString() + ". "
3220 * @see org.apache.hadoop.hbase.master.HMasterCommandLine
3222 public static void main(String
[] args
) {
3223 LOG
.info("STARTING service " + HMaster
.class.getSimpleName());
3224 VersionInfo
.logVersion();
3225 new HMasterCommandLine(HMaster
.class).doMain(args
);
3228 public HFileCleaner
getHFileCleaner() {
3229 return this.hfileCleaner
;
3232 public LogCleaner
getLogCleaner() {
3233 return this.logCleaner
;
3237 * @return the underlying snapshot manager
3240 public SnapshotManager
getSnapshotManager() {
3241 return this.snapshotManager
;
3245 * @return the underlying MasterProcedureManagerHost
3248 public MasterProcedureManagerHost
getMasterProcedureManagerHost() {
3253 public ClusterSchema
getClusterSchema() {
3254 return this.clusterSchemaService
;
3258 * Create a new Namespace.
3259 * @param namespaceDescriptor descriptor for new Namespace
3260 * @param nonceGroup Identifier for the source of the request, a client or process.
3261 * @param nonce A unique identifier for this operation from the client or process identified by
3262 * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3263 * @return procedure id
3265 long createNamespace(final NamespaceDescriptor namespaceDescriptor
, final long nonceGroup
,
3266 final long nonce
) throws IOException
{
3269 TableName
.isLegalNamespaceName(Bytes
.toBytes(namespaceDescriptor
.getName()));
3271 return MasterProcedureUtil
.submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this,
3272 nonceGroup
, nonce
) {
3274 protected void run() throws IOException
{
3275 getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor
);
3276 // We need to wait for the procedure to potentially fail due to "prepare" sanity
3277 // checks. This will block only the beginning of the procedure. See HBASE-19953.
3278 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
3279 LOG
.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor
);
3280 // Execute the operation synchronously - wait for the operation to complete before
3282 setProcId(getClusterSchema().createNamespace(namespaceDescriptor
, getNonceKey(), latch
));
3284 getMaster().getMasterCoprocessorHost().postCreateNamespace(namespaceDescriptor
);
3288 protected String
getDescription() {
3289 return "CreateNamespaceProcedure";
3295 * Modify an existing Namespace.
3296 * @param nonceGroup Identifier for the source of the request, a client or process.
3297 * @param nonce A unique identifier for this operation from the client or process identified by
3298 * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3299 * @return procedure id
3301 long modifyNamespace(final NamespaceDescriptor newNsDescriptor
, final long nonceGroup
,
3302 final long nonce
) throws IOException
{
3305 TableName
.isLegalNamespaceName(Bytes
.toBytes(newNsDescriptor
.getName()));
3307 return MasterProcedureUtil
.submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this,
3308 nonceGroup
, nonce
) {
3310 protected void run() throws IOException
{
3311 NamespaceDescriptor oldNsDescriptor
= getNamespace(newNsDescriptor
.getName());
3312 getMaster().getMasterCoprocessorHost().preModifyNamespace(oldNsDescriptor
, newNsDescriptor
);
3313 // We need to wait for the procedure to potentially fail due to "prepare" sanity
3314 // checks. This will block only the beginning of the procedure. See HBASE-19953.
3315 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
3316 LOG
.info(getClientIdAuditPrefix() + " modify " + newNsDescriptor
);
3317 // Execute the operation synchronously - wait for the operation to complete before
3319 setProcId(getClusterSchema().modifyNamespace(newNsDescriptor
, getNonceKey(), latch
));
3321 getMaster().getMasterCoprocessorHost().postModifyNamespace(oldNsDescriptor
,
3326 protected String
getDescription() {
3327 return "ModifyNamespaceProcedure";
3333 * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
3334 * @param nonceGroup Identifier for the source of the request, a client or process.
3335 * @param nonce A unique identifier for this operation from the client or process identified by
3336 * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3337 * @return procedure id
3339 long deleteNamespace(final String name
, final long nonceGroup
, final long nonce
)
3340 throws IOException
{
3343 return MasterProcedureUtil
.submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this,
3344 nonceGroup
, nonce
) {
3346 protected void run() throws IOException
{
3347 getMaster().getMasterCoprocessorHost().preDeleteNamespace(name
);
3348 LOG
.info(getClientIdAuditPrefix() + " delete " + name
);
3349 // Execute the operation synchronously - wait for the operation to complete before
3352 // We need to wait for the procedure to potentially fail due to "prepare" sanity
3353 // checks. This will block only the beginning of the procedure. See HBASE-19953.
3354 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
3355 setProcId(submitProcedure(
3356 new DeleteNamespaceProcedure(procedureExecutor
.getEnvironment(), name
, latch
)));
3358 // Will not be invoked in the face of Exception thrown by the Procedure's execution
3359 getMaster().getMasterCoprocessorHost().postDeleteNamespace(name
);
3363 protected String
getDescription() {
3364 return "DeleteNamespaceProcedure";
3371 * @param name Name of the Namespace
3372 * @return Namespace descriptor for <code>name</code>
3374 NamespaceDescriptor
getNamespace(String name
) throws IOException
{
3376 if (this.cpHost
!= null) this.cpHost
.preGetNamespaceDescriptor(name
);
3377 NamespaceDescriptor nsd
= this.clusterSchemaService
.getNamespace(name
);
3378 if (this.cpHost
!= null) this.cpHost
.postGetNamespaceDescriptor(nsd
);
3383 * Get all Namespaces
3384 * @return All Namespace descriptors
3386 List
<NamespaceDescriptor
> getNamespaces() throws IOException
{
3388 final List
<NamespaceDescriptor
> nsds
= new ArrayList
<>();
3389 if (cpHost
!= null) {
3390 cpHost
.preListNamespaceDescriptors(nsds
);
3392 nsds
.addAll(this.clusterSchemaService
.getNamespaces());
3393 if (this.cpHost
!= null) {
3394 this.cpHost
.postListNamespaceDescriptors(nsds
);
3400 * List namespace names
3401 * @return All namespace names
3403 public List
<String
> listNamespaces() throws IOException
{
3405 List
<String
> namespaces
= new ArrayList
<>();
3406 if (cpHost
!= null) {
3407 cpHost
.preListNamespaces(namespaces
);
3409 for (NamespaceDescriptor namespace
: clusterSchemaService
.getNamespaces()) {
3410 namespaces
.add(namespace
.getName());
3412 if (cpHost
!= null) {
3413 cpHost
.postListNamespaces(namespaces
);
3419 public List
<TableName
> listTableNamesByNamespace(String name
) throws IOException
{
3421 return listTableNames(name
, null, true);
3425 public List
<TableDescriptor
> listTableDescriptorsByNamespace(String name
) throws IOException
{
3427 return listTableDescriptors(name
, null, null, true);
3431 public boolean abortProcedure(final long procId
, final boolean mayInterruptIfRunning
)
3432 throws IOException
{
3433 if (cpHost
!= null) {
3434 cpHost
.preAbortProcedure(this.procedureExecutor
, procId
);
3437 final boolean result
= this.procedureExecutor
.abort(procId
, mayInterruptIfRunning
);
3439 if (cpHost
!= null) {
3440 cpHost
.postAbortProcedure();
3447 public List
<Procedure
<?
>> getProcedures() throws IOException
{
3448 if (cpHost
!= null) {
3449 cpHost
.preGetProcedures();
3452 @SuppressWarnings({ "unchecked", "rawtypes" })
3453 List
<Procedure
<?
>> procList
= (List
) this.procedureExecutor
.getProcedures();
3455 if (cpHost
!= null) {
3456 cpHost
.postGetProcedures(procList
);
3463 public List
<LockedResource
> getLocks() throws IOException
{
3464 if (cpHost
!= null) {
3465 cpHost
.preGetLocks();
3468 MasterProcedureScheduler procedureScheduler
=
3469 procedureExecutor
.getEnvironment().getProcedureScheduler();
3471 final List
<LockedResource
> lockedResources
= procedureScheduler
.getLocks();
3473 if (cpHost
!= null) {
3474 cpHost
.postGetLocks(lockedResources
);
3477 return lockedResources
;
3481 * Returns the list of table descriptors that match the specified request
3482 * @param namespace the namespace to query, or null if querying for all
3483 * @param regex The regular expression to match against, or null if querying for all
3484 * @param tableNameList the list of table names, or null if querying for all
3485 * @param includeSysTables False to match only against userspace tables
3486 * @return the list of table descriptors
3488 public List
<TableDescriptor
> listTableDescriptors(final String namespace
, final String regex
,
3489 final List
<TableName
> tableNameList
, final boolean includeSysTables
)
3490 throws IOException
{
3491 List
<TableDescriptor
> htds
= new ArrayList
<>();
3492 if (cpHost
!= null) {
3493 cpHost
.preGetTableDescriptors(tableNameList
, htds
, regex
);
3495 htds
= getTableDescriptors(htds
, namespace
, regex
, tableNameList
, includeSysTables
);
3496 if (cpHost
!= null) {
3497 cpHost
.postGetTableDescriptors(tableNameList
, htds
, regex
);
3503 * Returns the list of table names that match the specified request
3504 * @param regex The regular expression to match against, or null if querying for all
3505 * @param namespace the namespace to query, or null if querying for all
3506 * @param includeSysTables False to match only against userspace tables
3507 * @return the list of table names
3509 public List
<TableName
> listTableNames(final String namespace
, final String regex
,
3510 final boolean includeSysTables
) throws IOException
{
3511 List
<TableDescriptor
> htds
= new ArrayList
<>();
3512 if (cpHost
!= null) {
3513 cpHost
.preGetTableNames(htds
, regex
);
3515 htds
= getTableDescriptors(htds
, namespace
, regex
, null, includeSysTables
);
3516 if (cpHost
!= null) {
3517 cpHost
.postGetTableNames(htds
, regex
);
3519 List
<TableName
> result
= new ArrayList
<>(htds
.size());
3520 for (TableDescriptor htd
: htds
) result
.add(htd
.getTableName());
3525 * Return a list of table table descriptors after applying any provided filter parameters. Note
3526 * that the user-facing description of this filter logic is presented on the class-level javadoc
3527 * of {@link NormalizeTableFilterParams}.
3529 private List
<TableDescriptor
> getTableDescriptors(final List
<TableDescriptor
> htds
,
3530 final String namespace
, final String regex
, final List
<TableName
> tableNameList
,
3531 final boolean includeSysTables
)
3532 throws IOException
{
3533 if (tableNameList
== null || tableNameList
.isEmpty()) {
3534 // request for all TableDescriptors
3535 Collection
<TableDescriptor
> allHtds
;
3536 if (namespace
!= null && namespace
.length() > 0) {
3537 // Do a check on the namespace existence. Will fail if does not exist.
3538 this.clusterSchemaService
.getNamespace(namespace
);
3539 allHtds
= tableDescriptors
.getByNamespace(namespace
).values();
3541 allHtds
= tableDescriptors
.getAll().values();
3543 for (TableDescriptor desc
: allHtds
) {
3544 if (tableStateManager
.isTablePresent(desc
.getTableName())
3545 && (includeSysTables
|| !desc
.getTableName().isSystemTable())) {
3550 for (TableName s
: tableNameList
) {
3551 if (tableStateManager
.isTablePresent(s
)) {
3552 TableDescriptor desc
= tableDescriptors
.get(s
);
3560 // Retains only those matched by regular expression.
3561 if (regex
!= null) filterTablesByRegex(htds
, Pattern
.compile(regex
));
3566 * Removes the table descriptors that don't match the pattern.
3567 * @param descriptors list of table descriptors to filter
3568 * @param pattern the regex to use
3570 private static void filterTablesByRegex(final Collection
<TableDescriptor
> descriptors
,
3571 final Pattern pattern
) {
3572 final String defaultNS
= NamespaceDescriptor
.DEFAULT_NAMESPACE_NAME_STR
;
3573 Iterator
<TableDescriptor
> itr
= descriptors
.iterator();
3574 while (itr
.hasNext()) {
3575 TableDescriptor htd
= itr
.next();
3576 String tableName
= htd
.getTableName().getNameAsString();
3577 boolean matched
= pattern
.matcher(tableName
).matches();
3578 if (!matched
&& htd
.getTableName().getNamespaceAsString().equals(defaultNS
)) {
3579 matched
= pattern
.matcher(defaultNS
+ TableName
.NAMESPACE_DELIM
+ tableName
).matches();
3588 public long getLastMajorCompactionTimestamp(TableName table
) throws IOException
{
3589 return getClusterMetrics(EnumSet
.of(Option
.LIVE_SERVERS
))
3590 .getLastMajorCompactionTimestamp(table
);
3594 public long getLastMajorCompactionTimestampForRegion(byte[] regionName
) throws IOException
{
3595 return getClusterMetrics(EnumSet
.of(Option
.LIVE_SERVERS
))
3596 .getLastMajorCompactionTimestamp(regionName
);
3600 * Gets the mob file compaction state for a specific table.
3601 * Whether all the mob files are selected is known during the compaction execution, but
3602 * the statistic is done just before compaction starts, it is hard to know the compaction
3603 * type at that time, so the rough statistics are chosen for the mob file compaction. Only two
3604 * compaction states are available, CompactionState.MAJOR_AND_MINOR and CompactionState.NONE.
3605 * @param tableName The current table name.
3606 * @return If a given table is in mob file compaction now.
3608 public GetRegionInfoResponse
.CompactionState
getMobCompactionState(TableName tableName
) {
3609 AtomicInteger compactionsCount
= mobCompactionStates
.get(tableName
);
3610 if (compactionsCount
!= null && compactionsCount
.get() != 0) {
3611 return GetRegionInfoResponse
.CompactionState
.MAJOR_AND_MINOR
;
3613 return GetRegionInfoResponse
.CompactionState
.NONE
;
3616 public void reportMobCompactionStart(TableName tableName
) throws IOException
{
3617 IdLock
.Entry lockEntry
= null;
3619 lockEntry
= mobCompactionLock
.getLockEntry(tableName
.hashCode());
3620 AtomicInteger compactionsCount
= mobCompactionStates
.get(tableName
);
3621 if (compactionsCount
== null) {
3622 compactionsCount
= new AtomicInteger(0);
3623 mobCompactionStates
.put(tableName
, compactionsCount
);
3625 compactionsCount
.incrementAndGet();
3627 if (lockEntry
!= null) {
3628 mobCompactionLock
.releaseLockEntry(lockEntry
);
3633 public void reportMobCompactionEnd(TableName tableName
) throws IOException
{
3634 IdLock
.Entry lockEntry
= null;
3636 lockEntry
= mobCompactionLock
.getLockEntry(tableName
.hashCode());
3637 AtomicInteger compactionsCount
= mobCompactionStates
.get(tableName
);
3638 if (compactionsCount
!= null) {
3639 int count
= compactionsCount
.decrementAndGet();
3640 // remove the entry if the count is 0.
3642 mobCompactionStates
.remove(tableName
);
3646 if (lockEntry
!= null) {
3647 mobCompactionLock
.releaseLockEntry(lockEntry
);
3654 * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
3655 * false is returned.
3657 * @return The state of the load balancer, or false if the load balancer isn't defined.
3659 public boolean isBalancerOn() {
3660 return !isInMaintenanceMode()
3661 && loadBalancerTracker
!= null
3662 && loadBalancerTracker
.isBalancerOn();
3666 * Queries the state of the {@link RegionNormalizerTracker}. If it's not initialized,
3667 * false is returned.
3669 public boolean isNormalizerOn() {
3670 return !isInMaintenanceMode()
3671 && getRegionNormalizerManager().isNormalizerOn();
3675 * Queries the state of the {@link SplitOrMergeTracker}. If it is not initialized,
3676 * false is returned. If switchType is illegal, false will return.
3677 * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}
3678 * @return The state of the switch
3681 public boolean isSplitOrMergeEnabled(MasterSwitchType switchType
) {
3682 return !isInMaintenanceMode()
3683 && splitOrMergeTracker
!= null
3684 && splitOrMergeTracker
.isSplitOrMergeEnabled(switchType
);
3688 * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.
3690 * Notice that, the base load balancer will always be {@link RSGroupBasedLoadBalancer} now, so
3691 * this method will return the balancer used inside each rs group.
3692 * @return The name of the {@link LoadBalancer} in use.
3694 public String
getLoadBalancerClassName() {
3695 return conf
.get(HConstants
.HBASE_MASTER_LOADBALANCER_CLASS
,
3696 LoadBalancerFactory
.getDefaultLoadBalancerClass().getName());
3699 public SplitOrMergeTracker
getSplitOrMergeTracker() {
3700 return splitOrMergeTracker
;
3704 public RSGroupBasedLoadBalancer
getLoadBalancer() {
3709 public FavoredNodesManager
getFavoredNodesManager() {
3710 return balancer
.getFavoredNodesManager();
3713 private long executePeerProcedure(AbstractPeerProcedure
<?
> procedure
) throws IOException
{
3714 long procId
= procedureExecutor
.submitProcedure(procedure
);
3715 procedure
.getLatch().await();
3720 public long addReplicationPeer(String peerId
, ReplicationPeerConfig peerConfig
, boolean enabled
)
3721 throws ReplicationException
, IOException
{
3722 LOG
.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId
+ ", config=" +
3723 peerConfig
+ ", state=" + (enabled ?
"ENABLED" : "DISABLED"));
3724 return executePeerProcedure(new AddPeerProcedure(peerId
, peerConfig
, enabled
));
3728 public long removeReplicationPeer(String peerId
) throws ReplicationException
, IOException
{
3729 LOG
.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId
);
3730 return executePeerProcedure(new RemovePeerProcedure(peerId
));
3734 public long enableReplicationPeer(String peerId
) throws ReplicationException
, IOException
{
3735 LOG
.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId
);
3736 return executePeerProcedure(new EnablePeerProcedure(peerId
));
3740 public long disableReplicationPeer(String peerId
) throws ReplicationException
, IOException
{
3741 LOG
.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId
);
3742 return executePeerProcedure(new DisablePeerProcedure(peerId
));
3746 public ReplicationPeerConfig
getReplicationPeerConfig(String peerId
)
3747 throws ReplicationException
, IOException
{
3748 if (cpHost
!= null) {
3749 cpHost
.preGetReplicationPeerConfig(peerId
);
3751 LOG
.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId
);
3752 ReplicationPeerConfig peerConfig
= this.replicationPeerManager
.getPeerConfig(peerId
)
3753 .orElseThrow(() -> new ReplicationPeerNotFoundException(peerId
));
3754 if (cpHost
!= null) {
3755 cpHost
.postGetReplicationPeerConfig(peerId
);
3761 public long updateReplicationPeerConfig(String peerId
, ReplicationPeerConfig peerConfig
)
3762 throws ReplicationException
, IOException
{
3763 LOG
.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId
+
3764 ", config=" + peerConfig
);
3765 return executePeerProcedure(new UpdatePeerConfigProcedure(peerId
, peerConfig
));
3769 public List
<ReplicationPeerDescription
> listReplicationPeers(String regex
)
3770 throws ReplicationException
, IOException
{
3771 if (cpHost
!= null) {
3772 cpHost
.preListReplicationPeers(regex
);
3774 LOG
.debug("{} list replication peers, regex={}", getClientIdAuditPrefix(), regex
);
3775 Pattern pattern
= regex
== null ?
null : Pattern
.compile(regex
);
3776 List
<ReplicationPeerDescription
> peers
=
3777 this.replicationPeerManager
.listPeers(pattern
);
3778 if (cpHost
!= null) {
3779 cpHost
.postListReplicationPeers(regex
);
3785 public long transitReplicationPeerSyncReplicationState(String peerId
, SyncReplicationState state
)
3786 throws ReplicationException
, IOException
{
3788 getClientIdAuditPrefix() +
3789 " transit current cluster state to {} in a synchronous replication peer id={}",
3791 return executePeerProcedure(new TransitPeerSyncReplicationStateProcedure(peerId
, state
));
3795 * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
3796 * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0
3797 * @param servers Region servers to decommission.
3799 public void decommissionRegionServers(final List
<ServerName
> servers
, final boolean offload
)
3800 throws IOException
{
3801 List
<ServerName
> serversAdded
= new ArrayList
<>(servers
.size());
3802 // Place the decommission marker first.
3803 String parentZnode
= getZooKeeper().getZNodePaths().drainingZNode
;
3804 for (ServerName server
: servers
) {
3806 String node
= ZNodePaths
.joinZNode(parentZnode
, server
.getServerName());
3807 ZKUtil
.createAndFailSilent(getZooKeeper(), node
);
3808 } catch (KeeperException ke
) {
3809 throw new HBaseIOException(
3810 this.zooKeeper
.prefix("Unable to decommission '" + server
.getServerName() + "'."), ke
);
3812 if (this.serverManager
.addServerToDrainList(server
)) {
3813 serversAdded
.add(server
);
3816 // Move the regions off the decommissioned servers.
3818 final List
<ServerName
> destServers
= this.serverManager
.createDestinationServersList();
3819 for (ServerName server
: serversAdded
) {
3820 final List
<RegionInfo
> regionsOnServer
= this.assignmentManager
.getRegionsOnServer(server
);
3821 for (RegionInfo hri
: regionsOnServer
) {
3822 ServerName dest
= balancer
.randomAssignment(hri
, destServers
);
3824 throw new HBaseIOException("Unable to determine a plan to move " + hri
);
3826 RegionPlan rp
= new RegionPlan(hri
, server
, dest
);
3827 this.assignmentManager
.moveAsync(rp
);
3834 * List region servers marked as decommissioned (previously called 'draining') to not get regions
3836 * @return List of decommissioned servers.
3838 public List
<ServerName
> listDecommissionedRegionServers() {
3839 return this.serverManager
.getDrainingServersList();
3843 * Remove decommission marker (previously called 'draining') from a region server to allow regions
3844 * assignments. Load regions onto the server asynchronously if a list of regions is given
3845 * @param server Region server to remove decommission marker from.
3847 public void recommissionRegionServer(final ServerName server
,
3848 final List
<byte[]> encodedRegionNames
) throws IOException
{
3849 // Remove the server from decommissioned (draining) server list.
3850 String parentZnode
= getZooKeeper().getZNodePaths().drainingZNode
;
3851 String node
= ZNodePaths
.joinZNode(parentZnode
, server
.getServerName());
3853 ZKUtil
.deleteNodeFailSilent(getZooKeeper(), node
);
3854 } catch (KeeperException ke
) {
3855 throw new HBaseIOException(
3856 this.zooKeeper
.prefix("Unable to recommission '" + server
.getServerName() + "'."), ke
);
3858 this.serverManager
.removeServerFromDrainList(server
);
3860 // Load the regions onto the server if we are given a list of regions.
3861 if (encodedRegionNames
== null || encodedRegionNames
.isEmpty()) {
3864 if (!this.serverManager
.isServerOnline(server
)) {
3867 for (byte[] encodedRegionName
: encodedRegionNames
) {
3868 RegionState regionState
=
3869 assignmentManager
.getRegionStates().getRegionState(Bytes
.toString(encodedRegionName
));
3870 if (regionState
== null) {
3871 LOG
.warn("Unknown region " + Bytes
.toStringBinary(encodedRegionName
));
3874 RegionInfo hri
= regionState
.getRegion();
3875 if (server
.equals(regionState
.getServerName())) {
3876 LOG
.info("Skipping move of region " + hri
.getRegionNameAsString() +
3877 " because region already assigned to the same server " + server
+ ".");
3880 RegionPlan rp
= new RegionPlan(hri
, regionState
.getServerName(), server
);
3881 this.assignmentManager
.moveAsync(rp
);
3886 public LockManager
getLockManager() {
3890 public QuotaObserverChore
getQuotaObserverChore() {
3891 return this.quotaObserverChore
;
3894 public SpaceQuotaSnapshotNotifier
getSpaceQuotaSnapshotNotifier() {
3895 return this.spaceQuotaSnapshotNotifier
;
3898 @SuppressWarnings("unchecked")
3899 private RemoteProcedure
<MasterProcedureEnv
, ?
> getRemoteProcedure(long procId
) {
3900 Procedure
<?
> procedure
= procedureExecutor
.getProcedure(procId
);
3901 if (procedure
== null) {
3904 assert procedure
instanceof RemoteProcedure
;
3905 return (RemoteProcedure
<MasterProcedureEnv
, ?
>) procedure
;
3908 public void remoteProcedureCompleted(long procId
) {
3909 LOG
.debug("Remote procedure done, pid={}", procId
);
3910 RemoteProcedure
<MasterProcedureEnv
, ?
> procedure
= getRemoteProcedure(procId
);
3911 if (procedure
!= null) {
3912 procedure
.remoteOperationCompleted(procedureExecutor
.getEnvironment());
3916 public void remoteProcedureFailed(long procId
, RemoteProcedureException error
) {
3917 LOG
.debug("Remote procedure failed, pid={}", procId
, error
);
3918 RemoteProcedure
<MasterProcedureEnv
, ?
> procedure
= getRemoteProcedure(procId
);
3919 if (procedure
!= null) {
3920 procedure
.remoteOperationFailed(procedureExecutor
.getEnvironment(), error
);
3925 * Reopen regions provided in the argument
3927 * @param tableName The current table name
3928 * @param regionNames The region names of the regions to reopen
3929 * @param nonceGroup Identifier for the source of the request, a client or process
3930 * @param nonce A unique identifier for this operation from the client or process identified by
3931 * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3932 * @return procedure Id
3933 * @throws IOException if reopening region fails while running procedure
3935 long reopenRegions(final TableName tableName
, final List
<byte[]> regionNames
,
3936 final long nonceGroup
, final long nonce
)
3937 throws IOException
{
3939 return MasterProcedureUtil
3940 .submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
3943 protected void run() throws IOException
{
3944 submitProcedure(new ReopenTableRegionsProcedure(tableName
, regionNames
));
3948 protected String
getDescription() {
3949 return "ReopenTableRegionsProcedure";
3957 public ReplicationPeerManager
getReplicationPeerManager() {
3958 return replicationPeerManager
;
3961 public HashMap
<String
, List
<Pair
<ServerName
, ReplicationLoadSource
>>>
3962 getReplicationLoad(ServerName
[] serverNames
) {
3963 List
<ReplicationPeerDescription
> peerList
= this.getReplicationPeerManager().listPeers(null);
3964 if (peerList
== null) {
3967 HashMap
<String
, List
<Pair
<ServerName
, ReplicationLoadSource
>>> replicationLoadSourceMap
=
3968 new HashMap
<>(peerList
.size());
3970 .forEach(peer
-> replicationLoadSourceMap
.put(peer
.getPeerId(), new ArrayList
<>()));
3971 for (ServerName serverName
: serverNames
) {
3972 List
<ReplicationLoadSource
> replicationLoadSources
=
3973 getServerManager().getLoad(serverName
).getReplicationLoadSourceList();
3974 for (ReplicationLoadSource replicationLoadSource
: replicationLoadSources
) {
3975 List
<Pair
<ServerName
, ReplicationLoadSource
>> replicationLoadSourceList
=
3976 replicationLoadSourceMap
.get(replicationLoadSource
.getPeerID());
3977 if (replicationLoadSourceList
== null) {
3978 LOG
.debug("{} does not exist, but it exists "
3979 + "in znode(/hbase/replication/rs). when the rs restarts, peerId is deleted, so "
3980 + "we just need to ignore it", replicationLoadSource
.getPeerID());
3983 replicationLoadSourceList
.add(new Pair
<>(serverName
, replicationLoadSource
));
3986 for (List
<Pair
<ServerName
, ReplicationLoadSource
>> loads
: replicationLoadSourceMap
.values()) {
3987 if (loads
.size() > 0) {
3988 loads
.sort(Comparator
.comparingLong(load
-> (-1) * load
.getSecond().getReplicationLag()));
3991 return replicationLoadSourceMap
;
3995 * This method modifies the master's configuration in order to inject replication-related features
3997 @InterfaceAudience.Private
3998 public static void decorateMasterConfiguration(Configuration conf
) {
3999 String plugins
= conf
.get(HBASE_MASTER_LOGCLEANER_PLUGINS
);
4000 String cleanerClass
= ReplicationLogCleaner
.class.getCanonicalName();
4001 if (plugins
== null || !plugins
.contains(cleanerClass
)) {
4002 conf
.set(HBASE_MASTER_LOGCLEANER_PLUGINS
, plugins
+ "," + cleanerClass
);
4004 if (ReplicationUtils
.isReplicationForBulkLoadDataEnabled(conf
)) {
4005 plugins
= conf
.get(HFileCleaner
.MASTER_HFILE_CLEANER_PLUGINS
);
4006 cleanerClass
= ReplicationHFileCleaner
.class.getCanonicalName();
4007 if (!plugins
.contains(cleanerClass
)) {
4008 conf
.set(HFileCleaner
.MASTER_HFILE_CLEANER_PLUGINS
, plugins
+ "," + cleanerClass
);
4013 public SnapshotQuotaObserverChore
getSnapshotQuotaObserverChore() {
4014 return this.snapshotQuotaChore
;
4017 public ActiveMasterManager
getActiveMasterManager() {
4018 return activeMasterManager
;
4022 public SyncReplicationReplayWALManager
getSyncReplicationReplayWALManager() {
4023 return this.syncReplicationReplayWALManager
;
4026 public HbckChore
getHbckChore() {
4027 return this.hbckChore
;
4031 public void runReplicationBarrierCleaner() {
4032 ReplicationBarrierCleaner rbc
= this.replicationBarrierCleaner
;
4039 public RSGroupInfoManager
getRSGroupInfoManager() {
4040 return rsGroupInfoManager
;
4044 * Get the compaction state of the table
4046 * @param tableName The table name
4047 * @return CompactionState Compaction state of the table
4049 public CompactionState
getCompactionState(final TableName tableName
) {
4050 CompactionState compactionState
= CompactionState
.NONE
;
4052 List
<RegionInfo
> regions
=
4053 assignmentManager
.getRegionStates().getRegionsOfTable(tableName
);
4054 for (RegionInfo regionInfo
: regions
) {
4055 ServerName serverName
=
4056 assignmentManager
.getRegionStates().getRegionServerOfRegion(regionInfo
);
4057 if (serverName
== null) {
4060 ServerMetrics sl
= serverManager
.getLoad(serverName
);
4064 RegionMetrics regionMetrics
= sl
.getRegionMetrics().get(regionInfo
.getRegionName());
4065 if (regionMetrics
.getCompactionState() == CompactionState
.MAJOR
) {
4066 if (compactionState
== CompactionState
.MINOR
) {
4067 compactionState
= CompactionState
.MAJOR_AND_MINOR
;
4069 compactionState
= CompactionState
.MAJOR
;
4071 } else if (regionMetrics
.getCompactionState() == CompactionState
.MINOR
) {
4072 if (compactionState
== CompactionState
.MAJOR
) {
4073 compactionState
= CompactionState
.MAJOR_AND_MINOR
;
4075 compactionState
= CompactionState
.MINOR
;
4079 } catch (Exception e
) {
4080 compactionState
= null;
4081 LOG
.error("Exception when get compaction state for " + tableName
.getNameAsString(), e
);
4083 return compactionState
;
4087 public MetaLocationSyncer
getMetaLocationSyncer() {
4088 return metaLocationSyncer
;
4091 @RestrictedApi(explanation
= "Should only be called in tests", link
= "",
4092 allowedOnPath
= ".*/src/test/.*")
4093 public MasterRegion
getMasterRegion() {
4094 return masterRegion
;
4098 public void onConfigurationChange(Configuration newConf
) {
4100 Superusers
.initialize(newConf
);
4101 } catch (IOException e
) {
4102 LOG
.warn("Failed to initialize SuperUsers on reloading of the configuration");
4104 // append the quotas observer back to the master coprocessor key
4105 setQuotasObserver(newConf
);
4106 // update region server coprocessor if the configuration has changed.
4107 if (CoprocessorConfigurationUtil
.checkConfigurationChange(getConfiguration(), newConf
,
4108 CoprocessorHost
.MASTER_COPROCESSOR_CONF_KEY
) && !maintenanceMode
) {
4109 LOG
.info("Update the master coprocessor(s) because the configuration has changed");
4110 initializeCoprocessorHost(newConf
);
4115 protected NamedQueueRecorder
createNamedQueueRecord() {
4116 final boolean isBalancerDecisionRecording
= conf
4117 .getBoolean(BaseLoadBalancer
.BALANCER_DECISION_BUFFER_ENABLED
,
4118 BaseLoadBalancer
.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED
);
4119 final boolean isBalancerRejectionRecording
= conf
4120 .getBoolean(BaseLoadBalancer
.BALANCER_REJECTION_BUFFER_ENABLED
,
4121 BaseLoadBalancer
.DEFAULT_BALANCER_REJECTION_BUFFER_ENABLED
);
4122 if (isBalancerDecisionRecording
|| isBalancerRejectionRecording
) {
4123 return NamedQueueRecorder
.getInstance(conf
);
4130 protected boolean clusterMode() {
4134 public String
getClusterId() {
4138 return cachedClusterId
.getFromCacheOrFetch();
4141 public Optional
<ServerName
> getActiveMaster() {
4142 return activeMasterManager
.getActiveMasterServerName();
4145 public List
<ServerName
> getBackupMasters() {
4146 return activeMasterManager
.getBackupMasters();
4150 public Iterator
<ServerName
> getBootstrapNodes() {
4151 return regionServerTracker
.getRegionServers().iterator();
4155 public List
<HRegionLocation
> getMetaLocations() {
4156 return metaRegionLocationCache
.getMetaRegionLocations();
4159 public Collection
<ServerName
> getLiveRegionServers() {
4160 return regionServerTracker
.getRegionServers();
4163 @RestrictedApi(explanation
= "Should only be called in tests", link
= "",
4164 allowedOnPath
= ".*/src/test/.*")
4165 void setLoadBalancer(RSGroupBasedLoadBalancer loadBalancer
) {
4166 this.balancer
= loadBalancer
;
4169 @RestrictedApi(explanation
= "Should only be called in tests", link
= "",
4170 allowedOnPath
= ".*/src/test/.*")
4171 void setAssignmentManager(AssignmentManager assignmentManager
) {
4172 this.assignmentManager
= assignmentManager
;
4175 @RestrictedApi(explanation
= "Should only be called in tests", link
= "",
4176 allowedOnPath
= ".*/src/test/.*")
4177 static void setDisableBalancerChoreForTest(boolean disable
) {
4178 disableBalancerChoreForTest
= disable
;
4181 @RestrictedApi(explanation
= "Should only be called in tests", link
= "",
4182 allowedOnPath
= ".*/src/test/.*")
4183 public ConfigurationManager
getConfigurationManager() {
4184 return configurationManager
;
4188 private void setQuotasObserver(Configuration conf
) {
4189 // Add the Observer to delete quotas on table deletion before starting all CPs by
4190 // default with quota support, avoiding if user specifically asks to not load this Observer.
4191 if (QuotaUtil
.isQuotaEnabled(conf
)) {
4192 updateConfigurationForQuotasObserver(conf
);
4196 private void initializeCoprocessorHost(Configuration conf
) {
4197 // initialize master side coprocessors before we start handling requests
4198 this.cpHost
= new MasterCoprocessorHost(this, conf
);