2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org
.apache
.hadoop
.hbase
.master
;
20 import java
.io
.IOException
;
21 import java
.io
.InterruptedIOException
;
22 import java
.util
.Collections
;
23 import java
.util
.List
;
25 import java
.util
.concurrent
.ExecutorService
;
26 import java
.util
.concurrent
.Executors
;
27 import java
.util
.stream
.Collectors
;
28 import org
.apache
.hadoop
.hbase
.ServerMetrics
;
29 import org
.apache
.hadoop
.hbase
.ServerMetricsBuilder
;
30 import org
.apache
.hadoop
.hbase
.ServerName
;
31 import org
.apache
.hadoop
.hbase
.client
.VersionInfoUtil
;
32 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKListener
;
33 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
;
34 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKWatcher
;
35 import org
.apache
.yetus
.audience
.InterfaceAudience
;
36 import org
.apache
.zookeeper
.KeeperException
;
37 import org
.slf4j
.Logger
;
38 import org
.slf4j
.LoggerFactory
;
40 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Sets
;
41 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.util
.concurrent
.ThreadFactoryBuilder
;
42 import org
.apache
.hbase
.thirdparty
.org
.apache
.commons
.collections4
.CollectionUtils
;
44 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.ProtobufUtil
;
45 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.HBaseProtos
.RegionServerInfo
;
48 * Tracks the online region servers via ZK.
50 * Handling of new RSs checking in is done via RPC. This class is only responsible for watching for
51 * expired nodes. It handles listening for changes in the RS node list. The only exception is when
52 * master restart, we will use the list fetched from zk to construct the initial set of live region
55 * If an RS node gets deleted, this automatically handles calling of
56 * {@link ServerManager#expireServer(ServerName)}
58 @InterfaceAudience.Private
59 public class RegionServerTracker
extends ZKListener
{
60 private static final Logger LOG
= LoggerFactory
.getLogger(RegionServerTracker
.class);
61 // indicate whether we are active master
62 private boolean active
;
63 private volatile Set
<ServerName
> regionServers
= Collections
.emptySet();
64 private final MasterServices server
;
65 // As we need to send request to zk when processing the nodeChildrenChanged event, we'd better
66 // move the operation to a single threaded thread pool in order to not block the zk event
67 // processing since all the zk listener across HMaster will be called in one thread sequentially.
68 private final ExecutorService executor
;
70 public RegionServerTracker(ZKWatcher watcher
, MasterServices server
) {
73 this.executor
= Executors
.newSingleThreadExecutor(
74 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("RegionServerTracker-%d").build());
75 watcher
.registerListener(this);
79 private RegionServerInfo
getServerInfo(ServerName serverName
)
80 throws KeeperException
, IOException
{
81 String nodePath
= watcher
.getZNodePaths().getRsPath(serverName
);
84 data
= ZKUtil
.getData(watcher
, nodePath
);
85 } catch (InterruptedException e
) {
86 throw (InterruptedIOException
) new InterruptedIOException().initCause(e
);
89 // we should receive a children changed event later and then we will expire it, so we still
90 // need to add it to the region server set.
91 LOG
.warn("Server node {} does not exist, already dead?", serverName
);
94 if (data
.length
== 0 || !ProtobufUtil
.isPBMagicPrefix(data
)) {
95 // this should not happen actually, unless we have bugs or someone has messed zk up.
96 LOG
.warn("Invalid data for region server node {} on zookeeper, data length = {}", serverName
,
100 RegionServerInfo
.Builder builder
= RegionServerInfo
.newBuilder();
101 int magicLen
= ProtobufUtil
.lengthOfPBMagic();
102 ProtobufUtil
.mergeFrom(builder
, data
, magicLen
, data
.length
- magicLen
);
103 return builder
.build();
107 * Upgrade to active master mode, where besides tracking the changes of region server set, we will
108 * also started to add new region servers to ServerManager and also schedule SCP if a region
109 * server dies. Starts the tracking of online RegionServers. All RSes will be tracked after this
112 * In this method, we will also construct the region server sets in {@link ServerManager}. If a
113 * region server is dead between the crash of the previous master instance and the start of the
114 * current master instance, we will schedule a SCP for it. This is done in
115 * {@link ServerManager#findDeadServersAndProcess(Set, Set)}, we call it here under the lock
116 * protection to prevent concurrency issues with server expiration operation.
117 * @param deadServersFromPE the region servers which already have SCP associated.
118 * @param liveServersBeforeRestart the live region servers we recorded before master restarts.
119 * @param splittingServersFromWALDir Servers whose WALs are being actively 'split'.
121 public void upgrade(Set
<ServerName
> deadServersFromPE
, Set
<ServerName
> liveServersBeforeRestart
,
122 Set
<ServerName
> splittingServersFromWALDir
) throws KeeperException
, IOException
{
124 "Upgrading RegionServerTracker to active master mode; {} have existing" +
125 "ServerCrashProcedures, {} possibly 'live' servers, and {} 'splitting'.",
126 deadServersFromPE
.size(), liveServersBeforeRestart
.size(), splittingServersFromWALDir
.size());
127 // deadServersFromPE is made from a list of outstanding ServerCrashProcedures.
128 // splittingServersFromWALDir are being actively split -- the directory in the FS ends in
129 // '-SPLITTING'. Each splitting server should have a corresponding SCP. Log if not.
130 splittingServersFromWALDir
.stream().filter(s
-> !deadServersFromPE
.contains(s
)).
131 forEach(s
-> LOG
.error("{} has no matching ServerCrashProcedure", s
));
132 // create ServerNode for all possible live servers from wal directory
133 liveServersBeforeRestart
134 .forEach(sn
-> server
.getAssignmentManager().getRegionStates().getOrCreateServer(sn
));
135 ServerManager serverManager
= server
.getServerManager();
136 synchronized (this) {
137 Set
<ServerName
> liveServers
= regionServers
;
138 for (ServerName serverName
: liveServers
) {
139 RegionServerInfo info
= getServerInfo(serverName
);
140 ServerMetrics serverMetrics
= info
!= null ? ServerMetricsBuilder
.of(serverName
,
141 VersionInfoUtil
.getVersionNumber(info
.getVersionInfo()),
142 info
.getVersionInfo().getVersion()) : ServerMetricsBuilder
.of(serverName
);
143 serverManager
.checkAndRecordNewServer(serverName
, serverMetrics
);
145 serverManager
.findDeadServersAndProcess(deadServersFromPE
, liveServersBeforeRestart
);
151 executor
.shutdownNow();
154 public Set
<ServerName
> getRegionServers() {
155 return regionServers
;
158 // execute the operations which are only needed for active masters, such as expire old servers,
159 // add new servers, etc.
160 private void processAsActiveMaster(Set
<ServerName
> newServers
) {
161 Set
<ServerName
> oldServers
= regionServers
;
162 ServerManager serverManager
= server
.getServerManager();
163 // expire dead servers
164 for (ServerName crashedServer
: Sets
.difference(oldServers
, newServers
)) {
165 LOG
.info("RegionServer ephemeral node deleted, processing expiration [{}]", crashedServer
);
166 serverManager
.expireServer(crashedServer
);
168 // check whether there are new servers, log them
169 boolean newServerAdded
= false;
170 for (ServerName sn
: newServers
) {
171 if (!oldServers
.contains(sn
)) {
172 newServerAdded
= true;
173 LOG
.info("RegionServer ephemeral node created, adding [" + sn
+ "]");
176 if (newServerAdded
&& server
.isInitialized()) {
177 // Only call the check to move servers if a RegionServer was added to the cluster; in this
178 // case it could be a server with a new version so it makes sense to run the check.
179 server
.checkIfShouldMoveSystemRegionAsync();
183 private synchronized void refresh() {
186 names
= ZKUtil
.listChildrenAndWatchForNewChildren(watcher
, watcher
.getZNodePaths().rsZNode
);
187 } catch (KeeperException e
) {
188 // here we need to abort as we failed to set watcher on the rs node which means that we can
189 // not track the node deleted event any more.
190 server
.abort("Unexpected zk exception getting RS nodes", e
);
193 Set
<ServerName
> newServers
= CollectionUtils
.isEmpty(names
) ? Collections
.emptySet() :
194 names
.stream().map(ServerName
::parseServerName
)
195 .collect(Collectors
.collectingAndThen(Collectors
.toSet(), Collections
::unmodifiableSet
));
197 processAsActiveMaster(newServers
);
199 this.regionServers
= newServers
;
203 public void nodeChildrenChanged(String path
) {
204 if (path
.equals(watcher
.getZNodePaths().rsZNode
) && !server
.isAborted() &&
205 !server
.isStopped()) {
206 executor
.execute(this::refresh
);