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 2014/04/04 20:18:23 UTC

svn commit: r1584850 - in /hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver: TestHRegionClose.java TestHRegionDoubleClose.java

Author: liyin
Date: Fri Apr  4 18:18:22 2014
New Revision: 1584850

URL: http://svn.apache.org/r1584850
Log:
[HBASE-10901] Fix TestHRegionClose unstable bug

Author: daviddeng

Summary:
The bug was introduced by `HBaseTestingUtility.createMultiRegions` and his friends. Use `createTable` with multi regions to replace it.

Merge `TestHRegionClose` and `TestHRegionDoubleClose`

Test Plan: `TestHRegionClose`

Reviewers: manukranthk, gauravm, liyintang

Reviewed By: manukranthk

CC: hbase-eng@

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

Task ID: 3863945

Modified:
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionClose.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionDoubleClose.java

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionClose.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionClose.java?rev=1584850&r1=1584849&r2=1584850&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionClose.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionClose.java Fri Apr  4 18:18:22 2014
@@ -19,7 +19,11 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.hadoop.conf.Configuration;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.HTable;
@@ -30,14 +34,12 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
 import org.apache.zookeeper.data.Stat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestHRegionClose {
+  private static final Log LOG = LogFactory.getLog(TestHRegionClose.class);
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   protected static byte[][] FAMILIES = { Bytes.toBytes("f1"),
       Bytes.toBytes("f2"), Bytes.toBytes("f3"), Bytes.toBytes("f4") };
@@ -52,22 +54,21 @@ public class TestHRegionClose {
 
     // Build some data.
     byte[] tableName = Bytes.toBytes(getClass().getSimpleName());
-    TEST_UTIL.createTable(tableName, FAMILIES);
+    TEST_UTIL.createTable(tableName, FAMILIES, 1, Bytes.toBytes("bbb"),
+        Bytes.toBytes("yyy"), 25);
     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
     for (int i = 0; i < FAMILIES.length; i++) {
       byte[] columnFamily = FAMILIES[i];
-      TEST_UTIL.createMultiRegions(table, columnFamily);
       TEST_UTIL.loadTable(table, columnFamily);
     }
 
     // Pick a regionserver.
-    Configuration conf = TEST_UTIL.getConfiguration();
     server = TEST_UTIL.getHBaseCluster().getRegionServer(0);
 
     HRegion[] region = server.getOnlineRegionsAsArray();
     regionInfo = region[0].getRegionInfo();
 
-    // Some initializtion relevant to zk.
+    // Some initialization relevant to zk.
     zkWrapper = server.getZooKeeperWrapper();
     regionZNode = zkWrapper.getZNode(
         zkWrapper.getRegionInTransitionZNode(), regionInfo.getEncodedName());
@@ -95,7 +96,15 @@ public class TestHRegionClose {
   }
 
   @Test
-  public void mainTest() throws Exception {
+  public void singleClose() throws Exception {
+    tryCloseRegion();
+  }
+
+  @Test
+  public void doubleClose() throws Exception {
+    tryCloseRegion();
+    LOG.info("Trying to close the region again, to check that the RegionServer "
+        + "is idempotent. i.e. CLOSED -> CLOSING transition bug");
     tryCloseRegion();
   }
 

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionDoubleClose.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionDoubleClose.java?rev=1584850&r1=1584849&r2=1584850&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionDoubleClose.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionDoubleClose.java Fri Apr  4 18:18:22 2014
@@ -1,46 +0,0 @@
-/**
- * Copyright 2010 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * Test that RegionServer ignores a close request from Master
- * if it already successfully closed a region.  There was an
- * issue where it was changing the ZooKeeper state from
- * CLOSED to CLOSING
- */
-public class TestHRegionDoubleClose extends TestHRegionClose {
-  private static final Log LOG = LogFactory.getLog(TestHRegionDoubleClose.class);
-
-  @Override
-  public void testMemstoreCleanup() throws Exception {
-    // inherit to skip this redundant case
-  }
-
-  @Override
-  public void mainTest() throws Exception {
-    tryCloseRegion();
-    LOG.info("Trying to close the region again, to check that the RegionServer "
-        + "is idempotent. i.e. CLOSED -> CLOSING transition bug");
-    tryCloseRegion();
-  }
-}