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
.HashSet
;
23 import java
.util
.Iterator
;
24 import java
.util
.List
;
26 import java
.util
.concurrent
.ExecutorService
;
27 import java
.util
.concurrent
.Executors
;
28 import java
.util
.stream
.Collectors
;
29 import org
.apache
.hadoop
.hbase
.ServerMetrics
;
30 import org
.apache
.hadoop
.hbase
.ServerMetricsBuilder
;
31 import org
.apache
.hadoop
.hbase
.ServerName
;
32 import org
.apache
.hadoop
.hbase
.client
.VersionInfoUtil
;
33 import org
.apache
.hadoop
.hbase
.util
.Pair
;
34 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKListener
;
35 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
;
36 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKWatcher
;
37 import org
.apache
.hadoop
.hbase
.zookeeper
.ZNodePaths
;
38 import org
.apache
.yetus
.audience
.InterfaceAudience
;
39 import org
.apache
.zookeeper
.KeeperException
;
40 import org
.slf4j
.Logger
;
41 import org
.slf4j
.LoggerFactory
;
43 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.util
.concurrent
.ThreadFactoryBuilder
;
45 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.ProtobufUtil
;
46 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.HBaseProtos
.RegionServerInfo
;
49 * Tracks the online region servers via ZK.
51 * Handling of new RSs checking in is done via RPC. This class is only responsible for watching for
52 * expired nodes. It handles listening for changes in the RS node list. The only exception is when
53 * master restart, we will use the list fetched from zk to construct the initial set of live region
56 * If an RS node gets deleted, this automatically handles calling of
57 * {@link ServerManager#expireServer(ServerName)}
59 @InterfaceAudience.Private
60 public class RegionServerTracker
extends ZKListener
{
61 private static final Logger LOG
= LoggerFactory
.getLogger(RegionServerTracker
.class);
62 private final Set
<ServerName
> regionServers
= new HashSet
<>();
63 private final ServerManager serverManager
;
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
,
71 ServerManager serverManager
) {
74 this.serverManager
= serverManager
;
75 this.executor
= Executors
.newSingleThreadExecutor(
76 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("RegionServerTracker-%d").build());
79 private Pair
<ServerName
, RegionServerInfo
> getServerInfo(String name
)
80 throws KeeperException
, IOException
{
81 ServerName serverName
= ServerName
.parseServerName(name
);
82 String nodePath
= ZNodePaths
.joinZNode(watcher
.getZNodePaths().rsZNode
, name
);
85 data
= ZKUtil
.getData(watcher
, nodePath
);
86 } catch (InterruptedException e
) {
87 throw (InterruptedIOException
) new InterruptedIOException().initCause(e
);
90 // we should receive a children changed event later and then we will expire it, so we still
91 // need to add it to the region server set.
92 LOG
.warn("Server node {} does not exist, already dead?", name
);
93 return Pair
.newPair(serverName
, null);
95 if (data
.length
== 0 || !ProtobufUtil
.isPBMagicPrefix(data
)) {
96 // this should not happen actually, unless we have bugs or someone has messed zk up.
97 LOG
.warn("Invalid data for region server node {} on zookeeper, data length = {}", name
,
99 return Pair
.newPair(serverName
, null);
101 RegionServerInfo
.Builder builder
= RegionServerInfo
.newBuilder();
102 int magicLen
= ProtobufUtil
.lengthOfPBMagic();
103 ProtobufUtil
.mergeFrom(builder
, data
, magicLen
, data
.length
- magicLen
);
104 return Pair
.newPair(serverName
, builder
.build());
108 * Starts the tracking of online RegionServers. All RSes will be tracked after this method is
111 * In this method, we will also construct the region server sets in {@link ServerManager}. If a
112 * region server is dead between the crash of the previous master instance and the start of the
113 * current master instance, we will schedule a SCP for it. This is done in
114 * {@link ServerManager#findDeadServersAndProcess(Set, Set)}, we call it here under the lock
115 * protection to prevent concurrency issues with server expiration operation.
116 * @param deadServersFromPE the region servers which already have SCP associated.
117 * @param liveServersFromWALDir the live region servers from wal directory.
118 * @param splittingServersFromWALDir Servers whose WALs are being actively 'split'.
120 public void start(Set
<ServerName
> deadServersFromPE
, Set
<ServerName
> liveServersFromWALDir
,
121 Set
<ServerName
> splittingServersFromWALDir
)
122 throws KeeperException
, IOException
{
123 LOG
.info("Starting RegionServerTracker; {} have existing ServerCrashProcedures, {} " +
124 "possibly 'live' servers, and {} 'splitting'.", deadServersFromPE
.size(),
125 liveServersFromWALDir
.size(), splittingServersFromWALDir
.size());
126 // deadServersFromPE is made from a list of outstanding ServerCrashProcedures.
127 // splittingServersFromWALDir are being actively split -- the directory in the FS ends in
128 // '-SPLITTING'. Each splitting server should have a corresponding SCP. Log if not.
129 splittingServersFromWALDir
.stream().filter(s
-> !deadServersFromPE
.contains(s
)).
130 forEach(s
-> LOG
.error("{} has no matching ServerCrashProcedure", s
));
131 //create ServerNode for all possible live servers from wal directory
132 liveServersFromWALDir
.stream()
133 .forEach(sn
-> server
.getAssignmentManager().getRegionStates().getOrCreateServer(sn
));
134 watcher
.registerListener(this);
135 synchronized (this) {
136 List
<String
> servers
=
137 ZKUtil
.listChildrenAndWatchForNewChildren(watcher
, watcher
.getZNodePaths().rsZNode
);
138 for (String n
: servers
) {
139 Pair
<ServerName
, RegionServerInfo
> pair
= getServerInfo(n
);
140 ServerName serverName
= pair
.getFirst();
141 RegionServerInfo info
= pair
.getSecond();
142 regionServers
.add(serverName
);
143 ServerMetrics serverMetrics
= info
!= null ? ServerMetricsBuilder
.of(serverName
,
144 VersionInfoUtil
.getVersionNumber(info
.getVersionInfo()),
145 info
.getVersionInfo().getVersion()) : ServerMetricsBuilder
.of(serverName
);
146 serverManager
.checkAndRecordNewServer(serverName
, serverMetrics
);
148 serverManager
.findDeadServersAndProcess(deadServersFromPE
, liveServersFromWALDir
);
153 executor
.shutdownNow();
156 private synchronized void refresh() {
159 names
= ZKUtil
.listChildrenAndWatchForNewChildren(watcher
, watcher
.getZNodePaths().rsZNode
);
160 } catch (KeeperException e
) {
161 // here we need to abort as we failed to set watcher on the rs node which means that we can
162 // not track the node deleted evetnt any more.
163 server
.abort("Unexpected zk exception getting RS nodes", e
);
166 Set
<ServerName
> servers
=
167 names
.stream().map(ServerName
::parseServerName
).collect(Collectors
.toSet());
168 for (Iterator
<ServerName
> iter
= regionServers
.iterator(); iter
.hasNext();) {
169 ServerName sn
= iter
.next();
170 if (!servers
.contains(sn
)) {
171 LOG
.info("RegionServer ephemeral node deleted, processing expiration [{}]", sn
);
172 serverManager
.expireServer(sn
);
176 // here we do not need to parse the region server info as it is useless now, we only need the
178 boolean newServerAdded
= false;
179 for (ServerName sn
: servers
) {
180 if (regionServers
.add(sn
)) {
181 newServerAdded
= true;
182 LOG
.info("RegionServer ephemeral node created, adding [" + sn
+ "]");
185 if (newServerAdded
&& server
.isInitialized()) {
186 // Only call the check to move servers if a RegionServer was added to the cluster; in this
187 // case it could be a server with a new version so it makes sense to run the check.
188 server
.checkIfShouldMoveSystemRegionAsync();
193 public void nodeChildrenChanged(String path
) {
194 if (path
.equals(watcher
.getZNodePaths().rsZNode
) && !server
.isAborted() &&
195 !server
.isStopped()) {
196 executor
.execute(this::refresh
);