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.Collections;
023import java.util.HashSet;
024import java.util.Iterator;
025import java.util.List;
026import java.util.Set;
027import java.util.concurrent.ExecutorService;
028import java.util.concurrent.Executors;
029import java.util.stream.Collectors;
030import org.apache.hadoop.hbase.ServerMetrics;
031import org.apache.hadoop.hbase.ServerMetricsBuilder;
032import org.apache.hadoop.hbase.ServerName;
033import org.apache.hadoop.hbase.client.VersionInfoUtil;
034import org.apache.hadoop.hbase.util.Pair;
035import org.apache.hadoop.hbase.zookeeper.ZKListener;
036import org.apache.hadoop.hbase.zookeeper.ZKUtil;
037import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
038import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
039import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
040import org.apache.yetus.audience.InterfaceAudience;
041import org.apache.zookeeper.KeeperException;
042import org.slf4j.Logger;
043import org.slf4j.LoggerFactory;
044
045import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
046
047import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
048import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
049
050/**
051 * Tracks the online region servers via ZK.
052 * <p/>
053 * Handling of new RSs checking in is done via RPC. This class is only responsible for watching for
054 * expired nodes. It handles listening for changes in the RS node list. The only exception is when
055 * master restart, we will use the list fetched from zk to construct the initial set of live region
056 * servers.
057 * <p/>
058 * If an RS node gets deleted, this automatically handles calling of
059 * {@link ServerManager#expireServer(ServerName)}
060 */
061@InterfaceAudience.Private
062public class RegionServerTracker extends ZKListener {
063  private static final Logger LOG = LoggerFactory.getLogger(RegionServerTracker.class);
064  private final Set<ServerName> regionServers = new HashSet<>();
065  private final ServerManager serverManager;
066  private final MasterServices server;
067  // As we need to send request to zk when processing the nodeChildrenChanged event, we'd better
068  // move the operation to a single threaded thread pool in order to not block the zk event
069  // processing since all the zk listener across HMaster will be called in one thread sequentially.
070  private final ExecutorService executor;
071
072  public RegionServerTracker(ZKWatcher watcher, MasterServices server,
073      ServerManager serverManager) {
074    super(watcher);
075    this.server = server;
076    this.serverManager = serverManager;
077    this.executor = Executors.newSingleThreadExecutor(
078      new ThreadFactoryBuilder().setDaemon(true).setNameFormat("RegionServerTracker-%d").build());
079  }
080
081  private Pair<ServerName, RegionServerInfo> getServerInfo(String name)
082      throws KeeperException, IOException {
083    ServerName serverName = ServerName.parseServerName(name);
084    String nodePath = ZNodePaths.joinZNode(watcher.getZNodePaths().rsZNode, name);
085    byte[] data;
086    try {
087      data = ZKUtil.getData(watcher, nodePath);
088    } catch (InterruptedException e) {
089      throw (InterruptedIOException) new InterruptedIOException().initCause(e);
090    }
091    if (data == null) {
092      // we should receive a children changed event later and then we will expire it, so we still
093      // need to add it to the region server set.
094      LOG.warn("Server node {} does not exist, already dead?", name);
095      return Pair.newPair(serverName, null);
096    }
097    if (data.length == 0 || !ProtobufUtil.isPBMagicPrefix(data)) {
098      // this should not happen actually, unless we have bugs or someone has messed zk up.
099      LOG.warn("Invalid data for region server node {} on zookeeper, data length = {}", name,
100        data.length);
101      return Pair.newPair(serverName, null);
102    }
103    RegionServerInfo.Builder builder = RegionServerInfo.newBuilder();
104    int magicLen = ProtobufUtil.lengthOfPBMagic();
105    ProtobufUtil.mergeFrom(builder, data, magicLen, data.length - magicLen);
106    return Pair.newPair(serverName, builder.build());
107  }
108
109  /**
110   * Starts the tracking of online RegionServers. All RSes will be tracked after this method is
111   * called.
112   * <p/>
113   * In this method, we will also construct the region server sets in {@link ServerManager}. If a
114   * region server is dead between the crash of the previous master instance and the start of the
115   * current master instance, we will schedule a SCP for it. This is done in
116   * {@link ServerManager#findDeadServersAndProcess(Set, Set)}, we call it here under the lock
117   * protection to prevent concurrency issues with server expiration operation.
118   * @param deadServersFromPE the region servers which already have SCP associated.
119   * @param liveServersFromWALDir the live region servers from wal directory.
120   * @param splittingServersFromWALDir Servers whose WALs are being actively 'split'.
121   */
122  public void start(Set<ServerName> deadServersFromPE, Set<ServerName> liveServersFromWALDir,
123      Set<ServerName> splittingServersFromWALDir)
124      throws KeeperException, IOException {
125    LOG.info("Starting RegionServerTracker; {} have existing ServerCrashProcedures, {} " +
126        "possibly 'live' servers, and {} 'splitting'.", deadServersFromPE.size(),
127        liveServersFromWALDir.size(), splittingServersFromWALDir.size());
128    // deadServersFromPE is made from a list of outstanding ServerCrashProcedures.
129    // splittingServersFromWALDir are being actively split -- the directory in the FS ends in
130    // '-SPLITTING'. Each splitting server should have a corresponding SCP. Log if not.
131    splittingServersFromWALDir.stream().filter(s -> !deadServersFromPE.contains(s)).
132      forEach(s -> LOG.error("{} has no matching ServerCrashProcedure", s));
133    //create ServerNode for all possible live servers from wal directory
134    liveServersFromWALDir
135        .forEach(sn -> server.getAssignmentManager().getRegionStates().getOrCreateServer(sn));
136    watcher.registerListener(this);
137    synchronized (this) {
138      List<String> servers =
139        ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.getZNodePaths().rsZNode);
140      if (null != servers) {
141        for (String n : servers) {
142          Pair<ServerName, RegionServerInfo> pair = getServerInfo(n);
143          ServerName serverName = pair.getFirst();
144          RegionServerInfo info = pair.getSecond();
145          regionServers.add(serverName);
146          ServerMetrics serverMetrics = info != null ?
147            ServerMetricsBuilder.of(serverName, VersionInfoUtil.getVersionNumber(info.getVersionInfo()),
148              info.getVersionInfo().getVersion()) :
149            ServerMetricsBuilder.of(serverName);
150          serverManager.checkAndRecordNewServer(serverName, serverMetrics);
151        }
152      }
153      serverManager.findDeadServersAndProcess(deadServersFromPE, liveServersFromWALDir);
154    }
155  }
156
157  public void stop() {
158    executor.shutdownNow();
159  }
160
161  private synchronized void refresh() {
162    List<String> names;
163    try {
164      names = ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.getZNodePaths().rsZNode);
165    } catch (KeeperException e) {
166      // here we need to abort as we failed to set watcher on the rs node which means that we can
167      // not track the node deleted evetnt any more.
168      server.abort("Unexpected zk exception getting RS nodes", e);
169      return;
170    }
171    Set<ServerName> servers = CollectionUtils.isEmpty(names) ? Collections.emptySet() :
172      names.stream().map(ServerName::parseServerName).collect(Collectors.toSet());
173
174    for (Iterator<ServerName> iter = regionServers.iterator(); iter.hasNext();) {
175      ServerName sn = iter.next();
176      if (!servers.contains(sn)) {
177        LOG.info("RegionServer ephemeral node deleted, processing expiration [{}]", sn);
178        serverManager.expireServer(sn);
179        iter.remove();
180      }
181    }
182    // here we do not need to parse the region server info as it is useless now, we only need the
183    // server name.
184    boolean newServerAdded = false;
185    for (ServerName sn : servers) {
186      if (regionServers.add(sn)) {
187        newServerAdded = true;
188        LOG.info("RegionServer ephemeral node created, adding [" + sn + "]");
189      }
190    }
191    if (newServerAdded && server.isInitialized()) {
192      // Only call the check to move servers if a RegionServer was added to the cluster; in this
193      // case it could be a server with a new version so it makes sense to run the check.
194      server.checkIfShouldMoveSystemRegionAsync();
195    }
196  }
197
198  @Override
199  public void nodeChildrenChanged(String path) {
200    if (path.equals(watcher.getZNodePaths().rsZNode) && !server.isAborted() &&
201      !server.isStopped()) {
202      executor.execute(this::refresh);
203    }
204  }
205}