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

svn commit: r1340396 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/regionserver/HRegion.java test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java

Author: ramkrishna
Date: Sat May 19 09:57:48 2012
New Revision: 1340396

URL: http://svn.apache.org/viewvc?rev=1340396&view=rev
Log:
HBASE-5840 Open Region FAILED_OPEN doesn't clear the TaskMonitor Status, keeps showing the old status (Rajesh)

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1340396&r1=1340395&r2=1340396&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Sat May 19 09:57:48 2012
@@ -453,11 +453,27 @@ public class HRegion implements HeapSize
    * @throws IOException e
    */
   public long initialize(final CancelableProgressable reporter)
-  throws IOException {
+      throws IOException {
 
     MonitoredTask status = TaskMonitor.get().createStatus(
         "Initializing region " + this);
 
+    long nextSeqId = -1;
+    try {
+      nextSeqId = initializeRegionInternals(reporter, status);
+      return nextSeqId;
+    } finally {
+      // nextSeqid will be -1 if the initialization fails.
+      // At least it will be 0 otherwise.
+      if (nextSeqId == -1) {
+        status.abort("Exception during region " + this.getRegionNameAsString()
+            + " initialization.");
+      }
+    }
+  }
+
+  private long initializeRegionInternals(final CancelableProgressable reporter,
+      MonitoredTask status) throws IOException, UnsupportedEncodingException {
     if (coprocessorHost != null) {
       status.setStatus("Running coprocessor pre-open hook");
       coprocessorHost.preOpen();

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1340396&r1=1340395&r2=1340396&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Sat May 19 09:57:48 2012
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.filter.Pr
 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;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
@@ -88,6 +89,7 @@ import org.apache.hadoop.hbase.util.Pair
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 import com.google.common.collect.Lists;
 
@@ -3199,6 +3201,45 @@ public class TestHRegion extends HBaseTe
       this.region = null;
     }
   }
+  
+  /**
+   * Testcase to check state of region initialization task set to ABORTED or not if any exceptions
+   * during initialization
+   * 
+   * @throws Exception
+   */
+  @Test
+  public void testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization() throws Exception {
+    HRegionInfo info = null;
+    try {
+      FileSystem fs = Mockito.mock(FileSystem.class);
+      Mockito.when(fs.exists((Path) Mockito.anyObject())).thenThrow(new IOException());
+      HTableDescriptor htd = new HTableDescriptor(tableName);
+      htd.addFamily(new HColumnDescriptor("cf"));
+      info = new HRegionInfo(htd.getName(), HConstants.EMPTY_BYTE_ARRAY,
+          HConstants.EMPTY_BYTE_ARRAY, false);
+      Path path = new Path(DIR + "testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization");
+      // no where we are instantiating HStore in this test case so useTableNameGlobally is null. To
+      // avoid NullPointerException we are setting useTableNameGlobally to false.
+      SchemaMetrics.setUseTableNameInTest(false);
+      region = HRegion.newHRegion(path, null, fs, conf, info, htd, null);
+      // region initialization throws IOException and set task state to ABORTED.
+      region.initialize();
+      fail("Region initialization should fail due to IOException");
+    } catch (IOException io) {
+      List<MonitoredTask> tasks = TaskMonitor.get().getTasks();
+      for (MonitoredTask monitoredTask : tasks) {
+        if (!(monitoredTask instanceof MonitoredRPCHandler)
+            && monitoredTask.getDescription().contains(region.toString())) {
+          assertTrue("Region state should be ABORTED.",
+              monitoredTask.getState().equals(MonitoredTask.State.ABORTED));
+          break;
+        }
+      }
+    } finally {
+      HRegion.closeHRegion(region);
+    }
+  }
 
   public void testIndexesScanWithOneDeletedRow() throws IOException {
     byte[] tableName = Bytes.toBytes("testIndexesScanWithOneDeletedRow");