You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jd...@apache.org on 2010/08/19 22:57:05 UTC

svn commit: r987310 - in /hbase/trunk: ./ src/test/java/org/apache/hadoop/hbase/ src/test/java/org/apache/hadoop/hbase/client/ src/test/java/org/apache/hadoop/hbase/regionserver/wal/

Author: jdcryans
Date: Thu Aug 19 20:57:05 2010
New Revision: 987310

URL: http://svn.apache.org/viewvc?rev=987310&view=rev
Log:
HBASE-2924  TestLogRolling doesn't use the right HLog half the time

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=987310&r1=987309&r2=987310&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Thu Aug 19 20:57:05 2010
@@ -482,6 +482,7 @@ Release 0.21.0 - Unreleased
    HBASE-2927  BaseScanner gets stale HRegionInfo in some race cases
    HBASE-2928  Fault in logic in BinaryPrefixComparator leads to
                ArrayIndexOutOfBoundsException (pranav via jgray)
+   HBASE-2924  TestLogRolling doesn't use the right HLog half the time
 
   IMPROVEMENTS
    HBASE-1760  Cleanup TODOs in HTable

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=987310&r1=987309&r2=987310&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Thu Aug 19 20:57:05 2010
@@ -676,6 +676,26 @@ public class HBaseTestingUtility {
   }
 
   /**
+   * Tool to get the reference to the region server object that holds the
+   * region of the specified user table.
+   * It first searches for the meta rows that contain the region of the
+   * specified table, then gets the index of that RS, and finally retrieves
+   * the RS's reference.
+   * @param tableName user table to lookup in .META.
+   * @return region server that holds it, null if the row doesn't exist
+   * @throws IOException
+   */
+  public HRegionServer getRSForFirstRegionInTable(byte[] tableName)
+      throws IOException {
+    List<byte[]> metaRows = getMetaTableRows(tableName);
+    if (metaRows == null || metaRows.size() == 0) {
+      return null;
+    }
+    int index = hbaseCluster.getServerWith(metaRows.get(0));
+    return hbaseCluster.getRegionServerThreads().get(index).getRegionServer();
+  }
+
+  /**
    * Starts a <code>MiniMRCluster</code> with a default number of
    * <code>TaskTracker</code>'s.
    *

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java?rev=987310&r1=987309&r2=987310&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java Thu Aug 19 20:57:05 2010
@@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -114,9 +115,7 @@ public class TestScannerTimeout {
    */
   @Test
   public void test2772() throws Exception {
-    int rs = TEST_UTIL.getHBaseCluster().getServerWith(
-        TEST_UTIL.getHBaseCluster().getRegions(
-            TABLE_NAME).get(0).getRegionName());
+    HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
     Scan scan = new Scan();
     // 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.
@@ -128,7 +127,7 @@ public class TestScannerTimeout {
     HTable higherScanTimeoutTable = new HTable(conf, TABLE_NAME);
     ResultScanner r = higherScanTimeoutTable.getScanner(scan);
     // This takes way less than SCANNER_TIMEOUT*100
-    TEST_UTIL.getHBaseCluster().getRegionServer(rs).abort("die!");
+    rs.abort("die!");
     Result[] results = r.next(NB_ROWS);
     assertEquals(NB_ROWS, results.length);
     r.close();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java?rev=987310&r1=987309&r2=987310&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java Thu Aug 19 20:57:05 2010
@@ -154,14 +154,15 @@ public class TestLogRolling  {
   private void startAndWriteData() throws IOException {
     // When the META table can be opened, the region servers are running
     new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
-    this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
-    this.log = server.getLog();
 
     // Create the test table and open it
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc);
     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
+
+    server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
+    this.log = server.getLog();
     for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
       Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
       put.add(HConstants.CATALOG_FAMILY, null, value);
@@ -228,8 +229,6 @@ public class TestLogRolling  {
   @SuppressWarnings("null")
   DatanodeInfo[] getPipeline(HLog log) throws IllegalArgumentException,
       IllegalAccessException, InvocationTargetException {
-
-    // kill a datanode in the pipeline to force a log roll on the next sync()
     OutputStream stm = log.getOutputStream();
     Method getPipeline = null;
     for (Method m : stm.getClass().getDeclaredMethods()) {
@@ -263,7 +262,19 @@ public class TestLogRolling  {
         .getDefaultReplication() > 1);
     // When the META table can be opened, the region servers are running
     new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
-    this.server = cluster.getRegionServer(0);
+    // Create the test table and open it
+    String tableName = getName();
+    HTableDescriptor desc = new HTableDescriptor(tableName);
+    desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+
+    if (admin.tableExists(tableName)) {
+      admin.disableTable(tableName);
+      admin.deleteTable(tableName);
+    }
+    admin.createTable(desc);
+    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
+
+    server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
     this.log = server.getLog();
 
     assertTrue("Need HDFS-826 for this test", log.canGetCurReplicas());
@@ -277,18 +288,6 @@ public class TestLogRolling  {
     dfsCluster.waitActive();
     assertTrue(dfsCluster.getDataNodes().size() >= fs.getDefaultReplication() + 1);
 
-    // Create the test table and open it
-    String tableName = getName();
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-
-    if (admin.tableExists(tableName)) {
-      admin.disableTable(tableName);
-      admin.deleteTable(tableName);
-    }
-    admin.createTable(desc);
-
-    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
     writeData(table, 2);
 
     table.setAutoFlush(true);
@@ -303,6 +302,7 @@ public class TestLogRolling  {
     DatanodeInfo[] pipeline = getPipeline(log);
     assertTrue(pipeline.length == fs.getDefaultReplication());
 
+    // kill a datanode in the pipeline to force a log roll on the next sync()
     assertTrue(dfsCluster.stopDataNode(pipeline[0].getName()) != null);
     Thread.sleep(10000);
     // this write should succeed, but trigger a log roll