You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2013/08/03 20:18:22 UTC

svn commit: r1510049 - in /hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase: HBaseClusterTestCase.java HBaseTestingUtility.java TestEmptyMetaInfo.java master/TestRegionPlacement.java regionserver/TestStoreFile.java

Author: liyin
Date: Sat Aug  3 18:18:22 2013
New Revision: 1510049

URL: http://svn.apache.org/r1510049
Log:
[master] Fix fsimage locking issue for the unit tests

Author: liyintang

Summary:
There are multiple unit test failures caused by the FSImage file locked up by previous unclean shutdown. The current solution is to set different FSImage/FSEdit directory for each unit tests. And also make sure the MiniDFSCluster can be shutdown cleanly.

In the meanwhile, try to fix other issues for TestStoreFile as well as TestRegionPlacement

Test Plan: run all the unit tests through the jenkins

Reviewers: manukranthk

Reviewed By: manukranthk

CC: hbase-eng@

Differential Revision: https://phabricator.fb.com/D913231

Task ID: 2688608

Modified:
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseClusterTestCase.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestEmptyMetaInfo.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseClusterTestCase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseClusterTestCase.java?rev=1510049&r1=1510048&r2=1510049&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseClusterTestCase.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseClusterTestCase.java Sat Aug  3 18:18:22 2013
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,6 +33,8 @@ import org.apache.hadoop.hbase.client.HT
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.junit.After;
+import org.junit.Before;
 
 /**
  * Abstract base class for HBase cluster junit tests.  Spins up an hbase
@@ -119,13 +122,18 @@ public abstract class HBaseClusterTestCa
   }
 
   @Override
+  @Before
   protected void setUp() throws Exception {
     try {
       if (this.startDfs) {
-        // This spews a bunch of warnings about missing scheme. TODO: fix.
-        this.dfsCluster = new MiniDFSCluster(0, this.conf, 2, true, true, true,
-          null, null, null, null);
+        // This spews a bunch of warnings about missing scheme.
+        try {
+          this.dfsCluster = new HBaseTestingUtility().startMiniDFSCluster(2);
+        } catch (Exception e) {
+          LOG.error("Failed to restart the MiniDFSCluster ", e);
 
+
+        }
         // mangle the conf so that the fs parameter points to the minidfs we
         // just started up
         FileSystem filesystem = dfsCluster.getFileSystem();
@@ -164,34 +172,22 @@ public abstract class HBaseClusterTestCa
   }
 
   @Override
+  @After
   protected void tearDown() throws Exception {
-    if (!openMetaTable) {
-      // open the META table now to ensure cluster is running before shutdown.
-      new HTable(conf, HConstants.META_TABLE_NAME);
-    }
-    super.tearDown();
     try {
-      HConnectionManager.deleteConnectionInfo(conf, true);
-      if (this.cluster != null) {
-        try {
-          this.cluster.shutdown();
-        } catch (Exception e) {
-          LOG.warn("Closing mini dfs", e);
-        }
-        try {
-          this.zooKeeperCluster.shutdown();
-        } catch (IOException e) {
-          LOG.warn("Shutting down ZooKeeper cluster", e);
-        }
-      }
-      if (startDfs) {
-        shutdownDfs(dfsCluster);
+      if (!openMetaTable) {
+        // open the META table now to ensure cluster is running before shutdown.
+        new HTable(conf, HConstants.META_TABLE_NAME);
       }
+      super.tearDown();
+      HConnectionManager.deleteConnectionInfo(conf, true);
     } catch (Exception e) {
-      LOG.error(e);
+      LOG.error("Unexpected exception during shutdown ", e);
+    } finally {
+        try {this.cluster.shutdown();} catch (Throwable e) {}
+        try {this.dfsCluster.shutdown();} catch (Throwable e) {}
+        try {this.zooKeeperCluster.shutdown();} catch (Throwable e) {}
     }
-    // ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
-    //  "Temporary end-of-test thread dump debugging HADOOP-2040: " + getName());
   }
 
 

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1510049&r1=1510048&r2=1510049&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Sat Aug  3 18:18:22 2013
@@ -280,7 +280,8 @@ public class HBaseTestingUtility {
     createDirAndSetProperty("mapred_output", "mapred.output.dir");
     createDirAndSetProperty("mapred_local", "mapred.local.dir");
     createDirAndSetProperty("mapred_system", "mapred.system.dir");
-    createDirAndSetProperty("mapred_temp", "mapred.temp.dir");
+    createDirAndSetProperty("fsimage", "dfs.name.dir");
+    createDirAndSetProperty("fsedit", "dfs.name.edits.dir");
   }
 
   /**

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestEmptyMetaInfo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestEmptyMetaInfo.java?rev=1510049&r1=1510048&r2=1510049&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestEmptyMetaInfo.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestEmptyMetaInfo.java Sat Aug  3 18:18:22 2013
@@ -28,19 +28,46 @@ import org.apache.hadoop.hbase.client.Re
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests master cleanup of rows in meta table where there is no HRegionInfo
  */
-public class TestEmptyMetaInfo extends HBaseClusterTestCase {
+public class TestEmptyMetaInfo  {
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static int SLAVES = 3;
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster(SLAVES);
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+
   /**
    * Insert some bogus rows in meta. Master should clean them up.
    * @throws IOException
    */
+  @Test
   public void testEmptyMetaInfo() throws IOException {
-    HTable t = new HTable(conf, HConstants.META_TABLE_NAME);
+    HTable t = new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
     final int COUNT = 5;
-    final byte [] tableName = Bytes.toBytes(getName());
+    final byte [] tableName = Bytes.toBytes("testEmptyMetaInfo");
     for (int i = 0; i < COUNT; i++) {
       byte [] regionName = HRegionInfo.createRegionName(tableName,
         Bytes.toBytes(i == 0? "": Integer.toString(i)),
@@ -51,8 +78,8 @@ public class TestEmptyMetaInfo extends H
       t.put(put);
     }
     long sleepTime =
-      conf.getLong("hbase.master.meta.thread.rescanfrequency", 10000);
-    int tries = conf.getInt("hbase.client.retries.number", 5);
+      TEST_UTIL.getConfiguration().getLong("hbase.master.meta.thread.rescanfrequency", 10000);
+    int tries = TEST_UTIL.getConfiguration().getInt("hbase.client.retries.number", 5);
     int count = 0;
     do {
       tries -= 1;

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java?rev=1510049&r1=1510048&r2=1510049&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java Sat Aug  3 18:18:22 2013
@@ -104,7 +104,7 @@ public class TestRegionPlacement {
     assertTrue(loadBalancer instanceof AssignmentLoadBalancer);
   }
 
-  @Test(timeout = 180000)
+  @Test(timeout = 360000)
   public void testRegionPlacement() throws Exception {
     AssignmentPlan currentPlan;
     // ONLY meta regions, ROOT and META, are assigned at beginning.

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java?rev=1510049&r1=1510048&r2=1510049&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java Sat Aug  3 18:18:22 2013
@@ -74,7 +74,6 @@ public class TestStoreFile extends HBase
 
   @Override
   public void setUp() throws Exception {
-    super.setUp();
     startingMetrics = SchemaMetrics.getMetricsSnapshot();
     try {
       this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
@@ -85,6 +84,7 @@ public class TestStoreFile extends HBase
     } catch (IOException e) {
       shutdownDfs(cluster);
     }
+    super.setUp();
   }
 
   @Override