You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/07/12 19:26:46 UTC

svn commit: r1502628 - in /hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase: ./ backup/ client/ coprocessor/ io/encoding/ regionserver/ util/

Author: stack
Date: Fri Jul 12 17:26:45 2013
New Revision: 1502628

URL: http://svn.apache.org/r1502628
Log:
HBASE-8934 Fix bunch of flaky tests

Modified:
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1502628&r1=1502627&r2=1502628&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Fri Jul 12 17:26:45 2013
@@ -88,8 +88,8 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooKeeper;
 
 /**
@@ -842,6 +842,8 @@ public class HBaseTestingUtility {
       desc.addFamily(hcd);
     }
     getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
+    // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
+    waitUntilAllRegionsAssigned(tableName);
     return new HTable(getConfiguration(), tableName);
   }
 
@@ -861,6 +863,8 @@ public class HBaseTestingUtility {
       desc.addFamily(new HColumnDescriptor(family));
     }
     getHBaseAdmin().createTable(desc);
+    // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
+    waitUntilAllRegionsAssigned(tableName);
     return new HTable(c, tableName);
   }
 
@@ -883,6 +887,8 @@ public class HBaseTestingUtility {
       desc.addFamily(hcd);
     }
     getHBaseAdmin().createTable(desc);
+    // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
+    waitUntilAllRegionsAssigned(tableName);
     return new HTable(c, tableName);
   }
 
@@ -917,6 +923,8 @@ public class HBaseTestingUtility {
       desc.addFamily(hcd);
     }
     getHBaseAdmin().createTable(desc);
+    // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
+    waitUntilAllRegionsAssigned(tableName);
     return new HTable(new Configuration(getConfiguration()), tableName);
   }
 
@@ -938,6 +946,8 @@ public class HBaseTestingUtility {
       desc.addFamily(hcd);
     }
     getHBaseAdmin().createTable(desc);
+    // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
+    waitUntilAllRegionsAssigned(tableName);
     return new HTable(new Configuration(getConfiguration()), tableName);
   }
 
@@ -961,6 +971,8 @@ public class HBaseTestingUtility {
       i++;
     }
     getHBaseAdmin().createTable(desc);
+    // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
+    waitUntilAllRegionsAssigned(tableName);
     return new HTable(new Configuration(getConfiguration()), tableName);
   }
 
@@ -1065,7 +1077,7 @@ public class HBaseTestingUtility {
   throws IOException {
     return loadRegion(r, f, false);
   }
-  
+
   /**
    * Load region with rows from 'aaa' to 'zzz'.
    * @param r Region
@@ -1183,7 +1195,7 @@ public class HBaseTestingUtility {
    *
    * @param table  The table to use for the data.
    * @param columnFamily  The family to insert the data into.
-   * @param cleanupFS  True if a previous region should be remove from the FS  
+   * @param cleanupFS  True if a previous region should be remove from the FS
    * @return count of regions created.
    * @throws IOException When creating the regions fails.
    */
@@ -1220,7 +1232,7 @@ public class HBaseTestingUtility {
       final byte[] columnFamily, byte [][] startKeys) throws IOException {
     return createMultiRegions(c, table, columnFamily, startKeys, true);
   }
-  
+
   public int createMultiRegions(final Configuration c, final HTable table,
           final byte[] columnFamily, byte [][] startKeys, boolean cleanupFS)
   throws IOException {
@@ -2242,10 +2254,10 @@ public class HBaseTestingUtility {
         HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), htd);
     return region;
   }
-  
+
   /**
    * Create region split keys between startkey and endKey
-   * 
+   *
    * @param startKey
    * @param endKey
    * @param numRegions the number of regions to be created. it has to be greater than 3.

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java?rev=1502628&r1=1502627&r2=1502628&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java Fri Jul 12 17:26:45 2013
@@ -37,7 +37,6 @@ 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;
@@ -247,9 +246,23 @@ public class TestHFileArchiving {
     // then delete the table so the hfiles get archived
     UTIL.deleteTable(TABLE_NAME);
 
-    // then get the files in the archive directory.
+    assertArchiveFiles(fs, storeFiles, 30000);
+  }
+
+  private void assertArchiveFiles(FileSystem fs, List<String> storeFiles, long timeout) throws IOException {
+    long end = System.currentTimeMillis() + timeout;
     Path archiveDir = HFileArchiveUtil.getArchivePath(UTIL.getConfiguration());
-    List<String> archivedFiles = getAllFileNames(fs, archiveDir);
+    List<String> archivedFiles = new ArrayList<String>();
+
+    // We have to ensure that the DeleteTableHandler is finished. HBaseAdmin.deleteXXX() can return before all files
+    // are archived. We should fix HBASE-5487 and fix synchronous operations from admin.
+    while (System.currentTimeMillis() < end) {
+      archivedFiles = getAllFileNames(fs, archiveDir);
+      if (archivedFiles.size() >= storeFiles.size()) {
+        break;
+      }
+    }
+
     Collections.sort(storeFiles);
     Collections.sort(archivedFiles);
 
@@ -310,23 +323,7 @@ public class TestHFileArchiving {
     // then delete the table so the hfiles get archived
     UTIL.getHBaseAdmin().deleteColumn(TABLE_NAME, TEST_FAM);
 
-    // then get the files in the archive directory.
-    Path archiveDir = HFileArchiveUtil.getArchivePath(UTIL.getConfiguration());
-    List<String> archivedFiles = getAllFileNames(fs, archiveDir);
-    Collections.sort(storeFiles);
-    Collections.sort(archivedFiles);
-
-    LOG.debug("Store files:");
-    for (int i = 0; i < storeFiles.size(); i++) {
-      LOG.debug(i + " - " + storeFiles.get(i));
-    }
-    LOG.debug("Archive files:");
-    for (int i = 0; i < archivedFiles.size(); i++) {
-      LOG.debug(i + " - " + archivedFiles.get(i));
-    }
-
-    assertTrue("Archived files are missing some of the store files!",
-      archivedFiles.containsAll(storeFiles));
+    assertArchiveFiles(fs, storeFiles, 30000);
 
     UTIL.deleteTable(TABLE_NAME);
   }
@@ -423,7 +420,7 @@ public class TestHFileArchiving {
     return fileNames;
   }
 
-  private List<String> getRegionStoreFiles(final FileSystem fs, final Path regionDir) 
+  private List<String> getRegionStoreFiles(final FileSystem fs, final Path regionDir)
       throws IOException {
     List<String> storeFiles = getAllFileNames(fs, regionDir);
     // remove all the non-storefile named files for the region

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1502628&r1=1502627&r2=1502628&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Fri Jul 12 17:26:45 2013
@@ -39,7 +39,21 @@ import java.util.concurrent.atomic.Atomi
 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.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotEnabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
@@ -48,12 +62,16 @@ import org.apache.hadoop.hbase.master.Ma
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.wal.HLogUtilsForTests;
-import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.junit.*;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 
@@ -263,7 +281,7 @@ public class TestAdmin {
     } catch (IOException e) {
     }
   }
-  
+
   @Test
   public void testDisableAndEnableTables() throws IOException {
     final byte [] row = Bytes.toBytes("row");
@@ -604,7 +622,7 @@ public class TestAdmin {
     assertEquals("Table should have 16 region", 16, regions.size());
     ht5.close();
   }
- 
+
   @Test
   public void testCreateTableWithRegions() throws IOException, InterruptedException {
 
@@ -776,8 +794,8 @@ public class TestAdmin {
     }
     ladmin.close();
   }
-  
-    
+
+
   @Test
   public void testCreateTableWithOnlyEmptyStartRow() throws IOException {
     byte[] tableName = Bytes.toBytes("testCreateTableWithOnlyEmptyStartRow");
@@ -791,7 +809,7 @@ public class TestAdmin {
     } catch (IllegalArgumentException e) {
     }
   }
-    
+
   @Test
   public void testCreateTableWithEmptyRowInTheSplitKeys() throws IOException {
     byte[] tableName = Bytes
@@ -1058,7 +1076,7 @@ public class TestAdmin {
      new HColumnDescriptor("/cfamily/name");
   }
 
-  @Test(timeout=36000)
+  @Test(timeout=300000)
   public void testEnableDisableAddColumnDeleteColumn() throws Exception {
     ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
     byte [] tableName = Bytes.toBytes("testMasterAdmin");
@@ -1300,10 +1318,14 @@ public class TestAdmin {
             .getServerName().getServerName());
       }
     }
-    Thread.sleep(1000);
-    onlineRegions = rs.getOnlineRegions();
+    boolean isInList = rs.getOnlineRegions().contains(info);
+    long timeout = System.currentTimeMillis() + 10000;
+    while ((System.currentTimeMillis() < timeout) && (isInList)) {
+      Thread.sleep(100);
+      isInList = rs.getOnlineRegions().contains(info);
+    }
     assertFalse("The region should not be present in online regions list.",
-        onlineRegions.contains(info));
+        isInList);
   }
 
   @Test
@@ -1348,7 +1370,7 @@ public class TestAdmin {
     }
 
     boolean isInList = rs.getOnlineRegions().contains(info);
-    long timeout = System.currentTimeMillis() + 2000;
+    long timeout = System.currentTimeMillis() + 10000;
     while ((System.currentTimeMillis() < timeout) && (isInList)) {
       Thread.sleep(100);
       isInList = rs.getOnlineRegions().contains(info);
@@ -1634,7 +1656,7 @@ public class TestAdmin {
       ct.stop();
     }
   }
-  
+
   @Test
   public void testRootTableSplit() throws Exception {
     Scan s = new Scan();
@@ -1646,7 +1668,7 @@ public class TestAdmin {
     List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(HConstants.ROOT_TABLE_NAME);
     assertEquals("ROOT region should not be splitted.",1, regions.size());
   }
-  
+
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java?rev=1502628&r1=1502627&r2=1502628&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java Fri Jul 12 17:26:45 2013
@@ -25,7 +25,9 @@ import static org.junit.Assert.fail;
 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.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -147,7 +149,7 @@ public class TestScannerTimeout {
     LOG.info("END ************ test2772");
 
   }
-  
+
   /**
    * Test that scanner won't miss any rows if the region server it was reading
    * from failed. Before 3686, it would skip rows in the scan.
@@ -163,7 +165,14 @@ public class TestScannerTimeout {
     scan.setCaching(SCANNER_CACHING);
     LOG.info("************ TEST3686A");
     MetaReader.fullScanMetaAndPrint(TEST_UTIL.getHBaseCluster().getMaster().getCatalogTracker());
-    HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE_NAME);
+    // Set a very high timeout, we want to test what happens when a RS
+    // fails but the region is recovered before the lease times out.
+    // Since the RS is already created, this conf is client-side only for
+    // this new table
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    conf.setInt(
+        HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, SCANNER_TIMEOUT*100);
+    HTable table = new HTable(conf, TABLE_NAME);
     LOG.info("START ************ TEST3686A---22");
 
     ResultScanner r = table.getScanner(scan);
@@ -183,7 +192,7 @@ public class TestScannerTimeout {
     table.close();
     LOG.info("************ END TEST3686A");
   }
-  
+
   /**
    * Make sure that no rows are lost if the scanner timeout is longer on the
    * client than the server, and the scan times out on the server but not the

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java?rev=1502628&r1=1502627&r2=1502628&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java Fri Jul 12 17:26:45 2013
@@ -20,23 +20,26 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
-
 /**
  * Tests unhandled exceptions thrown by coprocessors running on regionserver.
  * Expected result is that the master will remove the buggy coprocessor from
@@ -77,7 +80,7 @@ public class TestRegionServerCoprocessor
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test(timeout=30000)
+  @Test(timeout=60000)
   public void testExceptionFromCoprocessorDuringPut()
       throws IOException {
     // Set watches on the zookeeper nodes for all of the regionservers in the

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java?rev=1502628&r1=1502627&r2=1502628&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java Fri Jul 12 17:26:45 2013
@@ -67,7 +67,7 @@ public class TestChangingEncoding {
       new HBaseTestingUtility();
   private static final Configuration conf = TEST_UTIL.getConfiguration();
 
-  private static final int TIMEOUT_MS = 240000;
+  private static final int TIMEOUT_MS = 600000;
 
   private HBaseAdmin admin;
   private HColumnDescriptor hcd;
@@ -239,7 +239,7 @@ public class TestChangingEncoding {
     LOG.debug("Compacting table " + tableName);
     admin.majorCompact(tableName);
     HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
-    
+
     // Waiting for the compaction to start, at least .5s.
     final long maxWaitime = System.currentTimeMillis() + 500;
     boolean cont;
@@ -247,7 +247,7 @@ public class TestChangingEncoding {
       cont = rs.compactSplitThread.getCompactionQueueSize() == 0;
       Threads.sleep(1);
     } while (cont && System.currentTimeMillis() < maxWaitime);
-    
+
     while (rs.compactSplitThread.getCompactionQueueSize() > 0) {
       Threads.sleep(5);
     }

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java?rev=1502628&r1=1502627&r2=1502628&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java Fri Jul 12 17:26:45 2013
@@ -58,22 +58,22 @@ public class TestCompactionState {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=600000)
   public void testMajorCompaction() throws IOException, InterruptedException {
     compaction("testMajorCompaction", 8, CompactionState.MAJOR, false);
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=600000)
   public void testMinorCompaction() throws IOException, InterruptedException {
     compaction("testMinorCompaction", 15, CompactionState.MINOR, false);
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=600000)
   public void testMajorCompactionOnFamily() throws IOException, InterruptedException {
     compaction("testMajorCompactionOnFamily", 8, CompactionState.MAJOR, true);
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=600000)
   public void testMinorCompactionOnFamily() throws IOException, InterruptedException {
     compaction("testMinorCompactionOnFamily", 15, CompactionState.MINOR, true);
   }
@@ -214,7 +214,7 @@ public class TestCompactionState {
     }
     return count;
   }
-  
+
   private static void loadData(final HTable ht, final byte[][] families,
       final int rows, final int flushes) throws IOException {
     List<Put> puts = new ArrayList<Put>(rows);
@@ -234,7 +234,7 @@ public class TestCompactionState {
       puts.clear();
     }
   }
-  
+
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java?rev=1502628&r1=1502627&r2=1502628&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestSizeBasedThrottler.java Fri Jul 12 17:26:45 2013
@@ -100,7 +100,7 @@ public class TestSizeBasedThrottler {
           15, // fail if throttler's value
               // exceeds 15
           1000, // use 1000 threads
-          500 // wait for 500ms
+          5000 // wait for 5 sec
           );
     }
   }
@@ -114,7 +114,7 @@ public class TestSizeBasedThrottler {
           4, // fail if throttler's value
              // exceeds 4
           1000, // use 1000 threads
-          500 // wait for 500ms
+          5000 // wait for 5 sec
           );
     }
   }
@@ -128,7 +128,7 @@ public class TestSizeBasedThrottler {
           2, // fail if throttler's value
              // exceeds 2
           1000, // use 1000 threads
-          500 // wait for 500ms
+          5000 // wait for 5 sec
           );
     }
   }