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/03/14 18:12:12 UTC

svn commit: r1456544 - in /hbase/branches/0.95/hbase-server/src: main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java

Author: stack
Date: Thu Mar 14 17:12:11 2013
New Revision: 1456544

URL: http://svn.apache.org/r1456544
Log:
HBASE-7876 Got exception when manually triggers a split on an empty region

Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java?rev=1456544&r1=1456543&r2=1456544&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java Thu Mar 14 17:12:11 2013
@@ -288,13 +288,6 @@ public class SplitTransaction {
       if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
       throw new IOException(exceptionToThrow);
     }
-
-
-    if (hstoreFilesToSplit.size() == 0) {
-      String errorMsg = "No store files to split for the region "+this.parent.getRegionInfo();
-      LOG.error(errorMsg);
-      throw new IOException(errorMsg);
-    }
     if (!testing) {
       services.removeFromOnlineRegions(this.parent, null);
     }
@@ -608,6 +601,10 @@ public class SplitTransaction {
     // there's files to split. It then fires up everything, waits for
     // completion and finally checks for any exception
     int nbFiles = hstoreFilesToSplit.size();
+    if (nbFiles == 0) {
+      // no file needs to be splitted.
+      return;
+    }
     ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
     builder.setNameFormat("StoreFileSplitter-%1$d");
     ThreadFactory factory = builder.build();

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java?rev=1456544&r1=1456543&r2=1456544&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java Thu Mar 14 17:12:11 2013
@@ -641,51 +641,6 @@ public class TestSplitTransactionOnClust
     admin.flush(tableName);
   }
 
-  @Test
-  public void testShouldThrowIOExceptionIfStoreFileSizeIsEmptyAndShouldSuccessfullyExecuteRollback()
-      throws Exception {
-    final byte[] tableName = Bytes.toBytes("testRollBackShudBeSuccessfulIfStoreFileIsEmpty");
-    // Create table then get the single region for our new table.
-    createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
-    List<HRegion> regions = cluster.getRegions(tableName);
-    HRegionInfo hri = getAndCheckSingleTableRegion(regions);
-    int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
-    int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
-    HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
-    // Turn off balancer so it doesn't cut in and mess up our placements.
-    this.admin.setBalancerRunning(false, true);
-    // Turn off the meta scanner so it don't remove parent on us.
-    cluster.getMaster().setCatalogJanitorEnabled(false);
-    try {
-      printOutRegions(regionServer, "Initial regions: ");
-
-      // find a splittable region.  Refresh the regions list
-      regions = cluster.getRegions(tableName);
-      final HRegion region = findSplittableRegion(regions);
-      assertTrue("not able to find a splittable region", region != null);
-
-      // Now split.
-      SplitTransaction st = new MockedSplitTransaction(region, Bytes.toBytes("row2"));
-      try {
-        st.prepare();
-        st.execute(regionServer, regionServer);
-      } catch (IOException e) {
-        List<HRegion> daughters = cluster.getRegions(tableName);
-        assertTrue(daughters.size() == 1);
-
-        String node = ZKAssign.getNodeName(regionServer.getZooKeeper(),
-            region.getRegionInfo().getEncodedName());
-        assertFalse(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
-        assertTrue(st.rollback(regionServer, regionServer));
-        assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
-      }
-    } finally {
-      admin.setBalancerRunning(true, false);
-      cluster.getMaster().setCatalogJanitorEnabled(true);
-    }
-
-  }
-
   private void testSplitBeforeSettingSplittingInZKInternals() throws Exception {
     final byte[] tableName = Bytes.toBytes("testSplitBeforeSettingSplittingInZK");
     try {