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/04/09 20:18:21 UTC

svn commit: r1466167 - in /hbase/branches/0.89-fb/src/main: java/org/apache/hadoop/hbase/client/ java/org/apache/hadoop/hbase/ipc/ java/org/apache/hadoop/hbase/master/ ruby/ ruby/hbase/ ruby/shell/commands/

Author: liyin
Date: Tue Apr  9 18:18:20 2013
New Revision: 1466167

URL: http://svn.apache.org/r1466167
Log:
[master] Add wait interval to region reopener.

Author: shaneh

Summary:
Added wait interval to region reopener and exposed it to the shell
commands.

Test Plan:
Created test table and confirmed the wait interval and concurrently
closed regions parameters were respected.

Reviewers: liyintang, nspiegelberg

Reviewed By: nspiegelberg

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

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ThrottledRegionReopener.java
    hbase/branches/0.89-fb/src/main/ruby/hbase.rb
    hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb
    hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter.rb
    hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1466167&r1=1466166&r2=1466167&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Tue Apr  9 18:18:20 2013
@@ -605,6 +605,28 @@ public class HBaseAdmin {
   }
 
   /**
+   * Sets the wait interval to wait between closing and reopening of Regions.
+   * Asynchronous operation.
+   *
+   * @param waitInterval The interval in milliseconds to wait
+   *                     in between closing and reopening regions
+   */
+  public void setCloseRegionWaitInterval(final String tableName, int waitInterval) {
+    this.master.setCloseRegionWaitInterval(tableName, waitInterval);
+  }
+
+  /**
+   * Sets the number of Regions to close concurrently.
+   * Asynchronous operation.
+   *
+   * @param numConcurrentClose The number of Regions to close at the same time
+   *
+   */
+  public void setNumConcurrentCloseRegions(final String tableName, int numConcurrentClose) {
+    this.master.setNumConcurrentCloseRegions(tableName, numConcurrentClose);
+  }
+
+  /**
    * Batch alter a table. Only takes regions offline once and performs a single
    * update to .META.
    * Asynchronous operation.
@@ -942,7 +964,7 @@ public class HBaseAdmin {
    * Compact a column family within a region.
    * Asynchronous operation.
    *
-   * @param regionName region to compact
+   * @param tableOrRegionName region to compact
    * @param columnFamily column family within the region to compact
    * @throws IOException if a remote or network exception occurs
    */
@@ -965,7 +987,7 @@ public class HBaseAdmin {
    * Compact a column family within a region.
    * Asynchronous operation.
    *
-   * @param regionName region to compact
+   * @param tableOrRegionName region to compact
    * @param columnFamily column family within the region to compact
    * @throws IOException if a remote or network exception occurs
    */
@@ -1009,7 +1031,7 @@ public class HBaseAdmin {
    * Major compacts a column family within a region or table.
    * Asynchronous operation.
    *
-   * @param regionName region to compact
+   * @param tableOrRegionName region to compact
    * @param columnFamily column family within the region to compact
    * @throws IOException if a remote or network exception occurs
    */
@@ -1032,7 +1054,7 @@ public class HBaseAdmin {
    * Major compacts a column family within a region or table.
    * Asynchronous operation.
    *
-   * @param regionName region to compact
+   * @param tableOrRegionName region to compact
    * @param columnFamily column family within the region to compact
    * @throws IOException if a remote or network exception occurs
    */
@@ -1292,9 +1314,6 @@ public class HBaseAdmin {
    *
    * @param hsa
    *          the address of the RegionServer to stop
-   * @param numThreads
-   *          the number of threads to be used for HDFS quorum reads
-   *          <= 0 will disable quorum Reads.
    * @throws IOException
    *           if a remote or network exception occurs
    */

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java?rev=1466167&r1=1466166&r2=1466167&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java Tue Apr  9 18:18:20 2013
@@ -68,6 +68,7 @@ public interface HMasterInterface extend
    * Batch adds, modifies, and deletes columns from the specified table.
    * Any of the lists may be null, in which case those types of alterations
    * will not occur.
+   *
    * @param tableName table to modify
    * @param columnAdditions column descriptors to add to the table
    * @param columnModifications pairs of column names with new descriptors
@@ -75,9 +76,9 @@ public interface HMasterInterface extend
    * @throws IOException e
    */
   public void alterTable(final byte [] tableName,
-      List<HColumnDescriptor> columnAdditions,
-      List<Pair<byte[], HColumnDescriptor>> columnModifications,
-      List<byte[]> columnDeletions) throws IOException;
+                         List<HColumnDescriptor> columnAdditions,
+                         List<Pair<byte [], HColumnDescriptor>> columnModifications,
+                         List<byte []> columnDeletions) throws IOException;
 
   /**
    * Adds a column to the specified table
@@ -131,10 +132,26 @@ public interface HMasterInterface extend
    * @param args arguments for operation
    * @throws IOException e
    */
-  public void modifyTable(byte[] tableName, HConstants.Modify op, Writable[] args)
+  public void modifyTable(byte [] tableName, HConstants.Modify op, Writable [] args)
     throws IOException;
 
   /**
+   * Set wait interval for close and reopen of regions
+   *
+   * @param tableName table to modify
+   * @param waitInterval the interval to wait in milliseconds
+   */
+  public void setCloseRegionWaitInterval(String tableName, int waitInterval);
+
+  /**
+   * Set the number of Regions to close concurrently.
+   *
+   * @param tableName table to modify
+   * @param numConcurrentClose the number of Regions to close at the same time.
+   */
+  void setNumConcurrentCloseRegions(String tableName, int numConcurrentClose);
+
+  /**
    * Shutdown an HBase cluster.
    * @throws IOException e
    */
@@ -161,7 +178,7 @@ public interface HMasterInterface extend
    *              getSecond() total number of regions of the table
    * @throws IOException
    */
-  public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
+  public Pair<Integer, Integer> getAlterStatus(byte [] tableName)
       throws IOException;
   
   public void enableLoadBalancer();

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1466167&r1=1466166&r2=1466167&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Tue Apr  9 18:18:20 2013
@@ -115,7 +115,6 @@ import org.apache.hadoop.hbase.util.Slee
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.hbase.zookeeper.LegacyRootZNodeUpdater;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
 import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Text;
@@ -1544,9 +1543,9 @@ public class HMaster extends HasThread i
 
   @Override
   public void alterTable(final byte [] tableName,
-      List<HColumnDescriptor> columnAdditions,
-      List<Pair<byte[], HColumnDescriptor>> columnModifications,
-      List<byte[]> columnDeletions) throws IOException {
+                         List<HColumnDescriptor> columnAdditions,
+                         List<Pair<byte [], HColumnDescriptor>> columnModifications,
+                         List<byte []> columnDeletions) throws IOException {
     lockTable(tableName, "alter");
     try {
       InjectionHandler.processEvent(InjectionEvent.HMASTER_ALTER_TABLE);
@@ -1561,7 +1560,7 @@ public class HMaster extends HasThread i
     }
   }
 
-  public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
+  public Pair<Integer, Integer> getAlterStatus(byte [] tableName)
       throws IOException {
     Pair <Integer, Integer> p = new Pair<Integer, Integer>(0,0);
     if (regionManager.getThrottledReopener(Bytes.toString(tableName)) != null) {
@@ -1958,7 +1957,7 @@ public class HMaster extends HasThread i
    * Returns null if none found (and logs fact that expected COL_REGIONINFO
    * was missing).  Utility method used by scanners of META tables.
    * @param row name of the row
-   * @param map Map to do lookup in.
+   * @param res Result to use to do lookup.
    * @return Null or found HRegionInfo.
    * @throws IOException
    */
@@ -2273,6 +2272,18 @@ public class HMaster extends HasThread i
     return -1;
   }
 
+  @Override
+  public void setCloseRegionWaitInterval(String tableName, int waitInterval) {
+    ThrottledRegionReopener reopener = this.regionManager.createThrottledReopener(tableName);
+    reopener.setRegionCloseWaitInterval(waitInterval);
+  }
+
+  @Override
+  public void setNumConcurrentCloseRegions(String tableName, int numConcurrentClose) {
+    ThrottledRegionReopener reopener = this.regionManager.createThrottledReopener(tableName);
+    reopener.setNumConcurrentCloseRegions(numConcurrentClose);
+  }
+
   String getZKWrapperName() {
     return getServerName();
   }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ThrottledRegionReopener.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ThrottledRegionReopener.java?rev=1466167&r1=1466166&r2=1466167&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ThrottledRegionReopener.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ThrottledRegionReopener.java Tue Apr  9 18:18:20 2013
@@ -21,6 +21,9 @@ public class ThrottledRegionReopener {
   private RegionManager regionManager;
   private String tableName;
   private int totalNoOfRegionsToReopen = 0;
+  private int regionCloseWaitInterval = 1000;
+  private long timeAllowedToProceed = 0;
+  private int numOfConcurrentClose = 0;
 
   ThrottledRegionReopener(String tn, HMaster m, RegionManager regMgr) {
     this.tableName = tn;
@@ -35,7 +38,7 @@ public class ThrottledRegionReopener {
    * region is closed it is removed from this set and added to the
    * alterTableReopeningRegions set.
    */
-  private final Set<HRegionInfo> regionsToBeReopend = new HashSet<HRegionInfo>();
+  private final Set<HRegionInfo> regionsToBeReopened = new HashSet<HRegionInfo>();
 
   /**
    * Set of regions that are currently being reopened by the master. Regions are
@@ -45,8 +48,8 @@ public class ThrottledRegionReopener {
   private final Set<HRegionInfo> regionsBeingReopened = new HashSet<HRegionInfo>();
 
   public synchronized void addRegionsToReopen(Set<HRegionInfo> regions) {
-    regionsToBeReopend.addAll(regions);
-    if (regionsToBeReopend.size() + regionsBeingReopened.size() == 0) {
+    regionsToBeReopened.addAll(regions);
+    if (regionsToBeReopened.size() + regionsBeingReopened.size() == 0) {
       totalNoOfRegionsToReopen = regions.size();
     } else {
       totalNoOfRegionsToReopen += regions.size();
@@ -54,8 +57,8 @@ public class ThrottledRegionReopener {
   }
 
   public synchronized void addRegionToReopen(HRegionInfo region) {
-    regionsToBeReopend.add(region);
-    if (regionsToBeReopend.size() + regionsBeingReopened.size() == 0) {
+    regionsToBeReopened.add(region);
+    if (regionsToBeReopened.size() + regionsBeingReopened.size() == 0) {
       totalNoOfRegionsToReopen = 1;
     } else {
       totalNoOfRegionsToReopen += 1;
@@ -69,7 +72,7 @@ public class ThrottledRegionReopener {
    */
   public synchronized Pair<Integer, Integer> getReopenStatus()
       throws IOException {
-    int pending = regionsToBeReopend.size() + regionsBeingReopened.size();
+    int pending = regionsToBeReopened.size() + regionsBeingReopened.size();
     return new Pair<Integer, Integer>(pending, totalNoOfRegionsToReopen);
   }
 
@@ -78,8 +81,7 @@ public class ThrottledRegionReopener {
    * regions.
    *
    * @param region
-   * @param serverName
-   *          on which the region reopened.
+   *
    */
   public synchronized void notifyRegionOpened(HRegionInfo region) {
     if (regionsBeingReopened.contains(region)) {
@@ -88,7 +90,8 @@ public class ThrottledRegionReopener {
 
       // Check if all the regions have reopened and log.
       if (closeSomeRegions() == 0) {
-        if (regionsToBeReopend.size() == 0 && regionsBeingReopened.size() == 0) {
+        if (regionsToBeReopened.size() == 0 && regionsBeingReopened.size() == 0) {
+          this.numOfConcurrentClose = 0;
           LOG.info("All regions of " + tableName + " reopened successfully.");
         } else {
           LOG.error("All regions of " + tableName
@@ -106,27 +109,44 @@ public class ThrottledRegionReopener {
    * which defines the percentage of regions of a table that the master may
    * reopen concurrently (defaults to 1).
    *
-   * @param serverName
-   *          Region server on which to close regions
    */
   public synchronized int closeSomeRegions() {
 
-    float percentConcurrentClose = this.master.getConfiguration().getFloat(
-        "hbase.regionserver.alterTable.concurrentReopen", 5);
-    // Find the number of regions you are allowed to close concurrently
-    float numOfConcurrentClose = (percentConcurrentClose / 100)
-        * totalNoOfRegionsToReopen;
-    // Close at least one region at a time
-    if (numOfConcurrentClose < 1 && numOfConcurrentClose > 0) {
-      numOfConcurrentClose = 1;
+    float localNumOfConcurrentClose = this.numOfConcurrentClose;
+
+    //Try to get the number from the config if class value is set to 0
+    if (this.numOfConcurrentClose == 0) {
+      float percentConcurrentClose = this.master.getConfiguration().getFloat(
+          "hbase.regionserver.alterTable.concurrentReopen", 5);
+      // Find the number of regions you are allowed to close concurrently
+      float configNumOfConcurrentClose = (percentConcurrentClose / 100)
+          * totalNoOfRegionsToReopen;
+      // Close at least one region at a time
+      if (configNumOfConcurrentClose < 1 && configNumOfConcurrentClose > 0) {
+        configNumOfConcurrentClose = 1;
+      }
+
+      localNumOfConcurrentClose = configNumOfConcurrentClose;
     }
 
-    numOfConcurrentClose -= regionsBeingReopened.size();
-    if (numOfConcurrentClose <= 0) {
+    localNumOfConcurrentClose -= regionsBeingReopened.size();
+    if (localNumOfConcurrentClose <= 0) {
       return 0;
     }
+
+    //If true we are not yet at the time when we can proceed
+    if ( this.timeAllowedToProceed > System.currentTimeMillis()) {
+      //Wait until we are allowed to try and close more regions
+      try {
+        long sleepLength = this.timeAllowedToProceed - System.currentTimeMillis();
+        Thread.sleep(sleepLength);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+    }
+
     int cnt = 0;
-    for (Iterator<HRegionInfo> iter = regionsToBeReopend.iterator(); iter
+    for (Iterator<HRegionInfo> iter = regionsToBeReopened.iterator(); iter
         .hasNext();) {
       HRegionInfo region = iter.next();
       // Get the server name on which this is currently deployed
@@ -144,11 +164,18 @@ public class ThrottledRegionReopener {
       iter.remove(); // Remove from regionsToBeReopened
       regionsBeingReopened.add(region);
       cnt++;
+
       // Close allowed number of regions, exit
-      if (cnt == (int) numOfConcurrentClose) {
+      if (cnt == localNumOfConcurrentClose) {
         break;
       }
     }
+
+    //If we closed some regions set the time we are allowed to close more regions
+    if (cnt > 0) {
+      this.timeAllowedToProceed = System.currentTimeMillis() + regionCloseWaitInterval;
+    }
+
     return cnt;
   }
 
@@ -212,4 +239,22 @@ public class ThrottledRegionReopener {
       }
     }
   }
+
+  public int getRegionCloseWaitInterval() {
+    return regionCloseWaitInterval;
+  }
+
+  public int getNumConcurrentCloseRegions() {
+    return numOfConcurrentClose;
+  }
+
+  public synchronized void setRegionCloseWaitInterval(int regionCloseWaitInterval) {
+    LOG.info("ThrottledRegionReopener: Setting wait interval " + regionCloseWaitInterval + " !!!");
+    //Wait interval is in milliseconds
+    this.regionCloseWaitInterval = regionCloseWaitInterval;
+  }
+
+  public synchronized void setNumConcurrentCloseRegions(int numConcurrentClose) {
+    this.numOfConcurrentClose = numConcurrentClose;
+  }
 }

Modified: hbase/branches/0.89-fb/src/main/ruby/hbase.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/hbase.rb?rev=1466167&r1=1466166&r2=1466167&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/hbase.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/hbase.rb Tue Apr  9 18:18:20 2013
@@ -58,6 +58,8 @@ module HBaseConstants
   FILTER = 'FILTER'
   SPLITALGO = 'SPLITALGO'
   NUMREGIONS = 'NUMREGIONS'
+  WAIT_INTERVAL = 'WAIT_INTERVAL'
+  NUM_CONCURRENT_CLOSE = 'NUM_CONCURRENT_CLOSE'
 
   # Load constants from hbase java API
   def self.promote_constants(constants)

Modified: hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb?rev=1466167&r1=1466166&r2=1466167&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb Tue Apr  9 18:18:20 2013
@@ -21,6 +21,7 @@
 include Java
 
 java_import java.util.ArrayList
+java_import java.util.concurrent.TimeUnit
 
 java_import org.apache.hadoop.hbase.client.HBaseAdmin
 java_import org.apache.zookeeper.ZooKeeperMain
@@ -127,21 +128,21 @@ module Hbase
       @admin.enableLoadBalancer
       get_loadbalancer
     end
-	
-	#----------------------------------------------------------------------------------------------
+
+  #----------------------------------------------------------------------------------------------
     # disable_loadbalancer
     def disable_loadbalancer
       @admin.disableLoadBalancer
       get_loadbalancer
     end
-	
-	#----------------------------------------------------------------------------------------------
+
+  #----------------------------------------------------------------------------------------------
     # Shuts hbase down
     def get_loadbalancer
       print "LoadBalacner disabled: %s \n " % @admin.isLoadBalancerDisabled
     end
-	
-	#----------------------------------------------------------------------------------------------
+
+  #----------------------------------------------------------------------------------------------
     # Shuts hbase down
     def shutdown
       @admin.shutdown
@@ -178,7 +179,6 @@ module Hbase
         unless arg.kind_of?(String) || arg.kind_of?(Hash)
           raise(ArgumentError, "#{arg.class} of #{arg.inspect} is not of Hash or String type")
         end
-
         if arg.kind_of?(String)
           # the arg is a string, default action is to add a column to the table
           htd.addFamily(hcd(arg, htd))
@@ -219,7 +219,6 @@ module Hbase
           end
         end
       end
-
       if num_regions.nil?
         # Perform the create table call
         @admin.createTable(htd)
@@ -337,6 +336,16 @@ module Hbase
         # Normalize args to support shortcut delete syntax
         arg = { METHOD => 'delete', NAME => arg['delete'] } if arg['delete']
 
+        # Now set regionCloseWaitInterval if specified
+        if arg[WAIT_INTERVAL]
+          @admin.setCloseRegionWaitInterval(table_name, arg[WAIT_INTERVAL])
+        end
+
+        # Now set the NumConcurrentCloseRegions if specified
+        if arg[NUM_CONCURRENT_CLOSE]
+          @admin.setNumConcurrentCloseRegions(table_name, arg[NUM_CONCURRENT_CLOSE])
+        end
+
         # No method parameter, try to use the args as a column definition
         unless method = arg.delete(METHOD)
           descriptor = hcd(arg, htd)
@@ -350,6 +359,7 @@ module Hbase
           end
           next
         end
+
         # Delete column family
         if method == "delete"
           raise(ArgumentError, "NAME parameter missing for delete method") unless arg[NAME]
@@ -379,6 +389,7 @@ module Hbase
         # Unknown method
         raise ArgumentError, "Unknown method: #{method}"
       end
+
       # now batch process alter requests
       @admin.alterTable(table_name, columnsToAdd, columnsToMod, columnsToDel)
       if wait == true

Modified: hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter.rb?rev=1466167&r1=1466166&r2=1466167&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter.rb Tue Apr  9 18:18:20 2013
@@ -30,21 +30,29 @@ module Shell
 
           To change or add the 'f1' column family in table 't1' from defaults
           to instead keep a maximum of 5 cell VERSIONS, do:
-          hbase> alter 't1', NAME => 'f1', VERSIONS => 5
+
+            hbase> alter 't1', NAME => 'f1', VERSIONS => 5
 
           To delete the 'f1' column family in table 't1', do:
           hbase> alter 't1', NAME => 'f1', METHOD => 'delete'
           or a shorter version:
-          hbase> alter 't1', 'delete' => 'f1'
+
+            hbase> alter 't1', 'delete' => 'f1'
 
           You can also change table-scope attributes like MAX_FILESIZE
           MEMSTORE_FLUSHSIZE, READONLY, and DEFERRED_LOG_FLUSH.
 
           For example, to change the max size of a family to 128MB, do:
-          hbase> alter 't1', METHOD => 'table_att', MAX_FILESIZE => '134217728'
+
+            hbase> alter 't1', METHOD => 'table_att', MAX_FILESIZE => '134217728'
 
           There could be more than one alteration in one command:
-          hbase> alter 't1', {NAME => 'f1'}, {NAME => 'f2', METHOD => 'delete'}
+
+            hbase> alter 't1', {NAME => 'f1'}, {NAME => 'f2', METHOD => 'delete'}
+
+          You can also specify the wait interval, in milliseconds, to pause for in between region restarts:
+
+            hbase> alter 't1', NAME => 'f1', METHOD => 'delete', WAIT_INTERVAL => 1000, NUM_CONCURRENT_CLOSE => 1
         EOF
       end
 

Modified: hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb?rev=1466167&r1=1466166&r2=1466167&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb Tue Apr  9 18:18:20 2013
@@ -46,11 +46,16 @@ module Shell
 
           For example, to change the max size of a family to 128MB, do:
 
-            hbase> alter 't1', METHOD => 'table_att', MAX_FILESIZE => '134217728'
+            hbase> alter_async 't1', METHOD => 'table_att', MAX_FILESIZE => '134217728'
 
           There could be more than one alteration in one command:
 
-            hbase> alter 't1', {NAME => 'f1'}, {NAME => 'f2', METHOD => 'delete'}
+            hbase> alter_async 't1', {NAME => 'f1'}, {NAME => 'f2', METHOD => 'delete'}
+
+          You can also specify the wait interval, in milliseconds, to pause for in between region restarts
+          and the simultaneous number of regions to close at a time:
+
+            hbase> alter_async 't1', NAME => 'f1', METHOD => 'delete', WAIT_INTERVAL => 1000, NUM_CONCURRENT_CLOSE => 1
 
           To check if all the regions have been updated, use alter_status <table_name>
         EOF