Skip to content

Commit

Permalink
HBASE-25323 Fix potential NPE when the zookeeper path of RegionServer…
Browse files Browse the repository at this point in the history
…Tracker does not exist when start (apache#2702)

Signed-off-by: Duo Zhang <[email protected]>
Signed-off-by: Guanghao Zhang <[email protected]>
  • Loading branch information
yuqi1129 authored and infraio committed Nov 25, 2020
1 parent 573a6f2 commit b710fbd
Show file tree
Hide file tree
Showing 2 changed files with 19 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
Expand All @@ -35,6 +36,7 @@
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
Expand Down Expand Up @@ -129,21 +131,24 @@ public void start(Set<ServerName> deadServersFromPE, Set<ServerName> liveServers
splittingServersFromWALDir.stream().filter(s -> !deadServersFromPE.contains(s)).
forEach(s -> LOG.error("{} has no matching ServerCrashProcedure", s));
//create ServerNode for all possible live servers from wal directory
liveServersFromWALDir.stream()
liveServersFromWALDir
.forEach(sn -> server.getAssignmentManager().getRegionStates().getOrCreateServer(sn));
watcher.registerListener(this);
synchronized (this) {
List<String> servers =
ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.getZNodePaths().rsZNode);
for (String n : servers) {
Pair<ServerName, RegionServerInfo> pair = getServerInfo(n);
ServerName serverName = pair.getFirst();
RegionServerInfo info = pair.getSecond();
regionServers.add(serverName);
ServerMetrics serverMetrics = info != null ? ServerMetricsBuilder.of(serverName,
VersionInfoUtil.getVersionNumber(info.getVersionInfo()),
info.getVersionInfo().getVersion()) : ServerMetricsBuilder.of(serverName);
serverManager.checkAndRecordNewServer(serverName, serverMetrics);
if (null != servers) {
for (String n : servers) {
Pair<ServerName, RegionServerInfo> pair = getServerInfo(n);
ServerName serverName = pair.getFirst();
RegionServerInfo info = pair.getSecond();
regionServers.add(serverName);
ServerMetrics serverMetrics = info != null ?
ServerMetricsBuilder.of(serverName, VersionInfoUtil.getVersionNumber(info.getVersionInfo()),
info.getVersionInfo().getVersion()) :
ServerMetricsBuilder.of(serverName);
serverManager.checkAndRecordNewServer(serverName, serverMetrics);
}
}
serverManager.findDeadServersAndProcess(deadServersFromPE, liveServersFromWALDir);
}
Expand All @@ -163,8 +168,9 @@ private synchronized void refresh() {
server.abort("Unexpected zk exception getting RS nodes", e);
return;
}
Set<ServerName> servers =
Set<ServerName> servers = CollectionUtils.isEmpty(names) ? Collections.emptySet() :
names.stream().map(ServerName::parseServerName).collect(Collectors.toSet());

for (Iterator<ServerName> iter = regionServers.iterator(); iter.hasNext();) {
ServerName sn = iter.next();
if (!servers.contains(sn)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -443,16 +443,15 @@ public static List<String> listChildrenAndWatchForNewChildren(
} catch(KeeperException.NoNodeException ke) {
LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
"because node does not exist (not an error)"));
return null;
} catch (KeeperException e) {
LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
zkw.keeperException(e);
return null;
} catch (InterruptedException e) {
LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
zkw.interruptedException(e);
return null;
}

return null;
}

/**
Expand Down

0 comments on commit b710fbd

Please sign in to comment.