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
.TableName
;
84 import org
.apache
.hadoop
.hbase
.TableNotDisabledException
;
85 import org
.apache
.hadoop
.hbase
.TableNotFoundException
;
86 import org
.apache
.hadoop
.hbase
.UnknownRegionException
;
87 import org
.apache
.hadoop
.hbase
.client
.BalanceRequest
;
88 import org
.apache
.hadoop
.hbase
.client
.BalanceResponse
;
89 import org
.apache
.hadoop
.hbase
.client
.ColumnFamilyDescriptor
;
90 import org
.apache
.hadoop
.hbase
.client
.CompactionState
;
91 import org
.apache
.hadoop
.hbase
.client
.MasterSwitchType
;
92 import org
.apache
.hadoop
.hbase
.client
.NormalizeTableFilterParams
;
93 import org
.apache
.hadoop
.hbase
.client
.Put
;
94 import org
.apache
.hadoop
.hbase
.client
.RegionInfo
;
95 import org
.apache
.hadoop
.hbase
.client
.RegionInfoBuilder
;
96 import org
.apache
.hadoop
.hbase
.client
.RegionStatesCount
;
97 import org
.apache
.hadoop
.hbase
.client
.ResultScanner
;
98 import org
.apache
.hadoop
.hbase
.client
.Scan
;
99 import org
.apache
.hadoop
.hbase
.client
.TableDescriptor
;
100 import org
.apache
.hadoop
.hbase
.client
.TableDescriptorBuilder
;
101 import org
.apache
.hadoop
.hbase
.client
.TableState
;
102 import org
.apache
.hadoop
.hbase
.coprocessor
.CoprocessorHost
;
103 import org
.apache
.hadoop
.hbase
.exceptions
.MasterStoppedException
;
104 import org
.apache
.hadoop
.hbase
.executor
.ExecutorType
;
105 import org
.apache
.hadoop
.hbase
.favored
.FavoredNodesManager
;
106 import org
.apache
.hadoop
.hbase
.http
.HttpServer
;
107 import org
.apache
.hadoop
.hbase
.http
.InfoServer
;
108 import org
.apache
.hadoop
.hbase
.ipc
.CoprocessorRpcUtils
;
109 import org
.apache
.hadoop
.hbase
.ipc
.RpcServer
;
110 import org
.apache
.hadoop
.hbase
.ipc
.ServerNotRunningYetException
;
111 import org
.apache
.hadoop
.hbase
.log
.HBaseMarkers
;
112 import org
.apache
.hadoop
.hbase
.master
.MasterRpcServices
.BalanceSwitchMode
;
113 import org
.apache
.hadoop
.hbase
.master
.assignment
.AssignmentManager
;
114 import org
.apache
.hadoop
.hbase
.master
.assignment
.MergeTableRegionsProcedure
;
115 import org
.apache
.hadoop
.hbase
.master
.assignment
.RegionStateNode
;
116 import org
.apache
.hadoop
.hbase
.master
.assignment
.RegionStateStore
;
117 import org
.apache
.hadoop
.hbase
.master
.assignment
.RegionStates
;
118 import org
.apache
.hadoop
.hbase
.master
.assignment
.TransitRegionStateProcedure
;
119 import org
.apache
.hadoop
.hbase
.master
.balancer
.BalancerChore
;
120 import org
.apache
.hadoop
.hbase
.master
.balancer
.BaseLoadBalancer
;
121 import org
.apache
.hadoop
.hbase
.master
.balancer
.ClusterStatusChore
;
122 import org
.apache
.hadoop
.hbase
.master
.balancer
.LoadBalancerFactory
;
123 import org
.apache
.hadoop
.hbase
.master
.balancer
.MaintenanceLoadBalancer
;
124 import org
.apache
.hadoop
.hbase
.master
.cleaner
.DirScanPool
;
125 import org
.apache
.hadoop
.hbase
.master
.cleaner
.HFileCleaner
;
126 import org
.apache
.hadoop
.hbase
.master
.cleaner
.LogCleaner
;
127 import org
.apache
.hadoop
.hbase
.master
.cleaner
.ReplicationBarrierCleaner
;
128 import org
.apache
.hadoop
.hbase
.master
.cleaner
.SnapshotCleanerChore
;
129 import org
.apache
.hadoop
.hbase
.master
.http
.MasterDumpServlet
;
130 import org
.apache
.hadoop
.hbase
.master
.http
.MasterRedirectServlet
;
131 import org
.apache
.hadoop
.hbase
.master
.http
.MasterStatusServlet
;
132 import org
.apache
.hadoop
.hbase
.master
.janitor
.CatalogJanitor
;
133 import org
.apache
.hadoop
.hbase
.master
.locking
.LockManager
;
134 import org
.apache
.hadoop
.hbase
.master
.migrate
.RollingUpgradeChore
;
135 import org
.apache
.hadoop
.hbase
.master
.normalizer
.RegionNormalizerFactory
;
136 import org
.apache
.hadoop
.hbase
.master
.normalizer
.RegionNormalizerManager
;
137 import org
.apache
.hadoop
.hbase
.master
.procedure
.CreateTableProcedure
;
138 import org
.apache
.hadoop
.hbase
.master
.procedure
.DeleteNamespaceProcedure
;
139 import org
.apache
.hadoop
.hbase
.master
.procedure
.DeleteTableProcedure
;
140 import org
.apache
.hadoop
.hbase
.master
.procedure
.DisableTableProcedure
;
141 import org
.apache
.hadoop
.hbase
.master
.procedure
.EnableTableProcedure
;
142 import org
.apache
.hadoop
.hbase
.master
.procedure
.InitMetaProcedure
;
143 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureConstants
;
144 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureEnv
;
145 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureScheduler
;
146 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureUtil
;
147 import org
.apache
.hadoop
.hbase
.master
.procedure
.MasterProcedureUtil
.NonceProcedureRunnable
;
148 import org
.apache
.hadoop
.hbase
.master
.procedure
.ModifyTableProcedure
;
149 import org
.apache
.hadoop
.hbase
.master
.procedure
.ProcedurePrepareLatch
;
150 import org
.apache
.hadoop
.hbase
.master
.procedure
.ProcedureSyncWait
;
151 import org
.apache
.hadoop
.hbase
.master
.procedure
.ReopenTableRegionsProcedure
;
152 import org
.apache
.hadoop
.hbase
.master
.procedure
.ServerCrashProcedure
;
153 import org
.apache
.hadoop
.hbase
.master
.procedure
.TruncateTableProcedure
;
154 import org
.apache
.hadoop
.hbase
.master
.region
.MasterRegion
;
155 import org
.apache
.hadoop
.hbase
.master
.region
.MasterRegionFactory
;
156 import org
.apache
.hadoop
.hbase
.master
.replication
.AbstractPeerProcedure
;
157 import org
.apache
.hadoop
.hbase
.master
.replication
.AddPeerProcedure
;
158 import org
.apache
.hadoop
.hbase
.master
.replication
.DisablePeerProcedure
;
159 import org
.apache
.hadoop
.hbase
.master
.replication
.EnablePeerProcedure
;
160 import org
.apache
.hadoop
.hbase
.master
.replication
.RemovePeerProcedure
;
161 import org
.apache
.hadoop
.hbase
.master
.replication
.ReplicationPeerManager
;
162 import org
.apache
.hadoop
.hbase
.master
.replication
.SyncReplicationReplayWALManager
;
163 import org
.apache
.hadoop
.hbase
.master
.replication
.TransitPeerSyncReplicationStateProcedure
;
164 import org
.apache
.hadoop
.hbase
.master
.replication
.UpdatePeerConfigProcedure
;
165 import org
.apache
.hadoop
.hbase
.master
.slowlog
.SlowLogMasterService
;
166 import org
.apache
.hadoop
.hbase
.master
.snapshot
.SnapshotManager
;
167 import org
.apache
.hadoop
.hbase
.master
.zksyncer
.MasterAddressSyncer
;
168 import org
.apache
.hadoop
.hbase
.master
.zksyncer
.MetaLocationSyncer
;
169 import org
.apache
.hadoop
.hbase
.mob
.MobFileCleanerChore
;
170 import org
.apache
.hadoop
.hbase
.mob
.MobFileCompactionChore
;
171 import org
.apache
.hadoop
.hbase
.monitoring
.MemoryBoundedLogMessageBuffer
;
172 import org
.apache
.hadoop
.hbase
.monitoring
.MonitoredTask
;
173 import org
.apache
.hadoop
.hbase
.monitoring
.TaskMonitor
;
174 import org
.apache
.hadoop
.hbase
.namequeues
.NamedQueueRecorder
;
175 import org
.apache
.hadoop
.hbase
.procedure
.MasterProcedureManagerHost
;
176 import org
.apache
.hadoop
.hbase
.procedure
.flush
.MasterFlushTableProcedureManager
;
177 import org
.apache
.hadoop
.hbase
.procedure2
.LockedResource
;
178 import org
.apache
.hadoop
.hbase
.procedure2
.Procedure
;
179 import org
.apache
.hadoop
.hbase
.procedure2
.ProcedureEvent
;
180 import org
.apache
.hadoop
.hbase
.procedure2
.ProcedureExecutor
;
181 import org
.apache
.hadoop
.hbase
.procedure2
.RemoteProcedureDispatcher
.RemoteProcedure
;
182 import org
.apache
.hadoop
.hbase
.procedure2
.RemoteProcedureException
;
183 import org
.apache
.hadoop
.hbase
.procedure2
.store
.ProcedureStore
;
184 import org
.apache
.hadoop
.hbase
.procedure2
.store
.ProcedureStore
.ProcedureStoreListener
;
185 import org
.apache
.hadoop
.hbase
.procedure2
.store
.region
.RegionProcedureStore
;
186 import org
.apache
.hadoop
.hbase
.quotas
.MasterQuotaManager
;
187 import org
.apache
.hadoop
.hbase
.quotas
.MasterQuotasObserver
;
188 import org
.apache
.hadoop
.hbase
.quotas
.QuotaObserverChore
;
189 import org
.apache
.hadoop
.hbase
.quotas
.QuotaTableUtil
;
190 import org
.apache
.hadoop
.hbase
.quotas
.QuotaUtil
;
191 import org
.apache
.hadoop
.hbase
.quotas
.SnapshotQuotaObserverChore
;
192 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshot
;
193 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshot
.SpaceQuotaStatus
;
194 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshotNotifier
;
195 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshotNotifierFactory
;
196 import org
.apache
.hadoop
.hbase
.quotas
.SpaceViolationPolicy
;
197 import org
.apache
.hadoop
.hbase
.regionserver
.HRegionServer
;
198 import org
.apache
.hadoop
.hbase
.regionserver
.NoSuchColumnFamilyException
;
199 import org
.apache
.hadoop
.hbase
.replication
.ReplicationException
;
200 import org
.apache
.hadoop
.hbase
.replication
.ReplicationLoadSource
;
201 import org
.apache
.hadoop
.hbase
.replication
.ReplicationPeerConfig
;
202 import org
.apache
.hadoop
.hbase
.replication
.ReplicationPeerDescription
;
203 import org
.apache
.hadoop
.hbase
.replication
.ReplicationUtils
;
204 import org
.apache
.hadoop
.hbase
.replication
.SyncReplicationState
;
205 import org
.apache
.hadoop
.hbase
.replication
.master
.ReplicationHFileCleaner
;
206 import org
.apache
.hadoop
.hbase
.replication
.master
.ReplicationLogCleaner
;
207 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupAdminEndpoint
;
208 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupBasedLoadBalancer
;
209 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupInfoManager
;
210 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupUtil
;
211 import org
.apache
.hadoop
.hbase
.security
.AccessDeniedException
;
212 import org
.apache
.hadoop
.hbase
.security
.SecurityConstants
;
213 import org
.apache
.hadoop
.hbase
.security
.Superusers
;
214 import org
.apache
.hadoop
.hbase
.security
.UserProvider
;
215 import org
.apache
.hadoop
.hbase
.util
.Addressing
;
216 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
217 import org
.apache
.hadoop
.hbase
.util
.CommonFSUtils
;
218 import org
.apache
.hadoop
.hbase
.util
.EnvironmentEdgeManager
;
219 import org
.apache
.hadoop
.hbase
.util
.FSTableDescriptors
;
220 import org
.apache
.hadoop
.hbase
.util
.FutureUtils
;
221 import org
.apache
.hadoop
.hbase
.util
.HBaseFsck
;
222 import org
.apache
.hadoop
.hbase
.util
.HFileArchiveUtil
;
223 import org
.apache
.hadoop
.hbase
.util
.IdLock
;
224 import org
.apache
.hadoop
.hbase
.util
.JVMClusterUtil
;
225 import org
.apache
.hadoop
.hbase
.util
.ModifyRegionUtils
;
226 import org
.apache
.hadoop
.hbase
.util
.Pair
;
227 import org
.apache
.hadoop
.hbase
.util
.RetryCounter
;
228 import org
.apache
.hadoop
.hbase
.util
.RetryCounterFactory
;
229 import org
.apache
.hadoop
.hbase
.util
.TableDescriptorChecker
;
230 import org
.apache
.hadoop
.hbase
.util
.Threads
;
231 import org
.apache
.hadoop
.hbase
.util
.VersionInfo
;
232 import org
.apache
.hadoop
.hbase
.zookeeper
.LoadBalancerTracker
;
233 import org
.apache
.hadoop
.hbase
.zookeeper
.MasterAddressTracker
;
234 import org
.apache
.hadoop
.hbase
.zookeeper
.MetaTableLocator
;
235 import org
.apache
.hadoop
.hbase
.zookeeper
.RegionNormalizerTracker
;
236 import org
.apache
.hadoop
.hbase
.zookeeper
.SnapshotCleanupTracker
;
237 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKClusterId
;
238 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
;
239 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKWatcher
;
240 import org
.apache
.hadoop
.hbase
.zookeeper
.ZNodePaths
;
241 import org
.apache
.yetus
.audience
.InterfaceAudience
;
242 import org
.apache
.zookeeper
.KeeperException
;
243 import org
.slf4j
.Logger
;
244 import org
.slf4j
.LoggerFactory
;
246 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Lists
;
247 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Maps
;
248 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Sets
;
249 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.io
.Closeables
;
250 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Descriptors
;
251 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Service
;
252 import org
.apache
.hbase
.thirdparty
.org
.eclipse
.jetty
.server
.Server
;
253 import org
.apache
.hbase
.thirdparty
.org
.eclipse
.jetty
.server
.ServerConnector
;
254 import org
.apache
.hbase
.thirdparty
.org
.eclipse
.jetty
.servlet
.ServletHolder
;
255 import org
.apache
.hbase
.thirdparty
.org
.eclipse
.jetty
.webapp
.WebAppContext
;
257 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.RequestConverter
;
258 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetRegionInfoResponse
;
259 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.SnapshotProtos
.SnapshotDescription
;
262 * HMaster is the "master server" for HBase. An HBase cluster has one active master. If many masters
263 * are started, all compete. Whichever wins goes on to run the cluster. All others park themselves
264 * in their constructor until master or cluster shutdown or until the active master loses its lease
265 * in zookeeper. Thereafter, all running master jostle to take over master role.
267 * The Master can be asked shutdown the cluster. See {@link #shutdown()}. In this case it will tell
268 * all regionservers to go down and then wait on them all reporting in that they are down. This
269 * master will then shut itself down.
271 * You can also shutdown just this master. Call {@link #stopMaster()}.
272 * @see org.apache.zookeeper.Watcher
274 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience
.TOOLS
)
275 public class HMaster
extends HBaseServerBase
<MasterRpcServices
> implements MasterServices
{
277 private static final Logger LOG
= LoggerFactory
.getLogger(HMaster
.class);
279 // MASTER is name of the webapp and the attribute name used stuffing this
280 //instance into web context.
281 public static final String MASTER
= "master";
283 // Manager and zk listener for master election
284 private final ActiveMasterManager activeMasterManager
;
285 // Region server tracker
286 private final RegionServerTracker regionServerTracker
;
287 // Draining region server tracker
288 private DrainingServerTracker drainingServerTracker
;
289 // Tracker for load balancer state
290 LoadBalancerTracker loadBalancerTracker
;
291 // Tracker for meta location, if any client ZK quorum specified
292 private MetaLocationSyncer metaLocationSyncer
;
293 // Tracker for active master location, if any client ZK quorum specified
294 @InterfaceAudience.Private
295 MasterAddressSyncer masterAddressSyncer
;
296 // Tracker for auto snapshot cleanup state
297 SnapshotCleanupTracker snapshotCleanupTracker
;
299 // Tracker for split and merge state
300 private SplitOrMergeTracker splitOrMergeTracker
;
302 private ClusterSchemaService clusterSchemaService
;
304 public static final String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
=
305 "hbase.master.wait.on.service.seconds";
306 public static final int DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
= 5 * 60;
308 public static final String HBASE_MASTER_CLEANER_INTERVAL
= "hbase.master.cleaner.interval";
310 public static final int DEFAULT_HBASE_MASTER_CLEANER_INTERVAL
= 600 * 1000;
312 private String clusterId
;
314 // Metrics for the HMaster
315 final MetricsMaster metricsMaster
;
316 // file system manager for the master FS operations
317 private MasterFileSystem fileSystemManager
;
318 private MasterWalManager walManager
;
320 // manager to manage procedure-based WAL splitting, can be null if current
321 // is zk-based WAL splitting. SplitWALManager will replace SplitLogManager
322 // and MasterWalManager, which means zk-based WAL splitting code will be
323 // useless after we switch to the procedure-based one. our eventual goal
324 // is to remove all the zk-based WAL splitting code.
325 private SplitWALManager splitWALManager
;
327 // server manager to deal with region server info
328 private volatile ServerManager serverManager
;
330 // manager of assignment nodes in zookeeper
331 private AssignmentManager assignmentManager
;
333 private RSGroupInfoManager rsGroupInfoManager
;
335 // manager of replication
336 private ReplicationPeerManager replicationPeerManager
;
338 private SyncReplicationReplayWALManager syncReplicationReplayWALManager
;
340 // buffer for "fatal error" notices from region servers
341 // in the cluster. This is only used for assisting
342 // operations/debugging.
343 MemoryBoundedLogMessageBuffer rsFatals
;
345 // flag set after we become the active master (used for testing)
346 private volatile boolean activeMaster
= false;
348 // flag set after we complete initialization once active
349 private final ProcedureEvent
<?
> initialized
= new ProcedureEvent
<>("master initialized");
351 // flag set after master services are started,
352 // initialization may have not completed yet.
353 volatile boolean serviceStarted
= false;
355 // Maximum time we should run balancer for
356 private final int maxBalancingTime
;
357 // Maximum percent of regions in transition when balancing
358 private final double maxRitPercent
;
360 private final LockManager lockManager
= new LockManager(this);
362 private RSGroupBasedLoadBalancer balancer
;
363 private BalancerChore balancerChore
;
364 private RegionNormalizerManager regionNormalizerManager
;
365 private ClusterStatusChore clusterStatusChore
;
366 private ClusterStatusPublisher clusterStatusPublisherChore
= null;
367 private SnapshotCleanerChore snapshotCleanerChore
= null;
369 private HbckChore hbckChore
;
370 CatalogJanitor catalogJanitorChore
;
371 // Threadpool for scanning the archive directory, used by the HFileCleaner
372 private DirScanPool hfileCleanerPool
;
373 // Threadpool for scanning the Old logs directory, used by the LogCleaner
374 private DirScanPool logCleanerPool
;
375 private LogCleaner logCleaner
;
376 private HFileCleaner hfileCleaner
;
377 private ReplicationBarrierCleaner replicationBarrierCleaner
;
378 private MobFileCleanerChore mobFileCleanerChore
;
379 private MobFileCompactionChore mobFileCompactionChore
;
380 private RollingUpgradeChore rollingUpgradeChore
;
381 // used to synchronize the mobCompactionStates
382 private final IdLock mobCompactionLock
= new IdLock();
383 // save the information of mob compactions in tables.
384 // the key is table name, the value is the number of compactions in that table.
385 private Map
<TableName
, AtomicInteger
> mobCompactionStates
= Maps
.newConcurrentMap();
387 MasterCoprocessorHost cpHost
;
389 private final boolean preLoadTableDescriptors
;
391 // Time stamps for when a hmaster became active
392 private long masterActiveTime
;
394 // Time stamp for when HMaster finishes becoming Active Master
395 private long masterFinishedInitializationTime
;
397 Map
<String
, Service
> coprocessorServiceHandlers
= Maps
.newHashMap();
399 // monitor for snapshot of hbase tables
400 SnapshotManager snapshotManager
;
401 // monitor for distributed procedures
402 private MasterProcedureManagerHost mpmHost
;
404 private RegionsRecoveryChore regionsRecoveryChore
= null;
406 private RegionsRecoveryConfigManager regionsRecoveryConfigManager
= null;
407 // it is assigned after 'initialized' guard set to true, so should be volatile
408 private volatile MasterQuotaManager quotaManager
;
409 private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier
;
410 private QuotaObserverChore quotaObserverChore
;
411 private SnapshotQuotaObserverChore snapshotQuotaChore
;
413 private ProcedureExecutor
<MasterProcedureEnv
> procedureExecutor
;
414 private ProcedureStore procedureStore
;
416 // the master local storage to store procedure data, meta region locations, etc.
417 private MasterRegion masterRegion
;
419 // handle table states
420 private TableStateManager tableStateManager
;
422 /** jetty server for master to redirect requests to regionserver infoServer */
423 private Server masterJettyServer
;
425 // Determine if we should do normal startup or minimal "single-user" mode with no region
426 // servers and no user tables. Useful for repair and recovery of hbase:meta
427 private final boolean maintenanceMode
;
428 static final String MAINTENANCE_MODE
= "hbase.master.maintenance_mode";
430 // the in process region server for carry system regions in maintenanceMode
431 private JVMClusterUtil
.RegionServerThread maintenanceRegionServer
;
433 // Cached clusterId on stand by masters to serve clusterID requests from clients.
434 private final CachedClusterId cachedClusterId
;
437 * Initializes the HMaster. The steps are as follows:
440 * <li>Initialize the local HRegionServer
441 * <li>Start the ActiveMasterManager.
444 * Remaining steps of initialization occur in
445 * {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the
448 public HMaster(final Configuration conf
) throws IOException
{
449 super(conf
, "Master");
451 if (conf
.getBoolean(MAINTENANCE_MODE
, false)) {
452 LOG
.info("Detected {}=true via configuration.", MAINTENANCE_MODE
);
453 maintenanceMode
= true;
454 } else if (Boolean
.getBoolean(MAINTENANCE_MODE
)) {
455 LOG
.info("Detected {}=true via environment variables.", MAINTENANCE_MODE
);
456 maintenanceMode
= true;
458 maintenanceMode
= false;
460 this.rsFatals
= new MemoryBoundedLogMessageBuffer(
461 conf
.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));
462 LOG
.info("hbase.rootdir={}, hbase.cluster.distributed={}",
463 CommonFSUtils
.getRootDir(this.conf
),
464 this.conf
.getBoolean(HConstants
.CLUSTER_DISTRIBUTED
, false));
466 // Disable usage of meta replicas in the master
467 this.conf
.setBoolean(HConstants
.USE_META_REPLICAS
, false);
469 decorateMasterConfiguration(this.conf
);
471 // Hack! Maps DFSClient => Master for logs. HDFS made this
472 // config param for task trackers, but we can piggyback off of it.
473 if (this.conf
.get("mapreduce.task.attempt.id") == null) {
474 this.conf
.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName
.toString());
477 this.metricsMaster
= new MetricsMaster(new MetricsMasterWrapperImpl(this));
479 // preload table descriptor at startup
480 this.preLoadTableDescriptors
= conf
.getBoolean("hbase.master.preload.tabledescriptors", true);
482 this.maxBalancingTime
= getMaxBalancingTime();
483 this.maxRitPercent
= conf
.getDouble(HConstants
.HBASE_MASTER_BALANCER_MAX_RIT_PERCENT
,
484 HConstants
.DEFAULT_HBASE_MASTER_BALANCER_MAX_RIT_PERCENT
);
486 // Do we publish the status?
488 boolean shouldPublish
= conf
.getBoolean(HConstants
.STATUS_PUBLISHED
,
489 HConstants
.STATUS_PUBLISHED_DEFAULT
);
490 Class
<?
extends ClusterStatusPublisher
.Publisher
> publisherClass
=
491 conf
.getClass(ClusterStatusPublisher
.STATUS_PUBLISHER_CLASS
,
492 ClusterStatusPublisher
.DEFAULT_STATUS_PUBLISHER_CLASS
,
493 ClusterStatusPublisher
.Publisher
.class);
496 if (publisherClass
== null) {
497 LOG
.warn(HConstants
.STATUS_PUBLISHED
+ " is true, but " +
498 ClusterStatusPublisher
.DEFAULT_STATUS_PUBLISHER_CLASS
+
499 " is not set - not publishing status");
501 clusterStatusPublisherChore
= new ClusterStatusPublisher(this, conf
, publisherClass
);
502 LOG
.debug("Created {}", this.clusterStatusPublisherChore
);
503 getChoreService().scheduleChore(clusterStatusPublisherChore
);
506 this.activeMasterManager
= createActiveMasterManager(zooKeeper
, serverName
, this);
507 cachedClusterId
= new CachedClusterId(this, conf
);
508 this.regionServerTracker
= new RegionServerTracker(zooKeeper
, this);
509 this.rpcServices
.start(zooKeeper
);
510 } catch (Throwable t
) {
511 // Make sure we log the exception. HMaster is often started via reflection and the
512 // cause of failed startup is lost.
513 LOG
.error("Failed construction of Master", t
);
519 * Protected to have custom implementations in tests override the default ActiveMaster
522 protected ActiveMasterManager
createActiveMasterManager(ZKWatcher zk
, ServerName sn
,
523 org
.apache
.hadoop
.hbase
.Server server
) throws InterruptedIOException
{
524 return new ActiveMasterManager(zk
, sn
, server
);
528 protected String
getUseThisHostnameInstead(Configuration conf
) {
529 return conf
.get(MASTER_HOSTNAME_KEY
);
532 private void registerConfigurationObservers() {
533 configurationManager
.registerObserver(this.rpcServices
);
534 configurationManager
.registerObserver(this);
537 // Main run loop. Calls through to the regionserver run loop AFTER becoming active Master; will
538 // block in here until then.
542 registerConfigurationObservers();
543 Threads
.setDaemonThreadRunning(new Thread(() -> {
545 int infoPort
= putUpJettyServer();
546 startActiveMasterManager(infoPort
);
547 } catch (Throwable t
) {
548 // Make sure we log the exception.
549 String error
= "Failed to become Active Master";
551 // Abort should have been called already.
556 }), getName() + ":becomeActiveMaster");
557 while (!isStopped() && !isAborted()) {
561 closeClusterConnection();
562 stopServiceThreads();
563 if (this.rpcServices
!= null) {
564 this.rpcServices
.stop();
568 if (this.clusterSchemaService
!= null) {
569 // If on way out, then we are no longer active master.
570 this.clusterSchemaService
.stopAsync();
572 this.clusterSchemaService
.awaitTerminated(
573 getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
,
574 DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
), TimeUnit
.SECONDS
);
575 } catch (TimeoutException te
) {
576 LOG
.warn("Failed shutdown of clusterSchemaService", te
);
579 this.activeMaster
= false;
583 // return the actual infoPort, -1 means disable info server.
584 private int putUpJettyServer() throws IOException
{
585 if (!conf
.getBoolean("hbase.master.infoserver.redirect", true)) {
588 final int infoPort
= conf
.getInt("hbase.master.info.port.orig",
589 HConstants
.DEFAULT_MASTER_INFOPORT
);
590 // -1 is for disabling info server, so no redirecting
591 if (infoPort
< 0 || infoServer
== null) {
594 if (infoPort
== infoServer
.getPort()) {
595 // server is already running
598 final String addr
= conf
.get("hbase.master.info.bindAddress", "0.0.0.0");
599 if (!Addressing
.isLocalAddress(InetAddress
.getByName(addr
))) {
601 "Failed to start redirecting jetty server. Address " + addr
602 + " does not belong to this host. Correct configuration parameter: "
603 + "hbase.master.info.bindAddress";
605 throw new IOException(msg
);
608 // TODO I'm pretty sure we could just add another binding to the InfoServer run by
609 // the RegionServer and have it run the RedirectServlet instead of standing up
610 // a second entire stack here.
611 masterJettyServer
= new Server();
612 final ServerConnector connector
= new ServerConnector(masterJettyServer
);
613 connector
.setHost(addr
);
614 connector
.setPort(infoPort
);
615 masterJettyServer
.addConnector(connector
);
616 masterJettyServer
.setStopAtShutdown(true);
617 masterJettyServer
.setHandler(HttpServer
.buildGzipHandler(masterJettyServer
.getHandler()));
619 final String redirectHostname
=
620 StringUtils
.isBlank(useThisHostnameInstead
) ?
null : useThisHostnameInstead
;
622 final MasterRedirectServlet redirect
= new MasterRedirectServlet(infoServer
, redirectHostname
);
623 final WebAppContext context
= new WebAppContext(null, "/", null, null, null, null, WebAppContext
.NO_SESSIONS
);
624 context
.addServlet(new ServletHolder(redirect
), "/*");
625 context
.setServer(masterJettyServer
);
628 masterJettyServer
.start();
629 } catch (Exception e
) {
630 throw new IOException("Failed to start redirecting jetty server", e
);
632 return connector
.getLocalPort();
636 * For compatibility, if failed with regionserver credentials, try the master one
639 protected void login(UserProvider user
, String host
) throws IOException
{
641 user
.login(SecurityConstants
.REGIONSERVER_KRB_KEYTAB_FILE
,
642 SecurityConstants
.REGIONSERVER_KRB_PRINCIPAL
, host
);
643 } catch (IOException ie
) {
644 user
.login(SecurityConstants
.MASTER_KRB_KEYTAB_FILE
, SecurityConstants
.MASTER_KRB_PRINCIPAL
,
649 public MasterRpcServices
getMasterRpcServices() {
653 public boolean balanceSwitch(final boolean b
) throws IOException
{
654 return getMasterRpcServices().switchBalancer(b
, BalanceSwitchMode
.ASYNC
);
658 protected String
getProcessName() {
663 protected boolean canCreateBaseZNode() {
668 protected boolean canUpdateTableDescriptor() {
673 protected boolean cacheTableDescriptor() {
677 protected MasterRpcServices
createRpcServices() throws IOException
{
678 return new MasterRpcServices(this);
682 protected void configureInfoServer(InfoServer infoServer
) {
683 infoServer
.addUnprivilegedServlet("master-status", "/master-status", MasterStatusServlet
.class);
684 infoServer
.setAttribute(MASTER
, this);
688 protected Class
<?
extends HttpServlet
> getDumpServlet() {
689 return MasterDumpServlet
.class;
693 public MetricsMaster
getMasterMetrics() {
694 return metricsMaster
;
698 * Initialize all ZK based system trackers. But do not include {@link RegionServerTracker}, it
699 * should have already been initialized along with {@link ServerManager}.
701 private void initializeZKBasedSystemTrackers()
702 throws IOException
, KeeperException
, ReplicationException
{
703 if (maintenanceMode
) {
704 // in maintenance mode, always use MaintenanceLoadBalancer.
705 conf
.unset(LoadBalancer
.HBASE_RSGROUP_LOADBALANCER_CLASS
);
706 conf
.setClass(HConstants
.HBASE_MASTER_LOADBALANCER_CLASS
, MaintenanceLoadBalancer
.class,
709 this.balancer
= new RSGroupBasedLoadBalancer();
710 this.loadBalancerTracker
= new LoadBalancerTracker(zooKeeper
, this);
711 this.loadBalancerTracker
.start();
713 this.regionNormalizerManager
=
714 RegionNormalizerFactory
.createNormalizerManager(conf
, zooKeeper
, this);
715 this.configurationManager
.registerObserver(regionNormalizerManager
);
716 this.regionNormalizerManager
.start();
718 this.splitOrMergeTracker
= new SplitOrMergeTracker(zooKeeper
, conf
, this);
719 this.splitOrMergeTracker
.start();
721 // This is for backwards compatible. We do not need the CP for rs group now but if user want to
722 // load it, we need to enable rs group.
723 String
[] cpClasses
= conf
.getStrings(MasterCoprocessorHost
.MASTER_COPROCESSOR_CONF_KEY
);
724 if (cpClasses
!= null) {
725 for (String cpClass
: cpClasses
) {
726 if (RSGroupAdminEndpoint
.class.getName().equals(cpClass
)) {
727 RSGroupUtil
.enableRSGroup(conf
);
732 this.rsGroupInfoManager
= RSGroupInfoManager
.create(this);
734 this.replicationPeerManager
= ReplicationPeerManager
.create(zooKeeper
, conf
, clusterId
);
736 this.drainingServerTracker
= new DrainingServerTracker(zooKeeper
, this, this.serverManager
);
737 this.drainingServerTracker
.start();
739 this.snapshotCleanupTracker
= new SnapshotCleanupTracker(zooKeeper
, this);
740 this.snapshotCleanupTracker
.start();
742 String clientQuorumServers
= conf
.get(HConstants
.CLIENT_ZOOKEEPER_QUORUM
);
743 boolean clientZkObserverMode
= conf
.getBoolean(HConstants
.CLIENT_ZOOKEEPER_OBSERVER_MODE
,
744 HConstants
.DEFAULT_CLIENT_ZOOKEEPER_OBSERVER_MODE
);
745 if (clientQuorumServers
!= null && !clientZkObserverMode
) {
746 // we need to take care of the ZK information synchronization
747 // if given client ZK are not observer nodes
748 ZKWatcher clientZkWatcher
= new ZKWatcher(conf
,
749 getProcessName() + ":" + rpcServices
.getSocketAddress().getPort() + "-clientZK", this,
751 this.metaLocationSyncer
= new MetaLocationSyncer(zooKeeper
, clientZkWatcher
, this);
752 this.metaLocationSyncer
.start();
753 this.masterAddressSyncer
= new MasterAddressSyncer(zooKeeper
, clientZkWatcher
, this);
754 this.masterAddressSyncer
.start();
755 // set cluster id is a one-go effort
756 ZKClusterId
.setClusterId(clientZkWatcher
, fileSystemManager
.getClusterId());
759 // Set the cluster as up. If new RSs, they'll be waiting on this before
760 // going ahead with their startup.
761 boolean wasUp
= this.clusterStatusTracker
.isClusterUp();
762 if (!wasUp
) this.clusterStatusTracker
.setClusterUp();
764 LOG
.info("Active/primary master=" + this.serverName
+
766 Long
.toHexString(this.zooKeeper
.getRecoverableZooKeeper().getSessionId()) +
767 ", setting cluster-up flag (Was=" + wasUp
+ ")");
769 // create/initialize the snapshot manager and other procedure managers
770 this.snapshotManager
= new SnapshotManager();
771 this.mpmHost
= new MasterProcedureManagerHost();
772 this.mpmHost
.register(this.snapshotManager
);
773 this.mpmHost
.register(new MasterFlushTableProcedureManager());
774 this.mpmHost
.loadProcedures(conf
);
775 this.mpmHost
.initialize(this, this.metricsMaster
);
778 // Will be overriden in test to inject customized AssignmentManager
779 @InterfaceAudience.Private
780 protected AssignmentManager
createAssignmentManager(MasterServices master
,
781 MasterRegion masterRegion
) {
782 return new AssignmentManager(master
, masterRegion
);
785 private void tryMigrateMetaLocationsFromZooKeeper() throws IOException
, KeeperException
{
786 // try migrate data from zookeeper
787 try (ResultScanner scanner
=
788 masterRegion
.getScanner(new Scan().addFamily(HConstants
.CATALOG_FAMILY
))) {
789 if (scanner
.next() != null) {
790 // notice that all replicas for a region are in the same row, so the migration can be
791 // done with in a one row put, which means if we have data in catalog family then we can
792 // make sure that the migration is done.
793 LOG
.info("The {} family in master local region already has data in it, skip migrating...",
794 HConstants
.CATALOG_FAMILY
);
799 byte[] row
= CatalogFamilyFormat
.getMetaKeyForRegion(RegionInfoBuilder
.FIRST_META_REGIONINFO
);
800 Put put
= new Put(row
);
801 List
<String
> metaReplicaNodes
= zooKeeper
.getMetaReplicaNodes();
802 StringBuilder info
= new StringBuilder("Migrating meta locations:");
803 for (String metaReplicaNode
: metaReplicaNodes
) {
804 int replicaId
= zooKeeper
.getZNodePaths().getMetaReplicaIdFromZNode(metaReplicaNode
);
805 RegionState state
= MetaTableLocator
.getMetaRegionState(zooKeeper
, replicaId
);
806 info
.append(" ").append(state
);
807 put
.setTimestamp(state
.getStamp());
808 MetaTableAccessor
.addRegionInfo(put
, state
.getRegion());
809 if (state
.getServerName() != null) {
810 MetaTableAccessor
.addLocation(put
, state
.getServerName(), HConstants
.NO_SEQNUM
, replicaId
);
812 put
.add(CellBuilderFactory
.create(CellBuilderType
.SHALLOW_COPY
).setRow(put
.getRow())
813 .setFamily(HConstants
.CATALOG_FAMILY
)
814 .setQualifier(RegionStateStore
.getStateColumn(replicaId
)).setTimestamp(put
.getTimestamp())
815 .setType(Cell
.Type
.Put
).setValue(Bytes
.toBytes(state
.getState().name())).build());
817 if (!put
.isEmpty()) {
818 LOG
.info(info
.toString());
819 masterRegion
.update(r
-> r
.put(put
));
821 LOG
.info("No meta location available on zookeeper, skip migrating...");
826 * Finish initialization of HMaster after becoming the primary master.
828 * The startup order is a bit complicated but very important, do not change it unless you know
829 * what you are doing.
831 * <li>Initialize file system based components - file system manager, wal manager, table
832 * descriptors, etc</li>
833 * <li>Publish cluster id</li>
834 * <li>Here comes the most complicated part - initialize server manager, assignment manager and
835 * region server tracker
837 * <li>Create server manager</li>
838 * <li>Create master local region</li>
839 * <li>Create procedure executor, load the procedures, but do not start workers. We will start it
840 * later after we finish scheduling SCPs to avoid scheduling duplicated SCPs for the same
842 * <li>Create assignment manager and start it, load the meta region state, but do not load data
843 * from meta region</li>
844 * <li>Start region server tracker, construct the online servers set and find out dead servers and
845 * schedule SCP for them. The online servers will be constructed by scanning zk, and we will also
846 * scan the wal directory to find out possible live region servers, and the differences between
847 * these two sets are the dead servers</li>
850 * <li>If this is a new deploy, schedule a InitMetaProcedure to initialize meta</li>
851 * <li>Start necessary service threads - balancer, catalog janitor, executor services, and also
852 * the procedure executor, etc. Notice that the balancer must be created first as assignment
853 * manager may use it when assigning regions.</li>
854 * <li>Wait for meta to be initialized if necessary, start table state manager.</li>
855 * <li>Wait for enough region servers to check-in</li>
856 * <li>Let assignment manager load data from meta and construct region states</li>
857 * <li>Start all other things such as chore services, etc</li>
860 * Notice that now we will not schedule a special procedure to make meta online(unless the first
861 * time where meta has not been created yet), we will rely on SCP to bring meta online.
863 private void finishActiveMasterInitialization(MonitoredTask status
) throws IOException
,
864 InterruptedException
, KeeperException
, ReplicationException
{
866 * We are active master now... go initialize components we need to run.
868 status
.setStatus("Initializing Master file system");
870 this.masterActiveTime
= EnvironmentEdgeManager
.currentTime();
871 // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
873 // always initialize the MemStoreLAB as we use a region to store data in master now, see
875 initializeMemStoreChunkCreator(null);
876 this.fileSystemManager
= new MasterFileSystem(conf
);
877 this.walManager
= new MasterWalManager(this);
879 // warm-up HTDs cache on master initialization
880 if (preLoadTableDescriptors
) {
881 status
.setStatus("Pre-loading table descriptors");
882 this.tableDescriptors
.getAll();
885 // Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
886 // only after it has checked in with the Master. At least a few tests ask Master for clusterId
887 // before it has called its run method and before RegionServer has done the reportForDuty.
888 ClusterId clusterId
= fileSystemManager
.getClusterId();
889 status
.setStatus("Publishing Cluster ID " + clusterId
+ " in ZooKeeper");
890 ZKClusterId
.setClusterId(this.zooKeeper
, fileSystemManager
.getClusterId());
891 this.clusterId
= clusterId
.toString();
893 // Precaution. Put in place the old hbck1 lock file to fence out old hbase1s running their
894 // hbck1s against an hbase2 cluster; it could do damage. To skip this behavior, set
895 // hbase.write.hbck1.lock.file to false.
896 if (this.conf
.getBoolean("hbase.write.hbck1.lock.file", true)) {
897 Pair
<Path
, FSDataOutputStream
> result
= null;
899 result
= HBaseFsck
.checkAndMarkRunningHbck(this.conf
,
900 HBaseFsck
.createLockRetryCounterFactory(this.conf
).create());
902 if (result
!= null) {
903 Closeables
.close(result
.getSecond(), true);
908 status
.setStatus("Initialize ServerManager and schedule SCP for crash servers");
909 // The below two managers must be created before loading procedures, as they will be used during
911 this.serverManager
= createServerManager(this);
912 this.syncReplicationReplayWALManager
= new SyncReplicationReplayWALManager(this);
913 if (!conf
.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK
,
914 DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK
)) {
915 this.splitWALManager
= new SplitWALManager(this);
918 // initialize master local region
919 masterRegion
= MasterRegionFactory
.create(this);
921 tryMigrateMetaLocationsFromZooKeeper();
923 createProcedureExecutor();
924 Map
<Class
<?
>, List
<Procedure
<MasterProcedureEnv
>>> procsByType
=
925 procedureExecutor
.getActiveProceduresNoCopy().stream()
926 .collect(Collectors
.groupingBy(p
-> p
.getClass()));
928 // Create Assignment Manager
929 this.assignmentManager
= createAssignmentManager(this, masterRegion
);
930 this.assignmentManager
.start();
931 // TODO: TRSP can perform as the sub procedure for other procedures, so even if it is marked as
932 // completed, it could still be in the procedure list. This is a bit strange but is another
933 // story, need to verify the implementation for ProcedureExecutor and ProcedureStore.
934 List
<TransitRegionStateProcedure
> ritList
=
935 procsByType
.getOrDefault(TransitRegionStateProcedure
.class, Collections
.emptyList()).stream()
936 .filter(p
-> !p
.isFinished()).map(p
-> (TransitRegionStateProcedure
) p
)
937 .collect(Collectors
.toList());
938 this.assignmentManager
.setupRIT(ritList
);
940 // Start RegionServerTracker with listing of servers found with exiting SCPs -- these should
941 // be registered in the deadServers set -- and with the list of servernames out on the
942 // filesystem that COULD BE 'alive' (we'll schedule SCPs for each and let SCP figure it out).
943 // We also pass dirs that are already 'splitting'... so we can do some checks down in tracker.
944 // TODO: Generate the splitting and live Set in one pass instead of two as we currently do.
945 this.regionServerTracker
.upgrade(
946 procsByType
.getOrDefault(ServerCrashProcedure
.class, Collections
.emptyList()).stream()
947 .map(p
-> (ServerCrashProcedure
) p
).map(p
-> p
.getServerName()).collect(Collectors
.toSet()),
948 walManager
.getLiveServersFromWALDir(), walManager
.getSplittingServersFromWALDir());
949 // This manager must be accessed AFTER hbase:meta is confirmed on line..
950 this.tableStateManager
= new TableStateManager(this);
952 status
.setStatus("Initializing ZK system trackers");
953 initializeZKBasedSystemTrackers();
954 status
.setStatus("Loading last flushed sequence id of regions");
956 this.serverManager
.loadLastFlushedSequenceIds();
957 } catch (IOException e
) {
958 LOG
.info("Failed to load last flushed sequence id of regions"
959 + " from file system", e
);
961 // Set ourselves as active Master now our claim has succeeded up in zk.
962 this.activeMaster
= true;
964 // Start the Zombie master detector after setting master as active, see HBASE-21535
965 Thread zombieDetector
= new Thread(new MasterInitializationMonitor(this),
966 "ActiveMasterInitializationMonitor-" + EnvironmentEdgeManager
.currentTime());
967 zombieDetector
.setDaemon(true);
968 zombieDetector
.start();
970 if (!maintenanceMode
) {
971 // Add the Observer to delete quotas on table deletion before starting all CPs by
972 // default with quota support, avoiding if user specifically asks to not load this Observer.
973 if (QuotaUtil
.isQuotaEnabled(conf
)) {
974 updateConfigurationForQuotasObserver(conf
);
976 // initialize master side coprocessors before we start handling requests
977 status
.setStatus("Initializing master coprocessors");
978 this.cpHost
= new MasterCoprocessorHost(this, this.conf
);
980 // start an in process region server for carrying system regions
981 maintenanceRegionServer
=
982 JVMClusterUtil
.createRegionServerThread(getConfiguration(), HRegionServer
.class, 0);
983 maintenanceRegionServer
.start();
986 // Checking if meta needs initializing.
987 status
.setStatus("Initializing meta table if this is a new deploy");
988 InitMetaProcedure initMetaProc
= null;
989 // Print out state of hbase:meta on startup; helps debugging.
990 if (!this.assignmentManager
.getRegionStates().hasTableRegionStates(TableName
.META_TABLE_NAME
)) {
991 Optional
<InitMetaProcedure
> optProc
= procedureExecutor
.getProcedures().stream()
992 .filter(p
-> p
instanceof InitMetaProcedure
).map(o
-> (InitMetaProcedure
) o
).findAny();
993 initMetaProc
= optProc
.orElseGet(() -> {
994 // schedule an init meta procedure if meta has not been deployed yet
995 InitMetaProcedure temp
= new InitMetaProcedure();
996 procedureExecutor
.submitProcedure(temp
);
1001 // initialize load balancer
1002 this.balancer
.setMasterServices(this);
1003 this.balancer
.initialize();
1004 this.balancer
.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1006 // start up all service threads.
1007 status
.setStatus("Initializing master service threads");
1008 startServiceThreads();
1009 // wait meta to be initialized after we start procedure executor
1010 if (initMetaProc
!= null) {
1011 initMetaProc
.await();
1013 // Wake up this server to check in
1014 sleeper
.skipSleepCycle();
1016 // Wait for region servers to report in.
1017 // With this as part of master initialization, it precludes our being able to start a single
1018 // server that is both Master and RegionServer. Needs more thought. TODO.
1019 String statusStr
= "Wait for region servers to report in";
1020 status
.setStatus(statusStr
);
1021 LOG
.info(Objects
.toString(status
));
1022 waitForRegionServers(status
);
1024 // Check if master is shutting down because issue initializing regionservers or balancer.
1029 status
.setStatus("Starting assignment manager");
1030 // FIRST HBASE:META READ!!!!
1031 // The below cannot make progress w/o hbase:meta being online.
1032 // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
1033 // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
1034 // if it is down. It may take a while to come online. So, wait here until meta if for sure
1035 // available. That's what waitForMetaOnline does.
1036 if (!waitForMetaOnline()) {
1040 TableDescriptor metaDescriptor
=
1041 tableDescriptors
.get(TableName
.META_TABLE_NAME
);
1042 final ColumnFamilyDescriptor tableFamilyDesc
=
1043 metaDescriptor
.getColumnFamily(HConstants
.TABLE_FAMILY
);
1044 final ColumnFamilyDescriptor replBarrierFamilyDesc
=
1045 metaDescriptor
.getColumnFamily(HConstants
.REPLICATION_BARRIER_FAMILY
);
1047 this.assignmentManager
.joinCluster();
1048 // The below depends on hbase:meta being online.
1049 this.assignmentManager
.processOfflineRegions();
1050 // this must be called after the above processOfflineRegions to prevent race
1051 this.assignmentManager
.wakeMetaLoadedEvent();
1053 // for migrating from a version without HBASE-25099, and also for honoring the configuration
1055 if (conf
.get(HConstants
.META_REPLICAS_NUM
) != null) {
1056 int replicasNumInConf
=
1057 conf
.getInt(HConstants
.META_REPLICAS_NUM
, HConstants
.DEFAULT_META_REPLICA_NUM
);
1058 TableDescriptor metaDesc
= tableDescriptors
.get(TableName
.META_TABLE_NAME
);
1059 if (metaDesc
.getRegionReplication() != replicasNumInConf
) {
1060 // it is possible that we already have some replicas before upgrading, so we must set the
1061 // region replication number in meta TableDescriptor directly first, without creating a
1062 // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas.
1063 int existingReplicasCount
=
1064 assignmentManager
.getRegionStates().getRegionsOfTable(TableName
.META_TABLE_NAME
).size();
1065 if (existingReplicasCount
> metaDesc
.getRegionReplication()) {
1066 LOG
.info("Update replica count of hbase:meta from {}(in TableDescriptor)" +
1067 " to {}(existing ZNodes)", metaDesc
.getRegionReplication(), existingReplicasCount
);
1068 metaDesc
= TableDescriptorBuilder
.newBuilder(metaDesc
)
1069 .setRegionReplication(existingReplicasCount
).build();
1070 tableDescriptors
.update(metaDesc
);
1072 // check again, and issue a ModifyTableProcedure if needed
1073 if (metaDesc
.getRegionReplication() != replicasNumInConf
) {
1075 "The {} config is {} while the replica count in TableDescriptor is {}" +
1076 " for hbase:meta, altering...",
1077 HConstants
.META_REPLICAS_NUM
, replicasNumInConf
, metaDesc
.getRegionReplication());
1078 procedureExecutor
.submitProcedure(new ModifyTableProcedure(
1079 procedureExecutor
.getEnvironment(), TableDescriptorBuilder
.newBuilder(metaDesc
)
1080 .setRegionReplication(replicasNumInConf
).build(),
1081 null, metaDesc
, false));
1085 // Initialize after meta is up as below scans meta
1086 FavoredNodesManager fnm
= getFavoredNodesManager();
1088 fnm
.initializeFromMeta();
1091 // set cluster status again after user regions are assigned
1092 this.balancer
.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1094 // Start balancer and meta catalog janitor after meta and regions have been assigned.
1095 status
.setStatus("Starting balancer and catalog janitor");
1096 this.clusterStatusChore
= new ClusterStatusChore(this, balancer
);
1097 getChoreService().scheduleChore(clusterStatusChore
);
1098 this.balancerChore
= new BalancerChore(this);
1099 getChoreService().scheduleChore(balancerChore
);
1100 if (regionNormalizerManager
!= null) {
1101 getChoreService().scheduleChore(regionNormalizerManager
.getRegionNormalizerChore());
1103 this.catalogJanitorChore
= new CatalogJanitor(this);
1104 getChoreService().scheduleChore(catalogJanitorChore
);
1105 this.hbckChore
= new HbckChore(this);
1106 getChoreService().scheduleChore(hbckChore
);
1107 this.serverManager
.startChore();
1109 // Only for rolling upgrade, where we need to migrate the data in namespace table to meta table.
1110 if (!waitForNamespaceOnline()) {
1113 status
.setStatus("Starting cluster schema service");
1115 initClusterSchemaService();
1116 } catch (IllegalStateException e
) {
1117 if (e
.getCause() != null && e
.getCause() instanceof NoSuchColumnFamilyException
1118 && tableFamilyDesc
== null && replBarrierFamilyDesc
== null) {
1119 LOG
.info("ClusterSchema service could not be initialized. This is "
1120 + "expected during HBase 1 to 2 upgrade", e
);
1126 if (this.cpHost
!= null) {
1128 this.cpHost
.preMasterInitialization();
1129 } catch (IOException e
) {
1130 LOG
.error("Coprocessor preMasterInitialization() hook failed", e
);
1134 status
.markComplete("Initialization successful");
1135 LOG
.info(String
.format("Master has completed initialization %.3fsec",
1136 (EnvironmentEdgeManager
.currentTime() - masterActiveTime
) / 1000.0f
));
1137 this.masterFinishedInitializationTime
= EnvironmentEdgeManager
.currentTime();
1138 configurationManager
.registerObserver(this.balancer
);
1139 configurationManager
.registerObserver(this.hfileCleanerPool
);
1140 configurationManager
.registerObserver(this.logCleanerPool
);
1141 configurationManager
.registerObserver(this.hfileCleaner
);
1142 configurationManager
.registerObserver(this.logCleaner
);
1143 configurationManager
.registerObserver(this.regionsRecoveryConfigManager
);
1144 // Set master as 'initialized'.
1145 setInitialized(true);
1147 if (tableFamilyDesc
== null && replBarrierFamilyDesc
== null) {
1148 // create missing CFs in meta table after master is set to 'initialized'.
1149 createMissingCFsInMetaDuringUpgrade(metaDescriptor
);
1151 // Throwing this Exception to abort active master is painful but this
1152 // seems the only way to add missing CFs in meta while upgrading from
1153 // HBase 1 to 2 (where HBase 2 has HBASE-23055 & HBASE-23782 checked-in).
1154 // So, why do we abort active master after adding missing CFs in meta?
1155 // When we reach here, we would have already bypassed NoSuchColumnFamilyException
1156 // in initClusterSchemaService(), meaning ClusterSchemaService is not
1157 // correctly initialized but we bypassed it. Similarly, we bypassed
1158 // tableStateManager.start() as well. Hence, we should better abort
1159 // current active master because our main task - adding missing CFs
1160 // in meta table is done (possible only after master state is set as
1161 // initialized) at the expense of bypassing few important tasks as part
1162 // of active master init routine. So now we abort active master so that
1163 // next active master init will not face any issues and all mandatory
1164 // services will be started during master init phase.
1165 throw new PleaseRestartMasterException("Aborting active master after missing"
1166 + " CFs are successfully added in meta. Subsequent active master "
1167 + "initialization should be uninterrupted");
1170 if (maintenanceMode
) {
1171 LOG
.info("Detected repair mode, skipping final initialization steps.");
1175 assignmentManager
.checkIfShouldMoveSystemRegionAsync();
1176 status
.setStatus("Starting quota manager");
1178 if (QuotaUtil
.isQuotaEnabled(conf
)) {
1179 // Create the quota snapshot notifier
1180 spaceQuotaSnapshotNotifier
= createQuotaSnapshotNotifier();
1181 spaceQuotaSnapshotNotifier
.initialize(getConnection());
1182 this.quotaObserverChore
= new QuotaObserverChore(this, getMasterMetrics());
1183 // Start the chore to read the region FS space reports and act on them
1184 getChoreService().scheduleChore(quotaObserverChore
);
1186 this.snapshotQuotaChore
= new SnapshotQuotaObserverChore(this, getMasterMetrics());
1187 // Start the chore to read snapshots and add their usage to table/NS quotas
1188 getChoreService().scheduleChore(snapshotQuotaChore
);
1190 final SlowLogMasterService slowLogMasterService
= new SlowLogMasterService(conf
, this);
1191 slowLogMasterService
.init();
1193 // clear the dead servers with same host name and port of online server because we are not
1194 // removing dead server with same hostname and port of rs which is trying to check in before
1195 // master initialization. See HBASE-5916.
1196 this.serverManager
.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
1198 // Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
1199 status
.setStatus("Checking ZNode ACLs");
1200 zooKeeper
.checkAndSetZNodeAcls();
1202 status
.setStatus("Initializing MOB Cleaner");
1205 status
.setStatus("Calling postStartMaster coprocessors");
1206 if (this.cpHost
!= null) {
1207 // don't let cp initialization errors kill the master
1209 this.cpHost
.postStartMaster();
1210 } catch (IOException ioe
) {
1211 LOG
.error("Coprocessor postStartMaster() hook failed", ioe
);
1215 zombieDetector
.interrupt();
1218 * After master has started up, lets do balancer post startup initialization. Since this runs
1219 * in activeMasterManager thread, it should be fine.
1221 long start
= EnvironmentEdgeManager
.currentTime();
1222 this.balancer
.postMasterStartupInitialize();
1223 if (LOG
.isDebugEnabled()) {
1224 LOG
.debug("Balancer post startup initialization complete, took " + (
1225 (EnvironmentEdgeManager
.currentTime() - start
) / 1000) + " seconds");
1228 this.rollingUpgradeChore
= new RollingUpgradeChore(this);
1229 getChoreService().scheduleChore(rollingUpgradeChore
);
1232 private void createMissingCFsInMetaDuringUpgrade(
1233 TableDescriptor metaDescriptor
) throws IOException
{
1234 TableDescriptor newMetaDesc
=
1235 TableDescriptorBuilder
.newBuilder(metaDescriptor
)
1236 .setColumnFamily(FSTableDescriptors
.getTableFamilyDescForMeta(conf
))
1237 .setColumnFamily(FSTableDescriptors
.getReplBarrierFamilyDescForMeta())
1239 long pid
= this.modifyTable(TableName
.META_TABLE_NAME
, () -> newMetaDesc
,
1242 while (!(getMasterProcedureExecutor().isFinished(pid
))
1243 && getMasterProcedureExecutor().isRunning() && tries
> 0) {
1246 } catch (InterruptedException e
) {
1247 throw new IOException("Wait interrupted", e
);
1252 throw new HBaseIOException(
1253 "Failed to add table and rep_barrier CFs to meta in a given time.");
1255 Procedure
<?
> result
= getMasterProcedureExecutor().getResult(pid
);
1256 if (result
!= null && result
.isFailed()) {
1257 throw new IOException(
1258 "Failed to add table and rep_barrier CFs to meta. "
1259 + MasterProcedureUtil
.unwrapRemoteIOException(result
));
1265 * Check hbase:meta is up and ready for reading. For use during Master startup only.
1266 * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
1267 * and we will hold here until operator intervention.
1269 @InterfaceAudience.Private
1270 public boolean waitForMetaOnline() {
1271 return isRegionOnline(RegionInfoBuilder
.FIRST_META_REGIONINFO
);
1275 * @return True if region is online and scannable else false if an error or shutdown (Otherwise
1276 * we just block in here holding up all forward-progess).
1278 private boolean isRegionOnline(RegionInfo ri
) {
1279 RetryCounter rc
= null;
1280 while (!isStopped()) {
1281 RegionState rs
= this.assignmentManager
.getRegionStates().getRegionState(ri
);
1282 if (rs
.isOpened()) {
1283 if (this.getServerManager().isServerOnline(rs
.getServerName())) {
1287 // Region is not OPEN.
1288 Optional
<Procedure
<MasterProcedureEnv
>> optProc
= this.procedureExecutor
.getProcedures().
1289 stream().filter(p
-> p
instanceof ServerCrashProcedure
).findAny();
1290 // TODO: Add a page to refguide on how to do repair. Have this log message point to it.
1291 // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
1292 // then how to assign including how to break region lock if one held.
1293 LOG
.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " +
1294 "progress, in holding-pattern until region onlined.",
1295 ri
.getRegionNameAsString(), rs
, optProc
.isPresent());
1296 // Check once-a-minute.
1298 rc
= new RetryCounterFactory(Integer
.MAX_VALUE
, 1000, 60_000
).create();
1300 Threads
.sleep(rc
.getBackoffTimeAndIncrementAttempts());
1306 * Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table
1308 * This is for rolling upgrading, later we will migrate the data in ns table to the ns family of
1309 * meta table. And if this is a new cluster, this method will return immediately as there will be
1310 * no namespace table/region.
1311 * @return True if namespace table is up/online.
1313 private boolean waitForNamespaceOnline() throws IOException
{
1314 TableState nsTableState
=
1315 MetaTableAccessor
.getTableState(getConnection(), TableName
.NAMESPACE_TABLE_NAME
);
1316 if (nsTableState
== null || nsTableState
.isDisabled()) {
1317 // this means we have already migrated the data and disabled or deleted the namespace table,
1318 // or this is a new deploy which does not have a namespace table from the beginning.
1321 List
<RegionInfo
> ris
=
1322 this.assignmentManager
.getRegionStates().getRegionsOfTable(TableName
.NAMESPACE_TABLE_NAME
);
1323 if (ris
.isEmpty()) {
1324 // maybe this will not happen any more, but anyway, no harm to add a check here...
1327 // Else there are namespace regions up in meta. Ensure they are assigned before we go on.
1328 for (RegionInfo ri
: ris
) {
1329 if (!isRegionOnline(ri
)) {
1337 * Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
1338 * automatically remove quotas for a table when that table is deleted.
1340 @InterfaceAudience.Private
1341 public void updateConfigurationForQuotasObserver(Configuration conf
) {
1342 // We're configured to not delete quotas on table deletion, so we don't need to add the obs.
1343 if (!conf
.getBoolean(
1344 MasterQuotasObserver
.REMOVE_QUOTA_ON_TABLE_DELETE
,
1345 MasterQuotasObserver
.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT
)) {
1348 String
[] masterCoprocs
= conf
.getStrings(CoprocessorHost
.MASTER_COPROCESSOR_CONF_KEY
);
1349 final int length
= null == masterCoprocs ?
0 : masterCoprocs
.length
;
1350 String
[] updatedCoprocs
= new String
[length
+ 1];
1352 System
.arraycopy(masterCoprocs
, 0, updatedCoprocs
, 0, masterCoprocs
.length
);
1354 updatedCoprocs
[length
] = MasterQuotasObserver
.class.getName();
1355 conf
.setStrings(CoprocessorHost
.MASTER_COPROCESSOR_CONF_KEY
, updatedCoprocs
);
1358 private void initMobCleaner() {
1359 this.mobFileCleanerChore
= new MobFileCleanerChore(this);
1360 getChoreService().scheduleChore(mobFileCleanerChore
);
1361 this.mobFileCompactionChore
= new MobFileCompactionChore(this);
1362 getChoreService().scheduleChore(mobFileCompactionChore
);
1367 * Create a {@link ServerManager} instance.
1370 * Will be overridden in tests.
1373 @InterfaceAudience.Private
1374 protected ServerManager
createServerManager(final MasterServices master
) throws IOException
{
1375 // We put this out here in a method so can do a Mockito.spy and stub it out
1376 // w/ a mocked up ServerManager.
1377 setupClusterConnection();
1378 return new ServerManager(master
);
1381 private void waitForRegionServers(final MonitoredTask status
)
1382 throws IOException
, InterruptedException
{
1383 this.serverManager
.waitForRegionServers(status
);
1386 // Will be overridden in tests
1387 @InterfaceAudience.Private
1388 protected void initClusterSchemaService() throws IOException
, InterruptedException
{
1389 this.clusterSchemaService
= new ClusterSchemaServiceImpl(this);
1390 this.clusterSchemaService
.startAsync();
1392 this.clusterSchemaService
.awaitRunning(getConfiguration().getInt(
1393 HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
,
1394 DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
), TimeUnit
.SECONDS
);
1395 } catch (TimeoutException toe
) {
1396 throw new IOException("Timedout starting ClusterSchemaService", toe
);
1400 private void initQuotaManager() throws IOException
{
1401 MasterQuotaManager quotaManager
= new MasterQuotaManager(this);
1402 quotaManager
.start();
1403 this.quotaManager
= quotaManager
;
1406 private SpaceQuotaSnapshotNotifier
createQuotaSnapshotNotifier() {
1407 SpaceQuotaSnapshotNotifier notifier
=
1408 SpaceQuotaSnapshotNotifierFactory
.getInstance().create(getConfiguration());
1412 public boolean isCatalogJanitorEnabled() {
1413 return catalogJanitorChore
!= null ? catalogJanitorChore
.getEnabled() : false;
1416 boolean isCleanerChoreEnabled() {
1417 boolean hfileCleanerFlag
= true, logCleanerFlag
= true;
1419 if (hfileCleaner
!= null) {
1420 hfileCleanerFlag
= hfileCleaner
.getEnabled();
1423 if (logCleaner
!= null) {
1424 logCleanerFlag
= logCleaner
.getEnabled();
1427 return (hfileCleanerFlag
&& logCleanerFlag
);
1431 public ServerManager
getServerManager() {
1432 return this.serverManager
;
1436 public MasterFileSystem
getMasterFileSystem() {
1437 return this.fileSystemManager
;
1441 public MasterWalManager
getMasterWalManager() {
1442 return this.walManager
;
1446 public SplitWALManager
getSplitWALManager() {
1447 return splitWALManager
;
1451 public TableStateManager
getTableStateManager() {
1452 return tableStateManager
;
1456 * Start up all services. If any of these threads gets an unhandled exception
1457 * then they just die with a logged message. This should be fine because
1458 * in general, we do not expect the master to get such unhandled exceptions
1459 * as OOMEs; it should be lightly loaded. See what HRegionServer does if
1460 * need to install an unexpected exception handler.
1462 private void startServiceThreads() throws IOException
{
1463 // Start the executor service pools
1464 final int masterOpenRegionPoolSize
= conf
.getInt(
1465 HConstants
.MASTER_OPEN_REGION_THREADS
, HConstants
.MASTER_OPEN_REGION_THREADS_DEFAULT
);
1466 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1467 ExecutorType
.MASTER_OPEN_REGION
).setCorePoolSize(masterOpenRegionPoolSize
));
1468 final int masterCloseRegionPoolSize
= conf
.getInt(
1469 HConstants
.MASTER_CLOSE_REGION_THREADS
, HConstants
.MASTER_CLOSE_REGION_THREADS_DEFAULT
);
1470 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1471 ExecutorType
.MASTER_CLOSE_REGION
).setCorePoolSize(masterCloseRegionPoolSize
));
1472 final int masterServerOpThreads
= conf
.getInt(HConstants
.MASTER_SERVER_OPERATIONS_THREADS
,
1473 HConstants
.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT
);
1474 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1475 ExecutorType
.MASTER_SERVER_OPERATIONS
).setCorePoolSize(masterServerOpThreads
));
1476 final int masterServerMetaOpsThreads
= conf
.getInt(
1477 HConstants
.MASTER_META_SERVER_OPERATIONS_THREADS
,
1478 HConstants
.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT
);
1479 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1480 ExecutorType
.MASTER_META_SERVER_OPERATIONS
).setCorePoolSize(masterServerMetaOpsThreads
));
1481 final int masterLogReplayThreads
= conf
.getInt(
1482 HConstants
.MASTER_LOG_REPLAY_OPS_THREADS
, HConstants
.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT
);
1483 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1484 ExecutorType
.M_LOG_REPLAY_OPS
).setCorePoolSize(masterLogReplayThreads
));
1485 final int masterSnapshotThreads
= conf
.getInt(
1486 SnapshotManager
.SNAPSHOT_POOL_THREADS_KEY
, SnapshotManager
.SNAPSHOT_POOL_THREADS_DEFAULT
);
1487 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1488 ExecutorType
.MASTER_SNAPSHOT_OPERATIONS
).setCorePoolSize(masterSnapshotThreads
)
1489 .setAllowCoreThreadTimeout(true));
1490 final int masterMergeDispatchThreads
= conf
.getInt(HConstants
.MASTER_MERGE_DISPATCH_THREADS
,
1491 HConstants
.MASTER_MERGE_DISPATCH_THREADS_DEFAULT
);
1492 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1493 ExecutorType
.MASTER_MERGE_OPERATIONS
).setCorePoolSize(masterMergeDispatchThreads
)
1494 .setAllowCoreThreadTimeout(true));
1496 // We depend on there being only one instance of this executor running
1497 // at a time. To do concurrency, would need fencing of enable/disable of
1499 // Any time changing this maxThreads to > 1, pls see the comment at
1500 // AccessController#postCompletedCreateTableAction
1501 executorService
.startExecutorService(executorService
.new ExecutorConfig().setExecutorType(
1502 ExecutorType
.MASTER_TABLE_OPERATIONS
).setCorePoolSize(1));
1503 startProcedureExecutor();
1505 // Create log cleaner thread pool
1506 logCleanerPool
= DirScanPool
.getLogCleanerScanPool(conf
);
1507 Map
<String
, Object
> params
= new HashMap
<>();
1508 params
.put(MASTER
, this);
1509 // Start log cleaner thread
1510 int cleanerInterval
=
1511 conf
.getInt(HBASE_MASTER_CLEANER_INTERVAL
, DEFAULT_HBASE_MASTER_CLEANER_INTERVAL
);
1512 this.logCleaner
= new LogCleaner(cleanerInterval
, this, conf
,
1513 getMasterWalManager().getFileSystem(), getMasterWalManager().getOldLogDir(),
1514 logCleanerPool
, params
);
1515 getChoreService().scheduleChore(logCleaner
);
1517 // start the hfile archive cleaner thread
1518 Path archiveDir
= HFileArchiveUtil
.getArchivePath(conf
);
1519 // Create archive cleaner thread pool
1520 hfileCleanerPool
= DirScanPool
.getHFileCleanerScanPool(conf
);
1521 this.hfileCleaner
= new HFileCleaner(cleanerInterval
, this, conf
,
1522 getMasterFileSystem().getFileSystem(), archiveDir
, hfileCleanerPool
, params
);
1523 getChoreService().scheduleChore(hfileCleaner
);
1525 // Regions Reopen based on very high storeFileRefCount is considered enabled
1526 // only if hbase.regions.recovery.store.file.ref.count has value > 0
1527 final int maxStoreFileRefCount
= conf
.getInt(
1528 HConstants
.STORE_FILE_REF_COUNT_THRESHOLD
,
1529 HConstants
.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD
);
1530 if (maxStoreFileRefCount
> 0) {
1531 this.regionsRecoveryChore
= new RegionsRecoveryChore(this, conf
, this);
1532 getChoreService().scheduleChore(this.regionsRecoveryChore
);
1534 LOG
.info("Reopening regions with very high storeFileRefCount is disabled. " +
1535 "Provide threshold value > 0 for {} to enable it.",
1536 HConstants
.STORE_FILE_REF_COUNT_THRESHOLD
);
1539 this.regionsRecoveryConfigManager
= new RegionsRecoveryConfigManager(this);
1541 replicationBarrierCleaner
= new ReplicationBarrierCleaner(conf
, this, getConnection(),
1542 replicationPeerManager
);
1543 getChoreService().scheduleChore(replicationBarrierCleaner
);
1545 final boolean isSnapshotChoreEnabled
= this.snapshotCleanupTracker
1546 .isSnapshotCleanupEnabled();
1547 this.snapshotCleanerChore
= new SnapshotCleanerChore(this, conf
, getSnapshotManager());
1548 if (isSnapshotChoreEnabled
) {
1549 getChoreService().scheduleChore(this.snapshotCleanerChore
);
1551 if (LOG
.isTraceEnabled()) {
1552 LOG
.trace("Snapshot Cleaner Chore is disabled. Not starting up the chore..");
1555 serviceStarted
= true;
1556 if (LOG
.isTraceEnabled()) {
1557 LOG
.trace("Started service threads");
1561 protected void stopServiceThreads() {
1562 if (masterJettyServer
!= null) {
1563 LOG
.info("Stopping master jetty server");
1565 masterJettyServer
.stop();
1566 } catch (Exception e
) {
1567 LOG
.error("Failed to stop master jetty server", e
);
1571 stopExecutorService();
1572 if (hfileCleanerPool
!= null) {
1573 hfileCleanerPool
.shutdownNow();
1574 hfileCleanerPool
= null;
1576 if (logCleanerPool
!= null) {
1577 logCleanerPool
.shutdownNow();
1578 logCleanerPool
= null;
1580 if (maintenanceRegionServer
!= null) {
1581 maintenanceRegionServer
.getRegionServer().stop(HBASE_MASTER_CLEANER_INTERVAL
);
1584 LOG
.debug("Stopping service threads");
1585 // stop procedure executor prior to other services such as server manager and assignment
1586 // manager, as these services are important for some running procedures. See HBASE-24117 for
1588 stopProcedureExecutor();
1590 if (regionNormalizerManager
!= null) {
1591 regionNormalizerManager
.stop();
1593 if (this.quotaManager
!= null) {
1594 this.quotaManager
.stop();
1597 if (this.activeMasterManager
!= null) {
1598 this.activeMasterManager
.stop();
1600 if (this.serverManager
!= null) {
1601 this.serverManager
.stop();
1603 if (this.assignmentManager
!= null) {
1604 this.assignmentManager
.stop();
1607 if (masterRegion
!= null) {
1608 masterRegion
.close(isAborted());
1610 if (this.walManager
!= null) {
1611 this.walManager
.stop();
1613 if (this.fileSystemManager
!= null) {
1614 this.fileSystemManager
.stop();
1616 if (this.mpmHost
!= null) {
1617 this.mpmHost
.stop("server shutting down.");
1619 if (this.regionServerTracker
!= null) {
1620 this.regionServerTracker
.stop();
1624 private void createProcedureExecutor() throws IOException
{
1625 MasterProcedureEnv procEnv
= new MasterProcedureEnv(this);
1627 new RegionProcedureStore(this, masterRegion
, new MasterProcedureEnv
.FsUtilsLeaseRecovery(this));
1628 procedureStore
.registerListener(new ProcedureStoreListener() {
1631 public void abortProcess() {
1632 abort("The Procedure Store lost the lease", null);
1635 MasterProcedureScheduler procedureScheduler
= procEnv
.getProcedureScheduler();
1636 procedureExecutor
= new ProcedureExecutor
<>(conf
, procEnv
, procedureStore
, procedureScheduler
);
1637 configurationManager
.registerObserver(procEnv
);
1639 int cpus
= Runtime
.getRuntime().availableProcessors();
1640 final int numThreads
= conf
.getInt(MasterProcedureConstants
.MASTER_PROCEDURE_THREADS
, Math
.max(
1641 (cpus
> 0 ? cpus
/ 4 : 0), MasterProcedureConstants
.DEFAULT_MIN_MASTER_PROCEDURE_THREADS
));
1642 final boolean abortOnCorruption
=
1643 conf
.getBoolean(MasterProcedureConstants
.EXECUTOR_ABORT_ON_CORRUPTION
,
1644 MasterProcedureConstants
.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION
);
1645 procedureStore
.start(numThreads
);
1646 // Just initialize it but do not start the workers, we will start the workers later by calling
1647 // startProcedureExecutor. See the javadoc for finishActiveMasterInitialization for more
1649 procedureExecutor
.init(numThreads
, abortOnCorruption
);
1650 if (!procEnv
.getRemoteDispatcher().start()) {
1651 throw new HBaseIOException("Failed start of remote dispatcher");
1655 // will be override in UT
1656 protected void startProcedureExecutor() throws IOException
{
1657 procedureExecutor
.startWorkers();
1661 * Turn on/off Snapshot Cleanup Chore
1663 * @param on indicates whether Snapshot Cleanup Chore is to be run
1665 void switchSnapshotCleanup(final boolean on
, final boolean synchronous
) {
1667 synchronized (this.snapshotCleanerChore
) {
1668 switchSnapshotCleanup(on
);
1671 switchSnapshotCleanup(on
);
1675 private void switchSnapshotCleanup(final boolean on
) {
1677 snapshotCleanupTracker
.setSnapshotCleanupEnabled(on
);
1679 getChoreService().scheduleChore(this.snapshotCleanerChore
);
1681 this.snapshotCleanerChore
.cancel();
1683 } catch (KeeperException e
) {
1684 LOG
.error("Error updating snapshot cleanup mode to {}", on
, e
);
1689 private void stopProcedureExecutor() {
1690 if (procedureExecutor
!= null) {
1691 configurationManager
.deregisterObserver(procedureExecutor
.getEnvironment());
1692 procedureExecutor
.getEnvironment().getRemoteDispatcher().stop();
1693 procedureExecutor
.stop();
1694 procedureExecutor
.join();
1695 procedureExecutor
= null;
1698 if (procedureStore
!= null) {
1699 procedureStore
.stop(isAborted());
1700 procedureStore
= null;
1704 protected void stopChores() {
1705 shutdownChore(mobFileCleanerChore
);
1706 shutdownChore(mobFileCompactionChore
);
1707 shutdownChore(balancerChore
);
1708 if (regionNormalizerManager
!= null) {
1709 shutdownChore(regionNormalizerManager
.getRegionNormalizerChore());
1711 shutdownChore(clusterStatusChore
);
1712 shutdownChore(catalogJanitorChore
);
1713 shutdownChore(clusterStatusPublisherChore
);
1714 shutdownChore(snapshotQuotaChore
);
1715 shutdownChore(logCleaner
);
1716 shutdownChore(hfileCleaner
);
1717 shutdownChore(replicationBarrierCleaner
);
1718 shutdownChore(snapshotCleanerChore
);
1719 shutdownChore(hbckChore
);
1720 shutdownChore(regionsRecoveryChore
);
1721 shutdownChore(rollingUpgradeChore
);
1725 * @return Get remote side's InetAddress
1727 InetAddress
getRemoteInetAddress(final int port
,
1728 final long serverStartCode
) throws UnknownHostException
{
1729 // Do it out here in its own little method so can fake an address when
1730 // mocking up in tests.
1731 InetAddress ia
= RpcServer
.getRemoteIp();
1733 // The call could be from the local regionserver,
1734 // in which case, there is no remote address.
1735 if (ia
== null && serverStartCode
== startcode
) {
1736 InetSocketAddress isa
= rpcServices
.getSocketAddress();
1737 if (isa
!= null && isa
.getPort() == port
) {
1738 ia
= isa
.getAddress();
1745 * @return Maximum time we should run balancer for
1747 private int getMaxBalancingTime() {
1748 // if max balancing time isn't set, defaulting it to period time
1749 int maxBalancingTime
= getConfiguration().getInt(HConstants
.HBASE_BALANCER_MAX_BALANCING
,
1751 .getInt(HConstants
.HBASE_BALANCER_PERIOD
, HConstants
.DEFAULT_HBASE_BALANCER_PERIOD
));
1752 return maxBalancingTime
;
1756 * @return Maximum number of regions in transition
1758 private int getMaxRegionsInTransition() {
1759 int numRegions
= this.assignmentManager
.getRegionStates().getRegionAssignments().size();
1760 return Math
.max((int) Math
.floor(numRegions
* this.maxRitPercent
), 1);
1764 * It first sleep to the next balance plan start time. Meanwhile, throttling by the max
1765 * number regions in transition to protect availability.
1766 * @param nextBalanceStartTime The next balance plan start time
1767 * @param maxRegionsInTransition max number of regions in transition
1768 * @param cutoffTime when to exit balancer
1770 private void balanceThrottling(long nextBalanceStartTime
, int maxRegionsInTransition
,
1772 boolean interrupted
= false;
1774 // Sleep to next balance plan start time
1775 // But if there are zero regions in transition, it can skip sleep to speed up.
1776 while (!interrupted
&& EnvironmentEdgeManager
.currentTime() < nextBalanceStartTime
1777 && this.assignmentManager
.getRegionStates().hasRegionsInTransition()) {
1780 } catch (InterruptedException ie
) {
1785 // Throttling by max number regions in transition
1787 && maxRegionsInTransition
> 0
1788 && this.assignmentManager
.getRegionStates().getRegionsInTransitionCount()
1789 >= maxRegionsInTransition
&& EnvironmentEdgeManager
.currentTime() <= cutoffTime
) {
1791 // sleep if the number of regions in transition exceeds the limit
1793 } catch (InterruptedException ie
) {
1798 if (interrupted
) Thread
.currentThread().interrupt();
1801 public BalanceResponse
balance() throws IOException
{
1802 return balance(BalanceRequest
.defaultInstance());
1806 * Trigger a normal balance, see {@link HMaster#balance()} . If the balance is not executed
1807 * this time, the metrics related to the balance will be updated.
1809 * When balance is running, related metrics will be updated at the same time. But if some
1810 * checking logic failed and cause the balancer exit early, we lost the chance to update
1811 * balancer metrics. This will lead to user missing the latest balancer info.
1813 public BalanceResponse
balanceOrUpdateMetrics() throws IOException
{
1814 synchronized (this.balancer
) {
1815 BalanceResponse response
= balance();
1816 if (!response
.isBalancerRan()) {
1817 Map
<TableName
, Map
<ServerName
, List
<RegionInfo
>>> assignments
=
1818 this.assignmentManager
.getRegionStates().getAssignmentsForBalancer(this.tableStateManager
,
1819 this.serverManager
.getOnlineServersList());
1820 for (Map
<ServerName
, List
<RegionInfo
>> serverMap
: assignments
.values()) {
1821 serverMap
.keySet().removeAll(this.serverManager
.getDrainingServersList());
1823 this.balancer
.updateBalancerLoadInfo(assignments
);
1830 * Checks master state before initiating action over region topology.
1831 * @param action the name of the action under consideration, for logging.
1832 * @return {@code true} when the caller should exit early, {@code false} otherwise.
1835 public boolean skipRegionManagementAction(final String action
) {
1836 // Note: this method could be `default` on MasterServices if but for logging.
1837 if (!isInitialized()) {
1838 LOG
.debug("Master has not been initialized, don't run {}.", action
);
1841 if (this.getServerManager().isClusterShutdown()) {
1842 LOG
.info("Cluster is shutting down, don't run {}.", action
);
1845 if (isInMaintenanceMode()) {
1846 LOG
.info("Master is in maintenance mode, don't run {}.", action
);
1852 public BalanceResponse
balance(BalanceRequest request
) throws IOException
{
1855 BalanceResponse
.Builder responseBuilder
= BalanceResponse
.newBuilder();
1857 if (loadBalancerTracker
== null
1858 || !(loadBalancerTracker
.isBalancerOn() || request
.isDryRun())) {
1859 return responseBuilder
.build();
1862 if (skipRegionManagementAction("balancer")) {
1863 return responseBuilder
.build();
1866 synchronized (this.balancer
) {
1867 // Only allow one balance run at at time.
1868 if (this.assignmentManager
.hasRegionsInTransition()) {
1869 List
<RegionStateNode
> regionsInTransition
= assignmentManager
.getRegionsInTransition();
1870 // if hbase:meta region is in transition, result of assignment cannot be recorded
1871 // ignore the force flag in that case
1872 boolean metaInTransition
= assignmentManager
.isMetaRegionInTransition();
1873 List
<RegionStateNode
> toPrint
= regionsInTransition
;
1875 boolean truncated
= false;
1876 if (regionsInTransition
.size() > max
) {
1877 toPrint
= regionsInTransition
.subList(0, max
);
1881 if (!request
.isIgnoreRegionsInTransition() || metaInTransition
) {
1882 LOG
.info("Not running balancer (ignoreRIT=false" + ", metaRIT=" + metaInTransition
+
1883 ") because " + regionsInTransition
.size() + " region(s) in transition: " + toPrint
1884 + (truncated?
"(truncated list)": ""));
1885 return responseBuilder
.build();
1888 if (this.serverManager
.areDeadServersInProgress()) {
1889 LOG
.info("Not running balancer because processing dead regionserver(s): " +
1890 this.serverManager
.getDeadServers());
1891 return responseBuilder
.build();
1894 if (this.cpHost
!= null) {
1896 if (this.cpHost
.preBalance(request
)) {
1897 LOG
.debug("Coprocessor bypassing balancer request");
1898 return responseBuilder
.build();
1900 } catch (IOException ioe
) {
1901 LOG
.error("Error invoking master coprocessor preBalance()", ioe
);
1902 return responseBuilder
.build();
1906 Map
<TableName
, Map
<ServerName
, List
<RegionInfo
>>> assignments
=
1907 this.assignmentManager
.getRegionStates()
1908 .getAssignmentsForBalancer(tableStateManager
, this.serverManager
.getOnlineServersList());
1909 for (Map
<ServerName
, List
<RegionInfo
>> serverMap
: assignments
.values()) {
1910 serverMap
.keySet().removeAll(this.serverManager
.getDrainingServersList());
1913 //Give the balancer the current cluster state.
1914 this.balancer
.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1916 List
<RegionPlan
> plans
= this.balancer
.balanceCluster(assignments
);
1918 responseBuilder
.setBalancerRan(true).setMovesCalculated(plans
== null ?
0 : plans
.size());
1920 if (skipRegionManagementAction("balancer")) {
1921 // make one last check that the cluster isn't shutting down before proceeding.
1922 return responseBuilder
.build();
1925 // For dry run we don't actually want to execute the moves, but we do want
1926 // to execute the coprocessor below
1927 List
<RegionPlan
> sucRPs
= request
.isDryRun()
1928 ? Collections
.emptyList()
1929 : executeRegionPlansWithThrottling(plans
);
1931 if (this.cpHost
!= null) {
1933 this.cpHost
.postBalance(request
, sucRPs
);
1934 } catch (IOException ioe
) {
1935 // balancing already succeeded so don't change the result
1936 LOG
.error("Error invoking master coprocessor postBalance()", ioe
);
1940 responseBuilder
.setMovesExecuted(sucRPs
.size());
1943 // If LoadBalancer did not generate any plans, it means the cluster is already balanced.
1944 // Return true indicating a success.
1945 return responseBuilder
.build();
1949 * Execute region plans with throttling
1950 * @param plans to execute
1951 * @return succeeded plans
1953 public List
<RegionPlan
> executeRegionPlansWithThrottling(List
<RegionPlan
> plans
) {
1954 List
<RegionPlan
> successRegionPlans
= new ArrayList
<>();
1955 int maxRegionsInTransition
= getMaxRegionsInTransition();
1956 long balanceStartTime
= EnvironmentEdgeManager
.currentTime();
1957 long cutoffTime
= balanceStartTime
+ this.maxBalancingTime
;
1958 int rpCount
= 0; // number of RegionPlans balanced so far
1959 if (plans
!= null && !plans
.isEmpty()) {
1960 int balanceInterval
= this.maxBalancingTime
/ plans
.size();
1961 LOG
.info("Balancer plans size is " + plans
.size() + ", the balance interval is "
1962 + balanceInterval
+ " ms, and the max number regions in transition is "
1963 + maxRegionsInTransition
);
1965 for (RegionPlan plan
: plans
) {
1966 LOG
.info("balance " + plan
);
1969 this.assignmentManager
.balance(plan
);
1970 } catch (HBaseIOException hioe
) {
1971 //should ignore failed plans here, avoiding the whole balance plans be aborted
1972 //later calls of balance() can fetch up the failed and skipped plans
1973 LOG
.warn("Failed balance plan {}, skipping...", plan
, hioe
);
1975 //rpCount records balance plans processed, does not care if a plan succeeds
1977 successRegionPlans
.add(plan
);
1979 if (this.maxBalancingTime
> 0) {
1980 balanceThrottling(balanceStartTime
+ rpCount
* balanceInterval
, maxRegionsInTransition
,
1984 // if performing next balance exceeds cutoff time, exit the loop
1985 if (this.maxBalancingTime
> 0 && rpCount
< plans
.size()
1986 && EnvironmentEdgeManager
.currentTime() > cutoffTime
) {
1987 // TODO: After balance, there should not be a cutoff time (keeping it as
1988 // a security net for now)
1989 LOG
.debug("No more balancing till next balance run; maxBalanceTime="
1990 + this.maxBalancingTime
);
1995 LOG
.debug("Balancer is going into sleep until next period in {}ms", getConfiguration()
1996 .getInt(HConstants
.HBASE_BALANCER_PERIOD
, HConstants
.DEFAULT_HBASE_BALANCER_PERIOD
));
1997 return successRegionPlans
;
2001 public RegionNormalizerManager
getRegionNormalizerManager() {
2002 return regionNormalizerManager
;
2006 public boolean normalizeRegions(
2007 final NormalizeTableFilterParams ntfp
,
2008 final boolean isHighPriority
2009 ) throws IOException
{
2010 if (regionNormalizerManager
== null || !regionNormalizerManager
.isNormalizerOn()) {
2011 LOG
.debug("Region normalization is disabled, don't run region normalizer.");
2014 if (skipRegionManagementAction("region normalizer")) {
2017 if (assignmentManager
.hasRegionsInTransition()) {
2021 final Set
<TableName
> matchingTables
= getTableDescriptors(new LinkedList
<>(),
2022 ntfp
.getNamespace(), ntfp
.getRegex(), ntfp
.getTableNames(), false)
2024 .map(TableDescriptor
::getTableName
)
2025 .collect(Collectors
.toSet());
2026 final Set
<TableName
> allEnabledTables
=
2027 tableStateManager
.getTablesInStates(TableState
.State
.ENABLED
);
2028 final List
<TableName
> targetTables
=
2029 new ArrayList
<>(Sets
.intersection(matchingTables
, allEnabledTables
));
2030 Collections
.shuffle(targetTables
);
2031 return regionNormalizerManager
.normalizeRegions(targetTables
, isHighPriority
);
2035 * @return Client info for use as prefix on an audit log string; who did an action
2038 public String
getClientIdAuditPrefix() {
2039 return "Client=" + RpcServer
.getRequestUserName().orElse(null)
2040 + "/" + RpcServer
.getRemoteAddress().orElse(null);
2044 * Switch for the background CatalogJanitor thread.
2045 * Used for testing. The thread will continue to run. It will just be a noop
2047 * @param b If false, the catalog janitor won't do anything.
2049 public void setCatalogJanitorEnabled(final boolean b
) {
2050 this.catalogJanitorChore
.setEnabled(b
);
2054 public long mergeRegions(
2055 final RegionInfo
[] regionsToMerge
,
2056 final boolean forcible
,
2058 final long nonce
) throws IOException
{
2061 if (!isSplitOrMergeEnabled(MasterSwitchType
.MERGE
)) {
2062 String regionsStr
= Arrays
.deepToString(regionsToMerge
);
2063 LOG
.warn("Merge switch is off! skip merge of " + regionsStr
);
2064 throw new DoNotRetryIOException("Merge of " + regionsStr
+
2065 " failed because merge switch is off");
2068 final String mergeRegionsStr
= Arrays
.stream(regionsToMerge
).map(RegionInfo
::getEncodedName
)
2069 .collect(Collectors
.joining(", "));
2070 return MasterProcedureUtil
.submitProcedure(new NonceProcedureRunnable(this, ng
, nonce
) {
2072 protected void run() throws IOException
{
2073 getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge
);
2074 String aid
= getClientIdAuditPrefix();
2075 LOG
.info("{} merge regions {}", aid
, mergeRegionsStr
);
2076 submitProcedure(new MergeTableRegionsProcedure(procedureExecutor
.getEnvironment(),
2077 regionsToMerge
, forcible
));
2078 getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge
);
2082 protected String
getDescription() {
2083 return "MergeTableProcedure";
2089 public long splitRegion(final RegionInfo regionInfo
, final byte[] splitRow
,
2090 final long nonceGroup
, final long nonce
)
2091 throws IOException
{
2094 if (!isSplitOrMergeEnabled(MasterSwitchType
.SPLIT
)) {
2095 LOG
.warn("Split switch is off! skip split of " + regionInfo
);
2096 throw new DoNotRetryIOException("Split region " + regionInfo
.getRegionNameAsString() +
2097 " failed due to split switch off");
2100 return MasterProcedureUtil
.submitProcedure(
2101 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2103 protected void run() throws IOException
{
2104 getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo
.getTable(), splitRow
);
2105 LOG
.info(getClientIdAuditPrefix() + " split " + regionInfo
.getRegionNameAsString());
2107 // Execute the operation asynchronously
2108 submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo
, splitRow
));
2112 protected String
getDescription() {
2113 return "SplitTableProcedure";
2118 private void warmUpRegion(ServerName server
, RegionInfo region
) {
2119 FutureUtils
.addListener(asyncClusterConnection
.getRegionServerAdmin(server
)
2120 .warmupRegion(RequestConverter
.buildWarmupRegionRequest(region
)), (r
, e
) -> {
2122 LOG
.warn("Failed to warm up region {} on server {}", region
, server
, e
);
2127 // Public so can be accessed by tests. Blocks until move is done.
2128 // Replace with an async implementation from which you can get
2129 // a success/failure result.
2130 @InterfaceAudience.Private
2131 public void move(final byte[] encodedRegionName
, byte[] destServerName
) throws IOException
{
2132 RegionState regionState
= assignmentManager
.getRegionStates().
2133 getRegionState(Bytes
.toString(encodedRegionName
));
2136 if (regionState
!= null) {
2137 hri
= regionState
.getRegion();
2139 throw new UnknownRegionException(Bytes
.toStringBinary(encodedRegionName
));
2143 List
<ServerName
> exclude
= hri
.getTable().isSystemTable() ? assignmentManager
.getExcludedServersForSystemTable()
2144 : new ArrayList
<>(1);
2145 if (destServerName
!= null && exclude
.contains(ServerName
.valueOf(Bytes
.toString(destServerName
)))) {
2147 Bytes
.toString(encodedRegionName
) + " can not move to " + Bytes
.toString(destServerName
)
2148 + " because the server is in exclude list");
2149 destServerName
= null;
2151 if (destServerName
== null || destServerName
.length
== 0) {
2152 LOG
.info("Passed destination servername is null/empty so " +
2153 "choosing a server at random");
2154 exclude
.add(regionState
.getServerName());
2155 final List
<ServerName
> destServers
= this.serverManager
.createDestinationServersList(exclude
);
2156 dest
= balancer
.randomAssignment(hri
, destServers
);
2158 LOG
.debug("Unable to determine a plan to assign " + hri
);
2162 ServerName candidate
= ServerName
.valueOf(Bytes
.toString(destServerName
));
2163 dest
= balancer
.randomAssignment(hri
, Lists
.newArrayList(candidate
));
2165 LOG
.debug("Unable to determine a plan to assign " + hri
);
2168 // TODO: deal with table on master for rs group.
2169 if (dest
.equals(serverName
)) {
2170 // To avoid unnecessary region moving later by balancer. Don't put user
2171 // regions on master.
2172 LOG
.debug("Skipping move of region " + hri
.getRegionNameAsString() +
2173 " to avoid unnecessary region moving later by load balancer," +
2174 " because it should not be on master");
2179 if (dest
.equals(regionState
.getServerName())) {
2180 LOG
.debug("Skipping move of region " + hri
.getRegionNameAsString()
2181 + " because region already assigned to the same server " + dest
+ ".");
2185 // Now we can do the move
2186 RegionPlan rp
= new RegionPlan(hri
, regionState
.getServerName(), dest
);
2187 assert rp
.getDestination() != null: rp
.toString() + " " + dest
;
2191 if (this.cpHost
!= null) {
2192 this.cpHost
.preMove(hri
, rp
.getSource(), rp
.getDestination());
2195 TransitRegionStateProcedure proc
=
2196 this.assignmentManager
.createMoveRegionProcedure(rp
.getRegionInfo(), rp
.getDestination());
2197 // Warmup the region on the destination before initiating the move.
2198 // A region server could reject the close request because it either does not
2199 // have the specified region or the region is being split.
2200 warmUpRegion(rp
.getDestination(), hri
);
2202 LOG
.info(getClientIdAuditPrefix() + " move " + rp
+ ", running balancer");
2203 Future
<byte[]> future
= ProcedureSyncWait
.submitProcedure(this.procedureExecutor
, proc
);
2205 // Is this going to work? Will we throw exception on error?
2206 // TODO: CompletableFuture rather than this stunted Future.
2208 } catch (InterruptedException
| ExecutionException e
) {
2209 throw new HBaseIOException(e
);
2211 if (this.cpHost
!= null) {
2212 this.cpHost
.postMove(hri
, rp
.getSource(), rp
.getDestination());
2214 } catch (IOException ioe
) {
2215 if (ioe
instanceof HBaseIOException
) {
2216 throw (HBaseIOException
)ioe
;
2218 throw new HBaseIOException(ioe
);
2223 public long createTable(final TableDescriptor tableDescriptor
, final byte[][] splitKeys
,
2224 final long nonceGroup
, final long nonce
) throws IOException
{
2226 TableDescriptor desc
= getMasterCoprocessorHost().preCreateTableRegionsInfos(tableDescriptor
);
2228 throw new IOException("Creation for " + tableDescriptor
+ " is canceled by CP");
2230 String namespace
= desc
.getTableName().getNamespaceAsString();
2231 this.clusterSchemaService
.getNamespace(namespace
);
2233 RegionInfo
[] newRegions
= ModifyRegionUtils
.createRegionInfos(desc
, splitKeys
);
2234 TableDescriptorChecker
.sanityCheck(conf
, desc
);
2236 return MasterProcedureUtil
2237 .submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2239 protected void run() throws IOException
{
2240 getMaster().getMasterCoprocessorHost().preCreateTable(desc
, newRegions
);
2242 LOG
.info(getClientIdAuditPrefix() + " create " + desc
);
2244 // TODO: We can handle/merge duplicate requests, and differentiate the case of
2245 // TableExistsException by saying if the schema is the same or not.
2247 // We need to wait for the procedure to potentially fail due to "prepare" sanity
2248 // checks. This will block only the beginning of the procedure. See HBASE-19953.
2249 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
2251 new CreateTableProcedure(procedureExecutor
.getEnvironment(), desc
, newRegions
, latch
));
2254 getMaster().getMasterCoprocessorHost().postCreateTable(desc
, newRegions
);
2258 protected String
getDescription() {
2259 return "CreateTableProcedure";
2265 public long createSystemTable(final TableDescriptor tableDescriptor
) throws IOException
{
2267 throw new MasterNotRunningException();
2270 TableName tableName
= tableDescriptor
.getTableName();
2271 if (!(tableName
.isSystemTable())) {
2272 throw new IllegalArgumentException(
2273 "Only system table creation can use this createSystemTable API");
2276 RegionInfo
[] newRegions
= ModifyRegionUtils
.createRegionInfos(tableDescriptor
, null);
2278 LOG
.info(getClientIdAuditPrefix() + " create " + tableDescriptor
);
2280 // This special create table is called locally to master. Therefore, no RPC means no need
2281 // to use nonce to detect duplicated RPC call.
2282 long procId
= this.procedureExecutor
.submitProcedure(
2283 new CreateTableProcedure(procedureExecutor
.getEnvironment(), tableDescriptor
, newRegions
));
2288 private void startActiveMasterManager(int infoPort
) throws KeeperException
{
2289 String backupZNode
= ZNodePaths
.joinZNode(
2290 zooKeeper
.getZNodePaths().backupMasterAddressesZNode
, serverName
.toString());
2292 * Add a ZNode for ourselves in the backup master directory since we
2293 * may not become the active master. If so, we want the actual active
2294 * master to know we are backup masters, so that it won't assign
2295 * regions to us if so configured.
2297 * If we become the active master later, ActiveMasterManager will delete
2298 * this node explicitly. If we crash before then, ZooKeeper will delete
2299 * this node for us since it is ephemeral.
2301 LOG
.info("Adding backup master ZNode " + backupZNode
);
2302 if (!MasterAddressTracker
.setMasterAddress(zooKeeper
, backupZNode
, serverName
, infoPort
)) {
2303 LOG
.warn("Failed create of " + backupZNode
+ " by " + serverName
);
2305 this.activeMasterManager
.setInfoPort(infoPort
);
2306 int timeout
= conf
.getInt(HConstants
.ZK_SESSION_TIMEOUT
, HConstants
.DEFAULT_ZK_SESSION_TIMEOUT
);
2307 // If we're a backup master, stall until a primary to write this address
2308 if (conf
.getBoolean(HConstants
.MASTER_TYPE_BACKUP
, HConstants
.DEFAULT_MASTER_TYPE_BACKUP
)) {
2309 LOG
.debug("HMaster started in backup mode. Stalling until master znode is written.");
2310 // This will only be a minute or so while the cluster starts up,
2311 // so don't worry about setting watches on the parent znode
2312 while (!activeMasterManager
.hasActiveMaster()) {
2313 LOG
.debug("Waiting for master address and cluster state znode to be written.");
2314 Threads
.sleep(timeout
);
2317 MonitoredTask status
= TaskMonitor
.get().createStatus("Master startup");
2318 status
.setDescription("Master startup");
2320 if (activeMasterManager
.blockUntilBecomingActiveMaster(timeout
, status
)) {
2321 finishActiveMasterInitialization(status
);
2323 } catch (Throwable t
) {
2324 status
.setStatus("Failed to become active: " + t
.getMessage());
2325 LOG
.error(HBaseMarkers
.FATAL
, "Failed to become active master", t
);
2326 // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
2327 if (t
instanceof NoClassDefFoundError
&& t
.getMessage().
2328 contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")) {
2329 // improved error message for this special case
2330 abort("HBase is having a problem with its Hadoop jars. You may need to recompile " +
2331 "HBase against Hadoop version " + org
.apache
.hadoop
.util
.VersionInfo
.getVersion() +
2332 " or change your hadoop jars to start properly", t
);
2334 abort("Unhandled exception. Starting shutdown.", t
);
2341 private static boolean isCatalogTable(final TableName tableName
) {
2342 return tableName
.equals(TableName
.META_TABLE_NAME
);
2346 public long deleteTable(
2347 final TableName tableName
,
2348 final long nonceGroup
,
2349 final long nonce
) throws IOException
{
2352 return MasterProcedureUtil
.submitProcedure(
2353 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2355 protected void run() throws IOException
{
2356 getMaster().getMasterCoprocessorHost().preDeleteTable(tableName
);
2358 LOG
.info(getClientIdAuditPrefix() + " delete " + tableName
);
2360 // TODO: We can handle/merge duplicate request
2362 // We need to wait for the procedure to potentially fail due to "prepare" sanity
2363 // checks. This will block only the beginning of the procedure. See HBASE-19953.
2364 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
2365 submitProcedure(new DeleteTableProcedure(procedureExecutor
.getEnvironment(),
2369 getMaster().getMasterCoprocessorHost().postDeleteTable(tableName
);
2373 protected String
getDescription() {
2374 return "DeleteTableProcedure";
2380 public long truncateTable(
2381 final TableName tableName
,
2382 final boolean preserveSplits
,
2383 final long nonceGroup
,
2384 final long nonce
) throws IOException
{
2387 return MasterProcedureUtil
.submitProcedure(
2388 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2390 protected void run() throws IOException
{
2391 getMaster().getMasterCoprocessorHost().preTruncateTable(tableName
);
2393 LOG
.info(getClientIdAuditPrefix() + " truncate " + tableName
);
2394 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createLatch(2, 0);
2395 submitProcedure(new TruncateTableProcedure(procedureExecutor
.getEnvironment(),
2396 tableName
, preserveSplits
, latch
));
2399 getMaster().getMasterCoprocessorHost().postTruncateTable(tableName
);
2403 protected String
getDescription() {
2404 return "TruncateTableProcedure";
2410 public long addColumn(final TableName tableName
, final ColumnFamilyDescriptor column
,
2411 final long nonceGroup
, final long nonce
) throws IOException
{
2413 checkTableExists(tableName
);
2415 return modifyTable(tableName
, new TableDescriptorGetter() {
2418 public TableDescriptor
get() throws IOException
{
2419 TableDescriptor old
= getTableDescriptors().get(tableName
);
2420 if (old
.hasColumnFamily(column
.getName())) {
2421 throw new InvalidFamilyOperationException("Column family '" + column
.getNameAsString()
2422 + "' in table '" + tableName
+ "' already exists so cannot be added");
2425 return TableDescriptorBuilder
.newBuilder(old
).setColumnFamily(column
).build();
2427 }, nonceGroup
, nonce
, true);
2431 * Implement to return TableDescriptor after pre-checks
2433 protected interface TableDescriptorGetter
{
2434 TableDescriptor
get() throws IOException
;
2438 public long modifyColumn(final TableName tableName
, final ColumnFamilyDescriptor descriptor
,
2439 final long nonceGroup
, final long nonce
) throws IOException
{
2441 checkTableExists(tableName
);
2442 return modifyTable(tableName
, new TableDescriptorGetter() {
2445 public TableDescriptor
get() throws IOException
{
2446 TableDescriptor old
= getTableDescriptors().get(tableName
);
2447 if (!old
.hasColumnFamily(descriptor
.getName())) {
2448 throw new InvalidFamilyOperationException("Family '" + descriptor
.getNameAsString()
2449 + "' does not exist, so it cannot be modified");
2452 return TableDescriptorBuilder
.newBuilder(old
).modifyColumnFamily(descriptor
).build();
2454 }, nonceGroup
, nonce
, true);
2458 public long deleteColumn(final TableName tableName
, final byte[] columnName
,
2459 final long nonceGroup
, final long nonce
) throws IOException
{
2461 checkTableExists(tableName
);
2463 return modifyTable(tableName
, new TableDescriptorGetter() {
2466 public TableDescriptor
get() throws IOException
{
2467 TableDescriptor old
= getTableDescriptors().get(tableName
);
2469 if (!old
.hasColumnFamily(columnName
)) {
2470 throw new InvalidFamilyOperationException("Family '" + Bytes
.toString(columnName
)
2471 + "' does not exist, so it cannot be deleted");
2473 if (old
.getColumnFamilyCount() == 1) {
2474 throw new InvalidFamilyOperationException("Family '" + Bytes
.toString(columnName
)
2475 + "' is the only column family in the table, so it cannot be deleted");
2477 return TableDescriptorBuilder
.newBuilder(old
).removeColumnFamily(columnName
).build();
2479 }, nonceGroup
, nonce
, true);
2483 public long enableTable(final TableName tableName
, final long nonceGroup
, final long nonce
)
2484 throws IOException
{
2487 return MasterProcedureUtil
.submitProcedure(
2488 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2490 protected void run() throws IOException
{
2491 getMaster().getMasterCoprocessorHost().preEnableTable(tableName
);
2493 // Normally, it would make sense for this authorization check to exist inside
2494 // AccessController, but because the authorization check is done based on internal state
2495 // (rather than explicit permissions) we'll do the check here instead of in the
2497 MasterQuotaManager quotaManager
= getMasterQuotaManager();
2498 if (quotaManager
!= null) {
2499 if (quotaManager
.isQuotaInitialized()) {
2500 SpaceQuotaSnapshot currSnapshotOfTable
=
2501 QuotaTableUtil
.getCurrentSnapshotFromQuotaTable(getConnection(), tableName
);
2502 if (currSnapshotOfTable
!= null) {
2503 SpaceQuotaStatus quotaStatus
= currSnapshotOfTable
.getQuotaStatus();
2504 if (quotaStatus
.isInViolation()
2505 && SpaceViolationPolicy
.DISABLE
== quotaStatus
.getPolicy().orElse(null)) {
2506 throw new AccessDeniedException("Enabling the table '" + tableName
2507 + "' is disallowed due to a violated space quota.");
2510 } else if (LOG
.isTraceEnabled()) {
2511 LOG
.trace("Unable to check for space quotas as the MasterQuotaManager is not enabled");
2515 LOG
.info(getClientIdAuditPrefix() + " enable " + tableName
);
2517 // Execute the operation asynchronously - client will check the progress of the operation
2518 // In case the request is from a <1.1 client before returning,
2519 // we want to make sure that the table is prepared to be
2520 // enabled (the table is locked and the table state is set).
2521 // Note: if the procedure throws exception, we will catch it and rethrow.
2522 final ProcedurePrepareLatch prepareLatch
= ProcedurePrepareLatch
.createLatch();
2523 submitProcedure(new EnableTableProcedure(procedureExecutor
.getEnvironment(),
2524 tableName
, prepareLatch
));
2525 prepareLatch
.await();
2527 getMaster().getMasterCoprocessorHost().postEnableTable(tableName
);
2531 protected String
getDescription() {
2532 return "EnableTableProcedure";
2538 public long disableTable(final TableName tableName
, final long nonceGroup
, final long nonce
)
2539 throws IOException
{
2542 return MasterProcedureUtil
.submitProcedure(
2543 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2545 protected void run() throws IOException
{
2546 getMaster().getMasterCoprocessorHost().preDisableTable(tableName
);
2548 LOG
.info(getClientIdAuditPrefix() + " disable " + tableName
);
2550 // Execute the operation asynchronously - client will check the progress of the operation
2551 // In case the request is from a <1.1 client before returning,
2552 // we want to make sure that the table is prepared to be
2553 // enabled (the table is locked and the table state is set).
2554 // Note: if the procedure throws exception, we will catch it and rethrow.
2556 // We need to wait for the procedure to potentially fail due to "prepare" sanity
2557 // checks. This will block only the beginning of the procedure. See HBASE-19953.
2558 final ProcedurePrepareLatch prepareLatch
= ProcedurePrepareLatch
.createBlockingLatch();
2559 submitProcedure(new DisableTableProcedure(procedureExecutor
.getEnvironment(),
2560 tableName
, false, prepareLatch
));
2561 prepareLatch
.await();
2563 getMaster().getMasterCoprocessorHost().postDisableTable(tableName
);
2567 protected String
getDescription() {
2568 return "DisableTableProcedure";
2573 private long modifyTable(final TableName tableName
,
2574 final TableDescriptorGetter newDescriptorGetter
, final long nonceGroup
, final long nonce
,
2575 final boolean shouldCheckDescriptor
) throws IOException
{
2576 return MasterProcedureUtil
2577 .submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2579 protected void run() throws IOException
{
2580 TableDescriptor oldDescriptor
= getMaster().getTableDescriptors().get(tableName
);
2581 TableDescriptor newDescriptor
= getMaster().getMasterCoprocessorHost()
2582 .preModifyTable(tableName
, oldDescriptor
, newDescriptorGetter
.get());
2583 TableDescriptorChecker
.sanityCheck(conf
, newDescriptor
);
2584 LOG
.info("{} modify table {} from {} to {}", getClientIdAuditPrefix(), tableName
,
2585 oldDescriptor
, newDescriptor
);
2587 // Execute the operation synchronously - wait for the operation completes before
2590 // We need to wait for the procedure to potentially fail due to "prepare" sanity
2591 // checks. This will block only the beginning of the procedure. See HBASE-19953.
2592 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
2593 submitProcedure(new ModifyTableProcedure(procedureExecutor
.getEnvironment(),
2594 newDescriptor
, latch
, oldDescriptor
, shouldCheckDescriptor
));
2597 getMaster().getMasterCoprocessorHost().postModifyTable(tableName
, oldDescriptor
,
2602 protected String
getDescription() {
2603 return "ModifyTableProcedure";
2610 public long modifyTable(final TableName tableName
, final TableDescriptor newDescriptor
,
2611 final long nonceGroup
, final long nonce
) throws IOException
{
2613 return modifyTable(tableName
, new TableDescriptorGetter() {
2615 public TableDescriptor
get() throws IOException
{
2616 return newDescriptor
;
2618 }, nonceGroup
, nonce
, false);
2622 public long restoreSnapshot(final SnapshotDescription snapshotDesc
, final long nonceGroup
,
2623 final long nonce
, final boolean restoreAcl
, final String customSFT
) throws IOException
{
2625 getSnapshotManager().checkSnapshotSupport();
2627 // Ensure namespace exists. Will throw exception if non-known NS.
2628 final TableName dstTable
= TableName
.valueOf(snapshotDesc
.getTable());
2629 getClusterSchema().getNamespace(dstTable
.getNamespaceAsString());
2631 return MasterProcedureUtil
.submitProcedure(
2632 new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
2634 protected void run() throws IOException
{
2636 getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc
, getNonceKey(), restoreAcl
,
2641 protected String
getDescription() {
2642 return "RestoreSnapshotProcedure";
2647 private void checkTableExists(final TableName tableName
)
2648 throws IOException
, TableNotFoundException
{
2649 if (!tableDescriptors
.exists(tableName
)) {
2650 throw new TableNotFoundException(tableName
);
2655 public void checkTableModifiable(final TableName tableName
)
2656 throws IOException
, TableNotFoundException
, TableNotDisabledException
{
2657 if (isCatalogTable(tableName
)) {
2658 throw new IOException("Can't modify catalog tables");
2660 checkTableExists(tableName
);
2661 TableState ts
= getTableStateManager().getTableState(tableName
);
2662 if (!ts
.isDisabled()) {
2663 throw new TableNotDisabledException("Not DISABLED; " + ts
);
2667 public ClusterMetrics
getClusterMetricsWithoutCoprocessor() throws InterruptedIOException
{
2668 return getClusterMetricsWithoutCoprocessor(EnumSet
.allOf(Option
.class));
2671 public ClusterMetrics
getClusterMetricsWithoutCoprocessor(EnumSet
<Option
> options
)
2672 throws InterruptedIOException
{
2673 ClusterMetricsBuilder builder
= ClusterMetricsBuilder
.newBuilder();
2674 // given that hbase1 can't submit the request with Option,
2675 // we return all information to client if the list of Option is empty.
2676 if (options
.isEmpty()) {
2677 options
= EnumSet
.allOf(Option
.class);
2680 for (Option opt
: options
) {
2682 case HBASE_VERSION
: builder
.setHBaseVersion(VersionInfo
.getVersion()); break;
2683 case CLUSTER_ID
: builder
.setClusterId(getClusterId()); break;
2684 case MASTER
: builder
.setMasterName(getServerName()); break;
2685 case BACKUP_MASTERS
: builder
.setBackerMasterNames(getBackupMasters()); break;
2686 case LIVE_SERVERS
: {
2687 if (serverManager
!= null) {
2688 builder
.setLiveServerMetrics(serverManager
.getOnlineServers().entrySet().stream()
2689 .collect(Collectors
.toMap(e
-> e
.getKey(), e
-> e
.getValue())));
2693 case DEAD_SERVERS
: {
2694 if (serverManager
!= null) {
2695 builder
.setDeadServerNames(new ArrayList
<>(
2696 serverManager
.getDeadServers().copyServerNames()));
2700 case MASTER_COPROCESSORS
: {
2701 if (cpHost
!= null) {
2702 builder
.setMasterCoprocessorNames(Arrays
.asList(getMasterCoprocessors()));
2706 case REGIONS_IN_TRANSITION
: {
2707 if (assignmentManager
!= null) {
2708 builder
.setRegionsInTransition(assignmentManager
.getRegionStates()
2709 .getRegionsStateInTransition());
2714 if (loadBalancerTracker
!= null) {
2715 builder
.setBalancerOn(loadBalancerTracker
.isBalancerOn());
2719 case MASTER_INFO_PORT
: {
2720 if (infoServer
!= null) {
2721 builder
.setMasterInfoPort(infoServer
.getPort());
2725 case SERVERS_NAME
: {
2726 if (serverManager
!= null) {
2727 builder
.setServerNames(serverManager
.getOnlineServersList());
2731 case TABLE_TO_REGIONS_COUNT
: {
2732 if (isActiveMaster() && isInitialized() && assignmentManager
!= null) {
2734 Map
<TableName
, RegionStatesCount
> tableRegionStatesCountMap
= new HashMap
<>();
2735 Map
<String
, TableDescriptor
> tableDescriptorMap
= getTableDescriptors().getAll();
2736 for (TableDescriptor tableDescriptor
: tableDescriptorMap
.values()) {
2737 TableName tableName
= tableDescriptor
.getTableName();
2738 RegionStatesCount regionStatesCount
= assignmentManager
2739 .getRegionStatesCount(tableName
);
2740 tableRegionStatesCountMap
.put(tableName
, regionStatesCount
);
2742 builder
.setTableRegionStatesCount(tableRegionStatesCountMap
);
2743 } catch (IOException e
) {
2744 LOG
.error("Error while populating TABLE_TO_REGIONS_COUNT for Cluster Metrics..", e
);
2751 return builder
.build();
2755 * @return cluster status
2757 public ClusterMetrics
getClusterMetrics() throws IOException
{
2758 return getClusterMetrics(EnumSet
.allOf(Option
.class));
2761 public ClusterMetrics
getClusterMetrics(EnumSet
<Option
> options
) throws IOException
{
2762 if (cpHost
!= null) {
2763 cpHost
.preGetClusterMetrics();
2765 ClusterMetrics status
= getClusterMetricsWithoutCoprocessor(options
);
2766 if (cpHost
!= null) {
2767 cpHost
.postGetClusterMetrics(status
);
2773 * @return info port of active master or 0 if any exception occurs.
2775 public int getActiveMasterInfoPort() {
2776 return activeMasterManager
.getActiveMasterInfoPort();
2780 * @param sn is ServerName of the backup master
2781 * @return info port of backup master or 0 if any exception occurs.
2783 public int getBackupMasterInfoPort(final ServerName sn
) {
2784 return activeMasterManager
.getBackupMasterInfoPort(sn
);
2788 * The set of loaded coprocessors is stored in a static set. Since it's
2789 * statically allocated, it does not require that HMaster's cpHost be
2790 * initialized prior to accessing it.
2791 * @return a String representation of the set of names of the loaded coprocessors.
2793 public static String
getLoadedCoprocessors() {
2794 return CoprocessorHost
.getLoadedCoprocessors().toString();
2798 * @return timestamp in millis when HMaster was started.
2800 public long getMasterStartTime() {
2805 * @return timestamp in millis when HMaster became the active master.
2807 public long getMasterActiveTime() {
2808 return masterActiveTime
;
2812 * @return timestamp in millis when HMaster finished becoming the active master
2814 public long getMasterFinishedInitializationTime() {
2815 return masterFinishedInitializationTime
;
2818 public int getNumWALFiles() {
2822 public ProcedureStore
getProcedureStore() {
2823 return procedureStore
;
2826 public int getRegionServerInfoPort(final ServerName sn
) {
2827 int port
= this.serverManager
.getInfoPort(sn
);
2828 return port
== 0 ? conf
.getInt(HConstants
.REGIONSERVER_INFO_PORT
,
2829 HConstants
.DEFAULT_REGIONSERVER_INFOPORT
) : port
;
2833 public String
getRegionServerVersion(ServerName sn
) {
2834 // Will return "0.0.0" if the server is not online to prevent move system region to unknown
2836 return this.serverManager
.getVersion(sn
);
2840 public void checkIfShouldMoveSystemRegionAsync() {
2841 assignmentManager
.checkIfShouldMoveSystemRegionAsync();
2845 * @return array of coprocessor SimpleNames.
2847 public String
[] getMasterCoprocessors() {
2848 Set
<String
> masterCoprocessors
= getMasterCoprocessorHost().getCoprocessors();
2849 return masterCoprocessors
.toArray(new String
[masterCoprocessors
.size()]);
2853 public void abort(String reason
, Throwable cause
) {
2854 if (!setAbortRequested() || isStopped()) {
2855 LOG
.debug("Abort called but aborted={}, stopped={}", isAborted(), isStopped());
2858 if (cpHost
!= null) {
2859 // HBASE-4014: dump a list of loaded coprocessors.
2860 LOG
.error(HBaseMarkers
.FATAL
, "Master server abort: loaded coprocessors are: " +
2861 getLoadedCoprocessors());
2863 String msg
= "***** ABORTING master " + this + ": " + reason
+ " *****";
2864 if (cause
!= null) {
2865 LOG
.error(HBaseMarkers
.FATAL
, msg
, cause
);
2867 LOG
.error(HBaseMarkers
.FATAL
, msg
);
2872 } catch (IOException e
) {
2873 LOG
.error("Exception occurred while stopping master", e
);
2878 public MasterCoprocessorHost
getMasterCoprocessorHost() {
2883 public MasterQuotaManager
getMasterQuotaManager() {
2884 return quotaManager
;
2888 public ProcedureExecutor
<MasterProcedureEnv
> getMasterProcedureExecutor() {
2889 return procedureExecutor
;
2893 public ServerName
getServerName() {
2894 return this.serverName
;
2898 public AssignmentManager
getAssignmentManager() {
2899 return this.assignmentManager
;
2903 public CatalogJanitor
getCatalogJanitor() {
2904 return this.catalogJanitorChore
;
2907 public MemoryBoundedLogMessageBuffer
getRegionServerFatalLogBuffer() {
2912 * Shutdown the cluster.
2913 * Master runs a coordinated stop of all RegionServers and then itself.
2915 public void shutdown() throws IOException
{
2916 if (cpHost
!= null) {
2917 cpHost
.preShutdown();
2920 // Tell the servermanager cluster shutdown has been called. This makes it so when Master is
2921 // last running server, it'll stop itself. Next, we broadcast the cluster shutdown by setting
2922 // the cluster status as down. RegionServers will notice this change in state and will start
2923 // shutting themselves down. When last has exited, Master can go down.
2924 if (this.serverManager
!= null) {
2925 this.serverManager
.shutdownCluster();
2927 if (this.clusterStatusTracker
!= null) {
2929 this.clusterStatusTracker
.setClusterDown();
2930 } catch (KeeperException e
) {
2931 LOG
.error("ZooKeeper exception trying to set cluster as down in ZK", e
);
2934 // Stop the procedure executor. Will stop any ongoing assign, unassign, server crash etc.,
2935 // processing so we can go down.
2936 if (this.procedureExecutor
!= null) {
2937 this.procedureExecutor
.stop();
2939 // Shutdown our cluster connection. This will kill any hosted RPCs that might be going on;
2940 // this is what we want especially if the Master is in startup phase doing call outs to
2941 // hbase:meta, etc. when cluster is down. Without ths connection close, we'd have to wait on
2942 // the rpc to timeout.
2943 if (this.asyncClusterConnection
!= null) {
2944 this.asyncClusterConnection
.close();
2948 public void stopMaster() throws IOException
{
2949 if (cpHost
!= null) {
2950 cpHost
.preStopMaster();
2952 stop("Stopped by " + Thread
.currentThread().getName());
2956 public void stop(String msg
) {
2957 if (!this.stopped
) {
2958 LOG
.info("***** STOPPING master '" + this + "' *****");
2959 this.stopped
= true;
2960 LOG
.info("STOPPED: " + msg
);
2961 // Wakes run() if it is sleeping
2962 sleeper
.skipSleepCycle();
2963 if (this.activeMasterManager
!= null) {
2964 this.activeMasterManager
.stop();
2969 protected void checkServiceStarted() throws ServerNotRunningYetException
{
2970 if (!serviceStarted
) {
2971 throw new ServerNotRunningYetException("Server is not running yet");
2975 void checkInitialized() throws PleaseHoldException
, ServerNotRunningYetException
,
2976 MasterNotRunningException
, MasterStoppedException
{
2977 checkServiceStarted();
2978 if (!isInitialized()) {
2979 throw new PleaseHoldException("Master is initializing");
2982 throw new MasterStoppedException();
2987 * Report whether this master is currently the active master or not.
2988 * If not active master, we are parked on ZK waiting to become active.
2990 * This method is used for testing.
2992 * @return true if active master, false if not.
2995 public boolean isActiveMaster() {
2996 return activeMaster
;
3000 * Report whether this master has completed with its initialization and is
3001 * ready. If ready, the master is also the active master. A standby master
3004 * This method is used for testing.
3006 * @return true if master is ready to go, false if not.
3009 public boolean isInitialized() {
3010 return initialized
.isReady();
3014 * Report whether this master is started
3016 * This method is used for testing.
3018 * @return true if master is ready to go, false if not.
3020 public boolean isOnline() {
3021 return serviceStarted
;
3025 * Report whether this master is in maintenance mode.
3027 * @return true if master is in maintenanceMode
3030 public boolean isInMaintenanceMode() {
3031 return maintenanceMode
;
3034 public void setInitialized(boolean isInitialized
) {
3035 procedureExecutor
.getEnvironment().setEventReady(initialized
, isInitialized
);
3039 public ProcedureEvent
<?
> getInitializedEvent() {
3044 * Compute the average load across all region servers.
3045 * Currently, this uses a very naive computation - just uses the number of
3046 * regions being served, ignoring stats about number of requests.
3047 * @return the average load
3049 public double getAverageLoad() {
3050 if (this.assignmentManager
== null) {
3054 RegionStates regionStates
= this.assignmentManager
.getRegionStates();
3055 if (regionStates
== null) {
3058 return regionStates
.getAverageLoad();
3062 public boolean registerService(Service instance
) {
3064 * No stacking of instances is allowed for a single service name
3066 Descriptors
.ServiceDescriptor serviceDesc
= instance
.getDescriptorForType();
3067 String serviceName
= CoprocessorRpcUtils
.getServiceName(serviceDesc
);
3068 if (coprocessorServiceHandlers
.containsKey(serviceName
)) {
3069 LOG
.error("Coprocessor service "+serviceName
+
3070 " already registered, rejecting request from "+instance
3075 coprocessorServiceHandlers
.put(serviceName
, instance
);
3076 if (LOG
.isDebugEnabled()) {
3077 LOG
.debug("Registered master coprocessor service: service="+serviceName
);
3083 * Utility for constructing an instance of the passed HMaster class.
3084 * @param masterClass
3085 * @return HMaster instance.
3087 public static HMaster
constructMaster(Class
<?
extends HMaster
> masterClass
,
3088 final Configuration conf
) {
3090 Constructor
<?
extends HMaster
> c
= masterClass
.getConstructor(Configuration
.class);
3091 return c
.newInstance(conf
);
3092 } catch(Exception e
) {
3093 Throwable error
= e
;
3094 if (e
instanceof InvocationTargetException
&&
3095 ((InvocationTargetException
)e
).getTargetException() != null) {
3096 error
= ((InvocationTargetException
)e
).getTargetException();
3098 throw new RuntimeException("Failed construction of Master: " + masterClass
.toString() + ". "
3104 * @see org.apache.hadoop.hbase.master.HMasterCommandLine
3106 public static void main(String
[] args
) {
3107 LOG
.info("STARTING service " + HMaster
.class.getSimpleName());
3108 VersionInfo
.logVersion();
3109 new HMasterCommandLine(HMaster
.class).doMain(args
);
3112 public HFileCleaner
getHFileCleaner() {
3113 return this.hfileCleaner
;
3116 public LogCleaner
getLogCleaner() {
3117 return this.logCleaner
;
3121 * @return the underlying snapshot manager
3124 public SnapshotManager
getSnapshotManager() {
3125 return this.snapshotManager
;
3129 * @return the underlying MasterProcedureManagerHost
3132 public MasterProcedureManagerHost
getMasterProcedureManagerHost() {
3137 public ClusterSchema
getClusterSchema() {
3138 return this.clusterSchemaService
;
3142 * Create a new Namespace.
3143 * @param namespaceDescriptor descriptor for new Namespace
3144 * @param nonceGroup Identifier for the source of the request, a client or process.
3145 * @param nonce A unique identifier for this operation from the client or process identified by
3146 * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3147 * @return procedure id
3149 long createNamespace(final NamespaceDescriptor namespaceDescriptor
, final long nonceGroup
,
3150 final long nonce
) throws IOException
{
3153 TableName
.isLegalNamespaceName(Bytes
.toBytes(namespaceDescriptor
.getName()));
3155 return MasterProcedureUtil
.submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this,
3156 nonceGroup
, nonce
) {
3158 protected void run() throws IOException
{
3159 getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor
);
3160 // We need to wait for the procedure to potentially fail due to "prepare" sanity
3161 // checks. This will block only the beginning of the procedure. See HBASE-19953.
3162 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
3163 LOG
.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor
);
3164 // Execute the operation synchronously - wait for the operation to complete before
3166 setProcId(getClusterSchema().createNamespace(namespaceDescriptor
, getNonceKey(), latch
));
3168 getMaster().getMasterCoprocessorHost().postCreateNamespace(namespaceDescriptor
);
3172 protected String
getDescription() {
3173 return "CreateNamespaceProcedure";
3179 * Modify an existing Namespace.
3180 * @param nonceGroup Identifier for the source of the request, a client or process.
3181 * @param nonce A unique identifier for this operation from the client or process identified by
3182 * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3183 * @return procedure id
3185 long modifyNamespace(final NamespaceDescriptor newNsDescriptor
, final long nonceGroup
,
3186 final long nonce
) throws IOException
{
3189 TableName
.isLegalNamespaceName(Bytes
.toBytes(newNsDescriptor
.getName()));
3191 return MasterProcedureUtil
.submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this,
3192 nonceGroup
, nonce
) {
3194 protected void run() throws IOException
{
3195 NamespaceDescriptor oldNsDescriptor
= getNamespace(newNsDescriptor
.getName());
3196 getMaster().getMasterCoprocessorHost().preModifyNamespace(oldNsDescriptor
, newNsDescriptor
);
3197 // We need to wait for the procedure to potentially fail due to "prepare" sanity
3198 // checks. This will block only the beginning of the procedure. See HBASE-19953.
3199 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
3200 LOG
.info(getClientIdAuditPrefix() + " modify " + newNsDescriptor
);
3201 // Execute the operation synchronously - wait for the operation to complete before
3203 setProcId(getClusterSchema().modifyNamespace(newNsDescriptor
, getNonceKey(), latch
));
3205 getMaster().getMasterCoprocessorHost().postModifyNamespace(oldNsDescriptor
,
3210 protected String
getDescription() {
3211 return "ModifyNamespaceProcedure";
3217 * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
3218 * @param nonceGroup Identifier for the source of the request, a client or process.
3219 * @param nonce A unique identifier for this operation from the client or process identified by
3220 * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3221 * @return procedure id
3223 long deleteNamespace(final String name
, final long nonceGroup
, final long nonce
)
3224 throws IOException
{
3227 return MasterProcedureUtil
.submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this,
3228 nonceGroup
, nonce
) {
3230 protected void run() throws IOException
{
3231 getMaster().getMasterCoprocessorHost().preDeleteNamespace(name
);
3232 LOG
.info(getClientIdAuditPrefix() + " delete " + name
);
3233 // Execute the operation synchronously - wait for the operation to complete before
3236 // We need to wait for the procedure to potentially fail due to "prepare" sanity
3237 // checks. This will block only the beginning of the procedure. See HBASE-19953.
3238 ProcedurePrepareLatch latch
= ProcedurePrepareLatch
.createBlockingLatch();
3239 setProcId(submitProcedure(
3240 new DeleteNamespaceProcedure(procedureExecutor
.getEnvironment(), name
, latch
)));
3242 // Will not be invoked in the face of Exception thrown by the Procedure's execution
3243 getMaster().getMasterCoprocessorHost().postDeleteNamespace(name
);
3247 protected String
getDescription() {
3248 return "DeleteNamespaceProcedure";
3255 * @param name Name of the Namespace
3256 * @return Namespace descriptor for <code>name</code>
3258 NamespaceDescriptor
getNamespace(String name
) throws IOException
{
3260 if (this.cpHost
!= null) this.cpHost
.preGetNamespaceDescriptor(name
);
3261 NamespaceDescriptor nsd
= this.clusterSchemaService
.getNamespace(name
);
3262 if (this.cpHost
!= null) this.cpHost
.postGetNamespaceDescriptor(nsd
);
3267 * Get all Namespaces
3268 * @return All Namespace descriptors
3270 List
<NamespaceDescriptor
> getNamespaces() throws IOException
{
3272 final List
<NamespaceDescriptor
> nsds
= new ArrayList
<>();
3273 if (cpHost
!= null) {
3274 cpHost
.preListNamespaceDescriptors(nsds
);
3276 nsds
.addAll(this.clusterSchemaService
.getNamespaces());
3277 if (this.cpHost
!= null) {
3278 this.cpHost
.postListNamespaceDescriptors(nsds
);
3284 * List namespace names
3285 * @return All namespace names
3287 public List
<String
> listNamespaces() throws IOException
{
3289 List
<String
> namespaces
= new ArrayList
<>();
3290 if (cpHost
!= null) {
3291 cpHost
.preListNamespaces(namespaces
);
3293 for (NamespaceDescriptor namespace
: clusterSchemaService
.getNamespaces()) {
3294 namespaces
.add(namespace
.getName());
3296 if (cpHost
!= null) {
3297 cpHost
.postListNamespaces(namespaces
);
3303 public List
<TableName
> listTableNamesByNamespace(String name
) throws IOException
{
3305 return listTableNames(name
, null, true);
3309 public List
<TableDescriptor
> listTableDescriptorsByNamespace(String name
) throws IOException
{
3311 return listTableDescriptors(name
, null, null, true);
3315 public boolean abortProcedure(final long procId
, final boolean mayInterruptIfRunning
)
3316 throws IOException
{
3317 if (cpHost
!= null) {
3318 cpHost
.preAbortProcedure(this.procedureExecutor
, procId
);
3321 final boolean result
= this.procedureExecutor
.abort(procId
, mayInterruptIfRunning
);
3323 if (cpHost
!= null) {
3324 cpHost
.postAbortProcedure();
3331 public List
<Procedure
<?
>> getProcedures() throws IOException
{
3332 if (cpHost
!= null) {
3333 cpHost
.preGetProcedures();
3336 @SuppressWarnings({ "unchecked", "rawtypes" })
3337 List
<Procedure
<?
>> procList
= (List
) this.procedureExecutor
.getProcedures();
3339 if (cpHost
!= null) {
3340 cpHost
.postGetProcedures(procList
);
3347 public List
<LockedResource
> getLocks() throws IOException
{
3348 if (cpHost
!= null) {
3349 cpHost
.preGetLocks();
3352 MasterProcedureScheduler procedureScheduler
=
3353 procedureExecutor
.getEnvironment().getProcedureScheduler();
3355 final List
<LockedResource
> lockedResources
= procedureScheduler
.getLocks();
3357 if (cpHost
!= null) {
3358 cpHost
.postGetLocks(lockedResources
);
3361 return lockedResources
;
3365 * Returns the list of table descriptors that match the specified request
3366 * @param namespace the namespace to query, or null if querying for all
3367 * @param regex The regular expression to match against, or null if querying for all
3368 * @param tableNameList the list of table names, or null if querying for all
3369 * @param includeSysTables False to match only against userspace tables
3370 * @return the list of table descriptors
3372 public List
<TableDescriptor
> listTableDescriptors(final String namespace
, final String regex
,
3373 final List
<TableName
> tableNameList
, final boolean includeSysTables
)
3374 throws IOException
{
3375 List
<TableDescriptor
> htds
= new ArrayList
<>();
3376 if (cpHost
!= null) {
3377 cpHost
.preGetTableDescriptors(tableNameList
, htds
, regex
);
3379 htds
= getTableDescriptors(htds
, namespace
, regex
, tableNameList
, includeSysTables
);
3380 if (cpHost
!= null) {
3381 cpHost
.postGetTableDescriptors(tableNameList
, htds
, regex
);
3387 * Returns the list of table names that match the specified request
3388 * @param regex The regular expression to match against, or null if querying for all
3389 * @param namespace the namespace to query, or null if querying for all
3390 * @param includeSysTables False to match only against userspace tables
3391 * @return the list of table names
3393 public List
<TableName
> listTableNames(final String namespace
, final String regex
,
3394 final boolean includeSysTables
) throws IOException
{
3395 List
<TableDescriptor
> htds
= new ArrayList
<>();
3396 if (cpHost
!= null) {
3397 cpHost
.preGetTableNames(htds
, regex
);
3399 htds
= getTableDescriptors(htds
, namespace
, regex
, null, includeSysTables
);
3400 if (cpHost
!= null) {
3401 cpHost
.postGetTableNames(htds
, regex
);
3403 List
<TableName
> result
= new ArrayList
<>(htds
.size());
3404 for (TableDescriptor htd
: htds
) result
.add(htd
.getTableName());
3409 * Return a list of table table descriptors after applying any provided filter parameters. Note
3410 * that the user-facing description of this filter logic is presented on the class-level javadoc
3411 * of {@link NormalizeTableFilterParams}.
3413 private List
<TableDescriptor
> getTableDescriptors(final List
<TableDescriptor
> htds
,
3414 final String namespace
, final String regex
, final List
<TableName
> tableNameList
,
3415 final boolean includeSysTables
)
3416 throws IOException
{
3417 if (tableNameList
== null || tableNameList
.isEmpty()) {
3418 // request for all TableDescriptors
3419 Collection
<TableDescriptor
> allHtds
;
3420 if (namespace
!= null && namespace
.length() > 0) {
3421 // Do a check on the namespace existence. Will fail if does not exist.
3422 this.clusterSchemaService
.getNamespace(namespace
);
3423 allHtds
= tableDescriptors
.getByNamespace(namespace
).values();
3425 allHtds
= tableDescriptors
.getAll().values();
3427 for (TableDescriptor desc
: allHtds
) {
3428 if (tableStateManager
.isTablePresent(desc
.getTableName())
3429 && (includeSysTables
|| !desc
.getTableName().isSystemTable())) {
3434 for (TableName s
: tableNameList
) {
3435 if (tableStateManager
.isTablePresent(s
)) {
3436 TableDescriptor desc
= tableDescriptors
.get(s
);
3444 // Retains only those matched by regular expression.
3445 if (regex
!= null) filterTablesByRegex(htds
, Pattern
.compile(regex
));
3450 * Removes the table descriptors that don't match the pattern.
3451 * @param descriptors list of table descriptors to filter
3452 * @param pattern the regex to use
3454 private static void filterTablesByRegex(final Collection
<TableDescriptor
> descriptors
,
3455 final Pattern pattern
) {
3456 final String defaultNS
= NamespaceDescriptor
.DEFAULT_NAMESPACE_NAME_STR
;
3457 Iterator
<TableDescriptor
> itr
= descriptors
.iterator();
3458 while (itr
.hasNext()) {
3459 TableDescriptor htd
= itr
.next();
3460 String tableName
= htd
.getTableName().getNameAsString();
3461 boolean matched
= pattern
.matcher(tableName
).matches();
3462 if (!matched
&& htd
.getTableName().getNamespaceAsString().equals(defaultNS
)) {
3463 matched
= pattern
.matcher(defaultNS
+ TableName
.NAMESPACE_DELIM
+ tableName
).matches();
3472 public long getLastMajorCompactionTimestamp(TableName table
) throws IOException
{
3473 return getClusterMetrics(EnumSet
.of(Option
.LIVE_SERVERS
))
3474 .getLastMajorCompactionTimestamp(table
);
3478 public long getLastMajorCompactionTimestampForRegion(byte[] regionName
) throws IOException
{
3479 return getClusterMetrics(EnumSet
.of(Option
.LIVE_SERVERS
))
3480 .getLastMajorCompactionTimestamp(regionName
);
3484 * Gets the mob file compaction state for a specific table.
3485 * Whether all the mob files are selected is known during the compaction execution, but
3486 * the statistic is done just before compaction starts, it is hard to know the compaction
3487 * type at that time, so the rough statistics are chosen for the mob file compaction. Only two
3488 * compaction states are available, CompactionState.MAJOR_AND_MINOR and CompactionState.NONE.
3489 * @param tableName The current table name.
3490 * @return If a given table is in mob file compaction now.
3492 public GetRegionInfoResponse
.CompactionState
getMobCompactionState(TableName tableName
) {
3493 AtomicInteger compactionsCount
= mobCompactionStates
.get(tableName
);
3494 if (compactionsCount
!= null && compactionsCount
.get() != 0) {
3495 return GetRegionInfoResponse
.CompactionState
.MAJOR_AND_MINOR
;
3497 return GetRegionInfoResponse
.CompactionState
.NONE
;
3500 public void reportMobCompactionStart(TableName tableName
) throws IOException
{
3501 IdLock
.Entry lockEntry
= null;
3503 lockEntry
= mobCompactionLock
.getLockEntry(tableName
.hashCode());
3504 AtomicInteger compactionsCount
= mobCompactionStates
.get(tableName
);
3505 if (compactionsCount
== null) {
3506 compactionsCount
= new AtomicInteger(0);
3507 mobCompactionStates
.put(tableName
, compactionsCount
);
3509 compactionsCount
.incrementAndGet();
3511 if (lockEntry
!= null) {
3512 mobCompactionLock
.releaseLockEntry(lockEntry
);
3517 public void reportMobCompactionEnd(TableName tableName
) throws IOException
{
3518 IdLock
.Entry lockEntry
= null;
3520 lockEntry
= mobCompactionLock
.getLockEntry(tableName
.hashCode());
3521 AtomicInteger compactionsCount
= mobCompactionStates
.get(tableName
);
3522 if (compactionsCount
!= null) {
3523 int count
= compactionsCount
.decrementAndGet();
3524 // remove the entry if the count is 0.
3526 mobCompactionStates
.remove(tableName
);
3530 if (lockEntry
!= null) {
3531 mobCompactionLock
.releaseLockEntry(lockEntry
);
3538 * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
3539 * false is returned.
3541 * @return The state of the load balancer, or false if the load balancer isn't defined.
3543 public boolean isBalancerOn() {
3544 return !isInMaintenanceMode()
3545 && loadBalancerTracker
!= null
3546 && loadBalancerTracker
.isBalancerOn();
3550 * Queries the state of the {@link RegionNormalizerTracker}. If it's not initialized,
3551 * false is returned.
3553 public boolean isNormalizerOn() {
3554 return !isInMaintenanceMode()
3555 && getRegionNormalizerManager().isNormalizerOn();
3559 * Queries the state of the {@link SplitOrMergeTracker}. If it is not initialized,
3560 * false is returned. If switchType is illegal, false will return.
3561 * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}
3562 * @return The state of the switch
3565 public boolean isSplitOrMergeEnabled(MasterSwitchType switchType
) {
3566 return !isInMaintenanceMode()
3567 && splitOrMergeTracker
!= null
3568 && splitOrMergeTracker
.isSplitOrMergeEnabled(switchType
);
3572 * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.
3574 * Notice that, the base load balancer will always be {@link RSGroupBasedLoadBalancer} now, so
3575 * this method will return the balancer used inside each rs group.
3576 * @return The name of the {@link LoadBalancer} in use.
3578 public String
getLoadBalancerClassName() {
3579 return conf
.get(HConstants
.HBASE_MASTER_LOADBALANCER_CLASS
,
3580 LoadBalancerFactory
.getDefaultLoadBalancerClass().getName());
3583 public SplitOrMergeTracker
getSplitOrMergeTracker() {
3584 return splitOrMergeTracker
;
3588 public RSGroupBasedLoadBalancer
getLoadBalancer() {
3593 public FavoredNodesManager
getFavoredNodesManager() {
3594 return balancer
.getFavoredNodesManager();
3597 private long executePeerProcedure(AbstractPeerProcedure
<?
> procedure
) throws IOException
{
3598 long procId
= procedureExecutor
.submitProcedure(procedure
);
3599 procedure
.getLatch().await();
3604 public long addReplicationPeer(String peerId
, ReplicationPeerConfig peerConfig
, boolean enabled
)
3605 throws ReplicationException
, IOException
{
3606 LOG
.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId
+ ", config=" +
3607 peerConfig
+ ", state=" + (enabled ?
"ENABLED" : "DISABLED"));
3608 return executePeerProcedure(new AddPeerProcedure(peerId
, peerConfig
, enabled
));
3612 public long removeReplicationPeer(String peerId
) throws ReplicationException
, IOException
{
3613 LOG
.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId
);
3614 return executePeerProcedure(new RemovePeerProcedure(peerId
));
3618 public long enableReplicationPeer(String peerId
) throws ReplicationException
, IOException
{
3619 LOG
.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId
);
3620 return executePeerProcedure(new EnablePeerProcedure(peerId
));
3624 public long disableReplicationPeer(String peerId
) throws ReplicationException
, IOException
{
3625 LOG
.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId
);
3626 return executePeerProcedure(new DisablePeerProcedure(peerId
));
3630 public ReplicationPeerConfig
getReplicationPeerConfig(String peerId
)
3631 throws ReplicationException
, IOException
{
3632 if (cpHost
!= null) {
3633 cpHost
.preGetReplicationPeerConfig(peerId
);
3635 LOG
.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId
);
3636 ReplicationPeerConfig peerConfig
= this.replicationPeerManager
.getPeerConfig(peerId
)
3637 .orElseThrow(() -> new ReplicationPeerNotFoundException(peerId
));
3638 if (cpHost
!= null) {
3639 cpHost
.postGetReplicationPeerConfig(peerId
);
3645 public long updateReplicationPeerConfig(String peerId
, ReplicationPeerConfig peerConfig
)
3646 throws ReplicationException
, IOException
{
3647 LOG
.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId
+
3648 ", config=" + peerConfig
);
3649 return executePeerProcedure(new UpdatePeerConfigProcedure(peerId
, peerConfig
));
3653 public List
<ReplicationPeerDescription
> listReplicationPeers(String regex
)
3654 throws ReplicationException
, IOException
{
3655 if (cpHost
!= null) {
3656 cpHost
.preListReplicationPeers(regex
);
3658 LOG
.debug("{} list replication peers, regex={}", getClientIdAuditPrefix(), regex
);
3659 Pattern pattern
= regex
== null ?
null : Pattern
.compile(regex
);
3660 List
<ReplicationPeerDescription
> peers
=
3661 this.replicationPeerManager
.listPeers(pattern
);
3662 if (cpHost
!= null) {
3663 cpHost
.postListReplicationPeers(regex
);
3669 public long transitReplicationPeerSyncReplicationState(String peerId
, SyncReplicationState state
)
3670 throws ReplicationException
, IOException
{
3672 getClientIdAuditPrefix() +
3673 " transit current cluster state to {} in a synchronous replication peer id={}",
3675 return executePeerProcedure(new TransitPeerSyncReplicationStateProcedure(peerId
, state
));
3679 * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
3680 * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0
3681 * @param servers Region servers to decommission.
3683 public void decommissionRegionServers(final List
<ServerName
> servers
, final boolean offload
)
3684 throws IOException
{
3685 List
<ServerName
> serversAdded
= new ArrayList
<>(servers
.size());
3686 // Place the decommission marker first.
3687 String parentZnode
= getZooKeeper().getZNodePaths().drainingZNode
;
3688 for (ServerName server
: servers
) {
3690 String node
= ZNodePaths
.joinZNode(parentZnode
, server
.getServerName());
3691 ZKUtil
.createAndFailSilent(getZooKeeper(), node
);
3692 } catch (KeeperException ke
) {
3693 throw new HBaseIOException(
3694 this.zooKeeper
.prefix("Unable to decommission '" + server
.getServerName() + "'."), ke
);
3696 if (this.serverManager
.addServerToDrainList(server
)) {
3697 serversAdded
.add(server
);
3700 // Move the regions off the decommissioned servers.
3702 final List
<ServerName
> destServers
= this.serverManager
.createDestinationServersList();
3703 for (ServerName server
: serversAdded
) {
3704 final List
<RegionInfo
> regionsOnServer
= this.assignmentManager
.getRegionsOnServer(server
);
3705 for (RegionInfo hri
: regionsOnServer
) {
3706 ServerName dest
= balancer
.randomAssignment(hri
, destServers
);
3708 throw new HBaseIOException("Unable to determine a plan to move " + hri
);
3710 RegionPlan rp
= new RegionPlan(hri
, server
, dest
);
3711 this.assignmentManager
.moveAsync(rp
);
3718 * List region servers marked as decommissioned (previously called 'draining') to not get regions
3720 * @return List of decommissioned servers.
3722 public List
<ServerName
> listDecommissionedRegionServers() {
3723 return this.serverManager
.getDrainingServersList();
3727 * Remove decommission marker (previously called 'draining') from a region server to allow regions
3728 * assignments. Load regions onto the server asynchronously if a list of regions is given
3729 * @param server Region server to remove decommission marker from.
3731 public void recommissionRegionServer(final ServerName server
,
3732 final List
<byte[]> encodedRegionNames
) throws IOException
{
3733 // Remove the server from decommissioned (draining) server list.
3734 String parentZnode
= getZooKeeper().getZNodePaths().drainingZNode
;
3735 String node
= ZNodePaths
.joinZNode(parentZnode
, server
.getServerName());
3737 ZKUtil
.deleteNodeFailSilent(getZooKeeper(), node
);
3738 } catch (KeeperException ke
) {
3739 throw new HBaseIOException(
3740 this.zooKeeper
.prefix("Unable to recommission '" + server
.getServerName() + "'."), ke
);
3742 this.serverManager
.removeServerFromDrainList(server
);
3744 // Load the regions onto the server if we are given a list of regions.
3745 if (encodedRegionNames
== null || encodedRegionNames
.isEmpty()) {
3748 if (!this.serverManager
.isServerOnline(server
)) {
3751 for (byte[] encodedRegionName
: encodedRegionNames
) {
3752 RegionState regionState
=
3753 assignmentManager
.getRegionStates().getRegionState(Bytes
.toString(encodedRegionName
));
3754 if (regionState
== null) {
3755 LOG
.warn("Unknown region " + Bytes
.toStringBinary(encodedRegionName
));
3758 RegionInfo hri
= regionState
.getRegion();
3759 if (server
.equals(regionState
.getServerName())) {
3760 LOG
.info("Skipping move of region " + hri
.getRegionNameAsString() +
3761 " because region already assigned to the same server " + server
+ ".");
3764 RegionPlan rp
= new RegionPlan(hri
, regionState
.getServerName(), server
);
3765 this.assignmentManager
.moveAsync(rp
);
3770 public LockManager
getLockManager() {
3774 public QuotaObserverChore
getQuotaObserverChore() {
3775 return this.quotaObserverChore
;
3778 public SpaceQuotaSnapshotNotifier
getSpaceQuotaSnapshotNotifier() {
3779 return this.spaceQuotaSnapshotNotifier
;
3782 @SuppressWarnings("unchecked")
3783 private RemoteProcedure
<MasterProcedureEnv
, ?
> getRemoteProcedure(long procId
) {
3784 Procedure
<?
> procedure
= procedureExecutor
.getProcedure(procId
);
3785 if (procedure
== null) {
3788 assert procedure
instanceof RemoteProcedure
;
3789 return (RemoteProcedure
<MasterProcedureEnv
, ?
>) procedure
;
3792 public void remoteProcedureCompleted(long procId
) {
3793 LOG
.debug("Remote procedure done, pid={}", procId
);
3794 RemoteProcedure
<MasterProcedureEnv
, ?
> procedure
= getRemoteProcedure(procId
);
3795 if (procedure
!= null) {
3796 procedure
.remoteOperationCompleted(procedureExecutor
.getEnvironment());
3800 public void remoteProcedureFailed(long procId
, RemoteProcedureException error
) {
3801 LOG
.debug("Remote procedure failed, pid={}", procId
, error
);
3802 RemoteProcedure
<MasterProcedureEnv
, ?
> procedure
= getRemoteProcedure(procId
);
3803 if (procedure
!= null) {
3804 procedure
.remoteOperationFailed(procedureExecutor
.getEnvironment(), error
);
3809 * Reopen regions provided in the argument
3811 * @param tableName The current table name
3812 * @param regionNames The region names of the regions to reopen
3813 * @param nonceGroup Identifier for the source of the request, a client or process
3814 * @param nonce A unique identifier for this operation from the client or process identified by
3815 * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3816 * @return procedure Id
3817 * @throws IOException if reopening region fails while running procedure
3819 long reopenRegions(final TableName tableName
, final List
<byte[]> regionNames
,
3820 final long nonceGroup
, final long nonce
)
3821 throws IOException
{
3823 return MasterProcedureUtil
3824 .submitProcedure(new MasterProcedureUtil
.NonceProcedureRunnable(this, nonceGroup
, nonce
) {
3827 protected void run() throws IOException
{
3828 submitProcedure(new ReopenTableRegionsProcedure(tableName
, regionNames
));
3832 protected String
getDescription() {
3833 return "ReopenTableRegionsProcedure";
3841 public ReplicationPeerManager
getReplicationPeerManager() {
3842 return replicationPeerManager
;
3845 public HashMap
<String
, List
<Pair
<ServerName
, ReplicationLoadSource
>>>
3846 getReplicationLoad(ServerName
[] serverNames
) {
3847 List
<ReplicationPeerDescription
> peerList
= this.getReplicationPeerManager().listPeers(null);
3848 if (peerList
== null) {
3851 HashMap
<String
, List
<Pair
<ServerName
, ReplicationLoadSource
>>> replicationLoadSourceMap
=
3852 new HashMap
<>(peerList
.size());
3854 .forEach(peer
-> replicationLoadSourceMap
.put(peer
.getPeerId(), new ArrayList
<>()));
3855 for (ServerName serverName
: serverNames
) {
3856 List
<ReplicationLoadSource
> replicationLoadSources
=
3857 getServerManager().getLoad(serverName
).getReplicationLoadSourceList();
3858 for (ReplicationLoadSource replicationLoadSource
: replicationLoadSources
) {
3859 List
<Pair
<ServerName
, ReplicationLoadSource
>> replicationLoadSourceList
=
3860 replicationLoadSourceMap
.get(replicationLoadSource
.getPeerID());
3861 if (replicationLoadSourceList
== null) {
3862 LOG
.debug("{} does not exist, but it exists "
3863 + "in znode(/hbase/replication/rs). when the rs restarts, peerId is deleted, so "
3864 + "we just need to ignore it", replicationLoadSource
.getPeerID());
3867 replicationLoadSourceList
.add(new Pair
<>(serverName
, replicationLoadSource
));
3870 for (List
<Pair
<ServerName
, ReplicationLoadSource
>> loads
: replicationLoadSourceMap
.values()) {
3871 if (loads
.size() > 0) {
3872 loads
.sort(Comparator
.comparingLong(load
-> (-1) * load
.getSecond().getReplicationLag()));
3875 return replicationLoadSourceMap
;
3879 * This method modifies the master's configuration in order to inject replication-related features
3881 @InterfaceAudience.Private
3882 public static void decorateMasterConfiguration(Configuration conf
) {
3883 String plugins
= conf
.get(HBASE_MASTER_LOGCLEANER_PLUGINS
);
3884 String cleanerClass
= ReplicationLogCleaner
.class.getCanonicalName();
3885 if (plugins
== null || !plugins
.contains(cleanerClass
)) {
3886 conf
.set(HBASE_MASTER_LOGCLEANER_PLUGINS
, plugins
+ "," + cleanerClass
);
3888 if (ReplicationUtils
.isReplicationForBulkLoadDataEnabled(conf
)) {
3889 plugins
= conf
.get(HFileCleaner
.MASTER_HFILE_CLEANER_PLUGINS
);
3890 cleanerClass
= ReplicationHFileCleaner
.class.getCanonicalName();
3891 if (!plugins
.contains(cleanerClass
)) {
3892 conf
.set(HFileCleaner
.MASTER_HFILE_CLEANER_PLUGINS
, plugins
+ "," + cleanerClass
);
3897 public SnapshotQuotaObserverChore
getSnapshotQuotaObserverChore() {
3898 return this.snapshotQuotaChore
;
3901 public ActiveMasterManager
getActiveMasterManager() {
3902 return activeMasterManager
;
3906 public SyncReplicationReplayWALManager
getSyncReplicationReplayWALManager() {
3907 return this.syncReplicationReplayWALManager
;
3910 public HbckChore
getHbckChore() {
3911 return this.hbckChore
;
3915 public void runReplicationBarrierCleaner() {
3916 ReplicationBarrierCleaner rbc
= this.replicationBarrierCleaner
;
3923 public RSGroupInfoManager
getRSGroupInfoManager() {
3924 return rsGroupInfoManager
;
3928 * Get the compaction state of the table
3930 * @param tableName The table name
3931 * @return CompactionState Compaction state of the table
3933 public CompactionState
getCompactionState(final TableName tableName
) {
3934 CompactionState compactionState
= CompactionState
.NONE
;
3936 List
<RegionInfo
> regions
=
3937 assignmentManager
.getRegionStates().getRegionsOfTable(tableName
);
3938 for (RegionInfo regionInfo
: regions
) {
3939 ServerName serverName
=
3940 assignmentManager
.getRegionStates().getRegionServerOfRegion(regionInfo
);
3941 if (serverName
== null) {
3944 ServerMetrics sl
= serverManager
.getLoad(serverName
);
3948 RegionMetrics regionMetrics
= sl
.getRegionMetrics().get(regionInfo
.getRegionName());
3949 if (regionMetrics
.getCompactionState() == CompactionState
.MAJOR
) {
3950 if (compactionState
== CompactionState
.MINOR
) {
3951 compactionState
= CompactionState
.MAJOR_AND_MINOR
;
3953 compactionState
= CompactionState
.MAJOR
;
3955 } else if (regionMetrics
.getCompactionState() == CompactionState
.MINOR
) {
3956 if (compactionState
== CompactionState
.MAJOR
) {
3957 compactionState
= CompactionState
.MAJOR_AND_MINOR
;
3959 compactionState
= CompactionState
.MINOR
;
3963 } catch (Exception e
) {
3964 compactionState
= null;
3965 LOG
.error("Exception when get compaction state for " + tableName
.getNameAsString(), e
);
3967 return compactionState
;
3971 public MetaLocationSyncer
getMetaLocationSyncer() {
3972 return metaLocationSyncer
;
3975 @RestrictedApi(explanation
= "Should only be called in tests", link
= "",
3976 allowedOnPath
= ".*/src/test/.*")
3977 MasterRegion
getMasterRegion() {
3978 return masterRegion
;
3982 public void onConfigurationChange(Configuration newConf
) {
3984 Superusers
.initialize(newConf
);
3985 } catch (IOException e
) {
3986 LOG
.warn("Failed to initialize SuperUsers on reloading of the configuration");
3991 protected NamedQueueRecorder
createNamedQueueRecord() {
3992 final boolean isBalancerDecisionRecording
= conf
3993 .getBoolean(BaseLoadBalancer
.BALANCER_DECISION_BUFFER_ENABLED
,
3994 BaseLoadBalancer
.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED
);
3995 final boolean isBalancerRejectionRecording
= conf
3996 .getBoolean(BaseLoadBalancer
.BALANCER_REJECTION_BUFFER_ENABLED
,
3997 BaseLoadBalancer
.DEFAULT_BALANCER_REJECTION_BUFFER_ENABLED
);
3998 if (isBalancerDecisionRecording
|| isBalancerRejectionRecording
) {
3999 return NamedQueueRecorder
.getInstance(conf
);
4006 protected boolean clusterMode() {
4010 public String
getClusterId() {
4014 return cachedClusterId
.getFromCacheOrFetch();
4017 public Optional
<ServerName
> getActiveMaster() {
4018 return activeMasterManager
.getActiveMasterServerName();
4021 public List
<ServerName
> getBackupMasters() {
4022 return activeMasterManager
.getBackupMasters();
4026 public Iterator
<ServerName
> getBootstrapNodes() {
4027 return regionServerTracker
.getRegionServers().iterator();
4031 public List
<HRegionLocation
> getMetaLocations() {
4032 return metaRegionLocationCache
.getMetaRegionLocations();
4035 public Collection
<ServerName
> getLiveRegionServers() {
4036 return regionServerTracker
.getRegionServers();