You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/14 13:58:21 UTC

svn commit: r1446147 [27/35] - in /hbase/branches/hbase-7290v2: ./ bin/ conf/ dev-support/ hbase-client/ hbase-common/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/ hbase-common/src/...

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Thu Feb 14 12:58:12 2013
@@ -52,6 +52,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -70,12 +71,12 @@ import org.apache.hadoop.hbase.io.hfile.
 import org.apache.hadoop.hbase.mapreduce.MapreduceTestingShim;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -188,7 +189,7 @@ public class HBaseTestingUtility extends
     List<Object[]> configurations = new ArrayList<Object[]>();
     for (Compression.Algorithm comprAlgo :
          HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
-      for (StoreFile.BloomType bloomType : StoreFile.BloomType.values()) {
+      for (BloomType bloomType : BloomType.values()) {
         configurations.add(new Object[] { comprAlgo, bloomType });
       }
     }
@@ -266,6 +267,9 @@ public class HBaseTestingUtility extends
       "mapred.local.dir",
       testPath, "mapred-local-dir");
 
+    createSubDir(
+      "hbase.local.dir",
+      testPath, "hbase-local-dir");
     return testPath;
   }
 
@@ -383,7 +387,7 @@ public class HBaseTestingUtility extends
     //file system, the tests should use getBaseTestDir, otherwise, we can use
     //the working directory, and create a unique sub dir there
     FileSystem fs = getTestFileSystem();
-    if (fs.getUri().getScheme().equals(fs.getLocal(conf).getUri().getScheme())) {
+    if (fs.getUri().getScheme().equals(FileSystem.getLocal(conf).getUri().getScheme())) {
       File dataTestDir = new File(getDataTestDir().toString());
       dataTestDirOnTestFS = new Path(dataTestDir.getAbsolutePath());
     } else {
@@ -677,6 +681,34 @@ public class HBaseTestingUtility extends
     return startMiniCluster(numMasters, numSlaves, null);
   }
 
+  /**
+   * Start up a minicluster of hbase, optionally dfs, and zookeeper.
+   * Modifies Configuration.  Homes the cluster data directory under a random
+   * subdirectory in a directory under System property test.build.data.
+   * Directory is cleaned up on exit.
+   * @param numMasters Number of masters to start up.  We'll start this many
+   * hbase masters.  If numMasters > 1, you can find the active/primary master
+   * with {@link MiniHBaseCluster#getMaster()}.
+   * @param numSlaves Number of slaves to start up.  We'll start this many
+   * regionservers. If dataNodeHosts == null, this also indicates the number of
+   * datanodes to start. If dataNodeHosts != null, the number of datanodes is
+   * based on dataNodeHosts.length.
+   * If numSlaves is > 1, then make sure
+   * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
+   * bind errors.
+   * @param dataNodeHosts hostnames DNs to run on.
+   * This is useful if you want to run datanode on distinct hosts for things
+   * like HDFS block location verification.
+   * If you start MiniDFSCluster without host names,
+   * all instances of the datanodes will have the same host name.
+   * @throws Exception
+   * @see {@link #shutdownMiniCluster()}
+   * @return Mini hbase cluster instance created.
+   */
+  public MiniHBaseCluster startMiniCluster(final int numMasters,
+      final int numSlaves, final String[] dataNodeHosts) throws Exception {
+    return startMiniCluster(numMasters, numSlaves, dataNodeHosts, null, null);
+  }
 
   /**
    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
@@ -698,12 +730,17 @@ public class HBaseTestingUtility extends
    * like HDFS block location verification.
    * If you start MiniDFSCluster without host names,
    * all instances of the datanodes will have the same host name.
+   * @param masterClass The class to use as HMaster, or null for default
+   * @param regionserverClass The class to use as HRegionServer, or null for
+   * default
    * @throws Exception
    * @see {@link #shutdownMiniCluster()}
    * @return Mini hbase cluster instance created.
    */
   public MiniHBaseCluster startMiniCluster(final int numMasters,
-    final int numSlaves, final String[] dataNodeHosts)
+    final int numSlaves, final String[] dataNodeHosts,
+    Class<? extends HMaster> masterClass,
+    Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
   throws Exception {
     int numDataNodes = numSlaves;
     if ( dataNodeHosts != null && dataNodeHosts.length != 0) {
@@ -732,7 +769,7 @@ public class HBaseTestingUtility extends
     }
 
     // Start the MiniHBaseCluster
-    return startMiniHBaseCluster(numMasters, numSlaves);
+    return startMiniHBaseCluster(numMasters, numSlaves, masterClass, regionserverClass);
   }
 
   public MiniHBaseCluster startMiniHBaseCluster(final int numMasters, final int numSlaves)
@@ -1271,6 +1308,18 @@ public class HBaseTestingUtility extends
     Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
   };
 
+  public static final byte[][] KEYS_FOR_HBA_CREATE_TABLE = {
+      Bytes.toBytes("bbb"),
+      Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
+      Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
+      Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
+      Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
+      Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
+      Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
+      Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
+      Bytes.toBytes("xxx"), Bytes.toBytes("yyy"), Bytes.toBytes("zzz")
+  };
+
   /**
    * Creates many regions names "aaa" to "zzz".
    * @param c Configuration to use.
@@ -1873,17 +1922,21 @@ public class HBaseTestingUtility extends
     throws IOException {
     boolean startedServer = ensureSomeRegionServersAvailable(num);
 
+    int nonStoppedServers = 0;
     for (JVMClusterUtil.RegionServerThread rst :
       getMiniHBaseCluster().getRegionServerThreads()) {
 
       HRegionServer hrs = rst.getRegionServer();
       if (hrs.isStopping() || hrs.isStopped()) {
         LOG.info("A region server is stopped or stopping:"+hrs);
-        LOG.info("Started new server=" + getMiniHBaseCluster().startRegionServer());
-        startedServer = true;
+      } else {
+        nonStoppedServers++;
       }
     }
-
+    for (int i=nonStoppedServers; i<num; ++i) {
+      LOG.info("Started new server=" + getMiniHBaseCluster().startRegionServer());
+      startedServer = true;
+    }
     return startedServer;
   }
 
@@ -2344,4 +2397,28 @@ public class HBaseTestingUtility extends
   public void setFileSystemURI(String fsURI) {
     FS_URI = fsURI;
   }
+  
+  /**
+   * Wrapper method for {@link Waiter#waitFor(Configuration, long, Predicate)}.
+   */
+  public <E extends Exception> long waitFor(long timeout, Predicate<E> predicate)
+      throws E {
+    return Waiter.waitFor(this.conf, timeout, predicate);
+  }
+
+  /**
+   * Wrapper method for {@link Waiter#waitFor(Configuration, long, long, Predicate)}.
+   */
+  public <E extends Exception> long waitFor(long timeout, long interval, Predicate<E> predicate)
+      throws E {
+    return Waiter.waitFor(this.conf, timeout, interval, predicate);
+  }
+
+  /**
+   * Wrapper method for {@link Waiter#waitFor(Configuration, long, long, boolean, Predicate)}.
+   */
+  public <E extends Exception> long waitFor(long timeout, long interval,
+      boolean failIfTimeout, Predicate<E> predicate) throws E {
+    return Waiter.waitFor(this.conf, timeout, interval, failIfTimeout, predicate);
+  }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java Thu Feb 14 12:58:12 2013
@@ -500,7 +500,7 @@ public class MiniHBaseCluster extends HB
     if (this.hbaseCluster != null) {
       this.hbaseCluster.shutdown();
     }
-    HConnectionManager.deleteAllConnections(false);
+    HConnectionManager.deleteAllConnections();
   }
 
   @Override

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java Thu Feb 14 12:58:12 2013
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -81,6 +82,9 @@ public class TestAcidGuarantees implemen
     // Set small flush size for minicluster so we exercise reseeking scanners
     Configuration conf = HBaseConfiguration.create();
     conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(128*1024));
+    // prevent aggressive region split
+    conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
+      ConstantSizeRegionSplitPolicy.class.getName());
     util = new HBaseTestingUtility(conf);
   }
   

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java Thu Feb 14 12:58:12 2013
@@ -27,8 +27,6 @@ import junit.framework.Assert;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.master.ServerManager;
@@ -36,8 +34,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -47,21 +44,21 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.google.protobuf.ServiceException;
 
 /**
  * Test the draining servers feature.
+ *
+ * This is typically an integration test: a unit test would be to check that the
+ * master does no assign regions to a regionserver marked as drained.
+ *
  * @see <a href="https://issues.apache.org/jira/browse/HBASE-4298">HBASE-4298</a>
  */
 @Category(MediumTests.class)
 public class TestDrainingServer {
   private static final Log LOG = LogFactory.getLog(TestDrainingServer.class);
-  private static final HBaseTestingUtility TEST_UTIL =
-    new HBaseTestingUtility();
-  private static final byte [] TABLENAME = Bytes.toBytes("t");
-  private static final byte [] FAMILY = Bytes.toBytes("f");
-  private static final int COUNT_OF_REGIONS = HBaseTestingUtility.KEYS.length;
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final int NB_SLAVES = 5;
+  private static final int COUNT_OF_REGIONS = NB_SLAVES * 2;
 
   /**
    * Spin up a cluster with a bunch of regions on it.
@@ -69,31 +66,25 @@ public class TestDrainingServer {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniCluster(NB_SLAVES);
+    TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster();
     TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
-    ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
-    HTableDescriptor htd = new HTableDescriptor(TABLENAME);
-    htd.addFamily(new HColumnDescriptor(FAMILY));
-    TEST_UTIL.createMultiRegionsInMeta(TEST_UTIL.getConfiguration(), htd,
-        HBaseTestingUtility.KEYS);
-    // Make a mark for the table in the filesystem.
-    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    FSTableDescriptors.
-      createTableDescriptor(fs, FSUtils.getRootDir(TEST_UTIL.getConfiguration()), htd);
-    // Assign out the regions we just created.
-    HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
-    MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
-    admin.disableTable(TABLENAME);
-    admin.enableTable(TABLENAME);
+
+    final List<String> families = new ArrayList<String>(1);
+    families.add("family");
+    TEST_UTIL.createRandomTable("table", families, 1, 0, 0, COUNT_OF_REGIONS, 0);
+
+    // Ensure a stable env
+    TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, false);
 
     boolean ready = false;
     while (!ready){
-      ZKAssign.blockUntilNoRIT(zkw);
+      waitForAllRegionsOnline();
 
       // Assert that every regionserver has some regions on it.
       int i = 0;
       ready = true;
       while (i < NB_SLAVES && ready){
-        HRegionServer hrs = cluster.getRegionServer(i);
+        HRegionServer hrs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(i);
         if (ProtobufUtil.getOnlineRegions(hrs).isEmpty()){
           ready = false;
         }
@@ -101,7 +92,9 @@ public class TestDrainingServer {
       }
 
       if (!ready){
-        admin.balancer();
+        TEST_UTIL.getHBaseAdmin().setBalancerRunning(true, true);
+        Assert.assertTrue("Can't start a balance!", TEST_UTIL.getHBaseAdmin().balancer());
+        TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, false);
         Thread.sleep(100);
       }
     }
@@ -135,12 +128,12 @@ public class TestDrainingServer {
   /**
    * Test adding server to draining servers and then move regions off it.
    * Make sure that no regions are moved back to the draining server.
-   * @throws IOException 
-   * @throws KeeperException 
+   * @throws IOException
+   * @throws KeeperException
    */
   @Test  // (timeout=30000)
   public void testDrainingServerOffloading()
-  throws IOException, KeeperException, ServiceException, DeserializationException {
+  throws Exception {
     // I need master in the below.
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
     HRegionInfo hriToMoveBack = null;
@@ -186,8 +179,6 @@ public class TestDrainingServer {
   public void testDrainingServerWithAbort() throws KeeperException, Exception {
     HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
 
-    // Ensure a stable env
-    TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, false);
     waitForAllRegionsOnline();
 
     final long regionCount = TEST_UTIL.getMiniHBaseCluster().countServedRegions();
@@ -215,19 +206,21 @@ public class TestDrainingServer {
       setDrainingServer(drainingServer);
 
       //wait for the master to receive and manage the event
-      while  (sm.createDestinationServersList().contains(drainingServer.getServerName())) ;
+      while  (sm.createDestinationServersList().contains(drainingServer.getServerName())) {
+        Thread.sleep(1);
+      }
 
       LOG.info("The available servers are: "+ sm.createDestinationServersList());
 
       Assert.assertEquals("Nothing should have happened here.", regionsOnDrainingServer,
         drainingServer.getNumberOfOnlineRegions());
-      Assert.assertFalse("We should not have regions in transition here. List is: "+
-        master.getAssignmentManager().getRegionStates().getRegionsInTransition(),
-        master.getAssignmentManager().getRegionStates().isRegionsInTransition());
+      Assert.assertFalse("We should not have regions in transition here. List is: " +
+          master.getAssignmentManager().getRegionStates().getRegionsInTransition(),
+          master.getAssignmentManager().getRegionStates().isRegionsInTransition());
 
       // Kill a few regionservers.
       for (int aborted = 0; aborted <= 2; aborted++) {
-        HRegionServer hrs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(aborted+1);
+        HRegionServer hrs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(aborted + 1);
         hrs.abort("Aborting");
       }
 
@@ -235,8 +228,8 @@ public class TestDrainingServer {
       waitForAllRegionsOnline();
 
       Collection<HRegion> regionsAfter =
-        drainingServer.getCopyOfOnlineRegionsSortedBySize().values();
-      LOG.info("Regions of drained server are: "+ regionsAfter );
+          drainingServer.getCopyOfOnlineRegionsSortedBySize().values();
+      LOG.info("Regions of drained server are: " + regionsAfter);
 
       Assert.assertEquals("Test conditions are not met: regions were" +
         " created/deleted during the test. ",
@@ -267,19 +260,37 @@ public class TestDrainingServer {
     }
   }
 
-  private void waitForAllRegionsOnline() {
+  private static void waitForAllRegionsOnline() throws Exception {
     // Wait for regions to come back on line again.
-    while (!isAllRegionsOnline()) {
-    }
 
-    while (TEST_UTIL.getMiniHBaseCluster().getMaster().
-      getAssignmentManager().getRegionStates().isRegionsInTransition()) {
+    boolean done = false;
+    while (!done) {
+      Thread.sleep(1);
+
+      // Nothing in ZK RIT for a start
+      ZKAssign.blockUntilNoRIT(TEST_UTIL.getZooKeeperWatcher());
+
+      // Then we want all the regions to be marked as available...
+      if (!isAllRegionsOnline()) continue;
+
+      // And without any work in progress on the master side
+      if (TEST_UTIL.getMiniHBaseCluster().getMaster().
+          getAssignmentManager().getRegionStates().isRegionsInTransition()) continue;
+
+      // nor on the region server side
+      done = true;
+      for (JVMClusterUtil.RegionServerThread rs :
+          TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) {
+        if (!rs.getRegionServer().getRegionsInTransitionInRS().isEmpty()) {
+          done = false;
+        }
+      }
     }
   }
 
-  private boolean isAllRegionsOnline() {
+  private static boolean isAllRegionsOnline() {
     return TEST_UTIL.getMiniHBaseCluster().countServedRegions() ==
-      (COUNT_OF_REGIONS + 2 /*catalog regions*/);
+        (COUNT_OF_REGIONS + 2 /*catalog regions*/);
   }
 
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java Thu Feb 14 12:58:12 2013
@@ -57,7 +57,7 @@ public class TestFullLogReconstruction {
     // faster failover with cluster.shutdown();fs.close() idiom
     c.setInt("ipc.client.connect.max.retries", 1);
     c.setInt("dfs.client.block.recovery.retries", 1);
-    c.setInt(HConstants.ZOOKEEPER_SESSION_TIMEOUT, 1000);
+    c.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000);
     TEST_UTIL.startMiniCluster(2);
   }
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java Thu Feb 14 12:58:12 2013
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTru
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.junit.experimental.categories.Category;
 
@@ -48,7 +49,7 @@ public class TestHColumnDescriptor {
     boolean inmemory = hcd.isInMemory();
     hcd.setScope(v);
     hcd.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
-    hcd.setBloomFilterType(StoreFile.BloomType.ROW);
+    hcd.setBloomFilterType(BloomType.ROW);
     hcd.setCompressionType(Algorithm.SNAPPY);
 
 
@@ -65,7 +66,7 @@ public class TestHColumnDescriptor {
     assertEquals(hcd.getScope(), deserializedHcd.getScope());
     assertTrue(deserializedHcd.getCompressionType().equals(Compression.Algorithm.SNAPPY));
     assertTrue(deserializedHcd.getDataBlockEncoding().equals(DataBlockEncoding.FAST_DIFF));
-    assertTrue(deserializedHcd.getBloomFilterType().equals(StoreFile.BloomType.ROW));
+    assertTrue(deserializedHcd.getBloomFilterType().equals(BloomType.ROW));
   }
 
   @Test
@@ -78,4 +79,18 @@ public class TestHColumnDescriptor {
       assertEquals("Family name can not be empty", e.getLocalizedMessage());
     }
   }
+
+  /**
+   * Test that we add and remove strings from configuration properly.
+   */
+  @Test
+  public void testAddGetRemoveConfiguration() throws Exception {
+    HColumnDescriptor desc = new HColumnDescriptor("foo");
+    String key = "Some";
+    String value = "value";
+    desc.setConfiguration(key, value);
+    assertEquals(value, desc.getConfigurationValue(key));
+    desc.removeConfiguration(key);
+    assertEquals(null, desc.getConfigurationValue(key));
+  }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java Thu Feb 14 12:58:12 2013
@@ -46,13 +46,16 @@ public class TestHRegionLocation {
       hsa1.getHostname(), hsa1.getPort());
     assertNotSame(hrl1, hrl3);
     // They are equal because they have same location even though they are
-    // carrying different regions.
+    // carrying different regions or timestamp.
     assertTrue(hrl1.equals(hrl3));
     ServerName hsa2 = new ServerName("localhost", 12345, -1L);
     HRegionLocation hrl4 = new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
         hsa2.getHostname(), hsa2.getPort());
     // These have same HRI but different locations so should be different.
     assertFalse(hrl3.equals(hrl4));
+    HRegionLocation hrl5 = new HRegionLocation(hrl4.getRegionInfo(),
+        hrl4.getHostname(), hrl4.getPort(), hrl4.getSeqNum() + 1);
+    assertTrue(hrl4.equals(hrl5));
   }
 
   @Test

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java Thu Feb 14 12:58:12 2013
@@ -28,6 +28,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -72,6 +73,43 @@ public class TestHTableDescriptor {
   }
 
   /**
+   * Test cps in the table description
+   * @throws Exception
+   */
+  @Test
+  public void testSetListRemoveCP() throws Exception {
+    HTableDescriptor desc = new HTableDescriptor("testGetSetRemoveCP");
+    // simple CP
+    String className1 = BaseRegionObserver.class.getName();
+    String className2 = SampleRegionWALObserver.class.getName();
+    // Check that any coprocessor is present.
+    assertTrue(desc.getCoprocessors().size() == 0);
+
+    // Add the 1 coprocessor and check if present.
+    desc.addCoprocessor(className1);
+    assertTrue(desc.getCoprocessors().size() == 1);
+    assertTrue(desc.getCoprocessors().contains(className1));
+
+    // Add the 2nd coprocessor and check if present.
+    // remove it and check that it is gone
+    desc.addCoprocessor(className2);
+    assertTrue(desc.getCoprocessors().size() == 2);
+    assertTrue(desc.getCoprocessors().contains(className2));
+
+    // Remove one and check
+    desc.removeCoprocessor(className1);
+    assertTrue(desc.getCoprocessors().size() == 1);
+    assertFalse(desc.getCoprocessors().contains(className1));
+    assertTrue(desc.getCoprocessors().contains(className2));
+
+    // Remove the last and check
+    desc.removeCoprocessor(className2);
+    assertTrue(desc.getCoprocessors().size() == 0);
+    assertFalse(desc.getCoprocessors().contains(className1));
+    assertFalse(desc.getCoprocessors().contains(className2));
+  }
+
+  /**
    * Test that we add and remove strings from settings properly.
    * @throws Exception
    */
@@ -123,4 +161,40 @@ public class TestHTableDescriptor {
       assertFalse(Pattern.matches(HTableDescriptor.VALID_USER_TABLE_REGEX, tn));
     }
   }
+
+    /**
+   * Test default value handling for maxFileSize
+   */
+  @Test
+  public void testGetMaxFileSize() {
+    HTableDescriptor desc = new HTableDescriptor("table");
+    assertEquals(-1, desc.getMaxFileSize());
+    desc.setMaxFileSize(1111L);
+    assertEquals(1111L, desc.getMaxFileSize());
+  }
+
+  /**
+   * Test default value handling for memStoreFlushSize
+   */
+  @Test
+  public void testGetMemStoreFlushSize() {
+    HTableDescriptor desc = new HTableDescriptor("table");
+    assertEquals(-1, desc.getMemStoreFlushSize());
+    desc.setMemStoreFlushSize(1111L);
+    assertEquals(1111L, desc.getMemStoreFlushSize());
+  }
+
+  /**
+   * Test that we add and remove strings from configuration properly.
+   */
+  @Test
+  public void testAddGetRemoveConfiguration() throws Exception {
+    HTableDescriptor desc = new HTableDescriptor("table");
+    String key = "Some";
+    String value = "value";
+    desc.setConfiguration(key, value);
+    assertEquals(value, desc.getConfigurationValue(key));
+    desc.removeConfiguration(key);
+    assertEquals(null, desc.getConfigurationValue(key));
+  }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java Thu Feb 14 12:58:12 2013
@@ -22,12 +22,7 @@ import static org.junit.Assert.*;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.zookeeper.KeeperException;
 
 import org.junit.Test;
@@ -40,46 +35,28 @@ public class TestLocalHBaseCluster {
   /**
    * Check that we can start a local HBase cluster specifying a custom master
    * and regionserver class and then cast back to those classes; also that
-   * the cluster will launch and terminate cleanly. See HBASE-6011.
+   * the cluster will launch and terminate cleanly. See HBASE-6011. Uses the
+   * HBaseTestingUtility facilities for creating a LocalHBaseCluster with
+   * custom master and regionserver classes.
    */
   @Test
   public void testLocalHBaseCluster() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    MiniZooKeeperCluster zkCluster = TEST_UTIL.startMiniZKCluster();
-    conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.toString(zkCluster.getClientPort()));
-    LocalHBaseCluster cluster = new LocalHBaseCluster(conf, 1, 1, MyHMaster.class,
-      MyHRegionServer.class);
+    TEST_UTIL.startMiniCluster(1, 1, null, MyHMaster.class, MyHRegionServer.class);
     // Can we cast back to our master class?
     try {
-      ((MyHMaster)cluster.getMaster(0)).setZKCluster(zkCluster);
+      int val = ((MyHMaster)TEST_UTIL.getHBaseCluster().getMaster(0)).echo(42);
+      assertEquals(42, val);
     } catch (ClassCastException e) {
       fail("Could not cast master to our class");
     }
     // Can we cast back to our regionserver class?
     try {
-      ((MyHRegionServer)cluster.getRegionServer(0)).echo(42);
+      int val = ((MyHRegionServer)TEST_UTIL.getHBaseCluster().getRegionServer(0)).echo(42);
+      assertEquals(42, val);
     } catch (ClassCastException e) {
       fail("Could not cast regionserver to our class");
     }
-    // Does the cluster start successfully?
-    try {
-      cluster.startup();
-      waitForClusterUp(conf);
-    } catch (IOException e) {
-      fail("LocalHBaseCluster did not start successfully");
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  private void waitForClusterUp(Configuration conf) throws IOException {
-    HTable t = new HTable(conf, HConstants.META_TABLE_NAME);
-    ResultScanner s = t.getScanner(new Scan());
-    while (s.next() != null) {
-      continue;
-    }
-    s.close();
-    t.close();
+    TEST_UTIL.shutdownMiniCluster();
   }
 
   /**
@@ -87,34 +64,20 @@ public class TestLocalHBaseCluster {
    * running in local mode.
    */
   public static class MyHMaster extends HMaster {
-    private MiniZooKeeperCluster zkcluster = null;
-
     public MyHMaster(Configuration conf) throws IOException, KeeperException,
         InterruptedException {
       super(conf);
     }
 
-    @Override
-    public void run() {
-      super.run();
-      if (this.zkcluster != null) {
-        try {
-          this.zkcluster.shutdown();
-        } catch (IOException e) {
-          e.printStackTrace();
-        }
-      }
-    }
-
-    void setZKCluster(final MiniZooKeeperCluster zkcluster) {
-      this.zkcluster = zkcluster;
+    public int echo(int val) {
+      return val;
     }
   }
 
   /**
    * A private regionserver class with a dummy method for testing casts
    */
-  public static class MyHRegionServer extends HRegionServer {
+  public static class MyHRegionServer extends MiniHBaseCluster.MiniHBaseClusterRegionServer {
 
     public MyHRegionServer(Configuration conf) throws IOException,
         InterruptedException {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java Thu Feb 14 12:58:12 2013
@@ -134,7 +134,7 @@ public class TestMultiVersions {
     final HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(contents));
     this.admin.createTable(desc);
-    Put put = new Put(row, timestamp1, null);
+    Put put = new Put(row, timestamp1);
     put.add(contents, contents, value1);
     HTable table = new HTable(UTIL.getConfiguration(), tableName);
     table.put(put);
@@ -147,7 +147,7 @@ public class TestMultiVersions {
     // is tied to an HConnection that has since gone stale.
     table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
     // Overwrite previous value
-    put = new Put(row, timestamp2, null);
+    put = new Put(row, timestamp2);
     put.add(contents, contents, value2);
     table.put(put);
     // Now verify that getRow(row, column, latest) works
@@ -218,7 +218,7 @@ public class TestMultiVersions {
     // Insert data
     for (int i = 0; i < locations.size(); i++) {
       for (int j = 0; j < timestamp.length; j++) {
-        Put put = new Put(rows[i], timestamp[j], null);
+        Put put = new Put(rows[i], timestamp[j]);
         put.add(HConstants.CATALOG_FAMILY, null, timestamp[j],
             Bytes.toBytes(timestamp[j]));
         table.put(put);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java Thu Feb 14 12:58:12 2013
@@ -33,7 +33,6 @@ import java.util.NavigableSet;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.RowLock;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -259,10 +258,8 @@ public class TestSerialization {
 
     long ts = System.currentTimeMillis();
     int maxVersions = 2;
-    long lockid = 5;
-    RowLock rowLock = new RowLock(lockid);
 
-    Get get = new Get(row, rowLock);
+    Get get = new Get(row);
     get.addColumn(fam, qf1);
     get.setTimeRange(ts, ts+1);
     get.setMaxVersions(maxVersions);
@@ -284,7 +281,6 @@ public class TestSerialization {
       }
     }
 
-    assertEquals(get.getLockId(), desGet.getLockId());
     assertEquals(get.getMaxVersions(), desGet.getMaxVersions());
     TimeRange tr = get.getTimeRange();
     TimeRange desTr = desGet.getTimeRange();
@@ -521,17 +517,6 @@ public class TestSerialization {
   }
   */
 
-  @Test public void testTimeRange() throws Exception{
-    TimeRange tr = new TimeRange(0,5);
-    byte [] mb = Writables.getBytes(tr);
-    TimeRange deserializedTr =
-      (TimeRange)Writables.getWritable(mb, new TimeRange());
-
-    assertEquals(tr.getMax(), deserializedTr.getMax());
-    assertEquals(tr.getMin(), deserializedTr.getMin());
-
-  }
-
   protected static final int MAXVERSIONS = 3;
   protected final static byte [] fam1 = Bytes.toBytes("colfamily1");
   protected final static byte [] fam2 = Bytes.toBytes("colfamily2");

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Thu Feb 14 12:58:12 2013
@@ -18,8 +18,8 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
@@ -27,9 +27,8 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.util.Map;
 import java.util.List;
-
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -58,6 +57,8 @@ import org.apache.zookeeper.KeeperExcept
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -83,7 +84,7 @@ public class TestZooKeeper {
     Configuration conf = TEST_UTIL.getConfiguration();
     TEST_UTIL.startMiniZKCluster();
     conf.setBoolean("dfs.support.append", true);
-    conf.setInt(HConstants.ZOOKEEPER_SESSION_TIMEOUT, 1000);
+    conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000);
     conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, MockLoadBalancer.class,
         LoadBalancer.class);
     TEST_UTIL.startMiniCluster(2);
@@ -122,7 +123,7 @@ public class TestZooKeeper {
    * @throws InterruptedException
    */
   // fails frequently, disabled for now, see HBASE-6406
-  // @Test
+  //@Test
   public void testClientSessionExpired() throws Exception {
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
 
@@ -230,7 +231,7 @@ public class TestZooKeeper {
    * Make sure we can use the cluster
    * @throws Exception
    */
-  public void testSanity() throws Exception{
+  private void testSanity() throws Exception{
     HBaseAdmin admin =
       new HBaseAdmin(TEST_UTIL.getConfiguration());
     String tableName = "test"+System.currentTimeMillis();
@@ -357,6 +358,10 @@ public class TestZooKeeper {
     ZooKeeperWatcher zk2 = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
       "testMasterAddressManagerFromZK", null);
 
+    // Save the previous ACL
+    Stat s =  new Stat();
+    List<ACL> oldACL =  zk.getACL("/", s);
+
     // I set this acl after the attempted creation of the cluster home node.
     // Add retries in case of retryable zk exceptions.
     while (true) {
@@ -396,6 +401,12 @@ public class TestZooKeeper {
     }
     zk.close();
     ZKUtil.createAndFailSilent(zk2, aclZnode);
+
+    // Restore the ACL
+    ZooKeeper zk3 = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
+    zk3.addAuthInfo("digest", "hbase:rox".getBytes());
+    zk3.setACL("/", oldACL, -1);
+    zk3.close();
  }
 
   /**
@@ -464,8 +475,8 @@ public class TestZooKeeper {
     ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 
-    Put p = null;
-    int numberOfPuts = 0;
+    Put p;
+    int numberOfPuts;
     for (numberOfPuts = 0; numberOfPuts < 6; numberOfPuts++) {
       p = new Put(Bytes.toBytes(numberOfPuts));
       p.add(Bytes.toBytes("col"), Bytes.toBytes("ql"), Bytes.toBytes("value" + numberOfPuts));

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java Thu Feb 14 12:58:12 2013
@@ -87,7 +87,7 @@ public class TimestampTestBase extends H
 
     Delete delete = new Delete(ROW);
     delete.deleteColumns(FAMILY_NAME, QUALIFIER_NAME, T2);
-    incommon.delete(delete, null, true);
+    incommon.delete(delete, true);
 
     // Should only be current value in set.  Assert this is so
     assertOnlyLatest(incommon, HConstants.LATEST_TIMESTAMP);
@@ -241,7 +241,7 @@ public class TimestampTestBase extends H
   public static void put(final Incommon loader, final byte [] bytes,
     final long ts)
   throws IOException {
-    Put put = new Put(ROW, ts, null);
+    Put put = new Put(ROW, ts);
     put.setWriteToWAL(false);
     put.add(FAMILY_NAME, QUALIFIER_NAME, bytes);
     loader.put(put);
@@ -265,9 +265,9 @@ public class TimestampTestBase extends H
       final long ts)
   throws IOException {
     Delete delete = ts == HConstants.LATEST_TIMESTAMP?
-      new Delete(ROW): new Delete(ROW, ts, null);
+      new Delete(ROW): new Delete(ROW, ts);
     delete.deleteColumn(FAMILY_NAME, QUALIFIER_NAME, ts);
-    loader.delete(delete, null, true);
+    loader.delete(delete, true);
   }
 
   public static Result get(final Incommon loader) throws IOException {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java Thu Feb 14 12:58:12 2013
@@ -37,12 +37,11 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
+import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveTestingUtil;
@@ -86,6 +85,10 @@ public class TestHFileArchiving {
     conf.setInt("hbase.hregion.memstore.flush.size", 25000);
     // disable major compactions
     conf.setInt(HConstants.MAJOR_COMPACTION_PERIOD, 0);
+
+    // prevent aggressive region split
+    conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
+      ConstantSizeRegionSplitPolicy.class.getName());
   }
 
   @Before
@@ -355,7 +358,7 @@ public class TestHFileArchiving {
 
         try {
           // Try to archive the file
-          HFileArchiver.archiveRegion(fs, rootDir,
+          HFileArchiver.archiveRegion(conf, fs, rootDir,
               sourceRegionDir.getParent(), sourceRegionDir);
 
           // The archiver succeded, the file is no longer in the original location

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java Thu Feb 14 12:58:12 2013
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTru
 
 import java.io.IOException;
 import java.net.ConnectException;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -115,7 +116,7 @@ public class TestCatalogTracker {
   private CatalogTracker constructAndStartCatalogTracker(final HConnection c)
   throws IOException, InterruptedException {
     CatalogTracker ct = new CatalogTracker(this.watcher, UTIL.getConfiguration(),
-      c, this.abortable, 0);
+      c, this.abortable);
     ct.start();
     return ct;
   }
@@ -177,7 +178,7 @@ public class TestCatalogTracker {
       // Join the thread... should exit shortly.
       t.join();
     } finally {
-      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
+      HConnectionManager.deleteConnection(UTIL.getConfiguration());
     }
   }
 
@@ -230,20 +231,20 @@ public class TestCatalogTracker {
         // So, do this in a thread and then reset meta location to break it out
         // of its wait after a bit of time.
         final AtomicBoolean metaSet = new AtomicBoolean(false);
+        final CountDownLatch latch = new CountDownLatch(1);
         Thread t = new Thread() {
           @Override
           public void run() {
             try {
-              metaSet.set(ct.waitForMetaServerConnectionDefault() !=  null);
-            } catch (NotAllMetaRegionsOnlineException e) {
-              throw new RuntimeException(e);
-            } catch (IOException e) {
+              latch.countDown();
+              metaSet.set(ct.waitForMeta(100000) !=  null);
+            } catch (Exception e) {
               throw new RuntimeException(e);
             }
           }
         };
         t.start();
-        while(!t.isAlive()) Threads.sleep(1);
+        latch.await();
         Threads.sleep(1);
         // Now reset the meta as though it were redeployed.
         ct.setMetaLocation(SN);
@@ -257,7 +258,7 @@ public class TestCatalogTracker {
       }
     } finally {
       // Clear out our doctored connection or could mess up subsequent tests.
-      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
+      HConnectionManager.deleteConnection(UTIL.getConfiguration());
     }
   }
 
@@ -284,7 +285,7 @@ public class TestCatalogTracker {
       }
     } finally {
       // Clear out our doctored connection or could mess up subsequent tests.
-      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
+      HConnectionManager.deleteConnection(UTIL.getConfiguration());
     }
   }
 
@@ -370,7 +371,7 @@ public class TestCatalogTracker {
       final CatalogTracker ct = constructAndStartCatalogTracker(connection);
       ct.waitForMeta(100);
     } finally {
-      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
+      HConnectionManager.deleteConnection(UTIL.getConfiguration());
     }
   }
 
@@ -453,7 +454,7 @@ public class TestCatalogTracker {
       // been assigned).
       String node = ct.getMetaNodeTracker().getNode();
       ZKUtil.createAndFailSilent(this.watcher, node);
-      MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, SN);
+      MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, SN, 0);
       ZKUtil.deleteNode(this.watcher, node);
       // Go get the new meta location. waitForMeta gets and verifies meta.
       Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
@@ -462,7 +463,7 @@ public class TestCatalogTracker {
       // Now meta is available.
       Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
     } finally {
-      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
+      HConnectionManager.deleteConnection(UTIL.getConfiguration());
     }
   }
 
@@ -477,7 +478,7 @@ public class TestCatalogTracker {
    * {@link HConnection#getAdmin(String, int)} is called, returns the passed
    * {@link ClientProtocol} instance when {@link HConnection#getClient(String, int)}
    * is called (Be sure call
-   * {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration, boolean)}
+   * {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration)}
    * when done with this mocked Connection.
    * @throws IOException
    */

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTrackerOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTrackerOnCluster.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTrackerOnCluster.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTrackerOnCluster.java Thu Feb 14 12:58:12 2013
@@ -46,6 +46,9 @@ public class TestCatalogTrackerOnCluster
     UTIL.startMiniCluster();
     // Shutdown hbase.
     UTIL.shutdownMiniHBaseCluster();
+    // Give the various ZKWatchers some time to settle their affairs.
+    Thread.sleep(1000);
+
     // Mess with the root location in the running zk.  Set it to be nonsense.
     ZooKeeperWatcher zookeeper = new ZooKeeperWatcher(UTIL.getConfiguration(),
       "Bad Root Location Writer", new Abortable() {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java Thu Feb 14 12:58:12 2013
@@ -109,6 +109,12 @@ public class TestMetaMigrationConverting
     }
     doFsCommand(shell,
       new String [] {"-put", untar.toURI().toString(), hbaseRootDir.toString()});
+
+    //windows fix: tgz file has .META. directory renamed as -META- since the original is an illegal
+    //name under windows. So we rename it back. See src/test/data//TestMetaMigrationConvertingToPB.README and
+    //https://issues.apache.org/jira/browse/HBASE-6821
+    doFsCommand(shell, new String [] {"-mv", new Path(hbaseRootDir, "-META-").toString(),
+      new Path(hbaseRootDir, ".META.").toString()});
     // See whats in minihdfs.
     doFsCommand(shell, new String [] {"-lsr", "/"});
     TEST_UTIL.startMiniHBaseCluster(1, 1);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java Thu Feb 14 12:58:12 2013
@@ -195,7 +195,7 @@ public class TestMetaReaderEditorNoClust
         when(connection).getClient(Mockito.anyString(), Mockito.anyInt());
 
       // Now start up the catalogtracker with our doctored Connection.
-      ct = new CatalogTracker(zkw, null, connection, ABORTABLE, 0);
+      ct = new CatalogTracker(zkw, null, connection, ABORTABLE);
       ct.start();
       // Scan meta for user tables and verify we got back expected answer.
       NavigableMap<HRegionInfo, Result> hris = MetaReader.getServerUserRegions(ct, sn);
@@ -208,7 +208,7 @@ public class TestMetaReaderEditorNoClust
         scan((RpcController)Mockito.any(), (ScanRequest)Mockito.any());
     } finally {
       if (ct != null) ct.stop();
-      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
+      HConnectionManager.deleteConnection(UTIL.getConfiguration());
       zkw.close();
     }
   }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Thu Feb 14 12:58:12 2013
@@ -32,6 +32,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -189,7 +190,8 @@ public class TestAdmin {
 
     // Now make it so at least the table exists and then do tests against a
     // nonexistent column family -- see if we get right exceptions.
-    final String tableName = "t";
+    final String tableName =
+        "testDeleteEditUnknownColumnFamilyAndOrTable" + System.currentTimeMillis();
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor("cf"));
     this.admin.createTable(htd);
@@ -200,7 +202,8 @@ public class TestAdmin {
       } catch (IOException e) {
         exception = e;
       }
-      assertTrue(exception instanceof InvalidFamilyOperationException);
+      assertTrue("found=" + exception.getClass().getName(),
+          exception instanceof InvalidFamilyOperationException);
 
       exception = null;
       try {
@@ -208,7 +211,8 @@ public class TestAdmin {
       } catch (IOException e) {
         exception = e;
       }
-      assertTrue(exception instanceof InvalidFamilyOperationException);
+      assertTrue("found=" + exception.getClass().getName(),
+          exception instanceof InvalidFamilyOperationException);
     } finally {
       this.admin.disableTable(tableName);
       this.admin.deleteTable(tableName);
@@ -341,7 +345,7 @@ public class TestAdmin {
 
   @Test
   public void testHColumnValidName() {
-       boolean exceptionThrown = false;
+       boolean exceptionThrown;
        try {
          new HColumnDescriptor("\\test\\abc");
        } catch(IllegalArgumentException iae) {
@@ -372,13 +376,16 @@ public class TestAdmin {
     assertTrue(htd.equals(copy));
     // Now amend the copy. Introduce differences.
     long newFlushSize = htd.getMemStoreFlushSize() / 2;
+    if (newFlushSize <=0) {
+      newFlushSize = HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE / 2;
+    }
     copy.setMemStoreFlushSize(newFlushSize);
     final String key = "anyoldkey";
     assertTrue(htd.getValue(key) == null);
     copy.setValue(key, key);
     boolean expectedException = false;
     try {
-      modifyTable(tableName, copy);
+      admin.modifyTable(tableName, copy);
     } catch (TableNotDisabledException re) {
       expectedException = true;
     }
@@ -390,7 +397,6 @@ public class TestAdmin {
     assertEquals(key, modifiedHtd.getValue(key));
 
     // Now work on column family changes.
-    htd = this.admin.getTableDescriptor(tableName);
     int countOfFamilies = modifiedHtd.getFamilies().size();
     assertTrue(countOfFamilies > 0);
     HColumnDescriptor hcd = modifiedHtd.getFamilies().iterator().next();
@@ -412,7 +418,6 @@ public class TestAdmin {
     // Try adding a column
     assertFalse(this.admin.isTableDisabled(tableName));
     final String xtracolName = "xtracol";
-    htd = this.admin.getTableDescriptor(tableName);
     HColumnDescriptor xtracol = new HColumnDescriptor(xtracolName);
     xtracol.setValue(xtracolName, xtracolName);
     expectedException = false;
@@ -460,42 +465,24 @@ public class TestAdmin {
     assertTrue(htd.equals(copy));
     // Now amend the copy. Introduce differences.
     long newFlushSize = htd.getMemStoreFlushSize() / 2;
+    if (newFlushSize <=0) {
+      newFlushSize = HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE / 2;
+    }
     copy.setMemStoreFlushSize(newFlushSize);
     final String key = "anyoldkey";
     assertTrue(htd.getValue(key) == null);
     copy.setValue(key, key);
     boolean expectedException = false;
     try {
-      modifyTable(tableName, copy);
+      admin.modifyTable(tableName, copy);
     } catch (TableNotDisabledException re) {
       expectedException = true;
     }
     assertTrue("Online schema update should not happen.", expectedException);
-  }
 
-  /**
-   * Modify table is async so wait on completion of the table operation in master.
-   * @param tableName
-   * @param htd
-   * @throws IOException
-   */
-  private void modifyTable(final byte [] tableName, final HTableDescriptor htd)
-  throws IOException {
-    MasterServices services = TEST_UTIL.getMiniHBaseCluster().getMaster();
-    ExecutorService executor = services.getExecutorService();
-    AtomicBoolean done = new AtomicBoolean(false);
-    executor.registerListener(EventType.C_M_MODIFY_TABLE, new DoneListener(done));
-    this.admin.modifyTable(tableName, htd);
-    while (!done.get()) {
-      synchronized (done) {
-        try {
-          done.wait(100);
-        } catch (InterruptedException e) {
-          e.printStackTrace();
-        }
-      }
-    }
-    executor.unregisterListener(EventType.C_M_MODIFY_TABLE);
+    // Reset the value for the other tests
+    TEST_UTIL.getMiniHBaseCluster().getMaster().getConfiguration().setBoolean(
+        "hbase.online.schema.update.enable", true);
   }
 
   /**
@@ -524,6 +511,7 @@ public class TestAdmin {
     }
   }
 
+  @SuppressWarnings("deprecation")
   protected void verifyRoundRobinDistribution(HTable ht, int expectedRegions) throws IOException {
     int numRS = ht.getConnection().getCurrentNrHRS();
     Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
@@ -750,7 +738,7 @@ public class TestAdmin {
   @Test
   public void testTableExist() throws IOException {
     final byte [] table = Bytes.toBytes("testTableExist");
-    boolean exist = false;
+    boolean exist;
     exist = this.admin.tableExists(table);
     assertEquals(false, exist);
     TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
@@ -806,8 +794,7 @@ public class TestAdmin {
         HConstants.META_TABLE_NAME);
     List<HRegionInfo> regionInfos = admin.getTableRegions(tableName);
     Map<String, Integer> serverMap = new HashMap<String, Integer>();
-    for (int i = 0, j = regionInfos.size(); i < j; i++) {
-      HRegionInfo hri = regionInfos.get(i);
+    for (HRegionInfo hri : regionInfos) {
       Get get = new Get(hri.getRegionName());
       Result result = metaTable.get(get);
       String server = Bytes.toString(result.getValue(HConstants.CATALOG_FAMILY,
@@ -872,125 +859,125 @@ public class TestAdmin {
     assertFalse(admin.tableExists(tableName));
     final HTable table = TEST_UTIL.createTable(tableName, familyNames,
       numVersions, blockSize);
-    try {
-      int rowCount = 0;
-      byte[] q = new byte[0];
-
-      // insert rows into column families. The number of rows that have values
-      // in a specific column family is decided by rowCounts[familyIndex]
-      for (int index = 0; index < familyNames.length; index++) {
-        ArrayList<Put> puts = new ArrayList<Put>(rowCounts[index]);
-        for (int i = 0; i < rowCounts[index]; i++) {
-          byte[] k = Bytes.toBytes(i);
-          Put put = new Put(k);
-          put.add(familyNames[index], q, k);
-          puts.add(put);
-        }
-        table.put(puts);
 
-        if ( rowCount < rowCounts[index] ) {
-          rowCount = rowCounts[index];
-        }
-      }
+    int rowCount = 0;
+    byte[] q = new byte[0];
 
-      // get the initial layout (should just be one region)
-      Map<HRegionInfo, ServerName> m = table.getRegionLocations();
-      System.out.println("Initial regions (" + m.size() + "): " + m);
-      assertTrue(m.size() == 1);
-
-      // Verify row count
-      Scan scan = new Scan();
-      ResultScanner scanner = table.getScanner(scan);
-      int rows = 0;
-      for(@SuppressWarnings("unused") Result result : scanner) {
-        rows++;
+    // insert rows into column families. The number of rows that have values
+    // in a specific column family is decided by rowCounts[familyIndex]
+    for (int index = 0; index < familyNames.length; index++) {
+      ArrayList<Put> puts = new ArrayList<Put>(rowCounts[index]);
+      for (int i = 0; i < rowCounts[index]; i++) {
+        byte[] k = Bytes.toBytes(i);
+        Put put = new Put(k);
+        put.add(familyNames[index], q, k);
+        puts.add(put);
+      }
+      table.put(puts);
+
+      if ( rowCount < rowCounts[index] ) {
+        rowCount = rowCounts[index];
       }
-      scanner.close();
-      assertEquals(rowCount, rows);
-
-      // Have an outstanding scan going on to make sure we can scan over splits.
-      scan = new Scan();
-      scanner = table.getScanner(scan);
-      // Scan first row so we are into first region before split happens.
-      scanner.next();
+    }
 
-      final AtomicInteger count = new AtomicInteger(0);
-      Thread t = new Thread("CheckForSplit") {
-        public void run() {
-          for (int i = 0; i < 20; i++) {
-            try {
-              sleep(1000);
-            } catch (InterruptedException e) {
-              continue;
-            }
-            // check again    table = new HTable(conf, tableName);
-            Map<HRegionInfo, ServerName> regions = null;
-            try {
-              regions = table.getRegionLocations();
-            } catch (IOException e) {
-              e.printStackTrace();
-            }
-            if (regions == null) continue;
-            count.set(regions.size());
-            if (count.get() >= 2) break;
-            LOG.debug("Cycle waiting on split");
+    // get the initial layout (should just be one region)
+    Map<HRegionInfo, ServerName> m = table.getRegionLocations();
+    System.out.println("Initial regions (" + m.size() + "): " + m);
+    assertTrue(m.size() == 1);
+
+    // Verify row count
+    Scan scan = new Scan();
+    ResultScanner scanner = table.getScanner(scan);
+    int rows = 0;
+    for(@SuppressWarnings("unused") Result result : scanner) {
+      rows++;
+    }
+    scanner.close();
+    assertEquals(rowCount, rows);
+
+    // Have an outstanding scan going on to make sure we can scan over splits.
+    scan = new Scan();
+    scanner = table.getScanner(scan);
+    // Scan first row so we are into first region before split happens.
+    scanner.next();
+
+    final AtomicInteger count = new AtomicInteger(0);
+    Thread t = new Thread("CheckForSplit") {
+      public void run() {
+        for (int i = 0; i < 20; i++) {
+          try {
+            sleep(1000);
+          } catch (InterruptedException e) {
+            continue;
           }
-        }
-      };
-      t.start();
-      // Split the table
-      this.admin.split(tableName, splitPoint);
-      t.join();
-
-      // Verify row count
-      rows = 1; // We counted one row above.
-      for (@SuppressWarnings("unused") Result result : scanner) {
-        rows++;
-        if (rows > rowCount) {
-          scanner.close();
-          assertTrue("Scanned more than expected (" + rowCount + ")", false);
+          // check again    table = new HTable(conf, tableName);
+          Map<HRegionInfo, ServerName> regions = null;
+          try {
+            regions = table.getRegionLocations();
+          } catch (IOException e) {
+            e.printStackTrace();
+          }
+          if (regions == null) continue;
+          count.set(regions.size());
+          if (count.get() >= 2) break;
+          LOG.debug("Cycle waiting on split");
         }
       }
-      scanner.close();
-      assertEquals(rowCount, rows);
-
-      Map<HRegionInfo, ServerName> regions = null;
-      try {
-        regions = table.getRegionLocations();
-      } catch (IOException e) {
-        e.printStackTrace();
+    };
+    t.start();
+    // Split the table
+    this.admin.split(tableName, splitPoint);
+    t.join();
+
+    // Verify row count
+    rows = 1; // We counted one row above.
+    for (@SuppressWarnings("unused") Result result : scanner) {
+      rows++;
+      if (rows > rowCount) {
+        scanner.close();
+        assertTrue("Scanned more than expected (" + rowCount + ")", false);
       }
-      assertEquals(2, regions.size());
-      HRegionInfo[] r = regions.keySet().toArray(new HRegionInfo[0]);
-      if (splitPoint != null) {
-        // make sure the split point matches our explicit configuration
-        assertEquals(Bytes.toString(splitPoint),
-            Bytes.toString(r[0].getEndKey()));
-        assertEquals(Bytes.toString(splitPoint),
-            Bytes.toString(r[1].getStartKey()));
-        LOG.debug("Properly split on " + Bytes.toString(splitPoint));
-      } else {
-        if (familyNames.length > 1) {
-          int splitKey = Bytes.toInt(r[0].getEndKey());
-          // check if splitKey is based on the largest column family
-          // in terms of it store size
-          int deltaForLargestFamily = Math.abs(rowCount/2 - splitKey);
-          for (int index = 0; index < familyNames.length; index++) {
-            int delta = Math.abs(rowCounts[index]/2 - splitKey);
-            assertTrue(delta >= deltaForLargestFamily);
-          }
+    }
+    scanner.close();
+    assertEquals(rowCount, rows);
+
+    Map<HRegionInfo, ServerName> regions = null;
+    try {
+      regions = table.getRegionLocations();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    assertEquals(2, regions.size());
+    Set<HRegionInfo> hRegionInfos = regions.keySet();
+    HRegionInfo[] r = hRegionInfos.toArray(new HRegionInfo[hRegionInfos.size()]);
+    if (splitPoint != null) {
+      // make sure the split point matches our explicit configuration
+      assertEquals(Bytes.toString(splitPoint),
+          Bytes.toString(r[0].getEndKey()));
+      assertEquals(Bytes.toString(splitPoint),
+          Bytes.toString(r[1].getStartKey()));
+      LOG.debug("Properly split on " + Bytes.toString(splitPoint));
+    } else {
+      if (familyNames.length > 1) {
+        int splitKey = Bytes.toInt(r[0].getEndKey());
+        // check if splitKey is based on the largest column family
+        // in terms of it store size
+        int deltaForLargestFamily = Math.abs(rowCount/2 - splitKey);
+        for (int index = 0; index < familyNames.length; index++) {
+          int delta = Math.abs(rowCounts[index]/2 - splitKey);
+          assertTrue(delta >= deltaForLargestFamily);
         }
       }
-    } finally {
-      TEST_UTIL.deleteTable(tableName);
-      table.close();
     }
+    TEST_UTIL.deleteTable(tableName);
+    table.close();
   }
 
   /**
    * HADOOP-2156
    * @throws IOException
    */
+  @SuppressWarnings("deprecation")
   @Test (expected=IllegalArgumentException.class)
   public void testEmptyHTableDescriptor() throws IOException {
     this.admin.createTable(new HTableDescriptor());
@@ -1159,11 +1146,11 @@ public class TestAdmin {
         HConstants.ROOT_TABLE_NAME,
         HConstants.META_TABLE_NAME
     };
-    for (int i = 0; i < illegalNames.length; i++) {
+    for (byte[] illegalName : illegalNames) {
       try {
-        new HTableDescriptor(illegalNames[i]);
+        new HTableDescriptor(illegalName);
         throw new IOException("Did not detect '" +
-          Bytes.toString(illegalNames[i]) + "' as an illegal user table name");
+            Bytes.toString(illegalName) + "' as an illegal user table name");
       } catch (IllegalArgumentException e) {
         // expected
       }
@@ -1226,6 +1213,8 @@ public class TestAdmin {
   public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
     new HTable(TEST_UTIL.getConfiguration(),"testTableNotFoundExceptionWithoutAnyTables");
   }
+
+
   @Test
   public void testShouldCloseTheRegionBasedOnTheEncodedRegionName()
       throws Exception {
@@ -1521,8 +1510,6 @@ public class TestAdmin {
     // When the META table can be opened, the region servers are running
     new HTable(
       TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME).close();
-    HRegionServer regionServer = TEST_UTIL.getHBaseCluster()
-        .getRegionServerThreads().get(0).getRegionServer();
 
     // Create the test table and open it
     HTableDescriptor desc = new HTableDescriptor(tableName);
@@ -1530,8 +1517,7 @@ public class TestAdmin {
     admin.createTable(desc);
     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 
-    regionServer = TEST_UTIL.getRSForFirstRegionInTable(Bytes
-        .toBytes(tableName));
+    HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
     for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
       Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
       put.add(HConstants.CATALOG_FAMILY, null, value);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java Thu Feb 14 12:58:12 2013
@@ -49,8 +49,6 @@ public class TestClientTimeouts {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    RandomTimeoutRpcEngine.setProtocolEngine(conf, MasterAdminProtocol.class);
-    RandomTimeoutRpcEngine.setProtocolEngine(conf, MasterMonitorProtocol.class);
     TEST_UTIL.startMiniCluster(SLAVES);
   }
 
@@ -73,26 +71,35 @@ public class TestClientTimeouts {
     HConnection lastConnection = null;
     boolean lastFailed = false;
     int initialInvocations = RandomTimeoutRpcEngine.getNumberOfInvocations();
-    for (int i = 0; i < 5 || (lastFailed && i < 100); ++i) {
-      lastFailed = false;
-      // Ensure the HBaseAdmin uses a new connection by changing Configuration.
-      Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
-      conf.setLong(HConstants.HBASE_CLIENT_PREFETCH_LIMIT, ++lastLimit);
-      try {
-        HBaseAdmin admin = new HBaseAdmin(conf);
-        HConnection connection = admin.getConnection();
-        assertFalse(connection == lastConnection);
-        // run some admin commands
-        HBaseAdmin.checkHBaseAvailable(conf);
-        admin.setBalancerRunning(false, false);
-      } catch (MasterNotRunningException ex) {
-        // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get
-        // a MasterNotRunningException.  It's a bug if we get other exceptions.
-        lastFailed = true;
+
+    RandomTimeoutRpcEngine engine = new RandomTimeoutRpcEngine(TEST_UTIL.getConfiguration());
+    try {
+      for (int i = 0; i < 5 || (lastFailed && i < 100); ++i) {
+        lastFailed = false;
+        // Ensure the HBaseAdmin uses a new connection by changing Configuration.
+        Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
+        conf.setLong(HConstants.HBASE_CLIENT_PREFETCH_LIMIT, ++lastLimit);
+        try {
+          HBaseAdmin admin = new HBaseAdmin(conf);
+          HConnection connection = admin.getConnection();
+          assertFalse(connection == lastConnection);
+          lastConnection = connection;
+          // override the connection's rpc engine for timeout testing
+          ((HConnectionManager.HConnectionImplementation)connection).setRpcEngine(engine);
+          // run some admin commands
+          HBaseAdmin.checkHBaseAvailable(conf);
+          admin.setBalancerRunning(false, false);
+        } catch (MasterNotRunningException ex) {
+          // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get
+          // a MasterNotRunningException.  It's a bug if we get other exceptions.
+          lastFailed = true;
+        }
       }
+      // Ensure the RandomTimeoutRpcEngine is actually being used.
+      assertFalse(lastFailed);
+      assertTrue(RandomTimeoutRpcEngine.getNumberOfInvocations() > initialInvocations);
+    } finally {
+      engine.close();
     }
-    // Ensure the RandomTimeoutRpcEngine is actually being used.
-    assertFalse(lastFailed);
-    assertTrue(RandomTimeoutRpcEngine.getNumberOfInvocations() > initialInvocations);
   }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Thu Feb 14 12:58:12 2013
@@ -184,10 +184,10 @@ public class TestFromClientSide {
      p.add(FAMILY, C0, T3);
      h.put(p);
 
-     Delete d = new Delete(T1, ts+3, null);
+     Delete d = new Delete(T1, ts+3);
      h.delete(d);
 
-     d = new Delete(T1, ts+3, null);
+     d = new Delete(T1, ts+3);
      d.deleteColumns(FAMILY, C0, ts+3);
      h.delete(d);
 
@@ -285,7 +285,7 @@ public class TestFromClientSide {
      z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
 
 
-     HConnectionManager.deleteConnection(newConfig, true);
+     HConnectionManager.deleteConnection(newConfig);
      try {
        z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false);
        assertTrue("We should not have a valid connection for z2", false);
@@ -296,7 +296,7 @@ public class TestFromClientSide {
      // We expect success here.
 
 
-     HConnectionManager.deleteConnection(newConfig2, true);
+     HConnectionManager.deleteConnection(newConfig2);
      try {
        z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
        assertTrue("We should not have a valid connection for z4", false);
@@ -4420,9 +4420,9 @@ public class TestFromClientSide {
 
     // Build a SynchronousQueue that we use for thread coordination
     final SynchronousQueue<Object> queue = new SynchronousQueue<Object>();
-    List<Thread> threads = new ArrayList<Thread>(5);
+    List<Runnable> tasks = new ArrayList<Runnable>(5);
     for (int i = 0; i < 5; i++) {
-      threads.add(new Thread() {
+      tasks.add(new Runnable() {
         public void run() {
           try {
             // The thread blocks here until we decide to let it go
@@ -4431,29 +4431,27 @@ public class TestFromClientSide {
         }
       });
     }
-    // First, add two threads and make sure the pool size follows
-    pool.submit(threads.get(0));
+    // First, add two tasks and make sure the pool size follows
+    pool.submit(tasks.get(0));
     assertEquals(1, pool.getPoolSize());
-    pool.submit(threads.get(1));
+    pool.submit(tasks.get(1));
     assertEquals(2, pool.getPoolSize());
 
-    // Next, terminate those threads and then make sure the pool is still the
+    // Next, terminate those tasks and then make sure the pool is still the
     // same size
     queue.put(new Object());
-    threads.get(0).join();
     queue.put(new Object());
-    threads.get(1).join();
     assertEquals(2, pool.getPoolSize());
 
-    //ensure that ThreadPoolExecutor knows that threads are finished.
+    //ensure that ThreadPoolExecutor knows that tasks are finished.
     while (pool.getCompletedTaskCount() < 2) {
       Threads.sleep(1);
     }
     // Now let's simulate adding a RS meaning that we'll go up to three
     // concurrent threads. The pool should not grow larger than three.
-    pool.submit(threads.get(2));
-    pool.submit(threads.get(3));
-    pool.submit(threads.get(4));
+    pool.submit(tasks.get(2));
+    pool.submit(tasks.get(3));
+    pool.submit(tasks.get(4));
     assertEquals(3, pool.getPoolSize());
     queue.put(new Object());
     queue.put(new Object());

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java Thu Feb 14 12:58:12 2013
@@ -19,9 +19,12 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 
@@ -51,6 +54,14 @@ public class TestFromClientSide3 {
   private static byte[] FAMILY = Bytes.toBytes("testFamily");
   private static Random random = new Random();
   private static int SLAVES = 3;
+  private static byte [] ROW = Bytes.toBytes("testRow");
+  private static final byte[] ANOTHERROW = Bytes.toBytes("anotherrow");
+  private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
+  private static byte [] VALUE = Bytes.toBytes("testValue");
+  private final static byte[] COL_QUAL = Bytes.toBytes("f1");
+  private final static byte[] VAL_BYTES = Bytes.toBytes("v1");
+  private final static byte[] ROW_BYTES = Bytes.toBytes("r1");
+
 
   /**
    * @throws java.lang.Exception
@@ -256,4 +267,144 @@ public class TestFromClientSide3 {
         "hbase.hstore.compaction.min"));
   }
 
+  @Test
+  public void testHTableExistsMethodSingleRegionSingleGet() throws Exception {
+
+    // Test with a single region table.
+
+    HTable table = TEST_UTIL.createTable(
+      Bytes.toBytes("testHTableExistsMethodSingleRegionSingleGet"), new byte[][] { FAMILY });
+
+    Put put = new Put(ROW);
+    put.add(FAMILY, QUALIFIER, VALUE);
+
+    Get get = new Get(ROW);
+
+    boolean exist = table.exists(get);
+    assertEquals(exist, false);
+
+    table.put(put);
+
+    exist = table.exists(get);
+    assertEquals(exist, true);
+  }
+
+  public void testHTableExistsMethodSingleRegionMultipleGets() throws Exception {
+
+    HTable table = TEST_UTIL.createTable(
+      Bytes.toBytes("testHTableExistsMethodSingleRegionMultipleGets"), new byte[][] { FAMILY });
+
+    Put put = new Put(ROW);
+    put.add(FAMILY, QUALIFIER, VALUE);
+    table.put(put);
+
+    List<Get> gets = new ArrayList<Get>();
+    gets.add(new Get(ROW));
+    gets.add(null);
+    gets.add(new Get(ANOTHERROW));
+
+    Boolean[] results = table.exists(gets);
+    assertEquals(results[0], true);
+    assertEquals(results[1], false);
+    assertEquals(results[2], false);
+  }
+
+  @Test
+  public void testHTableExistsMethodMultipleRegionsSingleGet() throws Exception {
+
+    HTable table = TEST_UTIL.createTable(
+      Bytes.toBytes("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY }, 1,
+      new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
+    Put put = new Put(ROW);
+    put.add(FAMILY, QUALIFIER, VALUE);
+
+    Get get = new Get(ROW);
+
+    boolean exist = table.exists(get);
+    assertEquals(exist, false);
+
+    table.put(put);
+
+    exist = table.exists(get);
+    assertEquals(exist, true);
+  }
+
+  @Test
+  public void testHTableExistsMethodMultipleRegionsMultipleGets() throws Exception {
+    HTable table = TEST_UTIL.createTable(
+      Bytes.toBytes("testHTableExistsMethodMultipleRegionsMultipleGets"), new byte[][] { FAMILY },
+      1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
+    Put put = new Put(ROW);
+    put.add(FAMILY, QUALIFIER, VALUE);
+    table.put (put);
+
+    List<Get> gets = new ArrayList<Get>();
+    gets.add(new Get(ANOTHERROW));
+    gets.add(new Get(Bytes.add(ROW, new byte[] { 0x00 })));
+    gets.add(new Get(ROW));
+    gets.add(new Get(Bytes.add(ANOTHERROW, new byte[] { 0x00 })));
+
+    Boolean[] results = table.exists(gets);
+    assertEquals(results[0], false);
+    assertEquals(results[1], false);
+    assertEquals(results[2], true);
+    assertEquals(results[3], false);
+
+    // Test with the first region.
+    put = new Put(new byte[] { 0x00 });
+    put.add(FAMILY, QUALIFIER, VALUE);
+    table.put(put);
+
+    gets = new ArrayList<Get>();
+    gets.add(new Get(new byte[] { 0x00 }));
+    gets.add(new Get(new byte[] { 0x00, 0x00 }));
+    results = table.exists(gets);
+    assertEquals(results[0], true);
+    assertEquals(results[1], false);
+
+    // Test with the last region
+    put = new Put(new byte[] { (byte) 0xff, (byte) 0xff });
+    put.add(FAMILY, QUALIFIER, VALUE);
+    table.put(put);
+
+    gets = new ArrayList<Get>();
+    gets.add(new Get(new byte[] { (byte) 0xff }));
+    gets.add(new Get(new byte[] { (byte) 0xff, (byte) 0xff }));
+    gets.add(new Get(new byte[] { (byte) 0xff, (byte) 0xff, (byte) 0xff }));
+    results = table.exists(gets);
+    assertEquals(results[0], false);
+    assertEquals(results[1], true);
+    assertEquals(results[2], false);
+  }
+
+  @Test
+  public void testGetEmptyRow() throws Exception {
+    //Create a table and put in 1 row
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor desc = new HTableDescriptor(Bytes.toBytes("test"));
+    desc.addFamily(new HColumnDescriptor(FAMILY));
+    admin.createTable(desc);
+    HTable table = new HTable(TEST_UTIL.getConfiguration(), "test");
+
+    Put put = new Put(ROW_BYTES);
+    put.add(FAMILY, COL_QUAL, VAL_BYTES);
+    table.put(put);
+    table.flushCommits();
+
+    //Try getting the row with an empty row key and make sure the other base cases work as well
+    Result res = table.get(new Get(new byte[0]));
+    assertTrue(res.isEmpty() == true);
+    res = table.get(new Get(Bytes.toBytes("r1-not-exist")));
+    assertTrue(res.isEmpty() == true);
+    res = table.get(new Get(ROW_BYTES));
+    assertTrue(Arrays.equals(res.getValue(FAMILY, COL_QUAL), VAL_BYTES));
+
+    //Now actually put in a row with an empty row key    
+    put = new Put(new byte[0]);
+    put.add(FAMILY, COL_QUAL, VAL_BYTES);
+    table.put(put);
+    table.flushCommits();
+    res = table.get(new Get(new byte[0]));
+    assertTrue(Arrays.equals(res.getValue(FAMILY, COL_QUAL), VAL_BYTES));
+  }
 }