You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/09/15 09:01:17 UTC

svn commit: r1385024 [2/2] - in /hbase/trunk: bin/ hbase-it/ hbase-it/src/ hbase-it/src/test/ hbase-it/src/test/java/ hbase-it/src/test/java/org/ hbase-it/src/test/java/org/apache/ hbase-it/src/test/java/org/apache/hadoop/ hbase-it/src/test/java/org/ap...

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java?rev=1385024&r1=1385023&r2=1385024&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java Sat Sep 15 07:01:15 2012
@@ -29,6 +29,8 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AdminProtocol;
+import org.apache.hadoop.hbase.client.ClientProtocol;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
@@ -36,11 +38,10 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.io.MapWritable;
 
 /**
  * This class creates a single process HBase cluster.
@@ -50,9 +51,8 @@ import org.apache.hadoop.io.MapWritable;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class MiniHBaseCluster {
+public class MiniHBaseCluster extends HBaseCluster {
   static final Log LOG = LogFactory.getLog(MiniHBaseCluster.class.getName());
-  private Configuration conf;
   public LocalHBaseCluster hbaseCluster;
   private static int index;
 
@@ -77,18 +77,17 @@ public class MiniHBaseCluster {
   public MiniHBaseCluster(Configuration conf, int numMasters,
                              int numRegionServers)
       throws IOException, InterruptedException {
-    this.conf = conf;
-    conf.set(HConstants.MASTER_PORT, "0");
-    init(numMasters, numRegionServers, null, null);
+    this(conf, numMasters, numRegionServers, null, null);
   }
 
   public MiniHBaseCluster(Configuration conf, int numMasters, int numRegionServers,
          Class<? extends HMaster> masterClass,
          Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
       throws IOException, InterruptedException {
-    this.conf = conf;
+    super(conf);
     conf.set(HConstants.MASTER_PORT, "0");
     init(numMasters, numRegionServers, masterClass, regionserverClass);
+    this.initialClusterStatus = getClusterStatus();
   }
 
   public Configuration getConfiguration() {
@@ -229,6 +228,54 @@ public class MiniHBaseCluster {
     }
   }
 
+  @Override
+  public void startRegionServer(String hostname) throws IOException {
+    this.startRegionServer();
+  }
+
+  @Override
+  public void killRegionServer(ServerName serverName) throws IOException {
+    HRegionServer server = getRegionServer(getRegionServerIndex(serverName));
+    if (server instanceof MiniHBaseClusterRegionServer) {
+      LOG.info("Killing " + server.toString());
+      ((MiniHBaseClusterRegionServer) server).kill();
+    } else {
+      abortRegionServer(getRegionServerIndex(serverName));
+    }
+  }
+
+  @Override
+  public void stopRegionServer(ServerName serverName) throws IOException {
+    stopRegionServer(getRegionServerIndex(serverName));
+  }
+
+  @Override
+  public void waitForRegionServerToStop(ServerName serverName, long timeout) throws IOException {
+    //ignore timeout for now
+    waitOnRegionServer(getRegionServerIndex(serverName));
+  }
+
+  @Override
+  public void startMaster(String hostname) throws IOException {
+    this.startMaster();
+  }
+
+  @Override
+  public void killMaster(ServerName serverName) throws IOException {
+    abortMaster(getMasterIndex(serverName));
+  }
+
+  @Override
+  public void stopMaster(ServerName serverName) throws IOException {
+    stopMaster(getMasterIndex(serverName));
+  }
+
+  @Override
+  public void waitForMasterToStop(ServerName serverName, long timeout) throws IOException {
+    //ignore timeout for now
+    waitOnMaster(getMasterIndex(serverName));
+  }
+
   /**
    * Starts a region server thread running
    *
@@ -324,6 +371,16 @@ public class MiniHBaseCluster {
     return t;
   }
 
+  @Override
+  public MasterAdminProtocol getMasterAdmin() {
+    return this.hbaseCluster.getActiveMaster();
+  }
+
+  @Override
+  public MasterMonitorProtocol getMasterMonitor() {
+    return this.hbaseCluster.getActiveMaster();
+  }
+
   /**
    * Returns the current active master, if available.
    * @return the active HMaster, null if none is active.
@@ -398,15 +455,18 @@ public class MiniHBaseCluster {
    *         masters left.
    * @throws InterruptedException
    */
-  public boolean waitForActiveAndReadyMaster() throws InterruptedException {
+  public boolean waitForActiveAndReadyMaster(long timeout) throws IOException {
     List<JVMClusterUtil.MasterThread> mts;
-    while (!(mts = getMasterThreads()).isEmpty()) {
+    long start = System.currentTimeMillis();
+    while (!(mts = getMasterThreads()).isEmpty()
+        && (System.currentTimeMillis() - start) < timeout) {
       for (JVMClusterUtil.MasterThread mt : mts) {
         if (mt.getMaster().isActiveMaster() && mt.getMaster().isInitialized()) {
           return true;
         }
       }
-      Thread.sleep(100);
+
+      Threads.sleep(100);
     }
     return false;
   }
@@ -443,6 +503,16 @@ public class MiniHBaseCluster {
     HConnectionManager.deleteAllConnections(false);
   }
 
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public ClusterStatus getClusterStatus() throws IOException {
+    HMaster master = getMaster();
+    return master == null ? null : master.getClusterStatus();
+  }
+
   /**
    * Call flushCache on all regions on all participating regionservers.
    * @throws IOException
@@ -565,6 +635,15 @@ public class MiniHBaseCluster {
     return index;
   }
 
+  @Override
+  public ServerName getServerHoldingRegion(byte[] regionName) throws IOException {
+    int index = getServerWith(regionName);
+    if (index < 0) {
+      return null;
+    }
+    return getRegionServer(index).getServerName();
+  }
+
   /**
    * Counts the total numbers of regions being served by the currently online
    * region servers by asking each how many regions they have.  Does not look
@@ -591,4 +670,40 @@ public class MiniHBaseCluster {
       masterThread.getMaster().abort("killAll", new Throwable());
     }
   }
+
+  @Override
+  public void waitUntilShutDown() {
+    this.hbaseCluster.join();
+  }
+
+  protected int getRegionServerIndex(ServerName serverName) {
+    //we have a small number of region servers, this should be fine for now.
+    List<RegionServerThread> servers = getRegionServerThreads();
+    for (int i=0; i < servers.size(); i++) {
+      if (servers.get(i).getRegionServer().getServerName().equals(serverName)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  protected int getMasterIndex(ServerName serverName) {
+    List<MasterThread> masters = getMasterThreads();
+    for (int i = 0; i < masters.size(); i++) {
+      if (masters.get(i).getMaster().getServerName().equals(serverName)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  @Override
+  public AdminProtocol getAdminProtocol(ServerName serverName) throws IOException {
+    return getRegionServer(getRegionServerIndex(serverName));
+  }
+
+  @Override
+  public ClientProtocol getClientProtocol(ServerName serverName) throws IOException {
+    return getRegionServer(getRegionServerIndex(serverName));
+  }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/SmallTests.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/SmallTests.java?rev=1385024&r1=1385023&r2=1385024&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/SmallTests.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/SmallTests.java Sat Sep 15 07:01:15 2012
@@ -28,6 +28,7 @@ package org.apache.hadoop.hbase;
  *
  * @see MediumTests
  * @see LargeTests
+ * @see IntegrationTests
  */
 public interface SmallTests {
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java?rev=1385024&r1=1385023&r2=1385024&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java Sat Sep 15 07:01:15 2012
@@ -18,11 +18,11 @@
 
 package org.apache.hadoop.hbase;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runners.Suite;
+import static junit.framework.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
+import java.io.FileFilter;
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -31,8 +31,9 @@ import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
 
-import static junit.framework.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.Suite;
 
 
 /**
@@ -41,8 +42,21 @@ import static org.junit.Assert.assertTru
 @Category(SmallTests.class)
 public class TestCheckTestClasses {
 
+  private FileFilter TEST_CLASS_FILE_FILTER = new FileFilter() {
+    @Override
+    public boolean accept(File file) {
+      return file.isDirectory() || isTestClassFile(file);
+
+    }
+    private boolean isTestClassFile(File file) {
+      String fileName = file.getName();
+      return fileName.endsWith(".class")
+          && (fileName.startsWith("Test") || fileName.startsWith("IntegrationTest"));
+    }
+  };
+
   /**
-   * Throws an assertion if we find a test class without category (small/medium/large).
+   * Throws an assertion if we find a test class without category (small/medium/large/integration).
    * List all the test classes without category in the assertion message.
    */
   @Test
@@ -50,7 +64,7 @@ public class TestCheckTestClasses {
     List<Class<?>> badClasses = new java.util.ArrayList<Class<?>>();
 
     for (Class<?> c : findTestClasses()) {
-      if (!existCategoryAnnotation(c)) {
+      if (!existCategoryAnnotation(c, null)) {
         badClasses.add(c);
       }
     }
@@ -59,9 +73,22 @@ public class TestCheckTestClasses {
       + badClasses, badClasses.isEmpty());
   }
 
+  /** Returns whether the class has @Category annotation having the xface value.
+   */
+  private boolean existCategoryAnnotation(Class<?> c, Class<?> xface) {
+    Category category = c.getAnnotation(Category.class);
 
-  private boolean existCategoryAnnotation(Class<?> c) {
-    return (c.getAnnotation(Category.class) != null);
+    if (category != null) {
+      if (xface == null) {
+        return true;
+      }
+      for (Class<?> cc : category.value()) {
+        if (cc.equals(xface)) {
+          return true;
+        }
+      }
+    }
+    return false;
   }
 
   /*
@@ -88,6 +115,19 @@ public class TestCheckTestClasses {
     return false;
   }
 
+  /**
+   * Finds test classes which are annotated with @Category having xface value
+   * @param xface the @Category value
+   */
+  public List<Class<?>> findTestClasses(Class<?> xface) throws ClassNotFoundException, IOException {
+    List<Class<?>> classes = new ArrayList<Class<?>>();
+    for (Class<?> c : findTestClasses()) {
+      if (existCategoryAnnotation(c, xface)) {
+        classes.add(c);
+      }
+    }
+    return classes;
+  }
 
   private List<Class<?>> findTestClasses() throws ClassNotFoundException, IOException {
     final String packageName = "org.apache.hadoop.hbase";
@@ -117,14 +157,14 @@ public class TestCheckTestClasses {
       return classes;
     }
 
-    File[] files = baseDirectory.listFiles();
+    File[] files = baseDirectory.listFiles(TEST_CLASS_FILE_FILTER);
     assertNotNull(files);
 
     for (File file : files) {
       final String fileName = file.getName();
       if (file.isDirectory()) {
         classes.addAll(findTestClasses(file, packageName + "." + fileName));
-      } else if (fileName.endsWith(".class") && fileName.startsWith("Test")) {
+      } else {
         Class<?> c = Class.forName(
           packageName + '.' + fileName.substring(0, fileName.length() - 6),
           false,

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java?rev=1385024&r1=1385023&r2=1385024&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java Sat Sep 15 07:01:15 2012
@@ -34,7 +34,19 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.RegionTransition;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -42,9 +54,9 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKTable;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -219,7 +231,7 @@ public class TestMasterFailover {
     enabledAndAssignedRegions.add(enabledRegions.remove(0));
     enabledAndAssignedRegions.add(enabledRegions.remove(0));
     enabledAndAssignedRegions.add(closingRegion);
-    
+
     List<HRegionInfo> disabledAndAssignedRegions = new ArrayList<HRegionInfo>();
     disabledAndAssignedRegions.add(disabledRegions.remove(0));
     disabledAndAssignedRegions.add(disabledRegions.remove(0));
@@ -457,18 +469,18 @@ public class TestMasterFailover {
     // Create a ZKW to use in the test
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
         "unittest", new Abortable() {
-          
+
           @Override
           public void abort(String why, Throwable e) {
             LOG.error("Fatal ZK Error: " + why, e);
             org.junit.Assert.assertFalse("Fatal ZK error", true);
           }
-          
+
           @Override
           public boolean isAborted() {
             return false;
           }
-          
+
     });
 
     // get all the master threads
@@ -895,8 +907,8 @@ public class TestMasterFailover {
     TEST_UTIL.shutdownMiniHBaseCluster();
 
     // Create a ZKW to use in the test
-    ZooKeeperWatcher zkw = 
-      HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL, 
+    ZooKeeperWatcher zkw =
+      HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
           metaRegion, regionServer.getServerName());
 
     LOG.info("Staring cluster for second time");
@@ -1042,10 +1054,10 @@ public class TestMasterFailover {
    * @param cluster
    * @return the new active master
    * @throws InterruptedException
-   * @throws MasterNotRunningException
+   * @throws IOException
    */
   private HMaster killActiveAndWaitForNewActive(MiniHBaseCluster cluster)
-  throws InterruptedException, MasterNotRunningException {
+  throws InterruptedException, IOException {
     int activeIndex = getActiveMasterIndex(cluster);
     HMaster active = cluster.getMaster();
     cluster.stopMaster(activeIndex);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java?rev=1385024&r1=1385023&r2=1385024&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java Sat Sep 15 07:01:15 2012
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.hadoop.hbase.util.StoppableImplementation;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -179,7 +180,7 @@ public class TestEndToEndSplitTransactio
     //for daughters.
     HTable table = TEST_UTIL.createTable(TABLENAME, FAMILY);
 
-    Stoppable stopper = new SimpleStoppable();
+    Stoppable stopper = new StoppableImplementation();
     RegionSplitter regionSplitter = new RegionSplitter(table);
     RegionChecker regionChecker = new RegionChecker(conf, stopper, TABLENAME);
 
@@ -202,20 +203,6 @@ public class TestEndToEndSplitTransactio
     regionChecker.verify();
   }
 
-  private static class SimpleStoppable implements Stoppable {
-    volatile boolean stopped = false;
-
-    @Override
-    public void stop(String why) {
-      this.stopped = true;
-    }
-
-    @Override
-    public boolean isStopped() {
-      return stopped;
-    }
-  }
-
   static class RegionSplitter extends Thread {
     Throwable ex;
     HTable table;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java?rev=1385024&r1=1385023&r2=1385024&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java Sat Sep 15 07:01:15 2012
@@ -289,7 +289,7 @@ public class LoadTestTool extends Abstra
   }
 
   @Override
-  protected void doWork() throws IOException {
+  protected int doWork() throws IOException {
     if (cmd.hasOption(OPT_ZK_QUORUM)) {
       conf.set(HConstants.ZOOKEEPER_QUORUM, cmd.getOptionValue(OPT_ZK_QUORUM));
     }
@@ -335,6 +335,16 @@ public class LoadTestTool extends Abstra
     if (isRead) {
       readerThreads.waitForFinish();
     }
+
+    boolean success = true;
+    if (isWrite) {
+      success = success && writerThreads.getNumWriteFailures() == 0;
+    }
+    if (isRead) {
+      success = success && readerThreads.getNumReadErrors() == 0
+          && readerThreads.getNumReadFailures() == 0;
+    }
+    return success ? 0 : 1;
   }
 
   public static void main(String[] args) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java?rev=1385024&r1=1385023&r2=1385024&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java Sat Sep 15 07:01:15 2012
@@ -86,7 +86,7 @@ public class RestartMetaTest extends Abs
   }
 
   @Override
-  protected void doWork() throws Exception {
+  protected int doWork() throws Exception {
     ProcessBasedLocalHBaseCluster hbaseCluster =
         new ProcessBasedLocalHBaseCluster(conf, NUM_DATANODES, numRegionServers);
     hbaseCluster.startMiniDFS();
@@ -128,6 +128,7 @@ public class RestartMetaTest extends Abs
           + Bytes.toStringBinary(result.getFamilyMap(HConstants.CATALOG_FAMILY)
               .get(HConstants.SERVER_QUALIFIER)));
     }
+    return 0;
   }
 
   @Override

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/StoppableImplementation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/StoppableImplementation.java?rev=1385024&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/StoppableImplementation.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/StoppableImplementation.java Sat Sep 15 07:01:15 2012
@@ -0,0 +1,40 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Stoppable;
+
+/**
+ * A base implementation for a Stoppable service
+ */
+@InterfaceAudience.Private
+public class StoppableImplementation implements Stoppable {
+  volatile boolean stopped = false;
+
+  @Override
+  public void stop(String why) {
+    this.stopped = true;
+  }
+
+  @Override
+  public boolean isStopped() {
+    return stopped;
+  }
+}
\ No newline at end of file