You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2011/06/23 03:38:54 UTC

svn commit: r1138707 - in /hbase/trunk: ./ src/main/java/org/apache/hadoop/hbase/client/ src/main/ruby/ src/main/ruby/hbase/ src/main/ruby/shell/commands/ src/test/java/org/apache/hadoop/hbase/client/

Author: tedyu
Date: Thu Jun 23 01:38:53 2011
New Revision: 1138707

URL: http://svn.apache.org/viewvc?rev=1138707&view=rev
Log:
HBASE-3506 Ability to disable, drop and enable tables using regex expression
  Joey Echeverria via Ted Yu


Added:
    hbase/trunk/src/main/ruby/shell/commands/disable_all.rb
    hbase/trunk/src/main/ruby/shell/commands/drop_all.rb
    hbase/trunk/src/main/ruby/shell/commands/enable_all.rb
Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/trunk/src/main/ruby/hbase/admin.rb
    hbase/trunk/src/main/ruby/shell.rb
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1138707&r1=1138706&r2=1138707&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Thu Jun 23 01:38:53 2011
@@ -272,6 +272,8 @@ Release 0.91.0 - Unreleased
    HBASE-3994  SplitTransaction has a window where clients can
                get RegionOfflineException
    HBASE-4010  HMaster.createTable could be heavily optimized
+   HBASE-3506  Ability to disable, drop and enable tables using regex expression
+               (Joey Echeverria via Ted Yu)
 
   TASKS
    HBASE-3559  Move report of split to master OFF the heartbeat channel

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1138707&r1=1138706&r2=1138707&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Thu Jun 23 01:38:53 2011
@@ -22,7 +22,9 @@ package org.apache.hadoop.hbase.client;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.LinkedList;
 import java.util.List;
+import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -190,6 +192,37 @@ public class HBaseAdmin implements Abort
     return this.connection.listTables();
   }
 
+  /**
+   * List all the userspace tables matching the given pattern.
+   *
+   * @param pattern The compiled regular expression to match against
+   * @return - returns an array of HTableDescriptors
+   * @throws IOException if a remote or network exception occurs
+   * @see #listTables()
+   */
+  public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
+    List<HTableDescriptor> matched = new LinkedList<HTableDescriptor>();
+    HTableDescriptor[] tables = listTables();
+    for (HTableDescriptor table : tables) {
+      if (pattern.matcher(table.getNameAsString()).matches()) {
+        matched.add(table);
+      }
+    }
+    return matched.toArray(new HTableDescriptor[matched.size()]);
+  }
+
+  /**
+   * List all the userspace tables matching the given regular expression.
+   *
+   * @param regex The regular expression to match against
+   * @return - returns an array of HTableDescriptors
+   * @throws IOException if a remote or network exception occurs
+   * @see #listTables(java.util.regex.Pattern)
+   */
+  public HTableDescriptor[] listTables(String regex) throws IOException {
+    return listTables(Pattern.compile(regex));
+  }
+
 
   /**
    * Method for getting the tableDescriptor
@@ -417,6 +450,48 @@ public class HBaseAdmin implements Abort
     LOG.info("Deleted " + Bytes.toString(tableName));
   }
 
+  /**
+   * Deletes tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.lang.String)} and
+   * {@link #deleteTable(byte[])}
+   *
+   * @param regex The regular expression to match table names against
+   * @return Table descriptors for tables that couldn't be deleted
+   * @throws IOException
+   * @see #deleteTables(java.util.regex.Pattern)
+   * @see #deleteTable(java.lang.String)
+   */
+  public HTableDescriptor[] deleteTables(String regex) throws IOException {
+    return deleteTables(Pattern.compile(regex));
+  }
+
+  /**
+   * Delete tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
+   * {@link #deleteTable(byte[])}
+   *
+   * @param pattern The pattern to match table names against
+   * @return Table descriptors for tables that couldn't be deleted
+   * @throws IOException
+   */
+  public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
+    List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
+    for (HTableDescriptor table : listTables(pattern)) {
+      try {
+        deleteTable(table.getName());
+      } catch (IOException ex) {
+        LOG.info("Failed to delete table " + table.getNameAsString(), ex);
+        failed.add(table);
+      }
+    }
+    return failed.toArray(new HTableDescriptor[failed.size()]);
+  }
+
+
   public void enableTable(final String tableName)
   throws IOException {
     enableTable(Bytes.toBytes(tableName));
@@ -489,6 +564,47 @@ public class HBaseAdmin implements Abort
     LOG.info("Started enable of " + Bytes.toString(tableName));
   }
 
+  /**
+   * Enable tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.lang.String)} and
+   * {@link #enableTable(byte[])}
+   *
+   * @param regex The regular expression to match table names against
+   * @throws IOException
+   * @see #enableTables(java.util.regex.Pattern)
+   * @see #enableTable(java.lang.String)
+   */
+  public HTableDescriptor[] enableTables(String regex) throws IOException {
+    return enableTables(Pattern.compile(regex));
+  }
+
+  /**
+   * Enable tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
+   * {@link #enableTable(byte[])}
+   *
+   * @param pattern The pattern to match table names against
+   * @throws IOException
+   */
+  public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
+    List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
+    for (HTableDescriptor table : listTables(pattern)) {
+      if (isTableDisabled(table.getName())) {
+        try {
+          enableTable(table.getName());
+        } catch (IOException ex) {
+          LOG.info("Failed to enable table " + table.getNameAsString(), ex);
+          failed.add(table);
+        }
+      }
+    }
+    return failed.toArray(new HTableDescriptor[failed.size()]);
+  }
+
   public void disableTableAsync(final String tableName) throws IOException {
     disableTableAsync(Bytes.toBytes(tableName));
   }
@@ -560,6 +676,49 @@ public class HBaseAdmin implements Abort
   }
 
   /**
+   * Disable tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.lang.String)} and
+   * {@link #disableTable(byte[])}
+   *
+   * @param regex The regular expression to match table names against
+   * @return Table descriptors for tables that couldn't be disabled
+   * @throws IOException
+   * @see #disableTables(java.util.regex.Pattern)
+   * @see #disableTable(java.lang.String)
+   */
+  public HTableDescriptor[] disableTables(String regex) throws IOException {
+    return disableTables(Pattern.compile(regex));
+  }
+
+  /**
+   * Disable tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
+   * {@link #disableTable(byte[])}
+   *
+   * @param pattern The pattern to match table names against
+   * @return Table descriptors for tables that couldn't be disabled
+   * @throws IOException
+   */
+  public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
+    List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
+    for (HTableDescriptor table : listTables(pattern)) {
+      if (isTableEnabled(table.getName())) {
+        try {
+          disableTable(table.getName());
+        } catch (IOException ex) {
+          LOG.info("Failed to disable table " + table.getNameAsString(), ex);
+          failed.add(table);
+        }
+      }
+    }
+    return failed.toArray(new HTableDescriptor[failed.size()]);
+  }
+
+  /**
    * @param tableName name of table to check
    * @return true if table is on-line
    * @throws IOException if a remote or network exception occurs

Modified: hbase/trunk/src/main/ruby/hbase/admin.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/hbase/admin.rb?rev=1138707&r1=1138706&r2=1138707&view=diff
==============================================================================
--- hbase/trunk/src/main/ruby/hbase/admin.rb (original)
+++ hbase/trunk/src/main/ruby/hbase/admin.rb Thu Jun 23 01:38:53 2011
@@ -92,6 +92,13 @@ module Hbase
     end
 
     #----------------------------------------------------------------------------------------------
+    # Enables all tables matching the given regex
+    def enable_all(regex)
+      regex = regex.to_s
+      @admin.enableTables(regex)
+    end
+
+    #----------------------------------------------------------------------------------------------
     # Disables a table
     def disable(table_name)
       tableExists(table_name)
@@ -99,6 +106,13 @@ module Hbase
       @admin.disableTable(table_name)
     end
 
+    #----------------------------------------------------------------------------------------------
+    # Disables all tables matching the given regex
+    def disable_all(regex)
+      regex = regex.to_s
+      @admin.disableTables(regex).map { |t| t.getNameAsString }
+    end
+
     #---------------------------------------------------------------------------------------------
     # Throw exception if table doesn't exist
     def tableExists(table_name)
@@ -123,6 +137,16 @@ module Hbase
     end
 
     #----------------------------------------------------------------------------------------------
+    # Drops a table
+    def drop_all(regex)
+      regex = regex.to_s
+      failed  = @admin.deleteTables(regex).map { |t| t.getNameAsString }
+      flush(org.apache.hadoop.hbase.HConstants::META_TABLE_NAME)
+      major_compact(org.apache.hadoop.hbase.HConstants::META_TABLE_NAME)
+      return failed
+    end
+
+    #----------------------------------------------------------------------------------------------
     # Returns ZooKeeper status dump
     def zk_dump
       org.apache.hadoop.hbase.zookeeper.ZKUtil::dump(@zk_wrapper)

Modified: hbase/trunk/src/main/ruby/shell.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell.rb?rev=1138707&r1=1138706&r2=1138707&view=diff
==============================================================================
--- hbase/trunk/src/main/ruby/shell.rb (original)
+++ hbase/trunk/src/main/ruby/shell.rb Thu Jun 23 01:38:53 2011
@@ -218,9 +218,12 @@ Shell.load_command_group(
     create
     describe
     disable
+    disable_all
     is_disabled
     drop
+    drop_all
     enable
+    enable_all
     is_enabled
     exists
     list

Added: hbase/trunk/src/main/ruby/shell/commands/disable_all.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell/commands/disable_all.rb?rev=1138707&view=auto
==============================================================================
--- hbase/trunk/src/main/ruby/shell/commands/disable_all.rb (added)
+++ hbase/trunk/src/main/ruby/shell/commands/disable_all.rb Thu Jun 23 01:38:53 2011
@@ -0,0 +1,50 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class DisableAll < Command
+      def help
+        return <<-EOF
+Disable all of tables matching the given regex:
+
+hbase> disable_all 't.*'
+EOF
+      end
+
+      def command(regex)
+        regex = /#{regex}/ unless regex.is_a?(Regexp)
+        list = admin.list.grep(regex)
+        count = list.size
+        list.each do |table|
+          formatter.row([ table ])
+        end
+        puts "\nDisable the above #{count} tables (y/n)?" unless count == 0
+        answer = 'n'
+        answer = gets.chomp unless count == 0
+        puts "No tables matched the regex #{regex.to_s}" if count == 0
+        return unless answer =~ /y.*/i
+        failed = admin.disable_all(regex)
+        puts "#{count - failed.size} tables successfully disabled"
+        puts "#{failed.size} tables not disabled due to an exception: #{failed.join ','}" unless failed.size == 0
+      end
+    end
+  end
+end

Added: hbase/trunk/src/main/ruby/shell/commands/drop_all.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell/commands/drop_all.rb?rev=1138707&view=auto
==============================================================================
--- hbase/trunk/src/main/ruby/shell/commands/drop_all.rb (added)
+++ hbase/trunk/src/main/ruby/shell/commands/drop_all.rb Thu Jun 23 01:38:53 2011
@@ -0,0 +1,50 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class DropAll < Command
+      def help
+        return <<-EOF
+Drop all of the tables matching the given regex:
+
+hbase> drop_all 't.*'
+EOF
+      end
+
+      def command(regex)
+        regex = /#{regex}/ unless regex.is_a?(Regexp)
+        list = admin.list.grep(regex)
+        count = list.size
+        list.each do |table|
+          formatter.row([ table ])
+        end
+        puts "\nDrop the above #{count} tables (y/n)?" unless count == 0
+        answer = 'n'
+        answer = gets.chomp unless count == 0
+        puts "No tables matched the regex #{regex.to_s}" if count == 0
+        return unless answer =~ /y.*/i
+        failed = admin.drop_all(regex)
+        puts "#{count - failed.size} tables successfully dropped"
+        puts "#{failed.size} tables not dropped due to an exception: #{failed.join ','}" unless failed.size == 0
+      end
+    end
+  end
+end

Added: hbase/trunk/src/main/ruby/shell/commands/enable_all.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell/commands/enable_all.rb?rev=1138707&view=auto
==============================================================================
--- hbase/trunk/src/main/ruby/shell/commands/enable_all.rb (added)
+++ hbase/trunk/src/main/ruby/shell/commands/enable_all.rb Thu Jun 23 01:38:53 2011
@@ -0,0 +1,50 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class EnableAll < Command
+      def help
+        return <<-EOF
+Enable all of the tables matching the given regex:
+
+hbase> enable_all 't.*'
+EOF
+      end
+
+      def command(regex)
+        regex = /#{regex}/ unless regex.is_a?(Regexp)
+        list = admin.list.grep(regex)
+        count = list.size
+        list.each do |table|
+          formatter.row([ table ])
+        end
+        puts "\nEnable the above #{count} tables (y/n)?" unless count == 0
+        answer = 'n'
+        answer = gets.chomp unless count == 0
+        puts "No tables matched the regex #{regex.to_s}" if count == 0
+        return unless answer =~ /y.*/i
+        failed = admin.enable_all(regex)
+        puts "#{count - failed.size} tables successfully enabled"
+        puts "#{failed.size} tables not enabled due to an exception: #{failed.join ','}" unless failed.size == 0
+      end
+    end
+  end
+end

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1138707&r1=1138706&r2=1138707&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Thu Jun 23 01:38:53 2011
@@ -124,6 +124,55 @@ public class TestAdmin {
   }
 
   @Test
+  public void testDisableAndEnableTables() throws IOException {
+    final byte [] row = Bytes.toBytes("row");
+    final byte [] qualifier = Bytes.toBytes("qualifier");
+    final byte [] value = Bytes.toBytes("value");
+    final byte [] table1 = Bytes.toBytes("testDisableAndEnableTable1");
+    final byte [] table2 = Bytes.toBytes("testDisableAndEnableTable2");
+    HTable ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY);
+    HTable ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY);
+    Put put = new Put(row);
+    put.add(HConstants.CATALOG_FAMILY, qualifier, value);
+    ht1.put(put);
+    ht2.put(put);
+    Get get = new Get(row);
+    get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
+    ht1.get(get);
+    ht2.get(get);
+
+    this.admin.disableTables("testDisableAndEnableTable.*");
+
+    // Test that tables are disabled
+    get = new Get(row);
+    get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
+    boolean ok = false;
+    try {
+      ht1.get(get);
+      ht2.get(get);
+    } catch (NotServingRegionException e) {
+      ok = true;
+    } catch (RetriesExhaustedException e) {
+      ok = true;
+    }
+    assertTrue(ok);
+    this.admin.enableTables("testDisableAndEnableTable.*");
+
+    // Test that tables are enabled
+    try {
+      ht1.get(get);
+    } catch (RetriesExhaustedException e) {
+      ok = false;
+    }
+    try {
+      ht2.get(get);
+    } catch (RetriesExhaustedException e) {
+      ok = false;
+    }
+    assertTrue(ok);
+  }
+
+  @Test
   public void testCreateTable() throws IOException {
     HTableDescriptor [] tables = admin.listTables();
     int numTables = tables.length;