You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2016/06/03 20:44:05 UTC

hbase git commit: HBASE-15845 Changes: - Renaming hbase.rb to hbase_constants.rb because there are two hbase.rb files right now which is confusing. - Remove omnipresence of formatter object since it is kind of a use-and-throw class. Commands should creat

Repository: hbase
Updated Branches:
  refs/heads/master f0c159b5f -> bdb46f01b


HBASE-15845 Changes:
- Renaming hbase.rb to hbase_constants.rb because there are two hbase.rb files right now which is confusing.
- Remove omnipresence of formatter object since it is kind of a use-and-throw class. Commands should create
  an instance, use it to format the output and discard it.
- Some refactoring

Change-Id: If9ea9873904e0a39d199a6aa10e23864b86a2f09


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/bdb46f01
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/bdb46f01
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/bdb46f01

Branch: refs/heads/master
Commit: bdb46f01b9645a0aa4e9827a7e627de76cd7983b
Parents: f0c159b
Author: Apekshit <ap...@gmail.com>
Authored: Mon Dec 28 14:50:50 2015 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Fri Jun 3 13:37:38 2016 -0700

----------------------------------------------------------------------
 bin/hirb.rb                                     |   8 +-
 hbase-shell/src/main/ruby/hbase.rb              | 109 -------------------
 hbase-shell/src/main/ruby/hbase/admin.rb        |  82 ++++++--------
 hbase-shell/src/main/ruby/hbase/hbase.rb        |  36 +++---
 hbase-shell/src/main/ruby/hbase/quotas.rb       |   3 +-
 .../src/main/ruby/hbase/replication_admin.rb    |   3 +-
 .../src/main/ruby/hbase/rsgroup_admin.rb        |   3 +-
 hbase-shell/src/main/ruby/hbase/security.rb     |   3 +-
 hbase-shell/src/main/ruby/hbase/table.rb        |   2 +-
 hbase-shell/src/main/ruby/hbase/taskmonitor.rb  |   3 +-
 .../src/main/ruby/hbase/visibility_labels.rb    |   5 +-
 hbase-shell/src/main/ruby/hbase_constants.rb    | 109 +++++++++++++++++++
 hbase-shell/src/main/ruby/shell.rb              |  25 ++---
 hbase-shell/src/main/ruby/shell/commands.rb     |   9 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   3 +-
 hbase-shell/src/test/ruby/hbase/hbase_test.rb   |  11 +-
 .../test/ruby/hbase/replication_admin_test.rb   |   3 +-
 .../src/test/ruby/hbase/security_admin_test.rb  |   3 +-
 hbase-shell/src/test/ruby/hbase/table_test.rb   |   2 +-
 .../src/test/ruby/hbase/taskmonitor_test.rb     |   2 +-
 .../ruby/hbase/visibility_labels_admin_test.rb  |   3 +-
 .../src/test/ruby/shell/commands_test.rb        |   2 +-
 .../src/test/ruby/shell/noninteractive_test.rb  |   6 +-
 .../src/test/ruby/shell/rsgroup_shell_test.rb   |  18 ++-
 hbase-shell/src/test/ruby/shell/shell_test.rb   |   6 +-
 hbase-shell/src/test/ruby/test_helper.rb        |  10 +-
 26 files changed, 218 insertions(+), 251 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/bin/hirb.rb
----------------------------------------------------------------------
diff --git a/bin/hirb.rb b/bin/hirb.rb
index 94b5cdb..d0295d6 100644
--- a/bin/hirb.rb
+++ b/bin/hirb.rb
@@ -115,7 +115,7 @@ org.apache.log4j.Logger.getLogger("org.apache.zookeeper").setLevel(log_level)
 org.apache.log4j.Logger.getLogger("org.apache.hadoop.hbase").setLevel(log_level)
 
 # Require HBase now after setting log levels
-require 'hbase'
+require 'hbase_constants'
 
 # Load hbase shell
 require 'shell'
@@ -123,15 +123,11 @@ require 'shell'
 # Require formatter
 require 'shell/formatter'
 
-# Presume console format.
-# Formatter takes an :output_stream parameter, if you don't want STDOUT.
-@formatter = Shell::Formatter::Console.new
-
 # Setup the HBase module.  Create a configuration.
 @hbase = Hbase::Hbase.new
 
 # Setup console
-@shell = Shell::Shell.new(@hbase, @formatter, interactive)
+@shell = Shell::Shell.new(@hbase, interactive)
 @shell.debug = @shell_debug
 
 # Add commands to this namespace

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase.rb b/hbase-shell/src/main/ruby/hbase.rb
deleted file mode 100644
index bc6f37c..0000000
--- a/hbase-shell/src/main/ruby/hbase.rb
+++ /dev/null
@@ -1,109 +0,0 @@
-#
-#
-# 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.
-#
-
-# HBase ruby classes.
-# Has wrapper classes for org.apache.hadoop.hbase.client.Admin
-# and for org.apache.hadoop.hbase.client.Table.  Classes take
-# Formatters on construction and outputs any results using
-# Formatter methods.  These classes are only really for use by
-# the hirb.rb HBase Shell script; they don't make much sense elsewhere.
-# For example, the exists method on Admin class prints to the formatter
-# whether the table exists and returns nil regardless.
-include Java
-
-include_class('java.lang.Integer') {|package,name| "J#{name}" }
-include_class('java.lang.Long') {|package,name| "J#{name}" }
-include_class('java.lang.Boolean') {|package,name| "J#{name}" }
-
-module HBaseConstants
-  COLUMN = "COLUMN"
-  COLUMNS = "COLUMNS"
-  TIMESTAMP = "TIMESTAMP"
-  TIMERANGE = "TIMERANGE"
-  NAME = org.apache.hadoop.hbase.HConstants::NAME
-  VERSIONS = org.apache.hadoop.hbase.HConstants::VERSIONS
-  IN_MEMORY = org.apache.hadoop.hbase.HConstants::IN_MEMORY
-  IN_MEMORY_COMPACTION = org.apache.hadoop.hbase.HColumnDescriptor::IN_MEMORY_COMPACTION
-  METADATA = org.apache.hadoop.hbase.HConstants::METADATA
-  STOPROW = "STOPROW"
-  STARTROW = "STARTROW"
-  ROWPREFIXFILTER = "ROWPREFIXFILTER"
-  ENDROW = STOPROW
-  RAW = "RAW"
-  LIMIT = "LIMIT"
-  METHOD = "METHOD"
-  MAXLENGTH = "MAXLENGTH"
-  CACHE_BLOCKS = "CACHE_BLOCKS"
-  ALL_METRICS = "ALL_METRICS"
-  METRICS = "METRICS"
-  REVERSED = "REVERSED"
-  REPLICATION_SCOPE = "REPLICATION_SCOPE"
-  INTERVAL = 'INTERVAL'
-  CACHE = 'CACHE'
-  FILTER = 'FILTER'
-  SPLITS = 'SPLITS'
-  SPLITS_FILE = 'SPLITS_FILE'
-  SPLITALGO = 'SPLITALGO'
-  NUMREGIONS = 'NUMREGIONS'
-  REGION_REPLICATION = 'REGION_REPLICATION'
-  REGION_REPLICA_ID = 'REGION_REPLICA_ID'
-  CONFIGURATION = org.apache.hadoop.hbase.HConstants::CONFIGURATION
-  ATTRIBUTES="ATTRIBUTES"
-  VISIBILITY="VISIBILITY"
-  AUTHORIZATIONS = "AUTHORIZATIONS"
-  SKIP_FLUSH = 'SKIP_FLUSH'
-  CONSISTENCY = "CONSISTENCY"
-  USER = 'USER'
-  TABLE = 'TABLE'
-  NAMESPACE = 'NAMESPACE'
-  TYPE = 'TYPE'
-  NONE = 'NONE'
-  VALUE = 'VALUE'
-  ENDPOINT_CLASSNAME = 'ENDPOINT_CLASSNAME'
-  CLUSTER_KEY = 'CLUSTER_KEY'
-  TABLE_CFS = 'TABLE_CFS'
-  CONFIG = 'CONFIG'
-  DATA = 'DATA'
-
-  # Load constants from hbase java API
-  def self.promote_constants(constants)
-    # The constants to import are all in uppercase
-    constants.each do |c|
-      next if c =~ /DEFAULT_.*/ || c != c.upcase
-      next if eval("defined?(#{c})")
-      eval("#{c} = '#{c}'")
-    end
-  end
-
-  promote_constants(org.apache.hadoop.hbase.HColumnDescriptor.constants)
-  promote_constants(org.apache.hadoop.hbase.HTableDescriptor.constants)
-end
-
-# Include classes definition
-require 'hbase/hbase'
-require 'hbase/admin'
-require 'hbase/taskmonitor'
-require 'hbase/table'
-require 'hbase/quotas'
-require 'hbase/replication_admin'
-require 'hbase/security'
-require 'hbase/visibility_labels'
-require 'hbase/rsgroup_admin'
-
-include HBaseQuotasConstants

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 35039af..f32376d 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -31,10 +31,10 @@ module Hbase
   class Admin
     include HBaseConstants
 
-    def initialize(admin, formatter)
-      @admin = admin
-      @connection = @admin.getConnection()
-      @formatter = formatter
+    def initialize(connection)
+      @connection = connection
+      # Java Admin instance
+      @admin = @connection.getAdmin
     end
 
     def close
@@ -309,12 +309,6 @@ module Hbase
     end
 
     #----------------------------------------------------------------------------------------------
-    # Parse arguments and update HTableDescriptor accordingly
-    def parse_htd_args(htd, arg)
-      htd.setNormalizationEnabled(JBoolean.valueOf(arg.delete(NORMALIZATION_ENABLED))) if arg[NORMALIZATION_ENABLED]
-    end
-
-    #----------------------------------------------------------------------------------------------
     # Creates a table
     def create(table_name, *args)
       # Fail if table name is not a string
@@ -392,24 +386,7 @@ module Hbase
         end
 
         # Done with splits; apply formerly-table_att parameters.
-        htd.setOwnerString(arg.delete(OWNER)) if arg[OWNER]
-        htd.setMaxFileSize(JLong.valueOf(arg.delete(MAX_FILESIZE))) if arg[MAX_FILESIZE]
-        htd.setReadOnly(JBoolean.valueOf(arg.delete(READONLY))) if arg[READONLY]
-        htd.setCompactionEnabled(JBoolean.valueOf(arg[COMPACTION_ENABLED])) if arg[COMPACTION_ENABLED]
-        htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(MEMSTORE_FLUSHSIZE))) if arg[MEMSTORE_FLUSHSIZE]
-        # DEFERRED_LOG_FLUSH is deprecated and was replaced by DURABILITY.  To keep backward compatible, it still exists.
-        # However, it has to be set before DURABILITY so that DURABILITY could overwrite if both args are set
-        if arg.include?(DEFERRED_LOG_FLUSH)
-          if arg.delete(DEFERRED_LOG_FLUSH).to_s.upcase == "TRUE"
-            htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("ASYNC_WAL"))
-          else
-            htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("SYNC_WAL"))
-          end
-        end
-        htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(DURABILITY))) if arg[DURABILITY]
-        parse_htd_args(htd, arg)
-        set_user_metadata(htd, arg.delete(METADATA)) if arg[METADATA]
-        set_descriptor_config(htd, arg.delete(CONFIGURATION)) if arg[CONFIGURATION]
+        update_htd_from_arg(htd, arg)
 
         arg.each_key do |ignored_key|
           puts("An argument ignored (unknown or overridden): %s" % [ ignored_key ])
@@ -653,26 +630,7 @@ module Hbase
         end
 
         # 3) Some args for the table, optionally with METHOD => table_att (deprecated)
-        raise(ArgumentError, "NAME argument in an unexpected place") if name
-        htd.setOwnerString(arg.delete(OWNER)) if arg[OWNER]
-        htd.setMaxFileSize(JLong.valueOf(arg.delete(MAX_FILESIZE))) if arg[MAX_FILESIZE]
-        htd.setReadOnly(JBoolean.valueOf(arg.delete(READONLY))) if arg[READONLY]
-        htd.setCompactionEnabled(JBoolean.valueOf(arg[COMPACTION_ENABLED])) if arg[COMPACTION_ENABLED]
-        parse_htd_args(htd, arg)
-        htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(MEMSTORE_FLUSHSIZE))) if arg[MEMSTORE_FLUSHSIZE]
-        # DEFERRED_LOG_FLUSH is deprecated and was replaced by DURABILITY.  To keep backward compatible, it still exists.
-        # However, it has to be set before DURABILITY so that DURABILITY could overwrite if both args are set
-        if arg.include?(DEFERRED_LOG_FLUSH)
-          if arg.delete(DEFERRED_LOG_FLUSH).to_s.upcase == "TRUE"
-            htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("ASYNC_WAL"))
-          else
-            htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("SYNC_WAL"))
-          end
-        end
-        htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(DURABILITY))) if arg[DURABILITY]
-        htd.setRegionReplication(JInteger.valueOf(arg.delete(REGION_REPLICATION))) if arg[REGION_REPLICATION]
-        set_user_metadata(htd, arg.delete(METADATA)) if arg[METADATA]
-        set_descriptor_config(htd, arg.delete(CONFIGURATION)) if arg[CONFIGURATION]
+        update_htd_from_arg(htd, arg)
 
         # set a coprocessor attribute
         valid_coproc_keys = []
@@ -764,7 +722,7 @@ module Hbase
             rLoadSink = sl.getReplicationLoadSink()
             rSinkString << " AgeOfLastAppliedOp=" + rLoadSink.getAgeOfLastAppliedOp().to_s
             rSinkString << ", TimeStampsOfLastAppliedOp=" +
-			    (java.util.Date.new(rLoadSink.getTimeStampsOfLastAppliedOp())).toString()
+                (java.util.Date.new(rLoadSink.getTimeStampsOfLastAppliedOp())).toString()
             rLoadSourceList = sl.getReplicationLoadSourceList()
             index = 0
             while index < rLoadSourceList.size()
@@ -773,7 +731,7 @@ module Hbase
               rSourceString << ", AgeOfLastShippedOp=" + rLoadSource.getAgeOfLastShippedOp().to_s
               rSourceString << ", SizeOfLogQueue=" + rLoadSource.getSizeOfLogQueue().to_s
               rSourceString << ", TimeStampsOfLastShippedOp=" +
-			      (java.util.Date.new(rLoadSource.getTimeStampOfLastShippedOp())).toString()
+                  (java.util.Date.new(rLoadSource.getTimeStampOfLastShippedOp())).toString()
               rSourceString << ", Replication Lag=" + rLoadSource.getReplicationLag().to_s
               index = index + 1
             end
@@ -1186,5 +1144,29 @@ module Hbase
     def list_procedures()
       @admin.listProcedures()
     end
+
+    # Parse arguments and update HTableDescriptor accordingly
+    def update_htd_from_arg(htd, arg)
+      htd.setOwnerString(arg.delete(OWNER)) if arg[OWNER]
+      htd.setMaxFileSize(JLong.valueOf(arg.delete(MAX_FILESIZE))) if arg[MAX_FILESIZE]
+      htd.setReadOnly(JBoolean.valueOf(arg.delete(READONLY))) if arg[READONLY]
+      htd.setCompactionEnabled(JBoolean.valueOf(arg[COMPACTION_ENABLED])) if arg[COMPACTION_ENABLED]
+      htd.setNormalizationEnabled(
+        JBoolean.valueOf(arg[NORMALIZATION_ENABLED])) if arg[NORMALIZATION_ENABLED]
+      htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(MEMSTORE_FLUSHSIZE))) if arg[MEMSTORE_FLUSHSIZE]
+      # DEFERRED_LOG_FLUSH is deprecated and was replaced by DURABILITY.  To keep backward compatible, it still exists.
+      # However, it has to be set before DURABILITY so that DURABILITY could overwrite if both args are set
+      if arg.include?(DEFERRED_LOG_FLUSH)
+        if arg.delete(DEFERRED_LOG_FLUSH).to_s.upcase == "TRUE"
+          htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("ASYNC_WAL"))
+        else
+          htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("SYNC_WAL"))
+        end
+      end
+      htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(DURABILITY))) if arg[DURABILITY]
+      htd.setRegionReplication(JInteger.valueOf(arg.delete(REGION_REPLICATION))) if arg[REGION_REPLICATION]
+      set_user_metadata(htd, arg.delete(METADATA)) if arg[METADATA]
+      set_descriptor_config(htd, arg.delete(CONFIGURATION)) if arg[CONFIGURATION]
+    end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/hbase.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/hbase.rb b/hbase-shell/src/main/ruby/hbase/hbase.rb
index 852f349..bc5a31d 100644
--- a/hbase-shell/src/main/ruby/hbase/hbase.rb
+++ b/hbase-shell/src/main/ruby/hbase/hbase.rb
@@ -18,6 +18,8 @@
 #
 
 include Java
+java_import org.apache.hadoop.hbase.client.ConnectionFactory
+java_import org.apache.hadoop.hbase.HBaseConfiguration
 
 require 'hbase/admin'
 require 'hbase/table'
@@ -35,25 +37,25 @@ module Hbase
       if config
         self.configuration = config
       else
-        self.configuration = org.apache.hadoop.hbase.HBaseConfiguration.create
+        self.configuration = HBaseConfiguration.create
         # Turn off retries in hbase and ipc.  Human doesn't want to wait on N retries.
         configuration.setInt("hbase.client.retries.number", 7)
         configuration.setInt("hbase.ipc.client.connect.max.retries", 3)
       end
-      @connection = org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(
-          self.configuration)
+      @connection = ConnectionFactory.createConnection(self.configuration)
     end
 
-    def admin(formatter)
-      ::Hbase::Admin.new(@connection.getAdmin, formatter)
+    # Returns ruby's Admin class from admin.rb
+    def admin()
+      ::Hbase::Admin.new(@connection)
     end
 
-    def rsgroup_admin(formatter)
-      ::Hbase::RSGroupAdmin.new(@connection, formatter)
+    def rsgroup_admin()
+      ::Hbase::RSGroupAdmin.new(@connection)
     end
 
-    def taskmonitor(formatter)
-      ::Hbase::TaskMonitor.new(configuration, formatter)
+    def taskmonitor()
+      ::Hbase::TaskMonitor.new(configuration)
     end
 
     # Create new one each time
@@ -61,20 +63,20 @@ module Hbase
       ::Hbase::Table.new(@connection.getTable(TableName.valueOf(table)), shell)
     end
 
-    def replication_admin(formatter)
-      ::Hbase::RepAdmin.new(configuration, formatter)
+    def replication_admin()
+      ::Hbase::RepAdmin.new(configuration)
     end
 
-    def security_admin(formatter)
-      ::Hbase::SecurityAdmin.new(@connection.getAdmin, formatter)
+    def security_admin()
+      ::Hbase::SecurityAdmin.new(@connection.getAdmin)
     end
 
-    def visibility_labels_admin(formatter)
-      ::Hbase::VisibilityLabelsAdmin.new(@connection.getAdmin, formatter)
+    def visibility_labels_admin()
+      ::Hbase::VisibilityLabelsAdmin.new(@connection.getAdmin)
     end
 
-    def quotas_admin(formatter)
-      ::Hbase::QuotasAdmin.new(@connection.getAdmin, formatter)
+    def quotas_admin()
+      ::Hbase::QuotasAdmin.new(@connection.getAdmin)
     end
 
     def shutdown

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/quotas.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/quotas.rb b/hbase-shell/src/main/ruby/hbase/quotas.rb
index 0be428d..bf2dc63 100644
--- a/hbase-shell/src/main/ruby/hbase/quotas.rb
+++ b/hbase-shell/src/main/ruby/hbase/quotas.rb
@@ -36,9 +36,8 @@ end
 
 module Hbase
   class QuotasAdmin
-    def initialize(admin, formatter)
+    def initialize(admin)
       @admin = admin
-      @formatter = formatter
     end
 
     def close

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index e91a4f7..7eae7af 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -31,10 +31,9 @@ module Hbase
   class RepAdmin
     include HBaseConstants
 
-    def initialize(configuration, formatter)
+    def initialize(configuration)
       @replication_admin = ReplicationAdmin.new(configuration)
       @configuration = configuration
-      @formatter = formatter
     end
 
     #----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb b/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb
index 51a4efb..c654f23 100644
--- a/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb
@@ -28,9 +28,8 @@ module Hbase
   class RSGroupAdmin
     include HBaseConstants
 
-    def initialize(connection, formatter)
+    def initialize(connection)
       @admin = org.apache.hadoop.hbase.rsgroup.RSGroupAdmin.newClient(connection)
-      @formatter = formatter
     end
 
     def close

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/security.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/security.rb b/hbase-shell/src/main/ruby/hbase/security.rb
index 7a31851..55519ed 100644
--- a/hbase-shell/src/main/ruby/hbase/security.rb
+++ b/hbase-shell/src/main/ruby/hbase/security.rb
@@ -24,10 +24,9 @@ module Hbase
   class SecurityAdmin
     include HBaseConstants
 
-    def initialize(admin, formatter)
+    def initialize(admin)
       @admin = admin
       @connection = @admin.getConnection()
-      @formatter = formatter
     end
 
     def close

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/table.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/table.rb b/hbase-shell/src/main/ruby/hbase/table.rb
index 6cb5dcb..e64b4ee 100644
--- a/hbase-shell/src/main/ruby/hbase/table.rb
+++ b/hbase-shell/src/main/ruby/hbase/table.rb
@@ -296,7 +296,7 @@ EOF
       # Parse arguments
       #
       unless args.kind_of?(Hash)
-        raise ArgumentError, "Failed parse of of #{args.inspect}, #{args.class}"
+        raise ArgumentError, "Failed parse of #{args.inspect}, #{args.class}"
       end
 
       # Get maxlength parameter if passed

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/taskmonitor.rb b/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
index df93848..d312558 100644
--- a/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
+++ b/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
@@ -71,9 +71,8 @@ module Hbase
     end
 
 
-    def initialize(configuration, formatter)
+    def initialize(configuration)
       @conf = configuration
-      @formatter = formatter
       @conn = org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(@conf)
       @admin = @conn.getAdmin()
     end

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
index 98bfb2c..8e6c93c 100644
--- a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
+++ b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
@@ -24,9 +24,8 @@ java_import org.apache.hadoop.hbase.util.Bytes
 module Hbase
   class VisibilityLabelsAdmin
 
-    def initialize(admin, formatter)
+    def initialize(admin)
       @admin = admin
-      @formatter = formatter
       @connection = @admin.getConnection()
     end
 
@@ -41,7 +40,7 @@ module Hbase
         labels = [ args ].flatten.compact
       end
       if labels.size() == 0
-      	raise(ArgumentError, "Arguments cannot be null")
+        raise(ArgumentError, "Arguments cannot be null")
       end
 
       begin

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/hbase_constants.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb
new file mode 100644
index 0000000..bc6f37c
--- /dev/null
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -0,0 +1,109 @@
+#
+#
+# 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.
+#
+
+# HBase ruby classes.
+# Has wrapper classes for org.apache.hadoop.hbase.client.Admin
+# and for org.apache.hadoop.hbase.client.Table.  Classes take
+# Formatters on construction and outputs any results using
+# Formatter methods.  These classes are only really for use by
+# the hirb.rb HBase Shell script; they don't make much sense elsewhere.
+# For example, the exists method on Admin class prints to the formatter
+# whether the table exists and returns nil regardless.
+include Java
+
+include_class('java.lang.Integer') {|package,name| "J#{name}" }
+include_class('java.lang.Long') {|package,name| "J#{name}" }
+include_class('java.lang.Boolean') {|package,name| "J#{name}" }
+
+module HBaseConstants
+  COLUMN = "COLUMN"
+  COLUMNS = "COLUMNS"
+  TIMESTAMP = "TIMESTAMP"
+  TIMERANGE = "TIMERANGE"
+  NAME = org.apache.hadoop.hbase.HConstants::NAME
+  VERSIONS = org.apache.hadoop.hbase.HConstants::VERSIONS
+  IN_MEMORY = org.apache.hadoop.hbase.HConstants::IN_MEMORY
+  IN_MEMORY_COMPACTION = org.apache.hadoop.hbase.HColumnDescriptor::IN_MEMORY_COMPACTION
+  METADATA = org.apache.hadoop.hbase.HConstants::METADATA
+  STOPROW = "STOPROW"
+  STARTROW = "STARTROW"
+  ROWPREFIXFILTER = "ROWPREFIXFILTER"
+  ENDROW = STOPROW
+  RAW = "RAW"
+  LIMIT = "LIMIT"
+  METHOD = "METHOD"
+  MAXLENGTH = "MAXLENGTH"
+  CACHE_BLOCKS = "CACHE_BLOCKS"
+  ALL_METRICS = "ALL_METRICS"
+  METRICS = "METRICS"
+  REVERSED = "REVERSED"
+  REPLICATION_SCOPE = "REPLICATION_SCOPE"
+  INTERVAL = 'INTERVAL'
+  CACHE = 'CACHE'
+  FILTER = 'FILTER'
+  SPLITS = 'SPLITS'
+  SPLITS_FILE = 'SPLITS_FILE'
+  SPLITALGO = 'SPLITALGO'
+  NUMREGIONS = 'NUMREGIONS'
+  REGION_REPLICATION = 'REGION_REPLICATION'
+  REGION_REPLICA_ID = 'REGION_REPLICA_ID'
+  CONFIGURATION = org.apache.hadoop.hbase.HConstants::CONFIGURATION
+  ATTRIBUTES="ATTRIBUTES"
+  VISIBILITY="VISIBILITY"
+  AUTHORIZATIONS = "AUTHORIZATIONS"
+  SKIP_FLUSH = 'SKIP_FLUSH'
+  CONSISTENCY = "CONSISTENCY"
+  USER = 'USER'
+  TABLE = 'TABLE'
+  NAMESPACE = 'NAMESPACE'
+  TYPE = 'TYPE'
+  NONE = 'NONE'
+  VALUE = 'VALUE'
+  ENDPOINT_CLASSNAME = 'ENDPOINT_CLASSNAME'
+  CLUSTER_KEY = 'CLUSTER_KEY'
+  TABLE_CFS = 'TABLE_CFS'
+  CONFIG = 'CONFIG'
+  DATA = 'DATA'
+
+  # Load constants from hbase java API
+  def self.promote_constants(constants)
+    # The constants to import are all in uppercase
+    constants.each do |c|
+      next if c =~ /DEFAULT_.*/ || c != c.upcase
+      next if eval("defined?(#{c})")
+      eval("#{c} = '#{c}'")
+    end
+  end
+
+  promote_constants(org.apache.hadoop.hbase.HColumnDescriptor.constants)
+  promote_constants(org.apache.hadoop.hbase.HTableDescriptor.constants)
+end
+
+# Include classes definition
+require 'hbase/hbase'
+require 'hbase/admin'
+require 'hbase/taskmonitor'
+require 'hbase/table'
+require 'hbase/quotas'
+require 'hbase/replication_admin'
+require 'hbase/security'
+require 'hbase/visibility_labels'
+require 'hbase/rsgroup_admin'
+
+include HBaseQuotasConstants

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 1927333..fa1f8b8 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -70,25 +70,23 @@ module Shell
   #----------------------------------------------------------------------
   class Shell
     attr_accessor :hbase
-    attr_accessor :formatter
     attr_accessor :interactive
     alias interactive? interactive
 
     @debug = false
     attr_accessor :debug
 
-    def initialize(hbase, formatter, interactive=true)
+    def initialize(hbase, interactive=true)
       self.hbase = hbase
-      self.formatter = formatter
       self.interactive = interactive
     end
 
     def hbase_admin
-      @hbase_admin ||= hbase.admin(formatter)
+      @hbase_admin ||= hbase.admin()
     end
 
     def hbase_taskmonitor
-      @hbase_taskmonitor ||= hbase.taskmonitor(formatter)
+      @hbase_taskmonitor ||= hbase.taskmonitor()
     end
 
     def hbase_table(name)
@@ -96,23 +94,23 @@ module Shell
     end
 
     def hbase_replication_admin
-      @hbase_replication_admin ||= hbase.replication_admin(formatter)
+      @hbase_replication_admin ||= hbase.replication_admin()
     end
 
     def hbase_security_admin
-      @hbase_security_admin ||= hbase.security_admin(formatter)
+      @hbase_security_admin ||= hbase.security_admin()
     end
 
     def hbase_visibility_labels_admin
-      @hbase_visibility_labels_admin ||= hbase.visibility_labels_admin(formatter)
+      @hbase_visibility_labels_admin ||= hbase.visibility_labels_admin()
     end
 
     def hbase_quotas_admin
-      @hbase_quotas_admin ||= hbase.quotas_admin(formatter)
+      @hbase_quotas_admin ||= hbase.quotas_admin()
     end
 
     def hbase_rsgroup_admin
-      @rsgroup_admin ||= hbase.rsgroup_admin(formatter)
+      @rsgroup_admin ||= hbase.rsgroup_admin()
     end
 
     def export_commands(where)
@@ -140,7 +138,7 @@ module Shell
       internal_command(command, :command, *args)
     end
 
-    #call a specific internal method in the command instance
+    # call a specific internal method in the command instance
     # command  - name of the command to call
     # method_name - name of the method on the command to call. Defaults to just 'command'
     # args - to be passed to the named method
@@ -149,8 +147,9 @@ module Shell
     end
 
     def print_banner
-      puts "HBase Shell; enter 'help<RETURN>' for list of supported commands."
-      puts 'Type "exit<RETURN>" to leave the HBase Shell'
+      puts 'HBase Shell'
+      puts 'Use "help" to get list of supported commands.'
+      puts 'Use "exit" to quit this interactive shell.'
       print 'Version '
       command('version')
       puts

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/main/ruby/shell/commands.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands.rb b/hbase-shell/src/main/ruby/shell/commands.rb
index b7b1705..b9446dd 100644
--- a/hbase-shell/src/main/ruby/shell/commands.rb
+++ b/hbase-shell/src/main/ruby/shell/commands.rb
@@ -26,7 +26,7 @@ module Shell
       end
 
       #wrap an execution of cmd to catch hbase exceptions
-      # cmd - command name to execture
+      # cmd - command name to execute
       # args - arguments to pass to the command
       def command_safe(debug, cmd = :command, *args)
         # send is internal ruby method to call 'cmd' with *args
@@ -50,6 +50,9 @@ module Shell
         end
       end
 
+      # Convenience functions to get different admins
+
+      # Returns HBase::Admin ruby class.
       def admin
         @shell.hbase_admin
       end
@@ -83,9 +86,9 @@ module Shell
       end
 
       #----------------------------------------------------------------------
-
+      # Creates formatter instance first time and then reuses it.
       def formatter
-        @shell.formatter
+        @formatter ||= ::Shell::Formatter::Console.new
       end
 
       def format_simple_command

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index 50a65d0..e2c3bc0 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -18,9 +18,8 @@
 #
 
 require 'shell'
-require 'shell/formatter'
 require 'stringio'
-require 'hbase'
+require 'hbase_constants'
 require 'hbase/hbase'
 require 'hbase/table'
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/hbase/hbase_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/hbase_test.rb b/hbase-shell/src/test/ruby/hbase/hbase_test.rb
index 185ec3e..0f19234 100644
--- a/hbase-shell/src/test/ruby/hbase/hbase_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/hbase_test.rb
@@ -17,12 +17,11 @@
 # limitations under the License.
 #
 
-require 'hbase'
+require 'hbase_constants'
 
 module Hbase
   class HbaseTest < Test::Unit::TestCase
     def setup
-      @formatter = Shell::Formatter::Console.new()
       @hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
     end
 
@@ -31,19 +30,19 @@ module Hbase
     end
 
     define_test "Hbase::Hbase#admin should create a new admin object when called the first time" do
-      assert_kind_of(::Hbase::Admin, @hbase.admin(@formatter))
+      assert_kind_of(::Hbase::Admin, @hbase.admin())
     end
 
     define_test "Hbase::Hbase#admin should create a new admin object every call" do
-      assert_not_same(@hbase.admin(@formatter), @hbase.admin(@formatter))
+      assert_not_same(@hbase.admin(), @hbase.admin())
     end
 
     define_test "Hbase::Hbase#table should create a new table object when called the first time" do
-      assert_kind_of(::Hbase::Table, @hbase.table('hbase:meta', @formatter))
+      assert_kind_of(::Hbase::Table, @hbase.table('hbase:meta', @shell))
     end
 
     define_test "Hbase::Hbase#table should create a new table object every call" do
-      assert_not_same(@hbase.table('hbase:meta', @formatter), @hbase.table('hbase:meta', @formatter))
+      assert_not_same(@hbase.table('hbase:meta', @shell), @hbase.table('hbase:meta', @shell))
     end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
index 0c026d6..d00dbc5 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -18,8 +18,7 @@
 #
 
 require 'shell'
-require 'shell/formatter'
-require 'hbase'
+require 'hbase_constants'
 require 'hbase/hbase'
 require 'hbase/table'
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/hbase/security_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/security_admin_test.rb b/hbase-shell/src/test/ruby/hbase/security_admin_test.rb
index 6ecfb98..be5bbae 100644
--- a/hbase-shell/src/test/ruby/hbase/security_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/security_admin_test.rb
@@ -18,8 +18,7 @@
 #
 
 require 'shell'
-require 'shell/formatter'
-require 'hbase'
+require 'hbase_constants'
 require 'hbase/hbase'
 require 'hbase/table'
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/hbase/table_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/table_test.rb b/hbase-shell/src/test/ruby/hbase/table_test.rb
index a617bc5..faf9827 100644
--- a/hbase-shell/src/test/ruby/hbase/table_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/table_test.rb
@@ -17,7 +17,7 @@
 # limitations under the License.
 #
 
-require 'hbase'
+require 'hbase_constants'
 
 include HBaseConstants
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/hbase/taskmonitor_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/taskmonitor_test.rb b/hbase-shell/src/test/ruby/hbase/taskmonitor_test.rb
index 78776d8..cdb91c7 100644
--- a/hbase-shell/src/test/ruby/hbase/taskmonitor_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/taskmonitor_test.rb
@@ -17,7 +17,7 @@
 # limitations under the License.
 #
 
-require 'hbase'
+require 'hbase_constants'
 
 module Hbase
   class TaskMonitorTest < Test::Unit::TestCase

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/hbase/visibility_labels_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/visibility_labels_admin_test.rb b/hbase-shell/src/test/ruby/hbase/visibility_labels_admin_test.rb
index 47ac292..0046909 100644
--- a/hbase-shell/src/test/ruby/hbase/visibility_labels_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/visibility_labels_admin_test.rb
@@ -18,8 +18,7 @@
 #
 
 require 'shell'
-require 'shell/formatter'
-require 'hbase'
+require 'hbase_constants'
 require 'hbase/hbase'
 require 'hbase/table'
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/shell/commands_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/commands_test.rb b/hbase-shell/src/test/ruby/shell/commands_test.rb
index 3f6a802..9fa291a 100644
--- a/hbase-shell/src/test/ruby/shell/commands_test.rb
+++ b/hbase-shell/src/test/ruby/shell/commands_test.rb
@@ -17,7 +17,7 @@
 # limitations under the License.
 #
 
-require 'hbase'
+require 'hbase_constants'
 require 'hbase/table'
 require 'shell'
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/shell/noninteractive_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/noninteractive_test.rb b/hbase-shell/src/test/ruby/shell/noninteractive_test.rb
index 14bdbc7..0fae4cb 100644
--- a/hbase-shell/src/test/ruby/shell/noninteractive_test.rb
+++ b/hbase-shell/src/test/ruby/shell/noninteractive_test.rb
@@ -14,15 +14,13 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-require 'hbase'
+require 'hbase_constants'
 require 'shell'
-require 'shell/formatter'
 
 class NonInteractiveTest < Test::Unit::TestCase
   def setup
-    @formatter = ::Shell::Formatter::Console.new()
     @hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
-    @shell = Shell::Shell.new(@hbase, @formatter, false)
+    @shell = Shell::Shell.new(@hbase, false)
   end
 
   define_test "Shell::Shell noninteractive mode should throw" do

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/shell/rsgroup_shell_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/rsgroup_shell_test.rb b/hbase-shell/src/test/ruby/shell/rsgroup_shell_test.rb
index 1040ed8..cb76c1f 100644
--- a/hbase-shell/src/test/ruby/shell/rsgroup_shell_test.rb
+++ b/hbase-shell/src/test/ruby/shell/rsgroup_shell_test.rb
@@ -17,16 +17,14 @@
 # limitations under the License.
 #
 
-require 'hbase'
+require 'hbase_constants'
 require 'shell'
-require 'shell/formatter'
 
 module Hbase
   class RSGroupShellTest < Test::Unit::TestCase
     def setup
-      @formatter = ::Shell::Formatter::Console.new
       @hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
-      @shell = Shell::Shell.new(@hbase, @formatter)
+      @shell = Shell::Shell.new(@hbase)
       connection = $TEST_CLUSTER.getConnection
       @rsgroup_admin =
           org.apache.hadoop.hbase.rsgroup.RSGroupAdmin.newClient(connection)
@@ -65,7 +63,7 @@ module Hbase
       assert_equal(1, @rsgroup_admin.getRSGroupInfo(group_name).getTables.count)
 
       count = 0
-      @hbase.rsgroup_admin(@formatter).get_rsgroup(group_name) do |line|
+      @hbase.rsgroup_admin().get_rsgroup(group_name) do |line|
         case count
         when 1
           assert_equal(hostPortStr, line)
@@ -77,22 +75,22 @@ module Hbase
       assert_equal(4, count)
 
       assert_equal(2,
-                   @hbase.rsgroup_admin(@formatter).list_rs_groups.count)
+                   @hbase.rsgroup_admin().list_rs_groups.count)
 
       # just run it to verify jruby->java api binding
-      @hbase.rsgroup_admin(@formatter).balance_rs_group(group_name)
+      @hbase.rsgroup_admin().balance_rs_group(group_name)
     end
 
     # we test exceptions that could be thrown by the ruby wrappers
     define_test 'Test bogus arguments' do
       assert_raise(ArgumentError) do
-        @hbase.rsgroup_admin(@formatter).get_rsgroup('foobar')
+        @hbase.rsgroup_admin().get_rsgroup('foobar')
       end
       assert_raise(ArgumentError) do
-        @hbase.rsgroup_admin(@formatter).get_rsgroup_of_server('foobar:123')
+        @hbase.rsgroup_admin().get_rsgroup_of_server('foobar:123')
       end
       assert_raise(ArgumentError) do
-        @hbase.rsgroup_admin(@formatter).get_rsgroup_of_table('foobar')
+        @hbase.rsgroup_admin().get_rsgroup_of_table('foobar')
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/shell/shell_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/shell_test.rb b/hbase-shell/src/test/ruby/shell/shell_test.rb
index 56b7dc8..f37f60c 100644
--- a/hbase-shell/src/test/ruby/shell/shell_test.rb
+++ b/hbase-shell/src/test/ruby/shell/shell_test.rb
@@ -17,15 +17,13 @@
 # limitations under the License.
 #
 
-require 'hbase'
+require 'hbase_constants'
 require 'shell'
-require 'shell/formatter'
 
 class ShellTest < Test::Unit::TestCase
   def setup
-    @formatter = ::Shell::Formatter::Console.new()
     @hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
-    @shell = Shell::Shell.new(@hbase, @formatter)
+    @shell = Shell::Shell.new(@hbase)
   end
 
   define_test "Shell::Shell#hbase_admin should return an admin instance" do

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdb46f01/hbase-shell/src/test/ruby/test_helper.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/test_helper.rb b/hbase-shell/src/test/ruby/test_helper.rb
index 3a5193a..11645d5 100644
--- a/hbase-shell/src/test/ruby/test_helper.rb
+++ b/hbase-shell/src/test/ruby/test_helper.rb
@@ -37,15 +37,13 @@ end
 
 module Hbase
   module TestHelpers
-    require 'hbase'
+    require 'hbase_constants'
     require 'hbase/hbase'
     require 'shell'
-    require 'shell/formatter'
 
     def setup_hbase
-      formatter = ::Shell::Formatter::Console.new
       hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
-      @shell = ::Shell::Shell.new(hbase, formatter)
+      @shell = ::Shell::Shell.new(hbase)
     end
     
     def shutdown
@@ -72,6 +70,10 @@ module Hbase
       @shell.hbase_visibility_labels_admin
     end
 
+    def quotas_admin
+      @shell.hbase_quotas_admin
+    end
+
     def replication_admin
       @shell.hbase_replication_admin
     end