HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / master / RegionServerTracker.java
blob63a3ec19facc08719d6abb11326a53714332c17a
1 /**
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;
24 import java.util.Set;
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;
47 /**
48 * Tracks the online region servers via ZK.
49 * <p/>
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
53 * servers.
54 * <p/>
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) {
71 super(watcher);
72 this.server = server;
73 this.executor = Executors.newSingleThreadExecutor(
74 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("RegionServerTracker-%d").build());
75 watcher.registerListener(this);
76 refresh();
79 private RegionServerInfo getServerInfo(ServerName serverName)
80 throws KeeperException, IOException {
81 String nodePath = watcher.getZNodePaths().getRsPath(serverName);
82 byte[] data;
83 try {
84 data = ZKUtil.getData(watcher, nodePath);
85 } catch (InterruptedException e) {
86 throw (InterruptedIOException) new InterruptedIOException().initCause(e);
88 if (data == null) {
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);
92 return null;
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,
97 data.length);
98 return null;
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
110 * method is called.
111 * <p/>
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 {
123 LOG.info(
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);
146 active = true;
150 public void stop() {
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() {
184 List<String> names;
185 try {
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);
191 return;
193 Set<ServerName> newServers = CollectionUtils.isEmpty(names) ? Collections.emptySet() :
194 names.stream().map(ServerName::parseServerName)
195 .collect(Collectors.collectingAndThen(Collectors.toSet(), Collections::unmodifiableSet));
196 if (active) {
197 processAsActiveMaster(newServers);
199 this.regionServers = newServers;
202 @Override
203 public void nodeChildrenChanged(String path) {
204 if (path.equals(watcher.getZNodePaths().rsZNode) && !server.isAborted() &&
205 !server.isStopped()) {
206 executor.execute(this::refresh);