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