HBASE-25323 Fix potential NPE when the zookeeper path of RegionServerTracker does not exist when start (#2702)
Signed-off-by: Duo Zhang <zhangduo@apache.org> Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
parent
6f07efb971
commit
3dd425abfa
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.master;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKListener;
|
|||
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;
|
||||
|
@ -129,21 +131,24 @@ public class RegionServerTracker extends ZKListener {
|
|||
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);
|
||||
}
|
||||
|
@ -163,8 +168,9 @@ public class RegionServerTracker extends ZKListener {
|
|||
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)) {
|
||||
|
|
|
@ -297,7 +297,7 @@ public class MasterAddressTracker extends ZKNodeTracker {
|
|||
public static List<ServerName> getBackupMastersAndRenewWatch(
|
||||
ZKWatcher zkw) throws InterruptedIOException {
|
||||
// Build Set of backup masters from ZK nodes
|
||||
List<String> backupMasterStrings = Collections.emptyList();
|
||||
List<String> backupMasterStrings = null;
|
||||
try {
|
||||
backupMasterStrings = ZKUtil.listChildrenAndWatchForNewChildren(zkw,
|
||||
zkw.getZNodePaths().backupMasterAddressesZNode);
|
||||
|
|
|
@ -445,16 +445,15 @@ public final class ZKUtil {
|
|||
} 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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue