You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/03/18 05:33:55 UTC

[GitHub] [hbase] saintstack opened a new pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

saintstack opened a new pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305
 
 
   …rdown
   
   Remove Region from RegionServer#regionsInTransitionInRS Map even on
   exception (because server is aborted).
   
   hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    Only dump metrics once on abort. Refactor closeRegion method removing overlap with
    CloseRegionHandler; i.e. call to CP.
   
   hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
    Refactor so we remove Region from regionsInTransitionRS on success or failure.
   
   hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
    Cleanup but also  made it so UnassignRegionHandler can use it closing Regions.
   
   hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
    Use CloseRegionHandler to close Region rather than dupe code.
   
   hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
    Bit more info when thread dumping...
   
   TestRegionServerNoMaster was changed because it presumes a behavior not
   present anymore (that CRH will leave danglng mention in regionsInTransitionInRS
   as it used to)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-601339743
 
 
   Ouch. I accidentally closed this.  No reopen. I just pushed a minimum change. Let me do again. Make a new PR. For comments here, will fold them into new  HBASE-24015  JIRA

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394685750
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
 
 Review comment:
   good.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394708650
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -38,9 +36,13 @@
 /**
  * Handles closing of a region on a region server.
  * <p/>
- * Now for regular close region request, we will use {@link UnassignRegionHandler} instead. But when
- * shutting down the region server, will also close regions and the related methods still use this
- * class so we keep it here.
+ * In normal operation, we use {@link UnassignRegionHandler} closing Regions but when shutting down
+ * the region server and closing out Regions, we use this handler instead; it does not expect to
 
 Review comment:
   This is for cluster shutdown only. Table disable is run by the Master so unassign procedures.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394700853
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
   private final RegionInfo regionInfo;
 
   // If true, the hosting server is aborting.  Region close process is different
   // when we are aborting.
   private final boolean abort;
-  private ServerName destination;
+
+  /**
+   * The target the Region is going to after successful close; usually null as it depends on
+   * context whether this is passed in or not.
+   */
+  private final ServerName destination;
 
   /**
    * This method used internally by the RegionServer to close out regions.
-   * @param server
-   * @param rsServices
-   * @param regionInfo
+   *
    * @param abort If the regionserver is aborting.
-   * @param destination
    */
-  public CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices,
-      final RegionInfo regionInfo, final boolean abort,
-      ServerName destination) {
-    this(server, rsServices, regionInfo, abort,
-      EventType.M_RS_CLOSE_REGION, destination);
+  public CloseRegionHandler(final RegionServerServices rsServices, final RegionInfo regionInfo,
+    final boolean abort, @Nullable ServerName destination) {
+    this(rsServices, regionInfo, abort, EventType.M_RS_CLOSE_REGION, destination);
   }
 
-  protected CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices, RegionInfo regionInfo,
-      boolean abort, EventType eventType, ServerName destination) {
-    super(server, eventType);
-    this.server = server;
-    this.rsServices = rsServices;
+  protected CloseRegionHandler(RegionServerServices rsServices, RegionInfo regionInfo,
+      boolean abort, EventType eventType, @Nullable ServerName destination) {
+    super(rsServices, eventType);
     this.regionInfo = regionInfo;
     this.abort = abort;
+    this.rss = rsServices;
     this.destination = destination;
   }
 
-  public RegionInfo getRegionInfo() {
-    return regionInfo;
-  }
-
-  @Override
-  public void process() {
+  @Override public void process() throws IOException {
     try {
-      String name = regionInfo.getEncodedName();
-      LOG.trace("Processing close of {}", name);
-      String encodedRegionName = regionInfo.getEncodedName();
       // Check that this region is being served here
-      HRegion region = (HRegion)rsServices.getRegion(encodedRegionName);
+      HRegion region = (HRegion)this.rss.getRegion(this.regionInfo.getEncodedName());
       if (region == null) {
-        LOG.warn("Received CLOSE for region {} but currently not serving - ignoring", name);
+        LOG.warn("Received CLOSE for {} but not ONLINE; ignoring CLOSE request",
+          this.regionInfo.getRegionNameAsString());
         // TODO: do better than a simple warning
         return;
       }
-
-      // Close the region
-      try {
-        if (region.close(abort) == null) {
-          // This region got closed.  Most likely due to a split.
-          // The split message will clean up the master state.
-          LOG.warn("Can't close region {}, was already closed during close()", name);
-          return;
-        }
-      } catch (IOException ioe) {
-        // An IOException here indicates that we couldn't successfully flush the
-        // memstore before closing. So, we need to abort the server and allow
-        // the master to split our logs in order to recover the data.
-        server.abort("Unrecoverable exception while closing region " +
-          regionInfo.getRegionNameAsString() + ", still finishing close", ioe);
-        throw new RuntimeException(ioe);
+      if (region.getCoprocessorHost() != null) {
+        // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
+        // there are exceptions thrown from the CP, we can not report the error to the master, and
 
 Review comment:
   Sorry, i'm at reviewer fatigue at this point. Will be back.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-600437808
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 52s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  7s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m 23s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  branch-2 passed  |
   | -1 :x: |  shadedjars  |   0m 12s |  branch has 7 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in branch-2 failed.  |
   ||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   3m  0s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 42s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 42s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   0m 10s |  patch has 7 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -0 :warning: |  unit  |   0m 42s |  hbase-server in the patch failed.  |
   |  |   |  15m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1305 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 635f5a9131b3 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / ebd37a314c |
   | Default Java | 2020-01-14 |
   | shadedjars | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/branch-shadedjars.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | mvninstall | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/testReport/ |
   | Max. process+thread count | 94 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/console |
   | versions | git=2.17.1 maven=2018-06-17T18:33:14Z) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394687610
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
 
 Review comment:
   ah, I see that you've merely replicated the comment from below. I'm now more curious how this is actually handled.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-600911418
 
 
   adding Duo and Guanghau for reviews.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395126162
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1457,6 +1458,9 @@ private void waitOnAllRegionsToClose(final boolean abort) {
                 " because some regions failed closing");
           }
           break;
+        } else {
+          LOG.trace("Waiting on {}", this.regionsInTransitionInRS.entrySet().stream().
 
 Review comment:
   My preference is not about preserving the work "region", it's about telling the operator what we're waiting for these objects to do -- the "transition" part. How about "Waiting on transitions to complete: {}"?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395105776
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
 
 Review comment:
   Oh shit, can we avoid going back to the old world? The reason for this class is to get rid of the CloseRegionHandler...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394709869
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
   private final RegionInfo regionInfo;
 
   // If true, the hosting server is aborting.  Region close process is different
   // when we are aborting.
   private final boolean abort;
-  private ServerName destination;
+
+  /**
+   * The target the Region is going to after successful close; usually null as it depends on
+   * context whether this is passed in or not.
+   */
+  private final ServerName destination;
 
   /**
    * This method used internally by the RegionServer to close out regions.
-   * @param server
-   * @param rsServices
-   * @param regionInfo
+   *
    * @param abort If the regionserver is aborting.
-   * @param destination
    */
-  public CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices,
-      final RegionInfo regionInfo, final boolean abort,
-      ServerName destination) {
-    this(server, rsServices, regionInfo, abort,
-      EventType.M_RS_CLOSE_REGION, destination);
+  public CloseRegionHandler(final RegionServerServices rsServices, final RegionInfo regionInfo,
+    final boolean abort, @Nullable ServerName destination) {
+    this(rsServices, regionInfo, abort, EventType.M_RS_CLOSE_REGION, destination);
   }
 
-  protected CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices, RegionInfo regionInfo,
-      boolean abort, EventType eventType, ServerName destination) {
-    super(server, eventType);
-    this.server = server;
-    this.rsServices = rsServices;
+  protected CloseRegionHandler(RegionServerServices rsServices, RegionInfo regionInfo,
+      boolean abort, EventType eventType, @Nullable ServerName destination) {
+    super(rsServices, eventType);
     this.regionInfo = regionInfo;
     this.abort = abort;
+    this.rss = rsServices;
     this.destination = destination;
   }
 
-  public RegionInfo getRegionInfo() {
-    return regionInfo;
-  }
-
-  @Override
-  public void process() {
+  @Override public void process() throws IOException {
     try {
-      String name = regionInfo.getEncodedName();
-      LOG.trace("Processing close of {}", name);
-      String encodedRegionName = regionInfo.getEncodedName();
       // Check that this region is being served here
-      HRegion region = (HRegion)rsServices.getRegion(encodedRegionName);
+      HRegion region = (HRegion)this.rss.getRegion(this.regionInfo.getEncodedName());
       if (region == null) {
-        LOG.warn("Received CLOSE for region {} but currently not serving - ignoring", name);
+        LOG.warn("Received CLOSE for {} but not ONLINE; ignoring CLOSE request",
+          this.regionInfo.getRegionNameAsString());
         // TODO: do better than a simple warning
         return;
       }
-
-      // Close the region
-      try {
-        if (region.close(abort) == null) {
-          // This region got closed.  Most likely due to a split.
-          // The split message will clean up the master state.
-          LOG.warn("Can't close region {}, was already closed during close()", name);
-          return;
-        }
-      } catch (IOException ioe) {
-        // An IOException here indicates that we couldn't successfully flush the
-        // memstore before closing. So, we need to abort the server and allow
-        // the master to split our logs in order to recover the data.
-        server.abort("Unrecoverable exception while closing region " +
-          regionInfo.getRegionNameAsString() + ", still finishing close", ioe);
-        throw new RuntimeException(ioe);
+      if (region.getCoprocessorHost() != null) {
+        // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
+        // there are exceptions thrown from the CP, we can not report the error to the master, and
 
 Review comment:
   Thanks Nick

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394694740
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 ##########
 @@ -105,50 +102,54 @@ public void process() throws IOException {
     if (previous != null) {
       if (previous) {
         // The region is opening and this maybe a retry on the rpc call, it is safe to ignore it.
-        LOG.info("Receiving OPEN for the region:{}, which we are already trying to OPEN" +
-          " - ignoring this new request for this region.", regionName);
+        LOG.info("Receiving OPEN for {} which is OPENING; ignoring request", regionName);
       } else {
         // The region is closing. This is possible as we will update the region state to CLOSED when
         // calling reportRegionStateTransition, so the HMaster will think the region is offline,
         // before we actually close the region, as reportRegionStateTransition is part of the
         // closing process.
         long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "Receiving OPEN for the region:{}, which we are trying to close, try again after {}ms",
+        LOG.info("Receiving OPEN for {} which we are trying to CLOSE, try again after {}ms",
           regionName, backoff);
         rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
       }
       return;
     }
     LOG.info("Open {}", regionName);
     HRegion region;
+    // Start a try. Above we registered region with rs.getRegionsInTransitionInRS. Need to clear
+    // it on success or error on the end.
     try {
-      TableDescriptor htd =
-        tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
-      if (htd == null) {
-        throw new IOException("Missing table descriptor for " + regionName);
+      try {
+        TableDescriptor htd =
+          tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
+        if (htd == null) {
+          throw new IOException("Missing table descriptor for " + regionName);
+        }
+        // pass null for the last parameter, which used to be a CancelableProgressable, as now the
+        // opening can not be interrupted by a close request any more.
+        region = HRegion
+          .openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(), rs, null);
+      } catch (IOException e) {
+        cleanUpAndReportFailure(e);
+        return;
       }
-      // pass null for the last parameter, which used to be a CancelableProgressable, as now the
-      // opening can not be interrupted by a close request any more.
-      region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(),
-        rs, null);
-    } catch (IOException e) {
-      cleanUpAndReportFailure(e);
-      return;
-    }
-    rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
-    rs.addRegion(region);
-    LOG.info("Opened {}", regionName);
-    // Cache the open region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(openProcId);
-    Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
-    if (current == null) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was NOT in transition. Region={}",
-        regionName);
-    } else if (!current) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was closing. Region={}", regionName);
+      rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
+      rs.addRegion(region);
+      LOG.info("Opened {}", regionName);
+    } finally {
+      // Remove from regionsInTransitionRS whether success or failure.
+      rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
+      Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
+      if (current == null) {
+        // Should NEVER happen, but let's be paranoid.
 
 Review comment:
   this is a lie. there's nothing stopping multiple of these executing concurrently, is there? the only guarantee our data structures give us is that changes to the RIT map are atomic. or do these handlers operate under a region lock at a higher level? I haven't notice one during this review.
   
   Based on above thinking, this should be logged at DEBUG, not ERROR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394703158
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
 
 Review comment:
   Let me remove.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394707772
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
 
 Review comment:
   Let me fix.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395309123
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
   private final RegionInfo regionInfo;
 
   // If true, the hosting server is aborting.  Region close process is different
   // when we are aborting.
   private final boolean abort;
-  private ServerName destination;
+
+  /**
+   * The target the Region is going to after successful close; usually null as it depends on
+   * context whether this is passed in or not.
+   */
+  private final ServerName destination;
 
   /**
    * This method used internally by the RegionServer to close out regions.
-   * @param server
-   * @param rsServices
-   * @param regionInfo
+   *
    * @param abort If the regionserver is aborting.
-   * @param destination
    */
-  public CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices,
-      final RegionInfo regionInfo, final boolean abort,
-      ServerName destination) {
-    this(server, rsServices, regionInfo, abort,
-      EventType.M_RS_CLOSE_REGION, destination);
+  public CloseRegionHandler(final RegionServerServices rsServices, final RegionInfo regionInfo,
+    final boolean abort, @Nullable ServerName destination) {
+    this(rsServices, regionInfo, abort, EventType.M_RS_CLOSE_REGION, destination);
   }
 
-  protected CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices, RegionInfo regionInfo,
-      boolean abort, EventType eventType, ServerName destination) {
-    super(server, eventType);
-    this.server = server;
-    this.rsServices = rsServices;
+  protected CloseRegionHandler(RegionServerServices rsServices, RegionInfo regionInfo,
+      boolean abort, EventType eventType, @Nullable ServerName destination) {
+    super(rsServices, eventType);
     this.regionInfo = regionInfo;
     this.abort = abort;
+    this.rss = rsServices;
     this.destination = destination;
   }
 
-  public RegionInfo getRegionInfo() {
-    return regionInfo;
-  }
-
-  @Override
-  public void process() {
+  @Override public void process() throws IOException {
     try {
-      String name = regionInfo.getEncodedName();
-      LOG.trace("Processing close of {}", name);
-      String encodedRegionName = regionInfo.getEncodedName();
       // Check that this region is being served here
-      HRegion region = (HRegion)rsServices.getRegion(encodedRegionName);
+      HRegion region = (HRegion)this.rss.getRegion(this.regionInfo.getEncodedName());
       if (region == null) {
-        LOG.warn("Received CLOSE for region {} but currently not serving - ignoring", name);
+        LOG.warn("Received CLOSE for {} but not ONLINE; ignoring CLOSE request",
+          this.regionInfo.getRegionNameAsString());
         // TODO: do better than a simple warning
         return;
       }
-
-      // Close the region
-      try {
-        if (region.close(abort) == null) {
-          // This region got closed.  Most likely due to a split.
-          // The split message will clean up the master state.
-          LOG.warn("Can't close region {}, was already closed during close()", name);
-          return;
-        }
-      } catch (IOException ioe) {
-        // An IOException here indicates that we couldn't successfully flush the
-        // memstore before closing. So, we need to abort the server and allow
-        // the master to split our logs in order to recover the data.
-        server.abort("Unrecoverable exception while closing region " +
-          regionInfo.getRegionNameAsString() + ", still finishing close", ioe);
-        throw new RuntimeException(ioe);
+      if (region.getCoprocessorHost() != null) {
+        // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
+        // there are exceptions thrown from the CP, we can not report the error to the master, and
+        // if here is, we just return without calling reportRegionStateTransition, the TRSP at
+        // master side will hang there for ever. So here if the CP throws an exception out, the only
+        // way is to abort the RS...
+        region.getCoprocessorHost().preClose(abort);
       }
-
-      this.rsServices.removeRegion(region, destination);
-      rsServices.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-        HConstants.NO_SEQNUM, Procedure.NO_PROC_ID, -1, regionInfo));
-
+      if (region.close(abort) == null) {
+        // This should not happen. Left over from old days when RegionServer
+        // ran the splits? Now splits are run by the Master.
+        LOG.warn("Can't close {}, already closed; split? (Unexpected state - investigate)",
+          this.regionInfo.getRegionNameAsString());
+        // Return null here. Makes more sense. We didnt do close on the Region.
+        return;
+      }
+      this.rss.removeRegion(region, destination);
+      boolean b = reportRegionStateTransition();
       // Done!  Region is closed on this RS
-      LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString());
+      LOG.debug("Closed {} (notified master={})", this.regionInfo.getRegionNameAsString(), b);
+      // We don't care if we are able to inform the Master successfully; try but ignore response.
     } finally {
-      this.rsServices.getRegionsInTransitionInRS().
-        remove(this.regionInfo.getEncodedNameAsBytes(), Boolean.FALSE);
+      // Presumption is that before the Close was run, the CLOSE operation had been registered
+      // with rss.getRegionsInTransitionInRS(). Here we do cleanup whether success or failure.
+      this.rss.getRegionsInTransitionInRS().remove(this.regionInfo.getEncodedNameAsBytes(),
 
 Review comment:
   This is more involved dancing. Yeah, you'd think all Handlers would do similar but they don't. TODO.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394666540
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
 
 Review comment:
   nit spelling, "NotServingRegionException"

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395159069
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
+          this.destination) {
+        @Override boolean reportRegionStateTransition() throws IOException {
 
 Review comment:
   I find this structure of creating an instance of a sibling class and overriding this method very surprising. I don't have a better suggestion ATM.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394704188
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
         // The master deletes the znode when it receives this exception.
-        throw new NotServingRegionException("The region " + encodedName +
-          " was opening but not yet served. Opening is cancelled.");
+        throw new NotServingRegionException(encodedRegionName +
+          " was opening but not yet served; cancelled by close");
       }
     } else if (previous == null) {
-      LOG.info("Received CLOSE for {}", encodedName);
+      LOG.info("Received CLOSE for {}", encodedRegionName);
     } else if (Boolean.FALSE.equals(previous)) {
-      LOG.info("Received CLOSE for the region: " + encodedName +
-        ", which we are already trying to CLOSE, but not completed yet");
+      LOG.info("Received CLOSE for {} which we are already trying to CLOSE", encodedRegionName);
       return true;
     }
+    // Above we've registered our CLOSE with this.regionsInTransitionInRS. Now run close handler.
+    // The close handler will undo the entry in this.regionsInTransitionInRS when dones
+    // (awkward but close handler runs in a different thread).
+    this.executorService.submit(region.getRegionInfo().isMetaRegion()?
 
 Review comment:
   I think the opposite. When operator at EOL, forces reader to continue reading... (Got it from 'Effective Java') 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395149773
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
         // The master deletes the znode when it receives this exception.
-        throw new NotServingRegionException("The region " + encodedName +
-          " was opening but not yet served. Opening is cancelled.");
+        throw new NotServingRegionException(encodedRegionName +
+          " was opening but not yet served; cancelled by close");
       }
     } else if (previous == null) {
-      LOG.info("Received CLOSE for {}", encodedName);
+      LOG.info("Received CLOSE for {}", encodedRegionName);
     } else if (Boolean.FALSE.equals(previous)) {
-      LOG.info("Received CLOSE for the region: " + encodedName +
-        ", which we are already trying to CLOSE, but not completed yet");
+      LOG.info("Received CLOSE for {} which we are already trying to CLOSE", encodedRegionName);
       return true;
     }
+    // Above we've registered our CLOSE with this.regionsInTransitionInRS. Now run close handler.
+    // The close handler will undo the entry in this.regionsInTransitionInRS when dones
+    // (awkward but close handler runs in a different thread).
+    this.executorService.submit(region.getRegionInfo().isMetaRegion()?
 
 Review comment:
   🤔 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394706569
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
 
 Review comment:
   Yeah, I'm not trying to change general behavior; just handle holes where we can exit early w/o cleaning up state and undoing some of the duplication.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394665180
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
 
 Review comment:
   an astute observation. this lacking abstraction accounts for quite a lot of our pain within the server...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394665941
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
 
 Review comment:
   nit: since the logic is to immediately throw, instead of assigning this variable multiple times, can declare it `final` below and simply null-check the method invocation up here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394683876
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
 
 Review comment:
   general code-style comment: it's not common to me to see the `.` of a method call attached to the invocation target; rather i'm accustomed to it being attached to the method name. Thus,
   
   ```java
   this.regionsInTransitionInRS
     .putIfAbsent(...)
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394705145
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1457,6 +1458,9 @@ private void waitOnAllRegionsToClose(final boolean abort) {
                 " because some regions failed closing");
           }
           break;
+        } else {
+          LOG.trace("Waiting on {}", this.regionsInTransitionInRS.entrySet().stream().
 
 Review comment:
   Will do

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack closed pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack closed pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394705231
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -2446,6 +2450,12 @@ public RSRpcServices getRSRpcServices() {
    */
   @Override
   public void abort(String reason, Throwable cause) {
+    if (isAborted()) {
+      // Don't run multiple aborts; fills logs w/ spew.
+      LOG.info("Abort called but server already aborted; by-passing; abort invoked because {}",
 
 Review comment:
   Will do

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395152110
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 ##########
 @@ -105,50 +102,54 @@ public void process() throws IOException {
     if (previous != null) {
       if (previous) {
         // The region is opening and this maybe a retry on the rpc call, it is safe to ignore it.
-        LOG.info("Receiving OPEN for the region:{}, which we are already trying to OPEN" +
-          " - ignoring this new request for this region.", regionName);
+        LOG.info("Receiving OPEN for {} which is OPENING; ignoring request", regionName);
       } else {
         // The region is closing. This is possible as we will update the region state to CLOSED when
         // calling reportRegionStateTransition, so the HMaster will think the region is offline,
         // before we actually close the region, as reportRegionStateTransition is part of the
         // closing process.
         long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "Receiving OPEN for the region:{}, which we are trying to close, try again after {}ms",
+        LOG.info("Receiving OPEN for {} which we are trying to CLOSE, try again after {}ms",
           regionName, backoff);
         rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
       }
       return;
     }
     LOG.info("Open {}", regionName);
     HRegion region;
+    // Start a try. Above we registered region with rs.getRegionsInTransitionInRS. Need to clear
+    // it on success or error on the end.
     try {
-      TableDescriptor htd =
-        tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
-      if (htd == null) {
-        throw new IOException("Missing table descriptor for " + regionName);
+      try {
+        TableDescriptor htd =
+          tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
+        if (htd == null) {
+          throw new IOException("Missing table descriptor for " + regionName);
+        }
+        // pass null for the last parameter, which used to be a CancelableProgressable, as now the
+        // opening can not be interrupted by a close request any more.
+        region = HRegion
+          .openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(), rs, null);
+      } catch (IOException e) {
+        cleanUpAndReportFailure(e);
+        return;
       }
-      // pass null for the last parameter, which used to be a CancelableProgressable, as now the
-      // opening can not be interrupted by a close request any more.
-      region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(),
-        rs, null);
-    } catch (IOException e) {
-      cleanUpAndReportFailure(e);
-      return;
-    }
-    rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
-    rs.addRegion(region);
-    LOG.info("Opened {}", regionName);
-    // Cache the open region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(openProcId);
-    Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
-    if (current == null) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was NOT in transition. Region={}",
-        regionName);
-    } else if (!current) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was closing. Region={}", regionName);
+      rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
+      rs.addRegion(region);
+      LOG.info("Opened {}", regionName);
+    } finally {
+      // Remove from regionsInTransitionRS whether success or failure.
+      rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
+      Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
+      if (current == null) {
+        // Should NEVER happen, but let's be paranoid.
+        LOG.error("Bad state: we've just opened {} and it was NOT in transition", regionName);
+      } else if (!current) {
+        // Should NEVER happen, but let's be paranoid.
 
 Review comment:
   Can you enlighten me mighty @apache9? By my read, any number of these handlers could be running concurrently, even on the same region. There's no mutex around access to the associated region's entry in the map, all we have is the promise of concurrent hashmap that concurrent edits won't tear. Thus, there's no guarantee that some change we made previously should be visible currently.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395180676
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -38,9 +36,13 @@
 /**
  * Handles closing of a region on a region server.
  * <p/>
- * Now for regular close region request, we will use {@link UnassignRegionHandler} instead. But when
- * shutting down the region server, will also close regions and the related methods still use this
- * class so we keep it here.
+ * In normal operation, we use {@link UnassignRegionHandler} closing Regions but when shutting down
+ * the region server and closing out Regions, we use this handler instead; it does not expect to
 
 Review comment:
   CRH is not for tests only. If you want to remove it, more work needs to be done first. Meantime, lets land this patch to fix a bug around cluster shutdown revealed by tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394656618
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -2446,6 +2450,12 @@ public RSRpcServices getRSRpcServices() {
    */
   @Override
   public void abort(String reason, Throwable cause) {
+    if (isAborted()) {
+      // Don't run multiple aborts; fills logs w/ spew.
+      LOG.info("Abort called but server already aborted; by-passing; abort invoked because {}",
 
 Review comment:
   nit: simplify the message. an abort has already been logged, so an operator will know that's the case. Something like
   
   ```java
   LOG.info("Redundant abort requested. Reason: {}", reason, cause);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394697778
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
 
 Review comment:
   Why? `rsServices` was more descriptive.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394616103
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
         // The master deletes the znode when it receives this exception.
-        throw new NotServingRegionException("The region " + encodedName +
-          " was opening but not yet served. Opening is cancelled.");
+        throw new NotServingRegionException(encodedRegionName +
+          " was opening but not yet served; cancelled by close");
       }
     } else if (previous == null) {
-      LOG.info("Received CLOSE for {}", encodedName);
+      LOG.info("Received CLOSE for {}", encodedRegionName);
     } else if (Boolean.FALSE.equals(previous)) {
-      LOG.info("Received CLOSE for the region: " + encodedName +
-        ", which we are already trying to CLOSE, but not completed yet");
+      LOG.info("Received CLOSE for {} which we are already trying to CLOSE", encodedRegionName);
       return true;
     }
+    // Above we've registered our CLOSE with this.regionsInTransitionInRS. Now run close handler.
+    // The close handler will undo the entry in this.regionsInTransitionInRS when dones
+    // (awkward but close handler runs in a different thread).
+    this.executorService.submit(region.getRegionInfo().isMetaRegion()?
 
 Review comment:
   nit: I prefer my ternary operators to show at the beginning of the line instead of show up as question marks at the end of the line. That way they read a little more like and if-then-else instead of an interrogation of the reader. So,
   
   ```
   this.executorService.submit(region.getRegionInfo().isMetaRegion()
     ? new CloseMetaHandler(this, region.getRegionInfo(), abort, destination)
     : new CloseRegionHandler(this, region.getRegionInfo(), abort, destination));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394708224
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 ##########
 @@ -105,50 +102,54 @@ public void process() throws IOException {
     if (previous != null) {
       if (previous) {
         // The region is opening and this maybe a retry on the rpc call, it is safe to ignore it.
-        LOG.info("Receiving OPEN for the region:{}, which we are already trying to OPEN" +
-          " - ignoring this new request for this region.", regionName);
+        LOG.info("Receiving OPEN for {} which is OPENING; ignoring request", regionName);
       } else {
         // The region is closing. This is possible as we will update the region state to CLOSED when
         // calling reportRegionStateTransition, so the HMaster will think the region is offline,
         // before we actually close the region, as reportRegionStateTransition is part of the
         // closing process.
         long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "Receiving OPEN for the region:{}, which we are trying to close, try again after {}ms",
+        LOG.info("Receiving OPEN for {} which we are trying to CLOSE, try again after {}ms",
           regionName, backoff);
         rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
       }
       return;
     }
     LOG.info("Open {}", regionName);
     HRegion region;
+    // Start a try. Above we registered region with rs.getRegionsInTransitionInRS. Need to clear
+    // it on success or error on the end.
     try {
-      TableDescriptor htd =
-        tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
-      if (htd == null) {
-        throw new IOException("Missing table descriptor for " + regionName);
+      try {
+        TableDescriptor htd =
+          tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
+        if (htd == null) {
+          throw new IOException("Missing table descriptor for " + regionName);
+        }
+        // pass null for the last parameter, which used to be a CancelableProgressable, as now the
+        // opening can not be interrupted by a close request any more.
+        region = HRegion
+          .openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(), rs, null);
+      } catch (IOException e) {
+        cleanUpAndReportFailure(e);
+        return;
       }
-      // pass null for the last parameter, which used to be a CancelableProgressable, as now the
-      // opening can not be interrupted by a close request any more.
-      region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(),
-        rs, null);
-    } catch (IOException e) {
-      cleanUpAndReportFailure(e);
-      return;
-    }
-    rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
-    rs.addRegion(region);
-    LOG.info("Opened {}", regionName);
-    // Cache the open region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(openProcId);
-    Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
-    if (current == null) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was NOT in transition. Region={}",
-        regionName);
-    } else if (!current) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was closing. Region={}", regionName);
+      rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
+      rs.addRegion(region);
+      LOG.info("Opened {}", regionName);
+    } finally {
+      // Remove from regionsInTransitionRS whether success or failure.
+      rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
+      Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
+      if (current == null) {
+        // Should NEVER happen, but let's be paranoid.
 
 Review comment:
   Let me look

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-600809623
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  6s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   7m 11s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m 11s |  branch-2 passed  |
   | -1 :x: |  shadedjars  |   0m  9s |  branch has 7 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in branch-2 failed.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m 51s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 11s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 11s |  the patch passed  |
   | -1 :x: |  shadedjars  |   0m  9s |  patch has 7 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -0 :warning: |  unit  | 101m 41s |  hbase-server in the patch failed.  |
   |  |   | 123m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1305 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 5ef94792cc44 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 58a0ef0cfe |
   | Default Java | 2020-01-14 |
   | shadedjars | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/artifact/yetus-jdk11-hadoop3-check/output/branch-shadedjars.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | shadedjars | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/testReport/ |
   | Max. process+thread count | 4114 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/console |
   | versions | git=2.17.1 maven=2018-06-17T18:33:14Z) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394707505
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
         // The master deletes the znode when it receives this exception.
-        throw new NotServingRegionException("The region " + encodedName +
-          " was opening but not yet served. Opening is cancelled.");
+        throw new NotServingRegionException(encodedRegionName +
+          " was opening but not yet served; cancelled by close");
 
 Review comment:
   Let me feed in benefit of your read.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394693392
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 ##########
 @@ -105,50 +102,54 @@ public void process() throws IOException {
     if (previous != null) {
       if (previous) {
         // The region is opening and this maybe a retry on the rpc call, it is safe to ignore it.
-        LOG.info("Receiving OPEN for the region:{}, which we are already trying to OPEN" +
-          " - ignoring this new request for this region.", regionName);
+        LOG.info("Receiving OPEN for {} which is OPENING; ignoring request", regionName);
       } else {
         // The region is closing. This is possible as we will update the region state to CLOSED when
         // calling reportRegionStateTransition, so the HMaster will think the region is offline,
         // before we actually close the region, as reportRegionStateTransition is part of the
         // closing process.
         long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "Receiving OPEN for the region:{}, which we are trying to close, try again after {}ms",
+        LOG.info("Receiving OPEN for {} which we are trying to CLOSE, try again after {}ms",
           regionName, backoff);
         rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
       }
       return;
     }
     LOG.info("Open {}", regionName);
     HRegion region;
+    // Start a try. Above we registered region with rs.getRegionsInTransitionInRS. Need to clear
+    // it on success or error on the end.
     try {
-      TableDescriptor htd =
-        tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
-      if (htd == null) {
-        throw new IOException("Missing table descriptor for " + regionName);
+      try {
+        TableDescriptor htd =
+          tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
+        if (htd == null) {
+          throw new IOException("Missing table descriptor for " + regionName);
+        }
+        // pass null for the last parameter, which used to be a CancelableProgressable, as now the
+        // opening can not be interrupted by a close request any more.
+        region = HRegion
+          .openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(), rs, null);
+      } catch (IOException e) {
+        cleanUpAndReportFailure(e);
+        return;
       }
-      // pass null for the last parameter, which used to be a CancelableProgressable, as now the
-      // opening can not be interrupted by a close request any more.
-      region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(),
-        rs, null);
-    } catch (IOException e) {
-      cleanUpAndReportFailure(e);
-      return;
-    }
-    rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
-    rs.addRegion(region);
-    LOG.info("Opened {}", regionName);
-    // Cache the open region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(openProcId);
-    Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
-    if (current == null) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was NOT in transition. Region={}",
-        regionName);
-    } else if (!current) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was closing. Region={}", regionName);
+      rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
+      rs.addRegion(region);
+      LOG.info("Opened {}", regionName);
+    } finally {
+      // Remove from regionsInTransitionRS whether success or failure.
 
 Review comment:
   this is the crux of your test fix -- that we clean up after ourselves properly?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395101717
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -38,9 +36,13 @@
 /**
  * Handles closing of a region on a region server.
  * <p/>
- * Now for regular close region request, we will use {@link UnassignRegionHandler} instead. But when
- * shutting down the region server, will also close regions and the related methods still use this
- * class so we keep it here.
+ * In normal operation, we use {@link UnassignRegionHandler} closing Regions but when shutting down
+ * the region server and closing out Regions, we use this handler instead; it does not expect to
 
 Review comment:
   The work is done by me before. What I could say is that, we have done a lot of hacks only for testing, in CloseRegionHandler, just like we want to do in this patch...
   
   That's why I introduced a new class called UnassignRegionHandler for implementing the region assignment, and still kept the class here. The first idea was to just remove this class but I finally gave up

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394705871
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -2446,6 +2450,12 @@ public RSRpcServices getRSRpcServices() {
    */
   @Override
   public void abort(String reason, Throwable cause) {
+    if (isAborted()) {
+      // Don't run multiple aborts; fills logs w/ spew.
+      LOG.info("Abort called but server already aborted; by-passing; abort invoked because {}",
 
 Review comment:
   Critical figuring how the abort came about. A 'reason' that was distinctive enough would be cool but would take some work; I could make new issue.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395098646
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 ##########
 @@ -105,50 +102,54 @@ public void process() throws IOException {
     if (previous != null) {
       if (previous) {
         // The region is opening and this maybe a retry on the rpc call, it is safe to ignore it.
-        LOG.info("Receiving OPEN for the region:{}, which we are already trying to OPEN" +
-          " - ignoring this new request for this region.", regionName);
+        LOG.info("Receiving OPEN for {} which is OPENING; ignoring request", regionName);
       } else {
         // The region is closing. This is possible as we will update the region state to CLOSED when
         // calling reportRegionStateTransition, so the HMaster will think the region is offline,
         // before we actually close the region, as reportRegionStateTransition is part of the
         // closing process.
         long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "Receiving OPEN for the region:{}, which we are trying to close, try again after {}ms",
+        LOG.info("Receiving OPEN for {} which we are trying to CLOSE, try again after {}ms",
           regionName, backoff);
         rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
       }
       return;
     }
     LOG.info("Open {}", regionName);
     HRegion region;
+    // Start a try. Above we registered region with rs.getRegionsInTransitionInRS. Need to clear
+    // it on success or error on the end.
     try {
-      TableDescriptor htd =
-        tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
-      if (htd == null) {
-        throw new IOException("Missing table descriptor for " + regionName);
+      try {
+        TableDescriptor htd =
+          tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
+        if (htd == null) {
+          throw new IOException("Missing table descriptor for " + regionName);
+        }
+        // pass null for the last parameter, which used to be a CancelableProgressable, as now the
+        // opening can not be interrupted by a close request any more.
+        region = HRegion
+          .openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(), rs, null);
+      } catch (IOException e) {
+        cleanUpAndReportFailure(e);
+        return;
       }
-      // pass null for the last parameter, which used to be a CancelableProgressable, as now the
-      // opening can not be interrupted by a close request any more.
-      region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(),
-        rs, null);
-    } catch (IOException e) {
-      cleanUpAndReportFailure(e);
-      return;
-    }
-    rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
-    rs.addRegion(region);
-    LOG.info("Opened {}", regionName);
-    // Cache the open region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(openProcId);
-    Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
-    if (current == null) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was NOT in transition. Region={}",
-        regionName);
-    } else if (!current) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was closing. Region={}", regionName);
+      rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
+      rs.addRegion(region);
+      LOG.info("Opened {}", regionName);
+    } finally {
+      // Remove from regionsInTransitionRS whether success or failure.
+      rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
+      Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
+      if (current == null) {
+        // Should NEVER happen, but let's be paranoid.
+        LOG.error("Bad state: we've just opened {} and it was NOT in transition", regionName);
+      } else if (!current) {
+        // Should NEVER happen, but let's be paranoid.
 
 Review comment:
   This is not a lie, if this happens, we will be in trouble as there will be critical bugs in the code.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394706224
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
 
 Review comment:
   duh

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394655426
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1457,6 +1458,9 @@ private void waitOnAllRegionsToClose(final boolean abort) {
                 " because some regions failed closing");
           }
           break;
+        } else {
+          LOG.trace("Waiting on {}", this.regionsInTransitionInRS.entrySet().stream().
 
 Review comment:
   Use `regionsInTransitionInRS.keySet().stream()` directly, simplify slightly.
   
   ```java
   LOG.trace("Waiting on regions in transition: {}", this.regionsInTransitionInRS.keySet()
     .stream()
     .map(Bytes::toString)
     .collect(Collectors.joining(", ")));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394699249
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
   private final RegionInfo regionInfo;
 
   // If true, the hosting server is aborting.  Region close process is different
   // when we are aborting.
   private final boolean abort;
-  private ServerName destination;
+
+  /**
+   * The target the Region is going to after successful close; usually null as it depends on
+   * context whether this is passed in or not.
+   */
+  private final ServerName destination;
 
   /**
    * This method used internally by the RegionServer to close out regions.
-   * @param server
-   * @param rsServices
-   * @param regionInfo
+   *
    * @param abort If the regionserver is aborting.
-   * @param destination
    */
-  public CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices,
-      final RegionInfo regionInfo, final boolean abort,
-      ServerName destination) {
-    this(server, rsServices, regionInfo, abort,
-      EventType.M_RS_CLOSE_REGION, destination);
+  public CloseRegionHandler(final RegionServerServices rsServices, final RegionInfo regionInfo,
+    final boolean abort, @Nullable ServerName destination) {
+    this(rsServices, regionInfo, abort, EventType.M_RS_CLOSE_REGION, destination);
   }
 
-  protected CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices, RegionInfo regionInfo,
-      boolean abort, EventType eventType, ServerName destination) {
-    super(server, eventType);
-    this.server = server;
-    this.rsServices = rsServices;
+  protected CloseRegionHandler(RegionServerServices rsServices, RegionInfo regionInfo,
+      boolean abort, EventType eventType, @Nullable ServerName destination) {
+    super(rsServices, eventType);
     this.regionInfo = regionInfo;
     this.abort = abort;
+    this.rss = rsServices;
     this.destination = destination;
   }
 
-  public RegionInfo getRegionInfo() {
-    return regionInfo;
-  }
-
-  @Override
-  public void process() {
+  @Override public void process() throws IOException {
     try {
-      String name = regionInfo.getEncodedName();
-      LOG.trace("Processing close of {}", name);
-      String encodedRegionName = regionInfo.getEncodedName();
       // Check that this region is being served here
-      HRegion region = (HRegion)rsServices.getRegion(encodedRegionName);
+      HRegion region = (HRegion)this.rss.getRegion(this.regionInfo.getEncodedName());
       if (region == null) {
-        LOG.warn("Received CLOSE for region {} but currently not serving - ignoring", name);
+        LOG.warn("Received CLOSE for {} but not ONLINE; ignoring CLOSE request",
+          this.regionInfo.getRegionNameAsString());
         // TODO: do better than a simple warning
         return;
       }
-
-      // Close the region
-      try {
-        if (region.close(abort) == null) {
-          // This region got closed.  Most likely due to a split.
-          // The split message will clean up the master state.
-          LOG.warn("Can't close region {}, was already closed during close()", name);
-          return;
-        }
-      } catch (IOException ioe) {
-        // An IOException here indicates that we couldn't successfully flush the
-        // memstore before closing. So, we need to abort the server and allow
-        // the master to split our logs in order to recover the data.
-        server.abort("Unrecoverable exception while closing region " +
-          regionInfo.getRegionNameAsString() + ", still finishing close", ioe);
-        throw new RuntimeException(ioe);
+      if (region.getCoprocessorHost() != null) {
+        // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
+        // there are exceptions thrown from the CP, we can not report the error to the master, and
 
 Review comment:
   😱 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394653235
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1010,7 +1011,7 @@ public void run() {
           } else if (!this.stopping) {
             this.stopping = true;
             LOG.info("Closing user regions");
-            closeUserRegions(this.abortRequested);
+            closeUserRegions(isAborted());
 
 Review comment:
   On using a public accessor vs. the private field directly, I am generally ambivalent. However, this is probably an improvement because it makes the master dependent on an interface instead of an implementation detail. +1.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395154921
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
+          this.destination) {
+        @Override boolean reportRegionStateTransition() throws IOException {
+          boolean b = super.reportRegionStateTransition();
+          if (!b) {
+            String name = getRegionInfo().getRegionNameAsString();
+            throw new IOException("Failed report close of " + name + " to master");
+          }
+          return b;
+        }
+
+        @Override long getClosePID() {
+          return UnassignRegionHandler.this.closeProcId;
+        }
+      };
+      crh.process();
+    } finally {
+      // Cache the close region procedure id
+      rss.finishRegionProcedure(closeProcId);
     }
-    // Cache the close region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(closeProcId);
-    rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-    LOG.info("Closed {}", regionName);
   }
 
   @Override
   protected void handleException(Throwable t) {
-    LOG.warn("Fatal error occurred while closing region {}, aborting...", encodedName, t);
-    getServer().abort("Failed to close region " + encodedName + " and can not recover", t);
+    LOG.warn("Fatal error occurred while closing {}, aborting...", this.encodedRegionName, t);
+    getServer().abort("Failed to close " + this.encodedRegionName, t);
 
 Review comment:
   Its aborting so seems plain to me that 'it can not recover' but no problem, will add it back.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394709830
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
 
 Review comment:
   rss is used in a bunch of places. regionServerServices would be better than rsServices but rss is short.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395152374
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
 ##########
 @@ -31,10 +32,8 @@
 public class CloseMetaHandler extends CloseRegionHandler {
 
   // Called when regionserver determines its to go down; not master orchestrated
-  public CloseMetaHandler(final Server server,
 
 Review comment:
   _nod_

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-600752271
 
 
   Fix findbugs, checkstyle, and good find by hadoopcheck

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395158526
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
 
 Review comment:
   Can you say more @Apache9 ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394653780
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1081,18 +1082,18 @@ public void run() {
 
     // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
     if (rspmHost != null) {
-      rspmHost.stop(this.abortRequested || this.killed);
 
 Review comment:
   I wonder if one day we'll want a `Killable` interface as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394708003
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 ##########
 @@ -105,50 +102,54 @@ public void process() throws IOException {
     if (previous != null) {
       if (previous) {
         // The region is opening and this maybe a retry on the rpc call, it is safe to ignore it.
-        LOG.info("Receiving OPEN for the region:{}, which we are already trying to OPEN" +
-          " - ignoring this new request for this region.", regionName);
+        LOG.info("Receiving OPEN for {} which is OPENING; ignoring request", regionName);
       } else {
         // The region is closing. This is possible as we will update the region state to CLOSED when
         // calling reportRegionStateTransition, so the HMaster will think the region is offline,
         // before we actually close the region, as reportRegionStateTransition is part of the
         // closing process.
         long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "Receiving OPEN for the region:{}, which we are trying to close, try again after {}ms",
+        LOG.info("Receiving OPEN for {} which we are trying to CLOSE, try again after {}ms",
           regionName, backoff);
         rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
       }
       return;
     }
     LOG.info("Open {}", regionName);
     HRegion region;
+    // Start a try. Above we registered region with rs.getRegionsInTransitionInRS. Need to clear
+    // it on success or error on the end.
     try {
-      TableDescriptor htd =
-        tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
-      if (htd == null) {
-        throw new IOException("Missing table descriptor for " + regionName);
+      try {
+        TableDescriptor htd =
+          tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
+        if (htd == null) {
+          throw new IOException("Missing table descriptor for " + regionName);
+        }
+        // pass null for the last parameter, which used to be a CancelableProgressable, as now the
+        // opening can not be interrupted by a close request any more.
+        region = HRegion
+          .openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(), rs, null);
+      } catch (IOException e) {
+        cleanUpAndReportFailure(e);
+        return;
       }
-      // pass null for the last parameter, which used to be a CancelableProgressable, as now the
-      // opening can not be interrupted by a close request any more.
-      region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(),
-        rs, null);
-    } catch (IOException e) {
-      cleanUpAndReportFailure(e);
-      return;
-    }
-    rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
-    rs.addRegion(region);
-    LOG.info("Opened {}", regionName);
-    // Cache the open region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(openProcId);
-    Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
-    if (current == null) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was NOT in transition. Region={}",
-        regionName);
-    } else if (!current) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was closing. Region={}", regionName);
+      rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
+      rs.addRegion(region);
+      LOG.info("Opened {}", regionName);
+    } finally {
+      // Remove from regionsInTransitionRS whether success or failure.
 
 Review comment:
   y

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394654963
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1457,6 +1458,9 @@ private void waitOnAllRegionsToClose(final boolean abort) {
                 " because some regions failed closing");
           }
           break;
+        } else {
+          LOG.trace("Waiting on {}", this.regionsInTransitionInRS.entrySet().stream().
 
 Review comment:
   nit: "Waiting on regions in transition: {}".

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-600813761
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 55s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  5s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m 43s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   5m 21s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m  7s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 54s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 102m 59s |  hbase-server in the patch passed.  |
   |  |   | 133m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/artifact/yetus-jdk8-hadoop2-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1305 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux cb1876241720 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 58a0ef0cfe |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/testReport/ |
   | Max. process+thread count | 3896 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/console |
   | versions | git=2.17.1 maven=2018-06-17T18:33:14Z) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394645601
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -22,6 +22,7 @@
 import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
 import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_MAX_SPLITTER;
 import static org.apache.hadoop.hbase.util.DNS.RS_HOSTNAME_KEY;
+import edu.umd.cs.findbugs.annotations.Nullable;
 
 Review comment:
   Do we actually make use of findbugs annotations? I haven't generally seen this for anything other than ignores.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395161763
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
+          this.destination) {
+        @Override boolean reportRegionStateTransition() throws IOException {
+          boolean b = super.reportRegionStateTransition();
+          if (!b) {
+            String name = getRegionInfo().getRegionNameAsString();
+            throw new IOException("Failed report close of " + name + " to master");
+          }
+          return b;
+        }
+
+        @Override long getClosePID() {
+          return UnassignRegionHandler.this.closeProcId;
+        }
+      };
+      crh.process();
+    } finally {
+      // Cache the close region procedure id
+      rss.finishRegionProcedure(closeProcId);
 
 Review comment:
   This is the old URH code: https://github.com/saintstack/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
   
   Any exception thrown calling the CP preclose or running the region close would exit the URH#process, get caught by the executor run method and then fed to the URH#handleException which would abort the RS. This refactor does not change this behavior or add to it; it just adds the cleanup of getRegionsInTransitionInRS references on exception -- of less use -- but on success too.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394689781
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
         // The master deletes the znode when it receives this exception.
-        throw new NotServingRegionException("The region " + encodedName +
-          " was opening but not yet served. Opening is cancelled.");
+        throw new NotServingRegionException(encodedRegionName +
+          " was opening but not yet served; cancelled by close");
 
 Review comment:
   this is also a lie: it wasn't "opening", and from my read of things, the action hasn't been "canceled." All we can say is what i mention on the above comment -- that this call of the CLOSE request for {region} could not be processed because it is not open. any existing state from an existing transition request remains intact.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-600444955
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 51s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   8m  4s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 28s |  branch-2 passed  |
   | +1 :green_heart: |  spotbugs  |   2m 38s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   7m 20s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 26s |  hbase-server: The patch generated 3 new + 67 unchanged - 5 fixed = 70 total (was 72)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | -1 :x: |  hadoopcheck  |  15m  9s |  The patch causes 20 errors with Hadoop v3.1.2.  |
   | -1 :x: |  spotbugs  |   2m 59s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 13s |  The patch does not generate ASF License warnings.  |
   |  |   |  43m  5s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 137] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1305 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux ab394a659015 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / ebd37a314c |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | hadoopcheck | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-general-check/output/patch-javac-3.1.2.txt |
   | spotbugs | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | Max. process+thread count | 83 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/console |
   | versions | git=2.17.1 maven=2018-06-17T18:33:14Z) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394691042
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
         // The master deletes the znode when it receives this exception.
-        throw new NotServingRegionException("The region " + encodedName +
-          " was opening but not yet served. Opening is cancelled.");
+        throw new NotServingRegionException(encodedRegionName +
+          " was opening but not yet served; cancelled by close");
       }
     } else if (previous == null) {
-      LOG.info("Received CLOSE for {}", encodedName);
+      LOG.info("Received CLOSE for {}", encodedRegionName);
     } else if (Boolean.FALSE.equals(previous)) {
-      LOG.info("Received CLOSE for the region: " + encodedName +
-        ", which we are already trying to CLOSE, but not completed yet");
+      LOG.info("Received CLOSE for {} which we are already trying to CLOSE", encodedRegionName);
       return true;
     }
+    // Above we've registered our CLOSE with this.regionsInTransitionInRS. Now run close handler.
+    // The close handler will undo the entry in this.regionsInTransitionInRS when dones
 
 Review comment:
   s/dones/finished/

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394695878
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
 ##########
 @@ -18,7 +18,8 @@
  */
 package org.apache.hadoop.hbase.regionserver.handler;
 
-import org.apache.hadoop.hbase.Server;
+import edu.umd.cs.findbugs.annotations.Nullable;
 
 Review comment:
   same question re `@Nullable`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394690833
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
 
 Review comment:
   This method doesn't actually do a close, it merely attempts to schedule the close request for background processing. Thus, "@return {@code true} when the close request is successfully scheduled or the close request is already scheduled, {@code false} otherwise."

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395261278
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1010,7 +1011,7 @@ public void run() {
           } else if (!this.stopping) {
             this.stopping = true;
             LOG.info("Closing user regions");
-            closeUserRegions(this.abortRequested);
+            closeUserRegions(isAborted());
 
 Review comment:
   Yeah, was thinking more the latter. Its probably public for tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395178485
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
 
 Review comment:
   Close of a Region on RS can happen in two main ways. One is via Procedure sent over by the Master.  It will ask RS run unassign and RS will queue a URH to run the close. The other way that a RS can close a Region is when it notices cluster shutdown. In this case the RS closes Regions itself; the close is NOT run via the Master. In both cases we have executors run bulk of close. For the latter case, we have CRH to pass to the executor. URH and CRH overlap. They do effectively the same thing only URH was missing a piece such that if a shutdown and a URH was run concurrently, it'd prevent RS going down. Ditto for ARH.
   
   If you want to get rid of CRH, thats fine, but more work is needed first.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-600774440
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 29s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m  7s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 17s |  branch-2 passed  |
   | +1 :green_heart: |  spotbugs  |   2m  8s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 42s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 16s |  hbase-server: The patch generated 0 new + 67 unchanged - 5 fixed = 67 total (was 72)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  17m  7s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  spotbugs  |   2m 17s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 11s |  The patch does not generate ASF License warnings.  |
   |  |   |  45m  3s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1305 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 1682208a7e1c 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 58a0ef0cfe |
   | Max. process+thread count | 83 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/2/console |
   | versions | git=2.17.1 maven=2018-06-17T18:33:14Z) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395261741
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -2446,6 +2450,12 @@ public RSRpcServices getRSRpcServices() {
    */
   @Override
   public void abort(String reason, Throwable cause) {
+    if (isAborted()) {
+      // Don't run multiple aborts; fills logs w/ spew.
+      LOG.info("Abort called but server already aborted; by-passing; abort invoked because {}",
 
 Review comment:
   This is better actually

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394704588
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1081,18 +1082,18 @@ public void run() {
 
     // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
     if (rspmHost != null) {
-      rspmHost.stop(this.abortRequested || this.killed);
 
 Review comment:
   We have kill in HBaseTestingUtility... And RS fakes 'killed'...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395260357
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
         // The master deletes the znode when it receives this exception.
-        throw new NotServingRegionException("The region " + encodedName +
-          " was opening but not yet served. Opening is cancelled.");
+        throw new NotServingRegionException(encodedRegionName +
+          " was opening but not yet served; cancelled by close");
       }
     } else if (previous == null) {
-      LOG.info("Received CLOSE for {}", encodedName);
+      LOG.info("Received CLOSE for {}", encodedRegionName);
     } else if (Boolean.FALSE.equals(previous)) {
-      LOG.info("Received CLOSE for the region: " + encodedName +
-        ", which we are already trying to CLOSE, but not completed yet");
+      LOG.info("Received CLOSE for {} which we are already trying to CLOSE", encodedRegionName);
       return true;
     }
+    // Above we've registered our CLOSE with this.regionsInTransitionInRS. Now run close handler.
+    // The close handler will undo the entry in this.regionsInTransitionInRS when dones
+    // (awkward but close handler runs in a different thread).
+    this.executorService.submit(region.getRegionInfo().isMetaRegion()?
 
 Review comment:
   I think intellij likes your way, not mine.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394652253
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -247,9 +249,8 @@
   public static boolean TEST_SKIP_REPORTING_TRANSITION = false;
 
   /**
-   * A map from RegionName to current action in progress. Boolean value indicates:
-   * true - if open region action in progress
-   * false - if close region action in progress
+   * A map from encoded region name as bytes to current action in progress. Boolean value indicates:
+   * TRUE if open region action in progress, and FALSE if close region action in progress
 
 Review comment:
   I wish this was a map to an enum, or something like `RegionState.State`, instead of just this boolean flag.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394706189
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
 
 Review comment:
   k
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394704294
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -247,9 +249,8 @@
   public static boolean TEST_SKIP_REPORTING_TRANSITION = false;
 
   /**
-   * A map from RegionName to current action in progress. Boolean value indicates:
-   * true - if open region action in progress
-   * false - if close region action in progress
+   * A map from encoded region name as bytes to current action in progress. Boolean value indicates:
+   * TRUE if open region action in progress, and FALSE if close region action in progress
 
 Review comment:
   nod

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395179414
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
 
 Review comment:
   CRH is not for tests only.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395108931
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
+          this.destination) {
+        @Override boolean reportRegionStateTransition() throws IOException {
+          boolean b = super.reportRegionStateTransition();
+          if (!b) {
+            String name = getRegionInfo().getRegionNameAsString();
+            throw new IOException("Failed report close of " + name + " to master");
+          }
+          return b;
+        }
+
+        @Override long getClosePID() {
+          return UnassignRegionHandler.this.closeProcId;
+        }
+      };
+      crh.process();
+    } finally {
+      // Cache the close region procedure id
+      rss.finishRegionProcedure(closeProcId);
     }
-    // Cache the close region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(closeProcId);
-    rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-    LOG.info("Closed {}", regionName);
   }
 
   @Override
   protected void handleException(Throwable t) {
-    LOG.warn("Fatal error occurred while closing region {}, aborting...", encodedName, t);
-    getServer().abort("Failed to close region " + encodedName + " and can not recover", t);
+    LOG.warn("Fatal error occurred while closing {}, aborting...", this.encodedRegionName, t);
+    getServer().abort("Failed to close " + this.encodedRegionName, t);
 
 Review comment:
   Please add the 'can not recover' back? It clearly tells why we must abort the region server.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395108150
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
+        if (previous) {
+          // This could happen as we will update the region state to OPEN when calling
+          // reportRegionStateTransition, so the HMaster will think the region is online, before
+          // we actually open the region, as reportRegionStateTransition is part of the opening
+          // process.
+          long backoff = this.retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Received CLOSE for {} which is being OPENED; try again after {}ms",
+            encodedRegionNameAsBytes, backoff);
+          rss.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
+        } else {
+          LOG.info("Received CLOSE for {}, which is CLOSING  but not complete yet",
+            encodedRegionNameAsBytes);
+        }
+        return;
       }
-      return;
-    }
-    HRegion region = rs.getRegion(encodedName);
-    if (region == null) {
-      LOG.debug(
-        "Received CLOSE for a region {} which is not online, and we're not opening/closing.",
-        encodedName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    String regionName = region.getRegionInfo().getEncodedName();
-    LOG.info("Close {}", regionName);
-    if (region.getCoprocessorHost() != null) {
-      // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
-      // there are exception thrown from the CP, we can not report the error to master, and if here
-      // we just return without calling reportRegionStateTransition, the TRSP at master side will
-      // hang there for ever. So here if the CP throws an exception out, the only way is to abort
-      // the RS...
-      region.getCoprocessorHost().preClose(abort);
-    }
-    if (region.close(abort) == null) {
-      // XXX: Is this still possible? The old comment says about split, but now split is done at
-      // master side, so...
-      LOG.warn("Can't close region {}, was already closed during close()", regionName);
-      rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
-      return;
-    }
-    rs.removeRegion(region, destination);
-    if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
-      throw new IOException("Failed to report close to master: " + regionName);
+      HRegion region = rss.getRegion(this.encodedRegionName);
+      // Can't unassign a Region that is not online.
+      if (region == null) {
+        LOG.debug("Received CLOSE for {} which is not online, and we're not opening/closing.",
+          this.encodedRegionName);
+        // Remove what we added above.
+        rss.getRegionsInTransitionInRS().remove(encodedRegionNameAsBytes, Boolean.FALSE);
+        return;
+      }
+      CloseRegionHandler crh = new CloseRegionHandler(rss, region.getRegionInfo(), this.abort,
+          this.destination) {
+        @Override boolean reportRegionStateTransition() throws IOException {
+          boolean b = super.reportRegionStateTransition();
+          if (!b) {
+            String name = getRegionInfo().getRegionNameAsString();
+            throw new IOException("Failed report close of " + name + " to master");
+          }
+          return b;
+        }
+
+        @Override long getClosePID() {
+          return UnassignRegionHandler.this.closeProcId;
+        }
+      };
+      crh.process();
+    } finally {
+      // Cache the close region procedure id
+      rss.finishRegionProcedure(closeProcId);
 
 Review comment:
   I do not like this, the design here is to abort the rs if there are exception thrown. In general, if the code is only for making UT pass, then we should change the UT...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395153280
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -38,9 +36,13 @@
 /**
  * Handles closing of a region on a region server.
  * <p/>
- * Now for regular close region request, we will use {@link UnassignRegionHandler} instead. But when
- * shutting down the region server, will also close regions and the related methods still use this
- * class so we keep it here.
+ * In normal operation, we use {@link UnassignRegionHandler} closing Regions but when shutting down
+ * the region server and closing out Regions, we use this handler instead; it does not expect to
 
 Review comment:
   Oof. That's rough @Apache9 .

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395260093
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
 
 Review comment:
   The @Nullable was in URH and ARH on this parameter so I just continued its use trying not to change too much. See https://github.com/saintstack/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395261579
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1457,6 +1458,9 @@ private void waitOnAllRegionsToClose(final boolean abort) {
                 " because some regions failed closing");
           }
           break;
+        } else {
+          LOG.trace("Waiting on {}", this.regionsInTransitionInRS.entrySet().stream().
 
 Review comment:
   Sure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395260533
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -22,6 +22,7 @@
 import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
 import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_MAX_SPLITTER;
 import static org.apache.hadoop.hbase.util.DNS.RS_HOSTNAME_KEY;
+import edu.umd.cs.findbugs.annotations.Nullable;
 
 Review comment:
   See above. I could purge in future.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394685649
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
 
 Review comment:
   Not your code, but javadoc is annoyingly vague as to why a call to `replace` might fail. Seems dangerous that we recurse in a non-tail-call position.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394690871
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
 
 Review comment:
   good.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395153759
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
 
 Review comment:
   I've some around to prefer longer and more verbose over short and terse. But that's just me.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395132768
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
 
 Review comment:
   I'm not against using it -- anything that enables more static analysis is better, IMHO. I just have't seen it done yet and suspect this was an error rather than intentional. If i'm wrong, please tell me to shove off ;)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394707871
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
         // The master deletes the znode when it receives this exception.
-        throw new NotServingRegionException("The region " + encodedName +
-          " was opening but not yet served. Opening is cancelled.");
+        throw new NotServingRegionException(encodedRegionName +
+          " was opening but not yet served; cancelled by close");
       }
     } else if (previous == null) {
-      LOG.info("Received CLOSE for {}", encodedName);
+      LOG.info("Received CLOSE for {}", encodedRegionName);
     } else if (Boolean.FALSE.equals(previous)) {
-      LOG.info("Received CLOSE for the region: " + encodedName +
-        ", which we are already trying to CLOSE, but not completed yet");
+      LOG.info("Received CLOSE for {} which we are already trying to CLOSE", encodedRegionName);
       return true;
     }
+    // Above we've registered our CLOSE with this.regionsInTransitionInRS. Now run close handler.
+    // The close handler will undo the entry in this.regionsInTransitionInRS when dones
 
 Review comment:
   k

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395131922
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -2446,6 +2450,12 @@ public RSRpcServices getRSRpcServices() {
    */
   @Override
   public void abort(String reason, Throwable cause) {
+    if (isAborted()) {
+      // Don't run multiple aborts; fills logs w/ spew.
+      LOG.info("Abort called but server already aborted; by-passing; abort invoked because {}",
 
 Review comment:
   _nod_

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394697608
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -38,9 +36,13 @@
 /**
  * Handles closing of a region on a region server.
  * <p/>
- * Now for regular close region request, we will use {@link UnassignRegionHandler} instead. But when
- * shutting down the region server, will also close regions and the related methods still use this
- * class so we keep it here.
+ * In normal operation, we use {@link UnassignRegionHandler} closing Regions but when shutting down
+ * the region server and closing out Regions, we use this handler instead; it does not expect to
 
 Review comment:
   It's really a shame if the only difference between `UnassignRegionHandler` and `CloseRegionHandler` is communication back to the master. should consolidate them if we can.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394611853
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
 
 Review comment:
   Do we actually use this findbugs `@Nullable` annotation in the project? I don't think we do. I know IntelliJ complains but if you look closely, that's it's own internal annotation, not this one.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394707303
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
 
 Review comment:
   I don't want to disturb what was there

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394707055
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
 
 Review comment:
   It seems to be a vestige of an old behavior. Work needs to be done to figure if it can be safely removed. Let me make it more explicit.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394687434
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
+    }
+    byte [] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+    // This getting from this.regionsInTransitionInRS and checking the return is done in a few
+    // places, mostly over in Handlers; this.regionsInTransitionInRS is how we ensure we don't
+    // clash opens/closes or duplicate closes when one is ongoing. FYI.
+    final Boolean previous = this.regionsInTransitionInRS.
+      putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
     if (Boolean.TRUE.equals(previous)) {
-      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
-          "trying to OPEN. Cancelling OPENING.");
-      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
+      LOG.info("Received CLOSE for {} which is OPENING. Cancelling OPENING.", encodedRegionName);
+      if (!this.regionsInTransitionInRS.replace(encodedRegionNameAsBytes, previous,
+          Boolean.FALSE)) {
         // The replace failed. That should be an exceptional case, but theoretically it can happen.
-        // We're going to try to do a standard close then.
-        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
-            " Doing a standard close now");
-        return closeRegion(encodedName, abort, sn);
+        // We're going to try to rerun the close.
+        LOG.warn("The opening of {} was done before we could cancel it; running a new close now",
+          encodedRegionName);
+        return closeRegion(encodedRegionName, abort, destination);
       }
       // Let's get the region from the online region list again
-      actualRegion = this.getRegion(encodedName);
-      if (actualRegion == null) { // If already online, we still need to close it.
-        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
+      region = this.getRegion(encodedRegionName);
+      if (region == null) {
+        LOG.info("The opening of {} has been cancelled by a CLOSE request", encodedRegionName);
 
 Review comment:
   also not your code, but "the opening of ...", I think this is an incorrect assumption. We don't know that the region opening was canceled.. could be a failed open, for example. All we know is that the region is not present in the list of online regions at this time.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394692563
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 ##########
 @@ -105,50 +102,54 @@ public void process() throws IOException {
     if (previous != null) {
 
 Review comment:
   nit: not your code, but i liked better the three-way if-elseif-else structure from `HRegionServer#closeRegion`. Makes it nice and obvious that there are three states possible from the return value. It would also be nice to have a log message in all three cases, rather than silently doing nothing for the `previous == null` case.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395158061
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 ##########
 @@ -76,75 +79,75 @@ private HRegionServer getServer() {
 
   @Override
   public void process() throws IOException {
-    HRegionServer rs = getServer();
-    byte[] encodedNameBytes = Bytes.toBytes(encodedName);
-    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.FALSE);
-    if (previous != null) {
-      if (previous) {
-        // This could happen as we will update the region state to OPEN when calling
-        // reportRegionStateTransition, so the HMaster will think the region is online, before we
-        // actually open the region, as reportRegionStateTransition is part of the opening process.
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.warn("Received CLOSE for the region: {}, which we are already " +
-          "trying to OPEN. try again after {}ms", encodedName, backoff);
-        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
-      } else {
-        LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
-          " but not completed yet", encodedName);
+    HRegionServer rss = getServer();
+    try {
+      byte[] encodedRegionNameAsBytes = Bytes.toBytes(this.encodedRegionName);
+      Boolean previous = rss.getRegionsInTransitionInRS().
+        putIfAbsent(encodedRegionNameAsBytes, Boolean.FALSE);
+      if (previous != null) {
 
 Review comment:
   nit: same wished about if-elseif-else structure vs. these nested ifs.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395155123
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
   private final RegionInfo regionInfo;
 
   // If true, the hosting server is aborting.  Region close process is different
   // when we are aborting.
   private final boolean abort;
-  private ServerName destination;
+
+  /**
+   * The target the Region is going to after successful close; usually null as it depends on
+   * context whether this is passed in or not.
+   */
+  private final ServerName destination;
 
   /**
    * This method used internally by the RegionServer to close out regions.
-   * @param server
-   * @param rsServices
-   * @param regionInfo
+   *
    * @param abort If the regionserver is aborting.
-   * @param destination
    */
-  public CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices,
-      final RegionInfo regionInfo, final boolean abort,
-      ServerName destination) {
-    this(server, rsServices, regionInfo, abort,
-      EventType.M_RS_CLOSE_REGION, destination);
+  public CloseRegionHandler(final RegionServerServices rsServices, final RegionInfo regionInfo,
+    final boolean abort, @Nullable ServerName destination) {
+    this(rsServices, regionInfo, abort, EventType.M_RS_CLOSE_REGION, destination);
   }
 
-  protected CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices, RegionInfo regionInfo,
-      boolean abort, EventType eventType, ServerName destination) {
-    super(server, eventType);
-    this.server = server;
-    this.rsServices = rsServices;
+  protected CloseRegionHandler(RegionServerServices rsServices, RegionInfo regionInfo,
+      boolean abort, EventType eventType, @Nullable ServerName destination) {
+    super(rsServices, eventType);
     this.regionInfo = regionInfo;
     this.abort = abort;
+    this.rss = rsServices;
     this.destination = destination;
   }
 
-  public RegionInfo getRegionInfo() {
-    return regionInfo;
-  }
-
-  @Override
-  public void process() {
+  @Override public void process() throws IOException {
     try {
-      String name = regionInfo.getEncodedName();
-      LOG.trace("Processing close of {}", name);
-      String encodedRegionName = regionInfo.getEncodedName();
       // Check that this region is being served here
-      HRegion region = (HRegion)rsServices.getRegion(encodedRegionName);
+      HRegion region = (HRegion)this.rss.getRegion(this.regionInfo.getEncodedName());
       if (region == null) {
-        LOG.warn("Received CLOSE for region {} but currently not serving - ignoring", name);
+        LOG.warn("Received CLOSE for {} but not ONLINE; ignoring CLOSE request",
+          this.regionInfo.getRegionNameAsString());
         // TODO: do better than a simple warning
         return;
       }
-
-      // Close the region
-      try {
-        if (region.close(abort) == null) {
-          // This region got closed.  Most likely due to a split.
-          // The split message will clean up the master state.
-          LOG.warn("Can't close region {}, was already closed during close()", name);
-          return;
-        }
-      } catch (IOException ioe) {
-        // An IOException here indicates that we couldn't successfully flush the
-        // memstore before closing. So, we need to abort the server and allow
-        // the master to split our logs in order to recover the data.
-        server.abort("Unrecoverable exception while closing region " +
-          regionInfo.getRegionNameAsString() + ", still finishing close", ioe);
-        throw new RuntimeException(ioe);
+      if (region.getCoprocessorHost() != null) {
+        // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
+        // there are exceptions thrown from the CP, we can not report the error to the master, and
+        // if here is, we just return without calling reportRegionStateTransition, the TRSP at
+        // master side will hang there for ever. So here if the CP throws an exception out, the only
+        // way is to abort the RS...
+        region.getCoprocessorHost().preClose(abort);
       }
-
-      this.rsServices.removeRegion(region, destination);
-      rsServices.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-        HConstants.NO_SEQNUM, Procedure.NO_PROC_ID, -1, regionInfo));
-
+      if (region.close(abort) == null) {
+        // This should not happen. Left over from old days when RegionServer
+        // ran the splits? Now splits are run by the Master.
+        LOG.warn("Can't close {}, already closed; split? (Unexpected state - investigate)",
+          this.regionInfo.getRegionNameAsString());
+        // Return null here. Makes more sense. We didnt do close on the Region.
+        return;
+      }
+      this.rss.removeRegion(region, destination);
+      boolean b = reportRegionStateTransition();
       // Done!  Region is closed on this RS
-      LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString());
+      LOG.debug("Closed {} (notified master={})", this.regionInfo.getRegionNameAsString(), b);
+      // We don't care if we are able to inform the Master successfully; try but ignore response.
     } finally {
-      this.rsServices.getRegionsInTransitionInRS().
-        remove(this.regionInfo.getEncodedNameAsBytes(), Boolean.FALSE);
+      // Presumption is that before the Close was run, the CLOSE operation had been registered
+      // with rss.getRegionsInTransitionInRS(). Here we do cleanup whether success or failure.
+      this.rss.getRegionsInTransitionInRS().remove(this.regionInfo.getEncodedNameAsBytes(),
 
 Review comment:
   We need to do the same dance around `remove` failure that's done elsewhere?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394681475
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -3185,71 +3195,61 @@ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
    *   If a close was in progress, this new request will be ignored, and an exception thrown.
    * </p>
    *
-   * @param encodedName Region to close
    * @param abort True if we are aborting
+   * @param destination Where the Region is being moved too... maybe null if unknown.
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    */
-  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
-      throws NotServingRegionException {
-    //Check for permissions to close.
-    HRegion actualRegion = this.getRegion(encodedName);
-    // Can be null if we're calling close on a region that's not online
-    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
-      try {
-        actualRegion.getCoprocessorHost().preClose(false);
-      } catch (IOException exp) {
-        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
-        return false;
-      }
-    }
-
-    // previous can come back 'null' if not in map.
-    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
-        Boolean.FALSE);
-
+  protected boolean closeRegion(String encodedRegionName, final boolean abort,
+      @Nullable final ServerName destination) throws NotServingRegionException {
+    // TODO: Check for perms to close.
+    // Ideally we'd shove a bunch of the below logic into CloseRegionHandler only we have to do
+    // some handling inline w/ the request such as throwing NotServervingRegionException if we
+    // are not hosting so client gets the message directly; can't do this from inside an
+    // executor service.
+    Region region = getRegion(encodedRegionName);
+    if (region == null) {
+      // Changed behavior. We'd proceed though Region not online.
+      // The master deletes the znode when it receives this exception.
+      throw new NotServingRegionException(encodedRegionName);
 
 Review comment:
   I don't see this exception handled in `RSProcedureDispatcher`. My read of `RSProcedureDispatcher$ExecuteProceduresRemoteCall#{run,scheduleForRetry}` is that when this exception is thrown, the master will simply retrying until the limit is exhausted.
   
   Should I be looking elsewhere?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r395157310
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -52,85 +54,101 @@
   // have a running queue of user regions to close?
   private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
 
-  private final RegionServerServices rsServices;
+  private final RegionServerServices rss;
   private final RegionInfo regionInfo;
 
   // If true, the hosting server is aborting.  Region close process is different
   // when we are aborting.
   private final boolean abort;
-  private ServerName destination;
+
+  /**
+   * The target the Region is going to after successful close; usually null as it depends on
+   * context whether this is passed in or not.
+   */
+  private final ServerName destination;
 
   /**
    * This method used internally by the RegionServer to close out regions.
-   * @param server
-   * @param rsServices
-   * @param regionInfo
+   *
    * @param abort If the regionserver is aborting.
-   * @param destination
    */
-  public CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices,
-      final RegionInfo regionInfo, final boolean abort,
-      ServerName destination) {
-    this(server, rsServices, regionInfo, abort,
-      EventType.M_RS_CLOSE_REGION, destination);
+  public CloseRegionHandler(final RegionServerServices rsServices, final RegionInfo regionInfo,
+    final boolean abort, @Nullable ServerName destination) {
+    this(rsServices, regionInfo, abort, EventType.M_RS_CLOSE_REGION, destination);
   }
 
-  protected CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices, RegionInfo regionInfo,
-      boolean abort, EventType eventType, ServerName destination) {
-    super(server, eventType);
-    this.server = server;
-    this.rsServices = rsServices;
+  protected CloseRegionHandler(RegionServerServices rsServices, RegionInfo regionInfo,
+      boolean abort, EventType eventType, @Nullable ServerName destination) {
+    super(rsServices, eventType);
     this.regionInfo = regionInfo;
     this.abort = abort;
+    this.rss = rsServices;
     this.destination = destination;
   }
 
-  public RegionInfo getRegionInfo() {
-    return regionInfo;
-  }
-
-  @Override
-  public void process() {
+  @Override public void process() throws IOException {
     try {
-      String name = regionInfo.getEncodedName();
-      LOG.trace("Processing close of {}", name);
-      String encodedRegionName = regionInfo.getEncodedName();
       // Check that this region is being served here
-      HRegion region = (HRegion)rsServices.getRegion(encodedRegionName);
+      HRegion region = (HRegion)this.rss.getRegion(this.regionInfo.getEncodedName());
       if (region == null) {
-        LOG.warn("Received CLOSE for region {} but currently not serving - ignoring", name);
+        LOG.warn("Received CLOSE for {} but not ONLINE; ignoring CLOSE request",
+          this.regionInfo.getRegionNameAsString());
         // TODO: do better than a simple warning
         return;
       }
-
-      // Close the region
-      try {
-        if (region.close(abort) == null) {
-          // This region got closed.  Most likely due to a split.
-          // The split message will clean up the master state.
-          LOG.warn("Can't close region {}, was already closed during close()", name);
-          return;
-        }
-      } catch (IOException ioe) {
-        // An IOException here indicates that we couldn't successfully flush the
-        // memstore before closing. So, we need to abort the server and allow
-        // the master to split our logs in order to recover the data.
-        server.abort("Unrecoverable exception while closing region " +
-          regionInfo.getRegionNameAsString() + ", still finishing close", ioe);
-        throw new RuntimeException(ioe);
+      if (region.getCoprocessorHost() != null) {
+        // XXX: The behavior is a bit broken. At master side there is no FAILED_CLOSE state, so if
+        // there are exceptions thrown from the CP, we can not report the error to the master, and
+        // if here is, we just return without calling reportRegionStateTransition, the TRSP at
+        // master side will hang there for ever. So here if the CP throws an exception out, the only
+        // way is to abort the RS...
+        region.getCoprocessorHost().preClose(abort);
       }
-
-      this.rsServices.removeRegion(region, destination);
-      rsServices.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-        HConstants.NO_SEQNUM, Procedure.NO_PROC_ID, -1, regionInfo));
-
+      if (region.close(abort) == null) {
+        // This should not happen. Left over from old days when RegionServer
+        // ran the splits? Now splits are run by the Master.
+        LOG.warn("Can't close {}, already closed; split? (Unexpected state - investigate)",
+          this.regionInfo.getRegionNameAsString());
+        // Return null here. Makes more sense. We didnt do close on the Region.
+        return;
+      }
+      this.rss.removeRegion(region, destination);
+      boolean b = reportRegionStateTransition();
       // Done!  Region is closed on this RS
-      LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString());
+      LOG.debug("Closed {} (notified master={})", this.regionInfo.getRegionNameAsString(), b);
+      // We don't care if we are able to inform the Master successfully; try but ignore response.
     } finally {
-      this.rsServices.getRegionsInTransitionInRS().
-        remove(this.regionInfo.getEncodedNameAsBytes(), Boolean.FALSE);
+      // Presumption is that before the Close was run, the CLOSE operation had been registered
+      // with rss.getRegionsInTransitionInRS(). Here we do cleanup whether success or failure.
+      this.rss.getRegionsInTransitionInRS().remove(this.regionInfo.getEncodedNameAsBytes(),
+        Boolean.FALSE);
     }
   }
+
+  /**
+   * @throws IOException Let subclasses throw IOE if they need to.
+   */
+  boolean reportRegionStateTransition() throws IOException {
+    return this.rss.reportRegionStateTransition(
+      new RegionStateTransitionContext(TransitionCode.CLOSED, HConstants.NO_SEQNUM,
+        getClosePID(), -1, this.regionInfo));
+  }
+
+  /**
+   * @return PID of Close to report to Master; default is Procedure.NO_PROC_ID.
+   */
+  long getClosePID() {
+    return Procedure.NO_PROC_ID;
+  }
+
+  @Override protected void handleException(Throwable t) {
+    LOG.warn("Fatal error occurred while closing region {}, aborting...",
 
 Review comment:
   base class is slightly more careful/different in handling the abort. Is it better to just customize the log here and the call the `super` method to handle abort? or maybe the `super` implementation need to abort more aggressively (abort on all exceptions instead of just the types it does)?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394705029
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -1457,6 +1458,9 @@ private void waitOnAllRegionsToClose(final boolean abort) {
                 " because some regions failed closing");
           }
           break;
+        } else {
+          LOG.trace("Waiting on {}", this.regionsInTransitionInRS.entrySet().stream().
 
 Review comment:
   I intentionally purged the word 'region' from logs that can only be about Region; redundant/padding. We are waiting on close here so let me add that rather generic 'transition'

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394709519
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -38,9 +36,13 @@
 /**
  * Handles closing of a region on a region server.
  * <p/>
- * Now for regular close region request, we will use {@link UnassignRegionHandler} instead. But when
- * shutting down the region server, will also close regions and the related methods still use this
- * class so we keep it here.
+ * In normal operation, we use {@link UnassignRegionHandler} closing Regions but when shutting down
+ * the region server and closing out Regions, we use this handler instead; it does not expect to
 
 Review comment:
   I tried. There are subtle differences ... one takes encoded region name only coming in from rpc. The other can assume it called by the hosting Region.  I played around.  A rewrite would disturb a bunch. I opened "HBASE-24015 Coverage for Assign and Unassign of Regions on RegionServer on failure". I think digging in on this issue will reveal issues around these handlers and will bring on refactor only we'll have extra coverage first so can be confident making change. At the moment afraid to change too much.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394697248
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
 ##########
 @@ -38,9 +36,13 @@
 /**
  * Handles closing of a region on a region server.
  * <p/>
- * Now for regular close region request, we will use {@link UnassignRegionHandler} instead. But when
- * shutting down the region server, will also close regions and the related methods still use this
- * class so we keep it here.
+ * In normal operation, we use {@link UnassignRegionHandler} closing Regions but when shutting down
+ * the region server and closing out Regions, we use this handler instead; it does not expect to
 
 Review comment:
   I guess a disable table also would call this handler?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394657691
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##########
 @@ -2446,6 +2450,12 @@ public RSRpcServices getRSRpcServices() {
    */
   @Override
   public void abort(String reason, Throwable cause) {
+    if (isAborted()) {
+      // Don't run multiple aborts; fills logs w/ spew.
+      LOG.info("Abort called but server already aborted; by-passing; abort invoked because {}",
 
 Review comment:
   I'm not a fan of logging exceptions except for `ERROR` or `FATAL`. Sometimes `DEBUG` is okay, but it's rarely helpful to an operator, and here I would consider it as "log spew" as well :) I should hope that the custom tailored `reason` message is enough.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394694903
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 ##########
 @@ -105,50 +102,54 @@ public void process() throws IOException {
     if (previous != null) {
       if (previous) {
         // The region is opening and this maybe a retry on the rpc call, it is safe to ignore it.
-        LOG.info("Receiving OPEN for the region:{}, which we are already trying to OPEN" +
-          " - ignoring this new request for this region.", regionName);
+        LOG.info("Receiving OPEN for {} which is OPENING; ignoring request", regionName);
       } else {
         // The region is closing. This is possible as we will update the region state to CLOSED when
         // calling reportRegionStateTransition, so the HMaster will think the region is offline,
         // before we actually close the region, as reportRegionStateTransition is part of the
         // closing process.
         long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "Receiving OPEN for the region:{}, which we are trying to close, try again after {}ms",
+        LOG.info("Receiving OPEN for {} which we are trying to CLOSE, try again after {}ms",
           regionName, backoff);
         rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
       }
       return;
     }
     LOG.info("Open {}", regionName);
     HRegion region;
+    // Start a try. Above we registered region with rs.getRegionsInTransitionInRS. Need to clear
+    // it on success or error on the end.
     try {
-      TableDescriptor htd =
-        tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
-      if (htd == null) {
-        throw new IOException("Missing table descriptor for " + regionName);
+      try {
+        TableDescriptor htd =
+          tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
+        if (htd == null) {
+          throw new IOException("Missing table descriptor for " + regionName);
+        }
+        // pass null for the last parameter, which used to be a CancelableProgressable, as now the
+        // opening can not be interrupted by a close request any more.
+        region = HRegion
+          .openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(), rs, null);
+      } catch (IOException e) {
+        cleanUpAndReportFailure(e);
+        return;
       }
-      // pass null for the last parameter, which used to be a CancelableProgressable, as now the
-      // opening can not be interrupted by a close request any more.
-      region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), rs.getConfiguration(),
-        rs, null);
-    } catch (IOException e) {
-      cleanUpAndReportFailure(e);
-      return;
-    }
-    rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
-    rs.addRegion(region);
-    LOG.info("Opened {}", regionName);
-    // Cache the open region procedure id after report region transition succeed.
-    rs.finishRegionProcedure(openProcId);
-    Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
-    if (current == null) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was NOT in transition. Region={}",
-        regionName);
-    } else if (!current) {
-      // Should NEVER happen, but let's be paranoid.
-      LOG.error("Bad state: we've just opened a region that was closing. Region={}", regionName);
+      rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
+      rs.addRegion(region);
+      LOG.info("Opened {}", regionName);
+    } finally {
+      // Remove from regionsInTransitionRS whether success or failure.
+      rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
+      Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
+      if (current == null) {
+        // Should NEVER happen, but let's be paranoid.
+        LOG.error("Bad state: we've just opened {} and it was NOT in transition", regionName);
+      } else if (!current) {
+        // Should NEVER happen, but let's be paranoid.
 
 Review comment:
   another lie, same thinking re: log level.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394696853
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
 ##########
 @@ -31,10 +32,8 @@
 public class CloseMetaHandler extends CloseRegionHandler {
 
   // Called when regionserver determines its to go down; not master orchestrated
-  public CloseMetaHandler(final Server server,
 
 Review comment:
   I actually prefer these long method argument lists being a single parameter per line, but our style guides don't agree.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#discussion_r394708425
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
 ##########
 @@ -31,10 +32,8 @@
 public class CloseMetaHandler extends CloseRegionHandler {
 
   // Called when regionserver determines its to go down; not master orchestrated
-  public CloseMetaHandler(final Server server,
 
 Review comment:
   Right. Mine agrees w/ general code base. Yours does not (smile).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1305: HBASE-23984 [Flakey Tests] TestMasterAbortAndRSGotKilled fails in tea…
URL: https://github.com/apache/hbase/pull/1305#issuecomment-600475344
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 45s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  7s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   7m 18s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m 10s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   5m 16s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m 12s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 56s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 108m 20s |  hbase-server in the patch failed.  |
   |  |   | 139m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk8-hadoop2-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1305 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 989c85733453 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / ebd37a314c |
   | Default Java | 1.8.0_232 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/artifact/yetus-jdk8-hadoop2-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/testReport/ |
   | Max. process+thread count | 3625 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1305/1/console |
   | versions | git=2.17.1 maven=2018-06-17T18:33:14Z) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services