You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2018/10/26 03:30:56 UTC

hbase git commit: HBASE-21380 Filter finished SCP at start

Repository: hbase
Updated Branches:
  refs/heads/branch-2.1 7de5f1d60 -> 127de9e63


HBASE-21380 Filter finished SCP at start


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/127de9e6
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/127de9e6
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/127de9e6

Branch: refs/heads/branch-2.1
Commit: 127de9e637932de246a5f01e20ecdbd5fa968a36
Parents: 7de5f1d
Author: Mike Drob <md...@apache.org>
Authored: Thu Oct 25 12:39:19 2018 -0500
Committer: Michael Stack <st...@apache.org>
Committed: Thu Oct 25 20:28:51 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/DeadServer.java   | 15 ++++++++++++---
 .../java/org/apache/hadoop/hbase/master/HMaster.java |  2 +-
 .../hadoop/hbase/master/RegionServerTracker.java     |  7 +++++--
 .../apache/hadoop/hbase/master/ServerManager.java    |  5 +++--
 4 files changed, 21 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/127de9e6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
index 4183201..de094c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
@@ -121,17 +121,26 @@ public class DeadServer {
     return clone;
   }
 
+
   /**
    * Adds the server to the dead server list if it's not there already.
    * @param sn the server name
    */
   public synchronized void add(ServerName sn) {
+    add(sn, true);
+  }
+
+  /**
+   * Adds the server to the dead server list if it's not there already.
+   * @param sn the server name
+   * @param processing whether there is an active SCP associated with the server
+   */
+  public synchronized void add(ServerName sn, boolean processing) {
     if (!deadServers.containsKey(sn)){
       deadServers.put(sn, EnvironmentEdgeManager.currentTime());
     }
-    boolean added = processingServers.add(sn);
-    if (LOG.isDebugEnabled() && added) {
-      LOG.debug("Added " + sn + "; numProcessing=" + processingServers.size());
+    if (processing && processingServers.add(sn)) {
+      LOG.debug("Added {}; numProcessing={}", sn, processingServers.size());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/127de9e6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index e8a9ef8..93d2b21 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -903,7 +903,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
     this.regionServerTracker.start(
       procedureExecutor.getProcedures().stream().filter(p -> p instanceof ServerCrashProcedure)
-        .map(p -> ((ServerCrashProcedure) p).getServerName()).collect(Collectors.toSet()),
+        .map(p -> ((ServerCrashProcedure) p)).collect(Collectors.toSet()),
       walManager.getLiveServersFromWALDir(), walManager.getSplittingServersFromWALDir());
     // This manager will be started AFTER hbase:meta is confirmed on line.
     // hbase.mirror.table.state.to.zookeeper is so hbase1 clients can connect. They read table

http://git-wip-us.apache.org/repos/asf/hbase/blob/127de9e6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
----------------------------------------------------------------------
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 f419732..2a3f8b9 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
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.VersionInfoUtil;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -117,7 +118,7 @@ public class RegionServerTracker extends ZKListener {
    * @param liveServersFromWALDir the live region servers from wal directory.
    * @param splittingServersFromWALDir Servers whose WALs are being actively 'split'.
    */
-  public void start(Set<ServerName> deadServersFromPE, Set<ServerName> liveServersFromWALDir,
+  public void start(Set<ServerCrashProcedure> deadServersFromPE, Set<ServerName> liveServersFromWALDir,
       Set<ServerName> splittingServersFromWALDir)
       throws KeeperException, IOException {
     LOG.info("Starting RegionServerTracker; {} have existing ServerCrashProcedures, {} " +
@@ -126,7 +127,9 @@ public class RegionServerTracker extends ZKListener {
     // deadServersFromPE is made from a list of outstanding ServerCrashProcedures.
     // splittingServersFromWALDir are being actively split -- the directory in the FS ends in
     // '-SPLITTING'. Each splitting server should have a corresponding SCP. Log if not.
-    splittingServersFromWALDir.stream().filter(s -> !deadServersFromPE.contains(s)).
+    Set<ServerName> deadServerNames = deadServersFromPE.stream()
+        .map(s -> s.getServerName()).collect(Collectors.toSet());
+    splittingServersFromWALDir.stream().filter(s -> !deadServerNames.contains(s)).
       forEach(s -> LOG.error("{} has no matching ServerCrashProcedure", s));
     watcher.registerListener(this);
     synchronized (this) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/127de9e6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 5489b1e..8282339 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -316,9 +317,9 @@ public class ServerManager {
    * @param deadServersFromPE the region servers which already have a SCP associated.
    * @param liveServersFromWALDir the live region servers from wal directory.
    */
-  void findDeadServersAndProcess(Set<ServerName> deadServersFromPE,
+  void findDeadServersAndProcess(Set<ServerCrashProcedure> deadServersFromPE,
       Set<ServerName> liveServersFromWALDir) {
-    deadServersFromPE.forEach(deadservers::add);
+    deadServersFromPE.forEach(scp -> deadservers.add(scp.getServerName(), !scp.isFinished()));
     liveServersFromWALDir.stream().filter(sn -> !onlineServers.containsKey(sn))
       .forEach(this::expireServer);
   }