You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2012/12/23 20:34:56 UTC

svn commit: r1425513 [6/7] - in /hbase/branches/0.94-test: ./ bin/ conf/ security/src/main/java/org/apache/hadoop/hbase/ipc/ security/src/main/java/org/apache/hadoop/hbase/security/access/ security/src/test/java/org/apache/hadoop/hbase/security/access/...

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java Sun Dec 23 19:34:53 2012
@@ -587,10 +587,8 @@ public class TestCompaction extends HBas
 
     List<StoreFile> storeFiles = store.getStorefiles();
     long maxId = StoreFile.getMaxSequenceIdInList(storeFiles);
-    Compactor tool = new Compactor(this.conf);
 
-    StoreFile.Writer compactedFile =
-      tool.compact(store, storeFiles, false, maxId);
+    StoreFile.Writer compactedFile = store.compactStore(storeFiles, false, maxId);
 
     // Now lets corrupt the compacted file.
     FileSystem fs = FileSystem.get(conf);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java Sun Dec 23 19:34:53 2012
@@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.client.Sc
 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;
@@ -171,7 +170,7 @@ public class TestEndToEndSplitTransactio
     //for daughters.
     HTable table = TEST_UTIL.createTable(TABLENAME, FAMILY);
 
-    Stoppable stopper = new StoppableImplementation();
+    Stoppable stopper = new SimpleStoppable();
     RegionSplitter regionSplitter = new RegionSplitter(table);
     RegionChecker regionChecker = new RegionChecker(conf, stopper, TABLENAME);
 
@@ -194,6 +193,20 @@ 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/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java Sun Dec 23 19:34:53 2012
@@ -16,9 +16,6 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.SmallTests;
-import org.apache.hadoop.hbase.MultithreadedTestUtil;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
@@ -33,32 +30,19 @@ import org.junit.experimental.categories
 
 import com.google.common.collect.Lists;
 
-/**
- * Test of HBASE-7051; that checkAndPuts and puts behave atomically with respect to each other.
- * Rather than perform a bunch of trials to verify atomicity, this test recreates a race condition
- * that causes the test to fail if checkAndPut doesn't wait for outstanding put transactions
- * to complete.  It does this by invasively overriding HRegion function to affect the timing of
- * the operations.
- */
 @Category(SmallTests.class)
 public class TestHBase7051 {
 
+  private static volatile boolean putCompleted = false;
   private static CountDownLatch latch = new CountDownLatch(1);
-  private enum TestStep {
-    INIT,                  // initial put of 10 to set value of the cell
-    PUT_STARTED,           // began doing a put of 50 to cell
-    PUT_COMPLETED,         // put complete (released RowLock, but may not have advanced MVCC).
-    CHECKANDPUT_STARTED,   // began checkAndPut: if 10 -> 11
-    CHECKANDPUT_COMPLETED  // completed checkAndPut
-    // NOTE: at the end of these steps, the value of the cell should be 50, not 11!
-  }
-  private static volatile TestStep testStep = TestStep.INIT;
-  private final String family = "f1";
-  	 
+  private boolean checkAndPutCompleted = false;
+  private static int count = 0;
+
   @Test
   public void testPutAndCheckAndPutInParallel() throws Exception {
 
     final String tableName = "testPutAndCheckAndPut";
+    final String family = "f1";
     Configuration conf = HBaseConfiguration.create();
     conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class);
     final MockHRegion region = (MockHRegion) TestHRegion.initHRegion(Bytes.toBytes(tableName),
@@ -73,16 +57,14 @@ public class TestHBase7051 {
 
     putsAndLocks.add(pair);
 
+    count++;
     region.batchMutate(putsAndLocks.toArray(new Pair[0]));
-    MultithreadedTestUtil.TestContext ctx =
-      new MultithreadedTestUtil.TestContext(conf);
-    ctx.addThread(new PutThread(ctx, region));
-    ctx.addThread(new CheckAndPutThread(ctx, region));
-    ctx.startThreads();
-    while (testStep != TestStep.CHECKANDPUT_COMPLETED) {
+    makeCheckAndPut(family, region);
+
+    makePut(family, region);
+    while (!checkAndPutCompleted) {
       Thread.sleep(100);
     }
-    ctx.stop();
     Scan s = new Scan();
     RegionScanner scanner = region.getScanner(s);
     List<KeyValue> results = new ArrayList<KeyValue>();
@@ -93,46 +75,54 @@ public class TestHBase7051 {
 
   }
 
-  private class PutThread extends TestThread {
-    private MockHRegion region;
-    PutThread(TestContext ctx, MockHRegion region) {
-      super(ctx);
-      this.region = region;
-    }
-
-    public void doWork() throws Exception {
-      List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
-      Put[] puts = new Put[1];
-      Put put = new Put(Bytes.toBytes("r1"));
-      put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
-      puts[0] = put;
-      Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
-      putsAndLocks.add(pair);
-      testStep = TestStep.PUT_STARTED;
-      region.batchMutate(putsAndLocks.toArray(new Pair[0]));
-    }
+  private void makePut(final String family, final MockHRegion region) {
+    new Thread() {
+      public void run() {
+        List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
+        Put[] puts = new Put[1];
+        Put put = new Put(Bytes.toBytes("r1"));
+        put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
+        puts[0] = put;
+        try {
+          Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
+          putsAndLocks.add(pair);
+          count++;
+          region.batchMutate(putsAndLocks.toArray(new Pair[0]));
+        } catch (IOException e) {
+          // TODO Auto-generated catch block
+          e.printStackTrace();
+        }
+      }
+    }.start();
   }
 
-  private class CheckAndPutThread extends TestThread {
-    private MockHRegion region;
-    CheckAndPutThread(TestContext ctx, MockHRegion region) {
-      super(ctx);
-      this.region = region;
-   }
-
-    public void doWork() throws Exception {
-      Put[] puts = new Put[1];
-      Put put = new Put(Bytes.toBytes("r1"));
-      put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("11"));
-      puts[0] = put;
-      while (testStep != TestStep.PUT_COMPLETED) {
-        Thread.sleep(100);
-      }
-      testStep = TestStep.CHECKANDPUT_STARTED;
-      region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
-        CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, null, true);
-      testStep = TestStep.CHECKANDPUT_COMPLETED;
-    }
+  private void makeCheckAndPut(final String family, final MockHRegion region) {
+    new Thread() {
+
+      public void run() {
+        Put[] puts = new Put[1];
+        Put put = new Put(Bytes.toBytes("r1"));
+        put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("11"));
+        puts[0] = put;
+        try {
+          while (putCompleted == false) {
+            try {
+              Thread.sleep(100);
+            } catch (InterruptedException e) {
+              // TODO Auto-generated catch block
+              e.printStackTrace();
+            }
+          }
+          count++;
+          region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
+              CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, null, true);
+          checkAndPutCompleted = true;
+        } catch (IOException e) {
+          // TODO Auto-generated catch block
+          e.printStackTrace();
+        }
+      }
+    }.start();
   }
 
   public static class MockHRegion extends HRegion {
@@ -144,39 +134,36 @@ public class TestHBase7051 {
 
     @Override
     public void releaseRowLock(Integer lockId) {
-      if (testStep == TestStep.INIT) {
+      if (count == 1) {
         super.releaseRowLock(lockId);
         return;
       }
 
-      if (testStep == TestStep.PUT_STARTED) {
+      if (count == 2) {
         try {
-          testStep = TestStep.PUT_COMPLETED;
+          putCompleted = true;
           super.releaseRowLock(lockId);
-          // put has been written to the memstore and the row lock has been released, but the
-          // MVCC has not been advanced.  Prior to fixing HBASE-7051, the following order of
-          // operations would cause the non-atomicity to show up:
-          // 1) Put releases row lock (where we are now)
-          // 2) CheckAndPut grabs row lock and reads the value prior to the put (10)
-          //    because the MVCC has not advanced
-          // 3) Put advances MVCC
-          // So, in order to recreate this order, we wait for the checkAndPut to grab the rowLock
-          // (see below), and then wait some more to give the checkAndPut time to read the old
-          // value.
           latch.await();
-          Thread.sleep(1000);
         } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
+          // TODO Auto-generated catch block
+          e.printStackTrace();
         }
       }
-      else if (testStep == TestStep.CHECKANDPUT_STARTED) {
+      if (count == 3) {
         super.releaseRowLock(lockId);
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          // TODO Auto-generated catch block
+          e.printStackTrace();
+        }
+        latch.countDown();
       }
     }
 
     @Override
     public Integer getLock(Integer lockid, byte[] row, boolean waitForLock) throws IOException {
-      if (testStep == TestStep.CHECKANDPUT_STARTED) {
+      if (count == 3) {
         latch.countDown();
       }
       return super.getLock(lockid, row, waitForLock);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Sun Dec 23 19:34:53 2012
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.regionse
 
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -52,15 +51,17 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.MultithreadedTestUtil;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
+import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 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.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
@@ -70,6 +71,8 @@ import org.apache.hadoop.hbase.filter.Fi
 import org.apache.hadoop.hbase.filter.NullComparator;
 import org.apache.hadoop.hbase.filter.PrefixFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -102,7 +105,6 @@ import com.google.common.collect.Lists;
  * HRegions or in the HBaseMaster, so only basic testing is possible.
  */
 @Category(MediumTests.class)
-@SuppressWarnings("deprecation")
 public class TestHRegion extends HBaseTestCase {
   // Do not spin up clusters in here.  If you need to spin up a cluster, do it
   // over in TestHRegionOnCluster.
@@ -155,6 +157,7 @@ public class TestHRegion extends HBaseTe
     String method = "testCompactionAffectedByScanners";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
+    Configuration conf = HBaseConfiguration.create();
     this.region = initHRegion(tableName, method, conf, family);
 
     Put put = new Put(Bytes.toBytes("r1"));
@@ -206,6 +209,7 @@ public class TestHRegion extends HBaseTe
     String method = "testToShowNPEOnRegionScannerReseek";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
+    Configuration conf = HBaseConfiguration.create();
     this.region = initHRegion(tableName, method, conf, family);
 
     Put put = new Put(Bytes.toBytes("r1"));
@@ -238,6 +242,7 @@ public class TestHRegion extends HBaseTe
     String method = "testSkipRecoveredEditsReplay";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
+    Configuration conf = HBaseConfiguration.create();
     this.region = initHRegion(tableName, method, conf, family);
     try {
       Path regiondir = region.getRegionDir();
@@ -283,7 +288,7 @@ public class TestHRegion extends HBaseTe
     String method = "testSkipRecoveredEditsReplaySomeIgnored";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
-    this.region = initHRegion(tableName, method, conf, family);
+    this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
     try {
       Path regiondir = region.getRegionDir();
       FileSystem fs = region.getFilesystem();
@@ -333,7 +338,7 @@ public class TestHRegion extends HBaseTe
     String method = "testSkipRecoveredEditsReplayAllIgnored";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
-    this.region = initHRegion(tableName, method, conf, family);
+    this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
     try {
       Path regiondir = region.getRegionDir();
       FileSystem fs = region.getFilesystem();
@@ -459,7 +464,7 @@ public class TestHRegion extends HBaseTe
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
-    this.region = initHRegion(TABLE, getName(), conf, FAMILIES);
+    this.region = initHRegion(TABLE, getName(), FAMILIES);
     try {
       String value = "this is the value";
       String value2 = "this is some other value";
@@ -580,7 +585,7 @@ public class TestHRegion extends HBaseTe
   public void testFamilyWithAndWithoutColon() throws Exception {
     byte [] b = Bytes.toBytes(getName());
     byte [] cf = Bytes.toBytes(COLUMN_FAMILY);
-    this.region = initHRegion(b, getName(), conf, cf);
+    this.region = initHRegion(b, getName(), cf);
     try {
       Put p = new Put(b);
       byte [] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
@@ -604,7 +609,7 @@ public class TestHRegion extends HBaseTe
     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
     byte[] qual = Bytes.toBytes("qual");
     byte[] val = Bytes.toBytes("val");
-    this.region = initHRegion(b, getName(), conf, cf);
+    this.region = initHRegion(b, getName(), cf);
     try {
       HLog.getSyncTime(); // clear counter from prior tests
       assertEquals(0, HLog.getSyncTime().count);
@@ -638,7 +643,7 @@ public class TestHRegion extends HBaseTe
       Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
 
       MultithreadedTestUtil.TestContext ctx =
-        new MultithreadedTestUtil.TestContext(conf);
+        new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
       final AtomicReference<OperationStatus[]> retFromThread =
         new AtomicReference<OperationStatus[]>();
       TestThread putter = new TestThread(ctx) {
@@ -705,7 +710,9 @@ public class TestHRegion extends HBaseTe
     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
     byte[] qual = Bytes.toBytes("qual");
     byte[] val = Bytes.toBytes("val");
-    Configuration conf = HBaseConfiguration.create(this.conf);
+
+    HBaseConfiguration conf = new HBaseConfiguration();
+
 
     // add data with a timestamp that is too recent for range. Ensure assert
     conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
@@ -752,7 +759,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting empty data in key
       Put put = new Put(row1);
@@ -827,7 +834,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting data in key
       Put put = new Put(row1);
@@ -861,7 +868,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting data in key
       Put put = new Put(row1);
@@ -899,7 +906,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in the key to check
       Put put = new Put(row1);
@@ -938,7 +945,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testCheckAndPut_wrongRowInPut() throws IOException {
-    this.region = initHRegion(tableName, this.getName(), conf, COLUMNS);
+    this.region = initHRegion(tableName, this.getName(), COLUMNS);
     try {
       Put put = new Put(row2);
       put.add(fam1, qual1, value1);
@@ -973,7 +980,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Put content
       Put put = new Put(row1);
@@ -1048,7 +1055,7 @@ public class TestHRegion extends HBaseTe
     put.add(fam1, qual, 2, value);
 
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       region.put(put);
 
@@ -1078,7 +1085,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1, fam2, fam3);
+    this.region = initHRegion(tableName, method, fam1, fam2, fam3);
     try {
       List<KeyValue> kvs  = new ArrayList<KeyValue>();
       kvs.add(new KeyValue(row1, fam4, null, null));
@@ -1116,7 +1123,7 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
 
@@ -1184,7 +1191,7 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       byte [] row = Bytes.toBytes("table_name");
       // column names
@@ -1227,7 +1234,7 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       byte [] row = Bytes.toBytes("row1");
       // column names
@@ -1280,7 +1287,7 @@ public class TestHRegion extends HBaseTe
     byte[] fam = Bytes.toBytes("info");
     byte[][] families = { fam };
     String method = this.getName();
-    Configuration conf = HBaseConfiguration.create(this.conf);
+    HBaseConfiguration conf = new HBaseConfiguration();
 
     // add data with a timestamp that is too recent for range. Ensure assert
     conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
@@ -1311,7 +1318,7 @@ public class TestHRegion extends HBaseTe
     byte [] tableName = Bytes.toBytes("test_table");
     byte [] fam1 = Bytes.toBytes("columnA");
     byte [] fam2 = Bytes.toBytes("columnB");
-    this.region = initHRegion(tableName, getName(), conf, fam1, fam2);
+    this.region = initHRegion(tableName, getName(), fam1, fam2);
     try {
       byte [] rowA = Bytes.toBytes("rowA");
       byte [] rowB = Bytes.toBytes("rowB");
@@ -1364,7 +1371,7 @@ public class TestHRegion extends HBaseTe
 
   public void doTestDelete_AndPostInsert(Delete delete)
       throws IOException, InterruptedException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
       Put put = new Put(row);
@@ -1417,7 +1424,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Building checkerList
       List<KeyValue> kvs  = new ArrayList<KeyValue>();
@@ -1457,7 +1464,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       Get get = new Get(row1);
       get.addColumn(fam2, col1);
@@ -1488,7 +1495,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Add to memstore
       Put put = new Put(row1);
@@ -1538,7 +1545,7 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("fam");
 
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam);
+    this.region = initHRegion(tableName, method, fam);
     try {
       Get get = new Get(row);
       get.addFamily(fam);
@@ -1558,8 +1565,7 @@ public class TestHRegion extends HBaseTe
   public void stestGet_Root() throws IOException {
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(HConstants.ROOT_TABLE_NAME,
-      method, conf, HConstants.CATALOG_FAMILY);
+    this.region = initHRegion(HConstants.ROOT_TABLE_NAME, method, HConstants.CATALOG_FAMILY);
     try {
       //Add to memstore
       Put put = new Put(HConstants.EMPTY_START_ROW);
@@ -1791,7 +1797,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       Scan scan = new Scan();
       scan.addFamily(fam1);
@@ -1816,7 +1822,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       Scan scan = new Scan();
       scan.addFamily(fam2);
@@ -1845,7 +1851,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
 
       //Putting data in Region
@@ -1893,7 +1899,7 @@ public class TestHRegion extends HBaseTe
     //Setting up region
     String method = this.getName();
     try {
-      this.region = initHRegion(tableName, method, conf, families);
+      this.region = initHRegion(tableName, method, families);
     } catch (IOException e) {
       e.printStackTrace();
       fail("Got IOException during initHRegion, " + e.getMessage());
@@ -1929,7 +1935,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       Put put = null;
@@ -1996,7 +2002,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       Put put = null;
@@ -2056,7 +2062,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       Put put = null;
@@ -2121,7 +2127,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
@@ -2203,7 +2209,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       //Putting data in Region
       Put put = null;
@@ -2264,7 +2270,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting data in Region
       Put put = null;
@@ -2315,7 +2321,7 @@ public class TestHRegion extends HBaseTe
   public void testScanner_StopRow1542() throws IOException {
     byte [] tableName = Bytes.toBytes("test_table");
     byte [] family = Bytes.toBytes("testFamily");
-    this.region = initHRegion(tableName, getName(), conf, family);
+    this.region = initHRegion(tableName, getName(), family);
     try {
       byte [] row1 = Bytes.toBytes("row111");
       byte [] row2 = Bytes.toBytes("row222");
@@ -2362,7 +2368,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrementColumnValue_UpdatingInPlace() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2390,7 +2396,7 @@ public class TestHRegion extends HBaseTe
   public void testIncrementColumnValue_BumpSnapshot() throws IOException {
     ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
     EnvironmentEdgeManagerTestHelper.injectEdge(mee);
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 42L;
       long incr = 44L;
@@ -2429,7 +2435,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrementColumnValue_ConcurrentFlush() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2463,7 +2469,7 @@ public class TestHRegion extends HBaseTe
   public void testIncrementColumnValue_heapSize() throws IOException {
     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
 
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long byAmount = 1L;
       long size;
@@ -2482,7 +2488,7 @@ public class TestHRegion extends HBaseTe
 
   public void testIncrementColumnValue_UpdatingInPlace_Negative()
     throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 3L;
       long amount = -1L;
@@ -2503,7 +2509,7 @@ public class TestHRegion extends HBaseTe
 
   public void testIncrementColumnValue_AddingNew()
     throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2532,7 +2538,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2560,7 +2566,7 @@ public class TestHRegion extends HBaseTe
 
   public void testIncrementColumnValue_AddingNewAfterSFCheck()
     throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2599,7 +2605,7 @@ public class TestHRegion extends HBaseTe
    * @throws IOException
    */
   public void testIncrementColumnValue_UpdatingInPlace_TimestampClobber() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       long value = 1L;
       long amount = 3L;
@@ -2647,7 +2653,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrementColumnValue_WrongInitialSize() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
       int row1Field1 = 0;
@@ -2675,7 +2681,7 @@ public class TestHRegion extends HBaseTe
   }
 
   public void testIncrement_WrongInitialSize() throws IOException {
-    this.region = initHRegion(tableName, getName(), conf, fam1);
+    this.region = initHRegion(tableName, getName(), fam1);
     try {
       byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
       long row1Field1 = 0;
@@ -2751,7 +2757,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, fam1);
+    this.region = initHRegion(tableName, method, fam1);
     try {
       //Putting data in Region
       KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
@@ -2966,7 +2972,7 @@ public class TestHRegion extends HBaseTe
     int compactInterval = 10 * flushAndScanInterval;
 
     String method = "testFlushCacheWhileScanning";
-    this.region = initHRegion(tableName,method, conf, family);
+    this.region = initHRegion(tableName,method, family);
     try {
       FlushThread flushThread = new FlushThread();
       flushThread.start();
@@ -3097,7 +3103,7 @@ public class TestHRegion extends HBaseTe
     }
 
     String method = "testWritesWhileScanning";
-    this.region = initHRegion(tableName, method, conf, families);
+    this.region = initHRegion(tableName, method, families);
     try {
       PutThread putThread = new PutThread(numRows, families, qualifiers);
       putThread.start();
@@ -3219,8 +3225,6 @@ public class TestHRegion extends HBaseTe
             }
             numPutsFinished++;
           }
-        } catch (InterruptedIOException e) {
-          // This is fine. It means we are done, or didn't get the lock on time
         } catch (IOException e) {
           LOG.error("error while putting records", e);
           error = e;
@@ -3257,9 +3261,8 @@ public class TestHRegion extends HBaseTe
       qualifiers[i] = Bytes.toBytes("qual" + i);
     }
 
-    Configuration conf = HBaseConfiguration.create(this.conf);
-
     String method = "testWritesWhileGetting";
+    Configuration conf = HBaseConfiguration.create();
     // This test flushes constantly and can cause many files to be created, possibly
     // extending over the ulimit.  Make sure compactions are aggressive in reducing
     // the number of HFiles created.
@@ -3268,7 +3271,7 @@ public class TestHRegion extends HBaseTe
     this.region = initHRegion(tableName, method, conf, families);
     PutThread putThread = null;
     MultithreadedTestUtil.TestContext ctx =
-      new MultithreadedTestUtil.TestContext(conf);
+      new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
     try {
       putThread = new PutThread(numRows, families, qualifiers);
       putThread.start();
@@ -3354,7 +3357,7 @@ public class TestHRegion extends HBaseTe
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method,
-        conf, family);
+        HBaseConfiguration.create(), family);
     try {
       byte[] rowNotServed = Bytes.toBytes("a");
       Get g = new Get(rowNotServed);
@@ -3418,7 +3421,7 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = "testIndexesScanWithOneDeletedRow";
-    this.region = initHRegion(tableName, method, conf, family);
+    this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
     try {
       Put put = new Put(Bytes.toBytes(1L));
       put.add(family, qual1, 1L, Bytes.toBytes(1L));
@@ -3871,6 +3874,7 @@ public class TestHRegion extends HBaseTe
    */
   @Test
   public void testParallelIncrementWithMemStoreFlush() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
     String method = "testParallelIncrementWithMemStoreFlush";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Incrementer.family;
@@ -4003,8 +4007,7 @@ public class TestHRegion extends HBaseTe
   }
 
   private Configuration initSplit() {
-    Configuration conf = HBaseConfiguration.create(this.conf);
-
+    Configuration conf = HBaseConfiguration.create();
     // Always compact if there is more than one store file.
     conf.setInt("hbase.hstore.compactionThreshold", 2);
 
@@ -4025,6 +4028,19 @@ public class TestHRegion extends HBaseTe
   /**
    * @param tableName
    * @param callingMethod
+   * @param families
+   * @return A region on which you must call {@link HRegion#closeHRegion(HRegion)} when done.
+   * @throws IOException
+   */
+  private static HRegion initHRegion (byte [] tableName, String callingMethod,
+    byte[] ... families)
+  throws IOException {
+    return initHRegion(tableName, callingMethod, HBaseConfiguration.create(), families);
+  }
+
+  /**
+   * @param tableName
+   * @param callingMethod
    * @param conf
    * @param families
    * @throws IOException

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestMXBean.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestMXBean.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestMXBean.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestMXBean.java Sun Dec 23 19:34:53 2012
@@ -21,13 +21,10 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.MediumTests;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
-@Category(MediumTests.class)
 public class TestMXBean {
 
   private static final HBaseTestingUtility TEST_UTIL =

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java Sun Dec 23 19:34:53 2012
@@ -113,8 +113,7 @@ public class TestQueryMatcher extends HB
     memstore.add(new KeyValue(row2, fam1, col1, data));
 
     List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
-    KeyValue k = memstore.get(0);
-    qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+    qm.setRow(memstore.get(0).getRow());
 
     for (KeyValue kv : memstore){
       actual.add(qm.match(kv));
@@ -159,8 +158,7 @@ public class TestQueryMatcher extends HB
 
     List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
 
-    KeyValue k = memstore.get(0);
-    qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+    qm.setRow(memstore.get(0).getRow());
 
     for(KeyValue kv : memstore) {
       actual.add(qm.match(kv));
@@ -212,8 +210,7 @@ public class TestQueryMatcher extends HB
         new KeyValue(row2, fam1, col1, now-10, data)
     };
 
-    KeyValue k = kvs[0];
-    qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+    qm.setRow(kvs[0].getRow());
 
     List<MatchCode> actual = new ArrayList<MatchCode>(kvs.length);
     for (KeyValue kv : kvs) {
@@ -265,8 +262,7 @@ public class TestQueryMatcher extends HB
         new KeyValue(row1, fam2, col5, now-10000, data),
         new KeyValue(row2, fam1, col1, now-10, data)
     };
-    KeyValue k = kvs[0];
-    qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+    qm.setRow(kvs[0].getRow());
 
     List<ScanQueryMatcher.MatchCode> actual =
         new ArrayList<ScanQueryMatcher.MatchCode>(kvs.length);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java Sun Dec 23 19:34:53 2012
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HColumnDe
 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.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -56,9 +55,7 @@ import org.apache.zookeeper.KeeperExcept
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
-@Category(LargeTests.class)
 public class TestRSKilledWhenMasterInitializing {
   private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
 

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java Sun Dec 23 19:34:53 2012
@@ -36,7 +36,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -728,44 +727,6 @@ public class TestSplitTransactionOnClust
     }
   }
 
-  @Test(timeout = 20000)
-  public void testTableExistsIfTheSpecifiedTableRegionIsSplitParent() throws Exception {
-    final byte[] tableName = 
-        Bytes.toBytes("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
-    HRegionServer regionServer = null;
-    List<HRegion> regions = null;
-    HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
-    try {
-      // Create table then get the single region for our new table.
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      htd.addFamily(new HColumnDescriptor("cf"));
-      admin.createTable(htd);
-      HTable t = new HTable(cluster.getConfiguration(), tableName);
-      regions = cluster.getRegions(tableName);
-      int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
-      regionServer = cluster.getRegionServer(regionServerIndex);
-      insertData(tableName, admin, t);
-      // Turn off balancer so it doesn't cut in and mess up our placements.
-      cluster.getMaster().setCatalogJanitorEnabled(false);
-      boolean tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
-          Bytes.toString(tableName));
-      assertEquals("The specified table should present.", true, tableExists);
-      SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("row2"));
-      try {
-        st.prepare();
-        st.createDaughters(regionServer, regionServer);
-      } catch (IOException e) {
-
-      }
-      tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
-          Bytes.toString(tableName));
-      assertEquals("The specified table should present.", true, tableExists);
-    } finally {
-      cluster.getMaster().setCatalogJanitorEnabled(true);
-      admin.close();
-    }
-  }
-  
   private void insertData(final byte[] tableName, HBaseAdmin admin, HTable t) throws IOException,
       InterruptedException {
     Put p = new Put(Bytes.toBytes("row1"));

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java Sun Dec 23 19:34:53 2012
@@ -49,9 +49,6 @@ public class FaultySequenceFileLogReader
         HLogKey key = HLog.newKey(conf);
         WALEdit val = new WALEdit();
         HLog.Entry e = new HLog.Entry(key, val);
-        if (compressionContext != null) {
-          e.setCompressionContext(compressionContext);
-        }
         b = this.reader.next(e.getKey(), e.getEdit());
         nextQueue.offer(e);
         numberOfFileEntries++;

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Sun Dec 23 19:34:53 2012
@@ -475,13 +475,15 @@ public class TestHLog  {
       throw t.exception;
 
     // Make sure you can read all the content
-    HLog.Reader reader = HLog.getReader(this.fs, walPath, this.conf);
+    SequenceFile.Reader reader
+      = new SequenceFile.Reader(this.fs, walPath, this.conf);
     int count = 0;
-    HLog.Entry entry = new HLog.Entry();
-    while (reader.next(entry) != null) {
+    HLogKey key = HLog.newKey(conf);
+    WALEdit val = new WALEdit();
+    while (reader.next(key, val)) {
       count++;
       assertTrue("Should be one KeyValue per WALEdit",
-                 entry.getEdit().getKeyValues().size() == 1);
+                 val.getKeyValues().size() == 1);
     }
     assertEquals(total, count);
     reader.close();

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java Sun Dec 23 19:34:53 2012
@@ -86,7 +86,7 @@ public class TestHLogSplit {
   private Configuration conf;
   private FileSystem fs;
 
-  protected final static HBaseTestingUtility
+  private final static HBaseTestingUtility
           TEST_UTIL = new HBaseTestingUtility();
 
 

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java Sun Dec 23 19:34:53 2012
@@ -61,7 +61,7 @@ public class TestReplication {
 
   private static final Log LOG = LogFactory.getLog(TestReplication.class);
 
-  protected static Configuration conf1 = HBaseConfiguration.create();
+  private static Configuration conf1;
   private static Configuration conf2;
   private static Configuration CONF_WITH_LOCALFS;
 
@@ -78,8 +78,8 @@ public class TestReplication {
   private static final int NB_ROWS_IN_BATCH = 100;
   private static final int NB_ROWS_IN_BIG_BATCH =
       NB_ROWS_IN_BATCH * 10;
-  private static final long SLEEP_TIME = 1500;
-  private static final int NB_RETRIES = 15;
+  private static final long SLEEP_TIME = 500;
+  private static final int NB_RETRIES = 10;
 
   private static final byte[] tableName = Bytes.toBytes("test");
   private static final byte[] famName = Bytes.toBytes("f");
@@ -91,6 +91,7 @@ public class TestReplication {
    */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
+    conf1 = HBaseConfiguration.create();
     conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
     // smaller block size and capacity to trigger more operations
     // and test them
@@ -519,7 +520,7 @@ public class TestReplication {
 
     // disable and start the peer
     admin.disablePeer("2");
-    utility2.startMiniHBaseCluster(1, 2);
+    utility2.startMiniHBaseCluster(1, 1);
     Get get = new Get(rowkey);
     for (int i = 0; i < NB_RETRIES; i++) {
       Result res = htable2.get(get);
@@ -716,7 +717,7 @@ public class TestReplication {
    */
   @Test(timeout=300000)
   public void queueFailover() throws Exception {
-    utility1.createMultiRegions(htable1, famName, false);
+    utility1.createMultiRegions(htable1, famName);
 
     // killing the RS with .META. can result into failed puts until we solve
     // IO fencing
@@ -759,8 +760,7 @@ public class TestReplication {
     int lastCount = 0;
 
     final long start = System.currentTimeMillis();
-    int i = 0;
-    while (true) {
+    for (int i = 0; i < NB_RETRIES; i++) {
       if (i==NB_RETRIES-1) {
         fail("Waited too much time for queueFailover replication. " +
           "Waited "+(System.currentTimeMillis() - start)+"ms.");
@@ -772,8 +772,6 @@ public class TestReplication {
       if (res2.length < initialCount) {
         if (lastCount < res2.length) {
           i--; // Don't increment timeout if we make progress
-        } else {
-          i++;
         }
         lastCount = res2.length;
         LOG.info("Only got " + lastCount + " rows instead of " +
@@ -793,7 +791,7 @@ public class TestReplication {
           Thread.sleep(timeout);
           utility.expireRegionServerSession(rs);
         } catch (Exception e) {
-          LOG.error("Couldn't kill a region server", e);
+          LOG.error(e);
         }
       }
     };

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java Sun Dec 23 19:34:53 2012
@@ -114,6 +114,7 @@ public class PerformanceEvaluation  {
 
   protected Map<String, CmdDescriptor> commands = new TreeMap<String, CmdDescriptor>();
   protected static Cluster cluster = new Cluster();
+  protected static String accessToken = null;
 
   volatile Configuration conf;
   private boolean nomapred = false;
@@ -448,7 +449,8 @@ public class PerformanceEvaluation  {
    */
   private boolean checkTable() throws IOException {
     HTableDescriptor tableDescriptor = getTableDescriptor();
-    RemoteAdmin admin = new RemoteAdmin(new Client(cluster), conf);
+    RemoteAdmin admin =
+      new RemoteAdmin(new Client(cluster), conf, accessToken);
     if (!admin.isTableAvailable(tableDescriptor.getName())) {
       admin.createTable(tableDescriptor);
       return true;
@@ -712,7 +714,8 @@ public class PerformanceEvaluation  {
     }
     
     void testSetup() throws IOException {
-      this.table = new RemoteHTable(new Client(cluster), conf, tableName);
+      this.table = new RemoteHTable(new Client(cluster), conf, tableName,
+        accessToken);
     }
 
     void testTakedown()  throws IOException {
@@ -1130,6 +1133,7 @@ public class PerformanceEvaluation  {
     System.err.println();
     System.err.println("Options:");
     System.err.println(" host          String. Specify Stargate endpoint.");
+    System.err.println(" token         String. API access token.");
     System.err.println(" rows          Integer. Rows each client runs. Default: One million");
     System.err.println(" rowsPerPut    Integer. Rows each Stargate (multi)Put. Default: 100");
     System.err.println(" nomapred      (Flag) Run multiple clients using threads " +
@@ -1204,6 +1208,12 @@ public class PerformanceEvaluation  {
           continue;
         }
 
+        final String token = "--token=";
+        if (cmd.startsWith(token)) {
+          accessToken = cmd.substring(token.length());
+          continue;
+        }
+
         Class<? extends Test> cmdClass = determineCommandClass(cmd);
         if (cmdClass != null) {
           getArgs(i + 1, args);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java Sun Dec 23 19:34:53 2012
@@ -101,7 +101,7 @@ public class TestRemoteTable {
     remoteTable = new RemoteHTable(
       new Client(new Cluster().add("localhost", 
           REST_TEST_UTIL.getServletPort())),
-        TEST_UTIL.getConfiguration(), TABLE);
+        TEST_UTIL.getConfiguration(), TABLE, null);
   }
 
   @AfterClass
@@ -222,45 +222,6 @@ public class TestRemoteTable {
   }
 
   @Test
-  public void testMultiGet() throws Exception {
-    ArrayList<Get> gets = new ArrayList<Get>();
-    gets.add(new Get(ROW_1));
-    gets.add(new Get(ROW_2));
-    Result[] results = remoteTable.get(gets);
-    assertNotNull(results);
-    assertEquals(2, results.length);
-    assertEquals(1, results[0].size());
-    assertEquals(2, results[1].size());
-
-    //Test Versions
-    gets = new ArrayList<Get>();
-    Get g = new Get(ROW_1);
-    g.setMaxVersions(3);
-    gets.add(g);
-    gets.add(new Get(ROW_2));
-    results = remoteTable.get(gets);
-    assertNotNull(results);
-    assertEquals(2, results.length);
-    assertEquals(1, results[0].size());
-    assertEquals(3, results[1].size());
-
-    //404
-    gets = new ArrayList<Get>();
-    gets.add(new Get(Bytes.toBytes("RESALLYREALLYNOTTHERE")));
-    results = remoteTable.get(gets);
-    assertNotNull(results);
-    assertEquals(0, results.length);
-
-    gets = new ArrayList<Get>();
-    gets.add(new Get(Bytes.toBytes("RESALLYREALLYNOTTHERE")));
-    gets.add(new Get(ROW_1));
-    gets.add(new Get(ROW_2));
-    results = remoteTable.get(gets);
-    assertNotNull(results);
-    assertEquals(0, results.length);
-  }
-
-  @Test
   public void testPut() throws IOException {
     Put put = new Put(ROW_3);
     put.add(COLUMN_1, QUALIFIER_1, VALUE_1);

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java Sun Dec 23 19:34:53 2012
@@ -95,8 +95,6 @@ public class LoadTestTool extends Abstra
   private static final String OPT_START_KEY = "start_key";
   private static final String OPT_TABLE_NAME = "tn";
   private static final String OPT_ZK_QUORUM = "zk";
-  private static final String OPT_SKIP_INIT = "skip_init";
-  private static final String OPT_INIT_ONLY = "init_only";
 
   private static final long DEFAULT_START_KEY = 0;
 
@@ -128,11 +126,6 @@ public class LoadTestTool extends Abstra
   private int maxReadErrors = MultiThreadedReader.DEFAULT_MAX_ERRORS;
   private int verifyPercent;
 
-  // TODO: refactor LoadTestToolImpl somewhere to make the usage from tests less bad,
-  //       console tool itself should only be used from console.
-  private boolean isSkipInit = false;
-  private boolean isInitOnly = false;
-
   private String[] splitColonSeparated(String option,
       int minNumCols, int maxNumCols) {
     String optVal = cmd.getOptionValue(option);
@@ -193,7 +186,6 @@ public class LoadTestTool extends Abstra
     addOptWithArg(OPT_TABLE_NAME, "The name of the table to read or write");
     addOptWithArg(OPT_WRITE, OPT_USAGE_LOAD);
     addOptWithArg(OPT_READ, OPT_USAGE_READ);
-    addOptNoArg(OPT_INIT_ONLY, "Initialize the test table only, don't do any loading");
     addOptWithArg(OPT_BLOOM, OPT_USAGE_BLOOM);
     addOptWithArg(OPT_COMPRESSION, OPT_USAGE_COMPRESSION);
     addOptWithArg(OPT_DATA_BLOCK_ENCODING, OPT_DATA_BLOCK_ENCODING_USAGE);
@@ -208,12 +200,10 @@ public class LoadTestTool extends Abstra
         "separate puts for every column in a row");
     addOptNoArg(OPT_ENCODE_IN_CACHE_ONLY, OPT_ENCODE_IN_CACHE_ONLY_USAGE);
 
-    addOptWithArg(OPT_NUM_KEYS, "The number of keys to read/write");
+    addRequiredOptWithArg(OPT_NUM_KEYS, "The number of keys to read/write");
     addOptWithArg(OPT_START_KEY, "The first key to read/write " +
         "(a 0-based index). The default value is " +
         DEFAULT_START_KEY + ".");
-    addOptNoArg(OPT_SKIP_INIT, "Skip the initialization; assume test table "
-        + "already exists");
   }
 
   @Override
@@ -222,35 +212,20 @@ public class LoadTestTool extends Abstra
 
     tableName = Bytes.toBytes(cmd.getOptionValue(OPT_TABLE_NAME,
         DEFAULT_TABLE_NAME));
+    startKey = parseLong(cmd.getOptionValue(OPT_START_KEY,
+        String.valueOf(DEFAULT_START_KEY)), 0, Long.MAX_VALUE);
+    long numKeys = parseLong(cmd.getOptionValue(OPT_NUM_KEYS), 1,
+        Long.MAX_VALUE - startKey);
+    endKey = startKey + numKeys;
 
     isWrite = cmd.hasOption(OPT_WRITE);
     isRead = cmd.hasOption(OPT_READ);
-    isInitOnly = cmd.hasOption(OPT_INIT_ONLY);
 
-    if (!isWrite && !isRead && !isInitOnly) {
+    if (!isWrite && !isRead) {
       throw new IllegalArgumentException("Either -" + OPT_WRITE + " or " +
           "-" + OPT_READ + " has to be specified");
     }
 
-    if (isInitOnly && (isRead || isWrite)) {
-      throw new IllegalArgumentException(OPT_INIT_ONLY + " cannot be specified with"
-          + " either -" + OPT_WRITE + " or -" + OPT_READ);
-    }
-
-    if (!isInitOnly) {
-      if (!cmd.hasOption(OPT_NUM_KEYS)) {
-        throw new IllegalArgumentException(OPT_NUM_KEYS + " must be specified in "
-            + "read or write mode");
-      }
-      startKey = parseLong(cmd.getOptionValue(OPT_START_KEY,
-          String.valueOf(DEFAULT_START_KEY)), 0, Long.MAX_VALUE);
-      long numKeys = parseLong(cmd.getOptionValue(OPT_NUM_KEYS), 1,
-          Long.MAX_VALUE - startKey);
-      endKey = startKey + numKeys;
-      isSkipInit = cmd.hasOption(OPT_SKIP_INIT);
-      System.out.println("Key range: [" + startKey + ".." + (endKey - 1) + "]");
-    }
-
     encodeInCacheOnly = cmd.hasOption(OPT_ENCODE_IN_CACHE_ONLY);
     parseColumnFamilyOptions(cmd);
 
@@ -299,6 +274,8 @@ public class LoadTestTool extends Abstra
       System.out.println("Percent of keys to verify: " + verifyPercent);
       System.out.println("Reader threads: " + numReaderThreads);
     }
+
+    System.out.println("Key range: [" + startKey + ".." + (endKey - 1) + "]");
   }
 
   private void parseColumnFamilyOptions(CommandLine cmd) {
@@ -319,27 +296,15 @@ public class LoadTestTool extends Abstra
         StoreFile.BloomType.valueOf(bloomStr);
   }
 
-  public void initTestTable() throws IOException {
-    HBaseTestingUtility.createPreSplitLoadTestTable(conf, tableName,
-        COLUMN_FAMILY, compressAlgo, dataBlockEncodingAlgo);
-    applyColumnFamilyOptions(tableName, COLUMN_FAMILIES);
-  }
-
   @Override
-  protected int doWork() throws IOException {
+  protected void doWork() throws IOException {
     if (cmd.hasOption(OPT_ZK_QUORUM)) {
       conf.set(HConstants.ZOOKEEPER_QUORUM, cmd.getOptionValue(OPT_ZK_QUORUM));
     }
 
-    if (isInitOnly) {
-      LOG.info("Initializing only; no reads or writes");
-      initTestTable();
-      return 0;
-    }
-
-    if (!isSkipInit) {
-      initTestTable();
-    }
+    HBaseTestingUtility.createPreSplitLoadTestTable(conf, tableName,
+        COLUMN_FAMILY, compressAlgo, dataBlockEncodingAlgo);
+    applyColumnFamilyOptions(tableName, COLUMN_FAMILIES);
 
     if (isWrite) {
       writerThreads = new MultiThreadedWriter(conf, tableName, COLUMN_FAMILY);
@@ -378,16 +343,6 @@ 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/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java Sun Dec 23 19:34:53 2012
@@ -89,7 +89,7 @@ public class RestartMetaTest extends Abs
   }
 
   @Override
-  protected int doWork() throws Exception {
+  protected void doWork() throws IOException {
     ProcessBasedLocalHBaseCluster hbaseCluster =
         new ProcessBasedLocalHBaseCluster(conf, hbaseHome, numRegionServers);
 
@@ -130,7 +130,6 @@ public class RestartMetaTest extends Abs
           + Bytes.toStringBinary(result.getFamilyMap(HConstants.CATALOG_FAMILY)
               .get(HConstants.SERVER_QUALIFIER)));
     }
-    return 0;
   }
 
   @Override

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java Sun Dec 23 19:34:53 2012
@@ -90,26 +90,20 @@ public class TestFSUtils {
       // given the default replication factor is 3, the same as the number of
       // datanodes; the locality index for each host should be 100%,
       // or getWeight for each host should be the same as getUniqueBlocksWeights
-      final long maxTime = System.currentTimeMillis() + 2000;
-      boolean ok;
-      do {
-        ok = true;
-        FileStatus status = fs.getFileStatus(testFile);
-        HDFSBlocksDistribution blocksDistribution =
-          FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
-        long uniqueBlocksTotalWeight =
-          blocksDistribution.getUniqueBlocksTotalWeight();
-        for (String host : hosts) {
-          long weight = blocksDistribution.getWeight(host);
-          ok = (ok && uniqueBlocksTotalWeight == weight);
-        }
-      } while (!ok && System.currentTimeMillis() < maxTime);
-      assertTrue(ok);
-      } finally {
+      FileStatus status = fs.getFileStatus(testFile);
+      HDFSBlocksDistribution blocksDistribution =
+        FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
+      long uniqueBlocksTotalWeight =
+        blocksDistribution.getUniqueBlocksTotalWeight();
+      for (String host : hosts) {
+        long weight = blocksDistribution.getWeight(host);
+        assertTrue(uniqueBlocksTotalWeight == weight);
+      }
+    } finally {
       htu.shutdownMiniDFSCluster();
     }
 
-
+    
     try {
       // set up a cluster with 4 nodes
       String hosts[] = new String[] { "host1", "host2", "host3", "host4" };
@@ -124,22 +118,16 @@ public class TestFSUtils {
       // given the default replication factor is 3, we will have total of 9
       // replica of blocks; thus the host with the highest weight should have
       // weight == 3 * DEFAULT_BLOCK_SIZE
-      final long maxTime = System.currentTimeMillis() + 2000;
-      long weight;
-      long uniqueBlocksTotalWeight;
-      do {
-        FileStatus status = fs.getFileStatus(testFile);
-        HDFSBlocksDistribution blocksDistribution =
-          FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
-        uniqueBlocksTotalWeight = blocksDistribution.getUniqueBlocksTotalWeight();
-
-        String tophost = blocksDistribution.getTopHosts().get(0);
-        weight = blocksDistribution.getWeight(tophost);
-
-        // NameNode is informed asynchronously, so we may have a delay. See HBASE-6175
-      } while (uniqueBlocksTotalWeight != weight && System.currentTimeMillis() < maxTime);
+      FileStatus status = fs.getFileStatus(testFile);
+      HDFSBlocksDistribution blocksDistribution =
+        FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
+      long uniqueBlocksTotalWeight =
+        blocksDistribution.getUniqueBlocksTotalWeight();
+      
+      String tophost = blocksDistribution.getTopHosts().get(0);
+      long weight = blocksDistribution.getWeight(tophost);
       assertTrue(uniqueBlocksTotalWeight == weight);
-
+      
     } finally {
       htu.shutdownMiniDFSCluster();
     }
@@ -158,16 +146,11 @@ public class TestFSUtils {
       
       // given the default replication factor is 3, we will have total of 3
       // replica of blocks; thus there is one host without weight
-      final long maxTime = System.currentTimeMillis() + 2000;
-      HDFSBlocksDistribution blocksDistribution;
-      do {
-        FileStatus status = fs.getFileStatus(testFile);
-        blocksDistribution = FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
-        // NameNode is informed asynchronously, so we may have a delay. See HBASE-6175
-      }
-      while (blocksDistribution.getTopHosts().size() != 3 && System.currentTimeMillis() < maxTime);
+      FileStatus status = fs.getFileStatus(testFile);
+      HDFSBlocksDistribution blocksDistribution =
+        FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
       assertEquals("Wrong number of hosts distributing blocks.", 3,
-        blocksDistribution.getTopHosts().size());
+          blocksDistribution.getTopHosts().size());
     } finally {
       htu.shutdownMiniDFSCluster();
     }

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java Sun Dec 23 19:34:53 2012
@@ -70,9 +70,6 @@ import org.apache.hadoop.hbase.ipc.HRegi
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
-import org.apache.hadoop.hbase.util.HBaseFsck.PrintingErrorReporter;
-import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
@@ -1316,160 +1313,6 @@ public class TestHBaseFsck {
   }
 
   /**
-   * Test -noHdfsChecking option can detect and fix assignments issue.
-   */
-  @Test
-  public void testFixAssignmentsAndNoHdfsChecking() throws Exception {
-    String table = "testFixAssignmentsAndNoHdfsChecking";
-    try {
-      setupTable(table);
-      assertEquals(ROWKEYS.length, countRows());
-
-      // Mess it up by closing a region
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
-        Bytes.toBytes("B"), true, false, false, false);
-
-      // verify there is no other errors
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-        ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
-
-      // verify that noHdfsChecking report the same errors
-      HBaseFsck fsck = new HBaseFsck(conf);
-      fsck.connect();
-      fsck.setDisplayFullReport(); // i.e. -details
-      fsck.setTimeLag(0);
-      fsck.setCheckHdfs(false);
-      fsck.onlineHbck();
-      assertErrors(fsck, new ERROR_CODE[] {
-        ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
-
-      // verify that fixAssignments works fine with noHdfsChecking
-      fsck = new HBaseFsck(conf);
-      fsck.connect();
-      fsck.setDisplayFullReport(); // i.e. -details
-      fsck.setTimeLag(0);
-      fsck.setCheckHdfs(false);
-      fsck.setFixAssignments(true);
-      fsck.onlineHbck();
-      assertTrue(fsck.shouldRerun());
-      fsck.onlineHbck();
-      assertNoErrors(fsck);
-
-      assertEquals(ROWKEYS.length, countRows());
-    } finally {
-      deleteTable(table);
-    }
-  }
-
-  /**
-   * Test -noHdfsChecking option can detect region is not in meta but deployed.
-   * However, it can not fix it without checking Hdfs because we need to get
-   * the region info from Hdfs in this case, then to patch the meta.
-   */
-  @Test
-  public void testFixMetaNotWorkingWithNoHdfsChecking() throws Exception {
-    String table = "testFixMetaNotWorkingWithNoHdfsChecking";
-    try {
-      setupTable(table);
-      assertEquals(ROWKEYS.length, countRows());
-
-      // Mess it up by deleting a region from the metadata
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
-        Bytes.toBytes("B"), false, true, false, false);
-
-      // verify there is no other errors
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-        ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN});
-
-      // verify that noHdfsChecking report the same errors
-      HBaseFsck fsck = new HBaseFsck(conf);
-      fsck.connect();
-      fsck.setDisplayFullReport(); // i.e. -details
-      fsck.setTimeLag(0);
-      fsck.setCheckHdfs(false);
-      fsck.onlineHbck();
-      assertErrors(fsck, new ERROR_CODE[] {
-        ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN});
-
-      // verify that fixMeta doesn't work with noHdfsChecking
-      fsck = new HBaseFsck(conf);
-      fsck.connect();
-      fsck.setDisplayFullReport(); // i.e. -details
-      fsck.setTimeLag(0);
-      fsck.setCheckHdfs(false);
-      fsck.setFixAssignments(true);
-      fsck.setFixMeta(true);
-      fsck.onlineHbck();
-      assertFalse(fsck.shouldRerun());
-      assertErrors(fsck, new ERROR_CODE[] {
-        ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN});
-    } finally {
-      deleteTable(table);
-    }
-  }
-
-  /**
-   * Test -fixHdfsHoles doesn't work with -noHdfsChecking option,
-   * and -noHdfsChecking can't detect orphan Hdfs region.
-   */
-  @Test
-  public void testFixHdfsHolesNotWorkingWithNoHdfsChecking() throws Exception {
-    String table = "testFixHdfsHolesNotWorkingWithNoHdfsChecking";
-    try {
-      setupTable(table);
-      assertEquals(ROWKEYS.length, countRows());
-
-      // Mess it up by creating an overlap in the metadata
-      TEST_UTIL.getHBaseAdmin().disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
-        Bytes.toBytes("B"), true, true, false, true);
-      TEST_UTIL.getHBaseAdmin().enableTable(table);
-
-      HRegionInfo hriOverlap = createRegion(conf, tbl.getTableDescriptor(),
-        Bytes.toBytes("A2"), Bytes.toBytes("B"));
-      TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
-      TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
-        .waitForAssignment(hriOverlap);
-
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-        ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-        ERROR_CODE.HOLE_IN_REGION_CHAIN});
-
-      // verify that noHdfsChecking can't detect ORPHAN_HDFS_REGION
-      HBaseFsck fsck = new HBaseFsck(conf);
-      fsck.connect();
-      fsck.setDisplayFullReport(); // i.e. -details
-      fsck.setTimeLag(0);
-      fsck.setCheckHdfs(false);
-      fsck.onlineHbck();
-      assertErrors(fsck, new ERROR_CODE[] {
-        ERROR_CODE.HOLE_IN_REGION_CHAIN});
-
-      // verify that fixHdfsHoles doesn't work with noHdfsChecking
-      fsck = new HBaseFsck(conf);
-      fsck.connect();
-      fsck.setDisplayFullReport(); // i.e. -details
-      fsck.setTimeLag(0);
-      fsck.setCheckHdfs(false);
-      fsck.setFixHdfsHoles(true);
-      fsck.setFixHdfsOverlaps(true);
-      fsck.setFixHdfsOrphans(true);
-      fsck.onlineHbck();
-      assertFalse(fsck.shouldRerun());
-      assertErrors(fsck, new ERROR_CODE[] {
-        ERROR_CODE.HOLE_IN_REGION_CHAIN});
-    } finally {
-      if (TEST_UTIL.getHBaseAdmin().isTableDisabled(table)) {
-        TEST_UTIL.getHBaseAdmin().enableTable(table);
-      }
-      deleteTable(table);
-    }
-  }
-
-  /**
    * We don't have an easy way to verify that a flush completed, so we loop until we find a
    * legitimate hfile and return it.
    * @param fs
@@ -1657,120 +1500,6 @@ public class TestHBaseFsck {
     doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
   }
 
-  /**
-   * Test fixing lingering reference file.
-   */
-  @Test
-  public void testLingeringReferenceFile() throws Exception {
-    String table = "testLingeringReferenceFile";
-    try {
-      setupTable(table);
-      assertEquals(ROWKEYS.length, countRows());
-
-      // Mess it up by creating a fake reference file
-      FileSystem fs = FileSystem.get(conf);
-      Path tableDir= FSUtils.getTablePath(FSUtils.getRootDir(conf), table);
-      Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
-      Path famDir = new Path(regionDir, FAM_STR);
-      Path fakeReferenceFile = new Path(famDir, "fbce357483ceea.12144538");
-      fs.create(fakeReferenceFile);
-
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.LINGERING_REFERENCE_HFILE });
-      // fix reference file
-      doFsck(conf, true);
-      // check that reference file fixed
-      assertNoErrors(doFsck(conf, false));
-    } finally {
-      deleteTable(table);
-    }
-  }
-
-  /**
-   * Test pluggable error reporter. It can be plugged in
-   * from system property or configuration.
-   */
-  @Test
-  public void testErrorReporter() throws Exception {
-    try {
-      MockErrorReporter.calledCount = 0;
-      doFsck(conf, false);
-      assertEquals(MockErrorReporter.calledCount, 0);
-
-      conf.set("hbasefsck.errorreporter", MockErrorReporter.class.getName());
-      doFsck(conf, false);
-      assertTrue(MockErrorReporter.calledCount > 20);
-    } finally {
-      conf.set("hbasefsck.errorreporter",
-        PrintingErrorReporter.class.getName());
-      MockErrorReporter.calledCount = 0;
-    }
-  }
-
-  static class MockErrorReporter implements ErrorReporter {
-    static int calledCount = 0;
-
-    public void clear() {
-      calledCount++;
-    }
-
-    public void report(String message) {
-      calledCount++;
-    }
-
-    public void reportError(String message) {
-      calledCount++;
-    }
-
-    public void reportError(ERROR_CODE errorCode, String message) {
-      calledCount++;
-    }
-
-    public void reportError(ERROR_CODE errorCode, String message, TableInfo table) {
-      calledCount++;
-    }
-
-    public void reportError(ERROR_CODE errorCode,
-        String message, TableInfo table, HbckInfo info) {
-      calledCount++;
-    }
-
-    public void reportError(ERROR_CODE errorCode, String message,
-        TableInfo table, HbckInfo info1, HbckInfo info2) {
-      calledCount++;
-    }
-
-    public int summarize() {
-      return ++calledCount;
-    }
-
-    public void detail(String details) {
-      calledCount++;
-    }
-
-    public ArrayList<ERROR_CODE> getErrorList() {
-      calledCount++;
-      return new ArrayList<ERROR_CODE>();
-    }
-
-    public void progress() {
-      calledCount++;
-    }
-
-    public void print(String message) {
-      calledCount++;
-    }
-
-    public void resetErrors() {
-      calledCount++;
-    }
-
-    public boolean tableHasErrors(TableInfo table) {
-      calledCount++;
-      return false;
-    }
-  }
-
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java Sun Dec 23 19:34:53 2012
@@ -100,7 +100,7 @@ public class TestSizeBasedThrottler {
           15, // fail if throttler's value
               // exceeds 15
           1000, // use 1000 threads
-          500 // wait for 500ms
+          200 // wait for 200ms
           );
     }
   }
@@ -114,7 +114,7 @@ public class TestSizeBasedThrottler {
           4, // fail if throttler's value
              // exceeds 4
           1000, // use 1000 threads
-          500 // wait for 500ms
+          200 // wait for 200ms
           );
     }
   }
@@ -128,7 +128,7 @@ public class TestSizeBasedThrottler {
           2, // fail if throttler's value
              // exceeds 2
           1000, // use 1000 threads
-          500 // wait for 500ms
+          200 // wait for 200ms
           );
     }
   }

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java Sun Dec 23 19:34:53 2012
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.util.HBas
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 
 public class HbckTestingUtil {
-  private static ExecutorService exec = new ScheduledThreadPoolExecutor(10);
   public static HBaseFsck doFsck(
       Configuration conf, boolean fix) throws Exception {
     return doFsck(conf, fix, null);
@@ -38,14 +37,14 @@ public class HbckTestingUtil {
 
   public static HBaseFsck doFsck(
       Configuration conf, boolean fix, String table) throws Exception {
-    return doFsck(conf, fix, fix, fix, fix,fix, fix, fix, fix, table);
+    return doFsck(conf, fix, fix, fix, fix,fix, fix, fix, table);
   }
 
   public static HBaseFsck doFsck(Configuration conf, boolean fixAssignments,
       boolean fixMeta, boolean fixHdfsHoles, boolean fixHdfsOverlaps,
       boolean fixHdfsOrphans, boolean fixTableOrphans, boolean fixVersionFile,
-      boolean fixReferenceFiles, String table) throws Exception {
-    HBaseFsck fsck = new HBaseFsck(conf, exec);
+      String table) throws Exception {
+    HBaseFsck fsck = new HBaseFsck(conf);
     fsck.connect();
     fsck.setDisplayFullReport(); // i.e. -details
     fsck.setTimeLag(0);
@@ -56,7 +55,6 @@ public class HbckTestingUtil {
     fsck.setFixHdfsOrphans(fixHdfsOrphans);
     fsck.setFixTableOrphans(fixTableOrphans);
     fsck.setFixVersionFile(fixVersionFile);
-    fsck.setFixReferenceFiles(fixReferenceFiles);
     if (table != null) {
       fsck.includeTable(table);
     }
@@ -73,6 +71,7 @@ public class HbckTestingUtil {
    */
   public static HBaseFsck doHFileQuarantine(Configuration conf, String table) throws Exception {
     String[] args = {"-sidelineCorruptHFiles", "-ignorePreCheckPermission", table};
+    ExecutorService exec = new ScheduledThreadPoolExecutor(10);
     HBaseFsck hbck = new HBaseFsck(conf, exec);
     hbck.exec(exec, args);
     return hbck;

Modified: hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java?rev=1425513&r1=1425512&r2=1425513&view=diff
==============================================================================
--- hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java (original)
+++ hbase/branches/0.94-test/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java Sun Dec 23 19:34:53 2012
@@ -27,7 +27,6 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.zookeeper.ZKTable.TableState;
 import org.apache.zookeeper.KeeperException;
@@ -111,9 +110,14 @@ public class TestZKTable {
     assertFalse(zkt.isTablePresent(name));
   }
 
-  private void runTest9294CompatibilityTest(String tableName, Configuration conf)
-  throws Exception {
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+  /**
+   * Test that ZK table writes table state in formats expected by 0.92 and 0.94 clients
+   */
+  @Test
+  public void test9294Compatibility() throws Exception {
+    final String tableName = "test9294Compatibility";
+
+    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
       tableName, abortable, true);
     ZKTable zkt = new ZKTable(zkw);
     zkt.setEnabledTable(tableName);
@@ -125,22 +129,6 @@ public class TestZKTable {
   }
 
   /**
-   * Test that ZK table writes table state in formats expected by 0.92 and 0.94 clients
-   */
-  @Test
-  public void test9294Compatibility() throws Exception {
-    // without useMulti
-    String tableName = "test9294Compatibility";
-    runTest9294CompatibilityTest(tableName, TEST_UTIL.getConfiguration());
-
-    // with useMulti
-    tableName = "test9294CompatibilityWithMulti";
-    Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
-    conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
-    runTest9294CompatibilityTest(tableName, conf);
-  }
-
-  /**
    * RecoverableZookeeper that throws a KeeperException after throwExceptionInNumOperations
    */
   class ThrowingRecoverableZookeeper extends RecoverableZooKeeper {
@@ -182,17 +170,14 @@ public class TestZKTable {
    * Because two ZooKeeper nodes are written for each table state transition
    * {@link ZooKeeperWatcher#masterTableZNode} and {@link ZooKeeperWatcher#masterTableZNode92}
    * it is possible that we fail in between the two operations and are left with
-   * inconsistent state (when hbase.zookeeper.useMulti is false).
-   * Check that we can get back to a consistent state by retrying the operation.
+   * inconsistent state.  Check that we can get back to a consistent state by
+   * retrying the operation.
    */
   @Test
   public void testDisableTableRetry() throws Exception {
     final String tableName = "testDisableTableRetry";
 
-    Configuration conf = TEST_UTIL.getConfiguration();
-    // test only relevant if useMulti is false
-    conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
       tableName, abortable, true);
     ThrowingRecoverableZookeeper throwing = new ThrowingRecoverableZookeeper(zkw);
     ZooKeeperWatcher spyZookeeperWatcher = Mockito.spy(zkw);
@@ -226,10 +211,7 @@ public class TestZKTable {
   public void testEnableTableRetry() throws Exception {
     final String tableName = "testEnableTableRetry";
 
-    Configuration conf = TEST_UTIL.getConfiguration();
-    // test only relevant if useMulti is false
-    conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
       tableName, abortable, true);
     ThrowingRecoverableZookeeper throwing = new ThrowingRecoverableZookeeper(zkw);
     ZooKeeperWatcher spyZookeeperWatcher = Mockito.spy(zkw);
@@ -263,10 +245,7 @@ public class TestZKTable {
   public void testDeleteTableRetry() throws Exception {
     final String tableName = "testEnableTableRetry";
 
-    Configuration conf = TEST_UTIL.getConfiguration();
-    // test only relevant if useMulti is false
-    conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
       tableName, abortable, true);
     ThrowingRecoverableZookeeper throwing = new ThrowingRecoverableZookeeper(zkw);
     ZooKeeperWatcher spyZookeeperWatcher = Mockito.spy(zkw);