You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/05/27 18:54:34 UTC

git commit: HBASE-11096 stop method of Master and RegionServer coprocessor is not invoked (Qiang Tian)

Repository: hbase
Updated Branches:
  refs/heads/0.94 a616bb42c -> 6ed3d0770


HBASE-11096 stop method of Master and RegionServer coprocessor is not invoked (Qiang Tian)


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

Branch: refs/heads/0.94
Commit: 6ed3d0770202a497e9884d0347efa46121dc2808
Parents: a616bb4
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue May 27 09:51:08 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue May 27 09:51:08 2014 -0700

----------------------------------------------------------------------
 .../hbase/coprocessor/CoprocessorHost.java      |   3 +
 .../hbase/master/MasterCoprocessorHost.java     |   4 +
 .../hbase/regionserver/HRegionServer.java       |  20 +--
 .../RegionServerCoprocessorHost.java            |   2 +
 .../hbase/coprocessor/TestCoprocessorStop.java  | 125 +++++++++++++++++++
 5 files changed, 145 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6ed3d077/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index c41d63c..24d7519 100644
--- a/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ b/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -260,6 +260,9 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
 
   public void shutdown(CoprocessorEnvironment e) {
     if (e instanceof Environment) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Stop coprocessor " + e.getInstance().getClass().getName());
+      }
       ((Environment)e).shutdown();
     } else {
       LOG.warn("Shutdown called on unknown environment: "+

http://git-wip-us.apache.org/repos/asf/hbase/blob/6ed3d077/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 936eb5a..bfc5983 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -595,6 +595,8 @@ public class MasterCoprocessorHost
           break;
         }
       }
+      // invoke coprocessor stop method
+      shutdown(env);
     }
   }
 
@@ -612,6 +614,8 @@ public class MasterCoprocessorHost
           break;
         }
       }
+      // invoke coprocessor stop method
+      shutdown(env);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6ed3d077/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index b1c3387..36fa0a3 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1840,16 +1840,18 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
 
   @Override
   public void stop(final String msg) {
-    try {
-      if (this.rsHost != null) {
-        this.rsHost.preStop(msg);
+    if (!this.stopped) {
+      try {
+        if (this.rsHost != null) {
+          this.rsHost.preStop(msg);
+        }
+        this.stopped = true;
+        LOG.info("STOPPED: " + msg);
+        // Wakes run() if it is sleeping
+        sleeper.skipSleepCycle();
+      } catch (IOException exp) {
+        LOG.warn("The region server did not stop", exp);
       }
-      this.stopped = true;
-      LOG.info("STOPPED: " + msg);
-      // Wakes run() if it is sleeping
-      sleeper.skipSleepCycle();
-    } catch (IOException exp) {
-      LOG.warn("The region server did not stop", exp);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6ed3d077/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java b/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
index 2302f9e..f7dafad 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
@@ -58,6 +58,8 @@ public class RegionServerCoprocessorHost extends
           break;
         }
       }
+      // invoke coprocessor stop method
+      shutdown(env);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6ed3d077/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorStop.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorStop.java b/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorStop.java
new file mode 100644
index 0000000..6800986
--- /dev/null
+++ b/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorStop.java
@@ -0,0 +1,125 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.coprocessor;
+
+import java.io.IOException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import static org.junit.Assert.assertTrue;
+
+
+/**
+ * Tests for master and regionserver coprocessor stop method
+ *
+ */
+@Category(MediumTests.class)
+public class TestCoprocessorStop {
+  private static final Log LOG = LogFactory.getLog(TestCoprocessorStop.class);
+  private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static final String MASTER_FILE =
+                              "master" + System.currentTimeMillis();
+  private static final String REGIONSERVER_FILE =
+                              "regionserver" + System.currentTimeMillis();
+
+  public static class FooCoprocessor implements Coprocessor {
+    @Override
+    public void start(CoprocessorEnvironment env) throws IOException {
+      String where = null;
+
+      if (env instanceof MasterCoprocessorEnvironment) {
+        // if running on HMaster
+        where = "master";
+      } else if (env instanceof RegionServerCoprocessorEnvironment) {
+        where = "regionserver";
+      } else if (env instanceof RegionCoprocessorEnvironment) {
+        LOG.error("on RegionCoprocessorEnvironment!!");
+      }
+      LOG.info("start coprocessor on " + where);
+    }
+
+    @Override
+    public void stop(CoprocessorEnvironment env) throws IOException {
+      String fileName = null;
+
+      if (env instanceof MasterCoprocessorEnvironment) {
+        // if running on HMaster
+        fileName = MASTER_FILE;
+      } else if (env instanceof RegionServerCoprocessorEnvironment) {
+        fileName = REGIONSERVER_FILE;
+      } else if (env instanceof RegionCoprocessorEnvironment) {
+        LOG.error("on RegionCoprocessorEnvironment!!");
+      }
+
+      Configuration conf = UTIL.getConfiguration();
+      Path resultFile = new Path(UTIL.getDataTestDir(), fileName);
+      FileSystem fs = FileSystem.get(conf);
+
+      boolean result = fs.createNewFile(resultFile);
+      LOG.info("create file " + resultFile + " return rc " + result);
+    }
+  }
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+      FooCoprocessor.class.getName());
+    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+      FooCoprocessor.class.getName());
+
+    UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testStopped() throws Exception {
+    //shutdown hbase only. then check flag file.
+    MiniHBaseCluster cluster = UTIL.getHBaseCluster();
+    LOG.info("shutdown hbase cluster...");
+    cluster.shutdown();
+    LOG.info("wait for the hbase cluster shutdown...");
+    cluster.waitUntilShutDown();
+
+    Configuration conf = UTIL.getConfiguration();
+    FileSystem fs = FileSystem.get(conf);
+
+    Path resultFile = new Path(UTIL.getDataTestDir(), MASTER_FILE);
+    assertTrue("Master flag file should have been created",fs.exists(resultFile));
+
+    resultFile = new Path(UTIL.getDataTestDir(), REGIONSERVER_FILE);
+    assertTrue("RegionServer flag file should have been created",fs.exists(resultFile));
+  }
+}