001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.master; 019 020import java.io.IOException; 021import java.io.InterruptedIOException; 022import java.util.HashSet; 023import java.util.Iterator; 024import java.util.List; 025import java.util.Set; 026import java.util.concurrent.ExecutorService; 027import java.util.concurrent.Executors; 028import java.util.stream.Collectors; 029import org.apache.hadoop.hbase.ServerMetrics; 030import org.apache.hadoop.hbase.ServerMetricsBuilder; 031import org.apache.hadoop.hbase.ServerName; 032import org.apache.hadoop.hbase.client.VersionInfoUtil; 033import org.apache.hadoop.hbase.util.Pair; 034import org.apache.hadoop.hbase.zookeeper.ZKListener; 035import org.apache.hadoop.hbase.zookeeper.ZKUtil; 036import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 037import org.apache.hadoop.hbase.zookeeper.ZNodePaths; 038import org.apache.yetus.audience.InterfaceAudience; 039import org.apache.zookeeper.KeeperException; 040import org.slf4j.Logger; 041import org.slf4j.LoggerFactory; 042 043import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 044 045import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 046import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo; 047 048/** 049 * Tracks the online region servers via ZK. 050 * <p/> 051 * Handling of new RSs checking in is done via RPC. This class is only responsible for watching for 052 * expired nodes. It handles listening for changes in the RS node list. The only exception is when 053 * master restart, we will use the list fetched from zk to construct the initial set of live region 054 * servers. 055 * <p/> 056 * If an RS node gets deleted, this automatically handles calling of 057 * {@link ServerManager#expireServer(ServerName)} 058 */ 059@InterfaceAudience.Private 060public class RegionServerTracker extends ZKListener { 061 private static final Logger LOG = LoggerFactory.getLogger(RegionServerTracker.class); 062 private final Set<ServerName> regionServers = new HashSet<>(); 063 private final ServerManager serverManager; 064 private final MasterServices server; 065 // As we need to send request to zk when processing the nodeChildrenChanged event, we'd better 066 // move the operation to a single threaded thread pool in order to not block the zk event 067 // processing since all the zk listener across HMaster will be called in one thread sequentially. 068 private final ExecutorService executor; 069 070 public RegionServerTracker(ZKWatcher watcher, MasterServices server, 071 ServerManager serverManager) { 072 super(watcher); 073 this.server = server; 074 this.serverManager = serverManager; 075 this.executor = Executors.newSingleThreadExecutor( 076 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("RegionServerTracker-%d").build()); 077 } 078 079 private Pair<ServerName, RegionServerInfo> getServerInfo(String name) 080 throws KeeperException, IOException { 081 ServerName serverName = ServerName.parseServerName(name); 082 String nodePath = ZNodePaths.joinZNode(watcher.getZNodePaths().rsZNode, name); 083 byte[] data; 084 try { 085 data = ZKUtil.getData(watcher, nodePath); 086 } catch (InterruptedException e) { 087 throw (InterruptedIOException) new InterruptedIOException().initCause(e); 088 } 089 if (data == null) { 090 // we should receive a children changed event later and then we will expire it, so we still 091 // need to add it to the region server set. 092 LOG.warn("Server node {} does not exist, already dead?", name); 093 return Pair.newPair(serverName, null); 094 } 095 if (data.length == 0 || !ProtobufUtil.isPBMagicPrefix(data)) { 096 // this should not happen actually, unless we have bugs or someone has messed zk up. 097 LOG.warn("Invalid data for region server node {} on zookeeper, data length = {}", name, 098 data.length); 099 return Pair.newPair(serverName, null); 100 } 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()); 105 } 106 107 /** 108 * Starts the tracking of online RegionServers. All RSes will be tracked after this method is 109 * called. 110 * <p/> 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'. 119 */ 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); 147 } 148 serverManager.findDeadServersAndProcess(deadServersFromPE, liveServersFromWALDir); 149 } 150 } 151 152 public void stop() { 153 executor.shutdownNow(); 154 } 155 156 private synchronized void refresh() { 157 List<String> names; 158 try { 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); 164 return; 165 } 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); 173 iter.remove(); 174 } 175 } 176 // here we do not need to parse the region server info as it is useless now, we only need the 177 // server name. 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 + "]"); 183 } 184 } 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(); 189 } 190 } 191 192 @Override 193 public void nodeChildrenChanged(String path) { 194 if (path.equals(watcher.getZNodePaths().rsZNode) && !server.isAborted() && 195 !server.isStopped()) { 196 executor.execute(this::refresh); 197 } 198 } 199}