From 66b8d0bcbea24a1aa1724a3cf9963c9b826a0ae5 Mon Sep 17 00:00:00 2001 From: Qi Yu Date: Wed, 25 Nov 2020 15:13:09 +0800 Subject: [PATCH] HBASE-25323 Fix potential NPE when the zookeeper path of RegionServerTracker does not exist when start (#2702) Signed-off-by: Duo Zhang Signed-off-by: Guanghao Zhang (cherry picked from commit b710fbd243bf306f2f84467b89caafb3278e5106) Change-Id: I8449e9eeb93715d598d644abb545e133f667b8b4 --- .../hbase/master/RegionServerTracker.java | 28 +++++++++++-------- .../apache/hadoop/hbase/zookeeper/ZKUtil.java | 5 ++-- 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java index 9d33a2120859..336f9dc04f8e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java @@ -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; @@ -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; @@ -129,21 +131,24 @@ public void start(Set deadServersFromPE, Set 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 servers = ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.getZNodePaths().rsZNode); - for (String n : servers) { - Pair 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 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 @@ private synchronized void refresh() { server.abort("Unexpected zk exception getting RS nodes", e); return; } - Set servers = + Set servers = CollectionUtils.isEmpty(names) ? Collections.emptySet() : names.stream().map(ServerName::parseServerName).collect(Collectors.toSet()); + for (Iterator iter = regionServers.iterator(); iter.hasNext();) { ServerName sn = iter.next(); if (!servers.contains(sn)) { diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 584cb5ffe30a..1b8d9294cf41 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -443,16 +443,15 @@ public static List 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; } /**