Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A129310ED0 for ; Mon, 23 Sep 2013 16:46:19 +0000 (UTC) Received: (qmail 59059 invoked by uid 500); 23 Sep 2013 16:46:16 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 58831 invoked by uid 500); 23 Sep 2013 16:46:14 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 58790 invoked by uid 99); 23 Sep 2013 16:46:14 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 23 Sep 2013 16:46:14 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_FILL_THIS_FORM_SHORT X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 23 Sep 2013 16:46:10 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id F35CB2388A32; Mon, 23 Sep 2013 16:45:49 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1525642 [2/4] - in /hbase/branches/0.96: ./ bin/ hbase-assembly/src/main/assembly/ hbase-it/ hbase-server/ hbase-server/src/main/ruby/ hbase-server/src/main/ruby/hbase/ hbase-server/src/main/ruby/irb/ hbase-server/src/main/ruby/shell/ hbas... Date: Mon, 23 Sep 2013 16:45:46 -0000 To: commits@hbase.apache.org From: nkeywal@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20130923164549.F35CB2388A32@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Added: hbase/branches/0.96/hbase-shell/src/main/ruby/hbase/security.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/hbase/security.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/hbase/security.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/hbase/security.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,233 @@ +# +# 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. +# + +include Java + +# Wrapper for org.apache.hadoop.hbase.client.HBaseAdmin + +module Hbase + class SecurityAdmin + include HBaseConstants + + def initialize(configuration, formatter) + @config = configuration + @admin = org.apache.hadoop.hbase.client.HBaseAdmin.new(configuration) + @formatter = formatter + end + + #---------------------------------------------------------------------------------------------- + def grant(user, permissions, table_name=nil, family=nil, qualifier=nil) + security_available? + + # TODO: need to validate user name + + begin + meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, + org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME) + service = meta_table.coprocessorService( + org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW) + + protocol = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos:: + AccessControlService.newBlockingStub(service) + perm = org.apache.hadoop.hbase.security.access.Permission.new( + permissions.to_java_bytes) + + # Verify that the specified permission is valid + if (permissions == nil || permissions.length == 0) + raise(ArgumentError, "Invalid permission: no actions associated with user") + end + + if (table_name != nil) + tablebytes=table_name.to_java_bytes + #check if the tablename passed is actually a namespace + if (isNamespace?(table_name)) + # Namespace should exist first. + namespace_name = table_name[1...table_name.length] + raise(ArgumentError, "Can't find a namespace: #{namespace_name}") unless namespace_exists?(namespace_name) + + #We pass the namespace name along with "@" so that we can differentiate a namespace from a table. + # invoke cp endpoint to perform access controlse + org.apache.hadoop.hbase.protobuf.ProtobufUtil.grant( + protocol, user, tablebytes, perm.getActions()) + else + # Table should exist + raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name) + + tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes) + htd = @admin.getTableDescriptor(tablebytes) + + if (family != nil) + raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes) + end + + fambytes = family.to_java_bytes if (family != nil) + qualbytes = qualifier.to_java_bytes if (qualifier != nil) + + # invoke cp endpoint to perform access controlse + org.apache.hadoop.hbase.protobuf.ProtobufUtil.grant( + protocol, user, tableName, fambytes, + qualbytes, perm.getActions()) + end + else + # invoke cp endpoint to perform access controlse + org.apache.hadoop.hbase.protobuf.ProtobufUtil.grant( + protocol, user, perm.getActions()) + end + + ensure + meta_table.close() + end + end + + #---------------------------------------------------------------------------------------------- + def revoke(user, table_name=nil, family=nil, qualifier=nil) + security_available? + + # TODO: need to validate user name + + begin + meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, + org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME) + service = meta_table.coprocessorService( + org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW) + + protocol = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos:: + AccessControlService.newBlockingStub(service) + + if (table_name != nil) + #check if the tablename passed is actually a namespace + if (isNamespace?(table_name)) + # Namespace should exist first. + namespace_name = table_name[1...table_name.length] + raise(ArgumentError, "Can't find a namespace: #{namespace_name}") unless namespace_exists?(namespace_name) + + #We pass the namespace name along with "@" so that we can differentiate a namespace from a table. + tablebytes=table_name.to_java_bytes + # invoke cp endpoint to perform access controlse + org.apache.hadoop.hbase.protobuf.ProtobufUtil.revoke( + protocol, user, tablebytes) + else + # Table should exist + raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name) + + tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes) + htd = @admin.getTableDescriptor(tableName) + + if (family != nil) + raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes) + end + + fambytes = family.to_java_bytes if (family != nil) + qualbytes = qualifier.to_java_bytes if (qualifier != nil) + + # invoke cp endpoint to perform access controlse + org.apache.hadoop.hbase.protobuf.ProtobufUtil.revoke( + protocol, user, tableName, fambytes, qualbytes) + end + else + # invoke cp endpoint to perform access controlse + perm = org.apache.hadoop.hbase.security.access.Permission.new(''.to_java_bytes) + org.apache.hadoop.hbase.protobuf.ProtobufUtil.revoke(protocol, user, perm.getActions()) + end + ensure + meta_table.close() + end + end + + #---------------------------------------------------------------------------------------------- + def user_permission(table_name=nil) + security_available? + + begin + meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, + org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME) + service = meta_table.coprocessorService( + org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW) + + protocol = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos:: + AccessControlService.newBlockingStub(service) + + if (table_name != nil) + #check if namespace is passed. + if (isNamespace?(table_name)) + # Namespace should exist first. + namespace_name = table_name[1...table_name.length] + raise(ArgumentError, "Can't find a namespace: #{namespace_name}") unless namespace_exists?(namespace_name) + # invoke cp endpoint to perform access controls + perms = org.apache.hadoop.hbase.protobuf.ProtobufUtil.getUserPermissions( + protocol, table_name.to_java_bytes) + else + raise(ArgumentError, "Can't find table: #{table_name}") unless exists?(table_name) + perms = org.apache.hadoop.hbase.protobuf.ProtobufUtil.getUserPermissions( + protocol, org.apache.hadoop.hbase.TableName.valueOf(table_name)) + end + else + perms = org.apache.hadoop.hbase.protobuf.ProtobufUtil.getUserPermissions(protocol) + end + ensure + meta_table.close() + end + + res = {} + count = 0 + perms.each do |value| + user_name = String.from_java_bytes(value.getUser) + table = (value.getTable != nil) ? value.getTable.toString() : '' + family = (value.getFamily != nil) ? org.apache.hadoop.hbase.util.Bytes::toStringBinary(value.getFamily) : '' + qualifier = (value.getQualifier != nil) ? org.apache.hadoop.hbase.util.Bytes::toStringBinary(value.getQualifier) : '' + + action = org.apache.hadoop.hbase.security.access.Permission.new value.getActions + + if block_given? + yield(user_name, "#{table},#{family},#{qualifier}: #{action.to_s}") + else + res[user_name] ||= {} + res[user_name][family + ":" +qualifier] = action + end + count += 1 + end + + return ((block_given?) ? count : res) + end + + # Does table exist? + def exists?(table_name) + @admin.tableExists(table_name) + end + + def isNamespace?(table_name) + table_name.start_with?('@') + end + + # Does Namespace exist + def namespace_exists?(namespace_name) + namespaceDesc = @admin.getNamespaceDescriptor(namespace_name) + if(namespaceDesc == nil) + return false + else + return true + end + end + + # Make sure that security tables are available + def security_available?() + raise(ArgumentError, "DISABLED: Security features are not available") \ + unless exists?(org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME) + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/hbase/table.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/hbase/table.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/hbase/table.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/hbase/table.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,539 @@ +# +# +# 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. +# + +include Java + +# Wrapper for org.apache.hadoop.hbase.client.HTable + +module Hbase + class Table + include HBaseConstants + + @@thread_pool = nil + + # Add the command 'name' to table s.t. the shell command also called via 'name' + # and has an internal method also called 'name'. + # + # e.g. name = scan, adds table.scan which calls Scan.scan + def self.add_shell_command(name) + self.add_command(name, name, name) + end + + # add a named command to the table instance + # + # name - name of the command that should added to the table + # (eg. sending 'scan' here would allow you to do table.scan) + # shell_command - name of the command in the shell + # internal_method_name - name of the method in the shell command to forward the call + def self.add_command(name, shell_command, internal_method_name) + method = name.to_sym + self.class_eval do + define_method method do |*args| + @shell.internal_command(shell_command, internal_method_name, self, *args) + end + end + end + + # General help for the table + # class level so we can call it from anywhere + def self.help + return <<-EOF +Help for table-reference commands. + +You can either create a table via 'create' and then manipulate the table via commands like 'put', 'get', etc. +See the standard help information for how to use each of these commands. + +However, as of 0.96, you can also get a reference to a table, on which you can invoke commands. +For instance, you can get create a table and keep around a reference to it via: + + hbase> t = create 't', 'cf' + +Or, if you have already created the table, you can get a reference to it: + + hbase> t = get_table 't' + +You can do things like call 'put' on the table: + + hbase> t.put 'r', 'cf:q', 'v' + +which puts a row 'r' with column family 'cf', qualifier 'q' and value 'v' into table t. + +To read the data out, you can scan the table: + + hbase> t.scan + +which will read all the rows in table 't'. + +Essentially, any command that takes a table name can also be done via table reference. +Other commands include things like: get, delete, deleteall, +get_all_columns, get_counter, count, incr. These functions, along with +the standard JRuby object methods are also available via tab completion. + +For more information on how to use each of these commands, you can also just type: + + hbase> t.help 'scan' + +which will output more information on how to use that command. + +You can also do general admin actions directly on a table; things like enable, disable, +flush and drop just by typing: + + hbase> t.enable + hbase> t.flush + hbase> t.disable + hbase> t.drop + +Note that after dropping a table, your reference to it becomes useless and further usage +is undefined (and not recommended). +EOF + end + + #--------------------------------------------------------------------------------------------- + + # let external objects read the underlying table object + attr_reader :table + # let external objects read the table name + attr_reader :name + + def initialize(configuration, table_name, shell) + if @@thread_pool then + @table = org.apache.hadoop.hbase.client.HTable.new(configuration, table_name.to_java_bytes, @@thread_pool) + else + @table = org.apache.hadoop.hbase.client.HTable.new(configuration, table_name) + @@thread_pool = @table.getPool() + end + @name = table_name + @shell = shell + @converters = Hash.new() + end + + # Note the below methods are prefixed with '_' to hide them from the average user, as + # they will be much less likely to tab complete to the 'dangerous' internal method + #---------------------------------------------------------------------------------------------- + # Put a cell 'value' at specified table/row/column + def _put_internal(row, column, value, timestamp = nil) + p = org.apache.hadoop.hbase.client.Put.new(row.to_s.to_java_bytes) + family, qualifier = parse_column_name(column) + if timestamp + p.add(family, qualifier, timestamp, value.to_s.to_java_bytes) + else + p.add(family, qualifier, value.to_s.to_java_bytes) + end + @table.put(p) + end + + #---------------------------------------------------------------------------------------------- + # Delete a cell + def _delete_internal(row, column, timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP) + _deleteall_internal(row, column, timestamp) + end + + #---------------------------------------------------------------------------------------------- + # Delete a row + def _deleteall_internal(row, column = nil, timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP) + raise ArgumentError, "Row Not Found" if _get_internal(row).nil? + d = org.apache.hadoop.hbase.client.Delete.new(row.to_s.to_java_bytes, timestamp) + if column + family, qualifier = parse_column_name(column) + d.deleteColumns(family, qualifier, timestamp) + end + @table.delete(d) + end + + #---------------------------------------------------------------------------------------------- + # Increment a counter atomically + def _incr_internal(row, column, value = nil) + value ||= 1 + family, qualifier = parse_column_name(column) + if qualifier.nil? + raise ArgumentError, "Failed to provide both column family and column qualifier for incr" + end + @table.incrementColumnValue(row.to_s.to_java_bytes, family, qualifier, value) + end + + #---------------------------------------------------------------------------------------------- + # Count rows in a table + def _count_internal(interval = 1000, caching_rows = 10) + # We can safely set scanner caching with the first key only filter + scan = org.apache.hadoop.hbase.client.Scan.new + scan.cache_blocks = false + scan.caching = caching_rows + scan.setFilter(org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter.new) + + # Run the scanner + scanner = @table.getScanner(scan) + count = 0 + iter = scanner.iterator + + # Iterate results + while iter.hasNext + row = iter.next + count += 1 + next unless (block_given? && count % interval == 0) + # Allow command modules to visualize counting process + yield(count, + org.apache.hadoop.hbase.util.Bytes::toStringBinary(row.getRow)) + end + + # Return the counter + return count + end + + #---------------------------------------------------------------------------------------------- + # Get from table + def _get_internal(row, *args) + get = org.apache.hadoop.hbase.client.Get.new(row.to_s.to_java_bytes) + maxlength = -1 + @converters.clear() + + # Normalize args + args = args.first if args.first.kind_of?(Hash) + if args.kind_of?(String) || args.kind_of?(Array) + columns = [ args ].flatten.compact + args = { COLUMNS => columns } + end + + # + # Parse arguments + # + unless args.kind_of?(Hash) + raise ArgumentError, "Failed parse of of #{args.inspect}, #{args.class}" + end + + # Get maxlength parameter if passed + maxlength = args.delete(MAXLENGTH) if args[MAXLENGTH] + filter = args.delete(FILTER) if args[FILTER] + + unless args.empty? + columns = args[COLUMN] || args[COLUMNS] + if args[VERSIONS] + vers = args[VERSIONS] + else + vers = 1 + end + if columns + # Normalize types, convert string to an array of strings + columns = [ columns ] if columns.is_a?(String) + + # At this point it is either an array or some unsupported stuff + unless columns.kind_of?(Array) + raise ArgumentError, "Failed parse column argument type #{args.inspect}, #{args.class}" + end + + # Get each column name and add it to the filter + columns.each do |column| + family, qualifier = parse_column_name(column.to_s) + if qualifier + get.addColumn(family, qualifier) + else + get.addFamily(family) + end + end + + # Additional params + get.setMaxVersions(vers) + get.setTimeStamp(args[TIMESTAMP]) if args[TIMESTAMP] + get.setTimeRange(args[TIMERANGE][0], args[TIMERANGE][1]) if args[TIMERANGE] + else + # May have passed TIMESTAMP and row only; wants all columns from ts. + unless ts = args[TIMESTAMP] || tr = args[TIMERANGE] + raise ArgumentError, "Failed parse of #{args.inspect}, #{args.class}" + end + + get.setMaxVersions(vers) + # Set the timestamp/timerange + get.setTimeStamp(ts.to_i) if args[TIMESTAMP] + get.setTimeRange(args[TIMERANGE][0], args[TIMERANGE][1]) if args[TIMERANGE] + end + end + + unless filter.class == String + get.setFilter(filter) + else + get.setFilter(org.apache.hadoop.hbase.filter.ParseFilter.new.parseFilterString(filter)) + end + + # Call hbase for the results + result = @table.get(get) + return nil if result.isEmpty + + # Print out results. Result can be Cell or RowResult. + res = {} + result.list.each do |kv| + family = String.from_java_bytes(kv.getFamily) + qualifier = org.apache.hadoop.hbase.util.Bytes::toStringBinary(kv.getQualifier) + + column = "#{family}:#{qualifier}" + value = to_string(column, kv, maxlength) + + if block_given? + yield(column, value) + else + res[column] = value + end + end + + # If block given, we've yielded all the results, otherwise just return them + return ((block_given?) ? nil : res) + end + + #---------------------------------------------------------------------------------------------- + # Fetches and decodes a counter value from hbase + def _get_counter_internal(row, column) + family, qualifier = parse_column_name(column.to_s) + # Format get request + get = org.apache.hadoop.hbase.client.Get.new(row.to_s.to_java_bytes) + get.addColumn(family, qualifier) + get.setMaxVersions(1) + + # Call hbase + result = @table.get(get) + return nil if result.isEmpty + + # Fetch cell value + cell = result.list[0] + org.apache.hadoop.hbase.util.Bytes::toLong(cell.getValue) + end + + #---------------------------------------------------------------------------------------------- + # Scans whole table or a range of keys and returns rows matching specific criteria + def _scan_internal(args = {}) + unless args.kind_of?(Hash) + raise ArgumentError, "Arguments should be a hash. Failed to parse #{args.inspect}, #{args.class}" + end + + limit = args.delete("LIMIT") || -1 + maxlength = args.delete("MAXLENGTH") || -1 + @converters.clear() + + if args.any? + filter = args["FILTER"] + startrow = args["STARTROW"] || '' + stoprow = args["STOPROW"] + timestamp = args["TIMESTAMP"] + columns = args["COLUMNS"] || args["COLUMN"] || [] + cache_blocks = args["CACHE_BLOCKS"] || true + cache = args["CACHE"] || 0 + versions = args["VERSIONS"] || 1 + timerange = args[TIMERANGE] + raw = args["RAW"] || false + + # Normalize column names + columns = [columns] if columns.class == String + unless columns.kind_of?(Array) + raise ArgumentError.new("COLUMNS must be specified as a String or an Array") + end + + scan = if stoprow + org.apache.hadoop.hbase.client.Scan.new(startrow.to_java_bytes, stoprow.to_java_bytes) + else + org.apache.hadoop.hbase.client.Scan.new(startrow.to_java_bytes) + end + + columns.each do |c| + family, qualifier = parse_column_name(c.to_s) + if qualifier + scan.addColumn(family, qualifier) + else + scan.addFamily(family) + end + end + + unless filter.class == String + scan.setFilter(filter) + else + scan.setFilter(org.apache.hadoop.hbase.filter.ParseFilter.new.parseFilterString(filter)) + end + + scan.setTimeStamp(timestamp) if timestamp + scan.setCacheBlocks(cache_blocks) + scan.setCaching(cache) if cache > 0 + scan.setMaxVersions(versions) if versions > 1 + scan.setTimeRange(timerange[0], timerange[1]) if timerange + scan.setRaw(raw) + else + scan = org.apache.hadoop.hbase.client.Scan.new + end + + # Start the scanner + scanner = @table.getScanner(scan) + count = 0 + res = {} + iter = scanner.iterator + + # Iterate results + while iter.hasNext + if limit > 0 && count >= limit + break + end + + row = iter.next + key = org.apache.hadoop.hbase.util.Bytes::toStringBinary(row.getRow) + + row.list.each do |kv| + family = String.from_java_bytes(kv.getFamily) + qualifier = org.apache.hadoop.hbase.util.Bytes::toStringBinary(kv.getQualifier) + + column = "#{family}:#{qualifier}" + cell = to_string(column, kv, maxlength) + + if block_given? + yield(key, "column=#{column}, #{cell}") + else + res[key] ||= {} + res[key][column] = cell + end + end + + # One more row processed + count += 1 + end + + return ((block_given?) ? count : res) + end + + #---------------------------- + # Add general administration utilities to the shell + # each of the names below adds this method name to the table + # by callling the corresponding method in the shell + # Add single method utilities to the current class + # Generally used for admin functions which just have one name and take the table name + def self.add_admin_utils(*args) + args.each do |method| + define_method method do |*method_args| + @shell.command(method, @name, *method_args) + end + end + end + + #Add the following admin utilities to the table + add_admin_utils :enable, :disable, :flush, :drop, :describe, :snapshot + + #---------------------------- + #give the general help for the table + # or the named command + def help (command = nil) + #if there is a command, get the per-command help from the shell + if command + begin + return @shell.help_command(command) + rescue NoMethodError + puts "Command \'#{command}\' does not exist. Please see general table help." + return nil + end + end + return @shell.help('table_help') + end + + # Table to string + def to_s + cl = self.class() + return "#{cl} - #{@name}" + end + + # Standard ruby call to get the return value for an object + # overriden here so we get sane semantics for printing a table on return + def inspect + to_s + end + + #---------------------------------------------------------------------------------------- + # Helper methods + + # Returns a list of column names in the table + def get_all_columns + @table.table_descriptor.getFamilies.map do |family| + "#{family.getNameAsString}:" + end + end + + # Checks if current table is one of the 'meta' tables + def is_meta_table? + tn = @table.table_name + org.apache.hadoop.hbase.util.Bytes.equals(tn, + org.apache.hadoop.hbase.TableName::META_TABLE_NAME.getName) + end + + # Returns family and (when has it) qualifier for a column name + def parse_column_name(column) + split = org.apache.hadoop.hbase.KeyValue.parseColumn(column.to_java_bytes) + set_converter(split) if split.length > 1 + return split[0], (split.length > 1) ? split[1] : nil + end + + # Make a String of the passed kv + # Intercept cells whose format we know such as the info:regioninfo in hbase:meta + def to_string(column, kv, maxlength = -1) + if is_meta_table? + if column == 'info:regioninfo' or column == 'info:splitA' or column == 'info:splitB' + hri = org.apache.hadoop.hbase.HRegionInfo.parseFromOrNull(kv.getValue) + return "timestamp=%d, value=%s" % [kv.getTimestamp, hri.toString] + end + if column == 'info:serverstartcode' + if kv.getValue.length > 0 + str_val = org.apache.hadoop.hbase.util.Bytes.toLong(kv.getValue) + else + str_val = org.apache.hadoop.hbase.util.Bytes.toStringBinary(kv.getValue) + end + return "timestamp=%d, value=%s" % [kv.getTimestamp, str_val] + end + end + + if kv.isDelete + val = "timestamp=#{kv.getTimestamp}, type=#{org.apache.hadoop.hbase.KeyValue::Type::codeToType(kv.getType)}" + else + val = "timestamp=#{kv.getTimestamp}, value=#{convert(column, kv)}" + end + (maxlength != -1) ? val[0, maxlength] : val + end + + def convert(column, kv) + #use org.apache.hadoop.hbase.util.Bytes as the default class + klazz_name = 'org.apache.hadoop.hbase.util.Bytes' + #use org.apache.hadoop.hbase.util.Bytes::toStringBinary as the default convertor + converter = 'toStringBinary' + if @converters.has_key?(column) + # lookup the CONVERTER for certain column - "cf:qualifier" + matches = /c\((.+)\)\.(.+)/.match(@converters[column]) + if matches.nil? + # cannot match the pattern of 'c(className).functionname' + # use the default klazz_name + converter = @converters[column] + else + klazz_name = matches[1] + converter = matches[2] + end + end + method = eval(klazz_name).method(converter) + return method.call(kv.getValue) # apply the converter + end + + # if the column spec contains CONVERTER information, to get rid of :CONVERTER info from column pair. + # 1. return back normal column pair as usual, i.e., "cf:qualifier[:CONVERTER]" to "cf" and "qualifier" only + # 2. register the CONVERTER information based on column spec - "cf:qualifier" + def set_converter(column) + family = String.from_java_bytes(column[0]) + parts = org.apache.hadoop.hbase.KeyValue.parseColumn(column[1]) + if parts.length > 1 + @converters["#{family}:#{String.from_java_bytes(parts[0])}"] = String.from_java_bytes(parts[1]) + column[1] = parts[0] + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/irb/hirb.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/irb/hirb.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/irb/hirb.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/irb/hirb.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,59 @@ +# +# +# 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. +# +require 'rbconfig' + +module IRB + WINDOZE = Config::CONFIG['host_os'] =~ /mswin|mingw/ + + # Subclass of IRB so can intercept methods + class HIRB < Irb + def initialize + # This is ugly. Our 'help' method above provokes the following message + # on irb construction: 'irb: warn: can't alias help from irb_help.' + # Below, we reset the output so its pointed at /dev/null during irb + # construction just so this message does not come out after we emit + # the banner. Other attempts at playing with the hash of methods + # down in IRB didn't seem to work. I think the worst thing that can + # happen is the shell exiting because of failed IRB construction with + # no error (though we're not blanking STDERR) + begin + # Map the '/dev/null' according to the runing platform + # Under Windows platform the 'dev/null' is not fully compliant with unix, + # and the 'NUL' object need to be use instead. + devnull = "/dev/null" + devnull = "NUL" if WINDOZE + f = File.open(devnull, "w") + $stdout = f + super + ensure + f.close() + $stdout = STDOUT + end + end + + def output_value + # Suppress output if last_value is 'nil' + # Otherwise, when user types help, get ugly 'nil' + # after all output. + if @context.last_value != nil + super + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,347 @@ +# +# +# 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. +# + +# Shell commands module +module Shell + @@commands = {} + def self.commands + @@commands + end + + @@command_groups = {} + def self.command_groups + @@command_groups + end + + def self.load_command(name, group) + return if commands[name] + + # Register command in the group + raise ArgumentError, "Unknown group: #{group}" unless command_groups[group] + command_groups[group][:commands] << name + + # Load command + begin + require "shell/commands/#{name}" + klass_name = name.to_s.gsub(/(?:^|_)(.)/) { $1.upcase } # camelize + commands[name] = eval("Commands::#{klass_name}") + rescue => e + raise "Can't load hbase shell command: #{name}. Error: #{e}\n#{e.backtrace.join("\n")}" + end + end + + def self.load_command_group(group, opts) + raise ArgumentError, "No :commands for group #{group}" unless opts[:commands] + + command_groups[group] = { + :commands => [], + :command_names => opts[:commands], + :full_name => opts[:full_name] || group, + :comment => opts[:comment] + } + + opts[:commands].each do |command| + load_command(command, group) + end + end + + #---------------------------------------------------------------------- + class Shell + attr_accessor :hbase + attr_accessor :formatter + + @debug = false + attr_accessor :debug + + def initialize(hbase, formatter) + self.hbase = hbase + self.formatter = formatter + end + + def hbase_admin + @hbase_admin ||= hbase.admin(formatter) + end + + def hbase_table(name) + hbase.table(name, self) + end + + def hbase_replication_admin + @hbase_replication_admin ||= hbase.replication_admin(formatter) + end + + def hbase_security_admin + @hbase_security_admin ||= hbase.security_admin(formatter) + end + + def export_commands(where) + ::Shell.commands.keys.each do |cmd| + # here where is the IRB namespace + # this method just adds the call to the specified command + # which just references back to 'this' shell object + # a decently extensible way to add commands + where.send :instance_eval, <<-EOF + def #{cmd}(*args) + ret = @shell.command('#{cmd}', *args) + puts + return ret + end + EOF + end + end + + def command_instance(command) + ::Shell.commands[command.to_s].new(self) + end + + #call the method 'command' on the specified command + def command(command, *args) + internal_command(command, :command, *args) + end + + #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 + def internal_command(command, method_name= :command, *args) + command_instance(command).command_safe(self.debug,method_name, *args) + end + + def print_banner + puts "HBase Shell; enter 'help' for list of supported commands." + puts 'Type "exit" to leave the HBase Shell' + print 'Version ' + command('version') + puts + end + + def help_multi_command(command) + puts "Command: #{command}" + puts command_instance(command).help + puts + return nil + end + + def help_command(command) + puts command_instance(command).help + return nil + end + + def help_group(group_name) + group = ::Shell.command_groups[group_name.to_s] + group[:commands].sort.each { |cmd| help_multi_command(cmd) } + if group[:comment] + puts '-' * 80 + puts + puts group[:comment] + puts + end + return nil + end + + def help(command = nil) + if command + return help_command(command) if ::Shell.commands[command.to_s] + return help_group(command) if ::Shell.command_groups[command.to_s] + puts "ERROR: Invalid command or command group name: #{command}" + puts + end + + puts help_header + puts + puts 'COMMAND GROUPS:' + ::Shell.command_groups.each do |name, group| + puts " Group name: " + name + puts " Commands: " + group[:command_names].sort.join(', ') + puts + end + unless command + puts 'SHELL USAGE:' + help_footer + end + return nil + end + + def help_header + return "HBase Shell, version #{org.apache.hadoop.hbase.util.VersionInfo.getVersion()}, " + + "r#{org.apache.hadoop.hbase.util.VersionInfo.getRevision()}, " + + "#{org.apache.hadoop.hbase.util.VersionInfo.getDate()}" + "\n" + + "Type 'help \"COMMAND\"', (e.g. 'help \"get\"' -- the quotes are necessary) for help on a specific command.\n" + + "Commands are grouped. Type 'help \"COMMAND_GROUP\"', (e.g. 'help \"general\"') for help on a command group." + end + + def help_footer + puts <<-HERE +Quote all names in HBase Shell such as table and column names. Commas delimit +command parameters. Type after entering a command to run it. +Dictionaries of configuration used in the creation and alteration of tables are +Ruby Hashes. They look like this: + + {'key1' => 'value1', 'key2' => 'value2', ...} + +and are opened and closed with curley-braces. Key/values are delimited by the +'=>' character combination. Usually keys are predefined constants such as +NAME, VERSIONS, COMPRESSION, etc. Constants do not need to be quoted. Type +'Object.constants' to see a (messy) list of all constants in the environment. + +If you are using binary keys or values and need to enter them in the shell, use +double-quote'd hexadecimal representation. For example: + + hbase> get 't1', "key\\x03\\x3f\\xcd" + hbase> get 't1', "key\\003\\023\\011" + hbase> put 't1', "test\\xef\\xff", 'f1:', "\\x01\\x33\\x40" + +The HBase shell is the (J)Ruby IRB with the above HBase-specific commands added. +For more on the HBase Shell, see http://hbase.apache.org/docs/current/book.html + HERE + end + end +end + +# Load commands base class +require 'shell/commands' + +# Load all commands +Shell.load_command_group( + 'general', + :full_name => 'GENERAL HBASE SHELL COMMANDS', + :commands => %w[ + status + version + table_help + whoami + ] +) + +Shell.load_command_group( + 'ddl', + :full_name => 'TABLES MANAGEMENT COMMANDS', + :commands => %w[ + alter + create + describe + disable + disable_all + is_disabled + drop + drop_all + enable + enable_all + is_enabled + exists + list + show_filters + alter_status + alter_async + get_table + ] +) + +Shell.load_command_group( + 'namespace', + :full_name => 'NAMESPACE MANAGEMENT COMMANDS', + :commands => %w[ + create_namespace + drop_namespace + alter_namespace + describe_namespace + list_namespace + list_namespace_tables + ] +) + +Shell.load_command_group( + 'dml', + :full_name => 'DATA MANIPULATION COMMANDS', + :commands => %w[ + count + delete + deleteall + get + get_counter + incr + put + scan + truncate + truncate_preserve + ] +) + +Shell.load_command_group( + 'tools', + :full_name => 'HBASE SURGERY TOOLS', + :comment => "WARNING: Above commands are for 'experts'-only as misuse can damage an install", + :commands => %w[ + assign + balancer + balance_switch + close_region + compact + flush + major_compact + move + split + merge_region + unassign + zk_dump + hlog_roll + catalogjanitor_run + catalogjanitor_switch + catalogjanitor_enabled + ] +) + +Shell.load_command_group( + 'replication', + :full_name => 'CLUSTER REPLICATION TOOLS', + :comment => "In order to use these tools, hbase.replication must be true.", + :commands => %w[ + add_peer + remove_peer + list_peers + enable_peer + disable_peer + list_replicated_tables + ] +) + +Shell.load_command_group( + 'snapshot', + :full_name => 'CLUSTER SNAPSHOT TOOLS', + :commands => %w[ + snapshot + clone_snapshot + restore_snapshot + rename_snapshot + delete_snapshot + list_snapshots + ] +) + +Shell.load_command_group( + 'security', + :full_name => 'SECURITY TOOLS', + :comment => "NOTE: Above commands are only applicable if running with the AccessController coprocessor", + :commands => %w[ + grant + revoke + user_permission + ] +) + Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,98 @@ +# +# +# 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 Command + + def initialize(shell) + @shell = shell + end + + #wrap an execution of cmd to catch hbase exceptions + # cmd - command name to execture + # args - arguments to pass to the command + def command_safe(debug, cmd = :command, *args) + # send is internal ruby method to call 'cmd' with *args + #(everything is a message, so this is just the formal semantics to support that idiom) + translate_hbase_exceptions(*args) { send(cmd,*args) } + rescue => e + rootCause = e + while rootCause != nil && rootCause.respond_to?(:cause) && rootCause.cause != nil + rootCause = rootCause.cause + end + puts + puts "ERROR: #{rootCause}" + puts "Backtrace: #{rootCause.backtrace.join("\n ")}" if debug + puts + puts "Here is some help for this command:" + puts help + puts + end + + def admin + @shell.hbase_admin + end + + def table(name) + @shell.hbase_table(name) + end + + def replication_admin + @shell.hbase_replication_admin + end + + def security_admin + @shell.hbase_security_admin + end + + #---------------------------------------------------------------------- + + def formatter + @shell.formatter + end + + def format_simple_command + now = Time.now + yield + formatter.header + formatter.footer(now) + end + + def format_and_return_simple_command + now = Time.now + ret = yield + formatter.header + formatter.footer(now) + return ret + end + + def translate_hbase_exceptions(*args) + yield + rescue org.apache.hadoop.hbase.TableNotFoundException + raise "Unknown table #{args.first}!" + rescue org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException + valid_cols = table(args.first).get_all_columns.map { |c| c + '*' } + raise "Unknown column family! Valid column names: #{valid_cols.join(", ")}" + rescue org.apache.hadoop.hbase.TableExistsException => e + raise "Table already exists: #{args.first}!" + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/add_peer.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/add_peer.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/add_peer.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/add_peer.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,43 @@ +# +# +# 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 AddPeer< Command + def help + return <<-EOF +Add a peer cluster to replicate to, the id must be a short and +the cluster key is composed like this: +hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent +This gives a full path for HBase to connect to another cluster. +Examples: + + hbase> add_peer '1', "server1.cie.com:2181:/hbase" + hbase> add_peer '2', "zk1,zk2,zk3:2182:/hbase-prod" +EOF + end + + def command(id, cluster_key) + format_simple_command do + replication_admin.add_peer(id, cluster_key) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,92 @@ +# +# +# 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 Alter < Command + def help + return <<-EOF +Alter a table. Depending on the HBase setting ("hbase.online.schema.update.enable"), +the table must be disabled or not to be altered (see help 'disable'). +You can add/modify/delete column families, as well as change table +configuration. Column families work similarly to create; column family +spec can either be a name string, or a dictionary with NAME attribute. +Dictionaries are described on the main help command output. + +For example, to change or add the 'f1' column family in table 't1' from +current value to keep a maximum of 5 cell VERSIONS, do: + + hbase> alter 't1', NAME => 'f1', VERSIONS => 5 + +You can operate on several column families: + + hbase> alter 't1', 'f1', {NAME => 'f2', IN_MEMORY => true}, {NAME => 'f3', VERSIONS => 5} + +To delete the 'f1' column family in table 't1', use one of: + + hbase> alter 't1', NAME => 'f1', METHOD => 'delete' + hbase> alter 't1', 'delete' => 'f1' + +You can also change table-scope attributes like MAX_FILESIZE, READONLY, +MEMSTORE_FLUSHSIZE, DEFERRED_LOG_FLUSH, etc. These can be put at the end; +for example, to change the max size of a region to 128MB, do: + + hbase> alter 't1', MAX_FILESIZE => '134217728' + +You can add a table coprocessor by setting a table coprocessor attribute: + + hbase> alter 't1', + 'coprocessor'=>'hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2' + +Since you can have multiple coprocessors configured for a table, a +sequence number will be automatically appended to the attribute name +to uniquely identify it. + +The coprocessor attribute must match the pattern below in order for +the framework to understand how to load the coprocessor classes: + + [coprocessor jar file location] | class name | [priority] | [arguments] + +You can also set configuration settings specific to this table or column family: + + hbase> alter 't1', CONFIGURATION => {'hbase.hregion.scan.loadColumnFamiliesOnDemand' => 'true'} + hbase> alter 't1', {NAME => 'f2', CONFIGURATION => {'hbase.hstore.blockingStoreFiles' => '10'}} + +You can also remove a table-scope attribute: + + hbase> alter 't1', METHOD => 'table_att_unset', NAME => 'MAX_FILESIZE' + + hbase> alter 't1', METHOD => 'table_att_unset', NAME => 'coprocessor$1' + +There could be more than one alteration in one command: + + hbase> alter 't1', { NAME => 'f1', VERSIONS => 3 }, + { MAX_FILESIZE => '134217728' }, { METHOD => 'delete', NAME => 'f2' }, + OWNER => 'johndoe', METADATA => { 'mykey' => 'myvalue' } +EOF + end + + def command(table, *args) + format_simple_command do + admin.alter(table, true, *args) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_async.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_async.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_async.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_async.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,65 @@ +# +# +# 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 AlterAsync < Command + def help + return <<-EOF +Alter column family schema, does not wait for all regions to receive the +schema changes. Pass table name and a dictionary specifying new column +family schema. Dictionaries are described on the main help command output. +Dictionary must include name of column family to alter. For example, + +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_async 't1', NAME => 'f1', VERSIONS => 5 + +To delete the 'f1' column family in table 't1', do: + + hbase> alter_async 't1', NAME => 'f1', METHOD => 'delete' + +or a shorter version: + + hbase> alter_async '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' + +There could be more than one alteration in one command: + + hbase> alter 't1', {NAME => 'f1'}, {NAME => 'f2', METHOD => 'delete'} + +To check if all the regions have been updated, use alter_status +EOF + end + + def command(table, *args) + format_simple_command do + admin.alter(table, false, *args) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_namespace.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_namespace.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_namespace.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_namespace.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,44 @@ +# +# +# 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 AlterNamespace < Command + def help + return <<-EOF +Alter namespace properties. + +To add/modify a property: + + hbase> alter_namespace 'ns1', {METHOD => 'set', 'PROERTY_NAME' => 'PROPERTY_VALUE'} + +To delete a property: + + hbase> alter_namespace 'ns1', {METHOD => 'unset', NAME=>'PROERTY_NAME'} +EOF + end + + def command(namespace, *args) + format_simple_command do + admin.alter_namespace(namespace, *args) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_status.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_status.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_status.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/alter_status.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,37 @@ +# +# +# 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 AlterStatus < Command + def help + return <<-EOF +Get the status of the alter command. Indicates the number of regions of the +table that have received the updated schema +Pass table name. + +hbase> alter_status 't1' +EOF + end + def command(table) + admin.alter_status(table) + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/assign.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/assign.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/assign.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/assign.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,41 @@ +# +# +# 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 Assign < Command + def help + return <<-EOF +Assign a region. Use with caution. If region already assigned, +this command will do a force reassign. For experts only. +Examples: + + hbase> assign 'REGIONNAME' + hbase> assign 'ENCODED_REGIONNAME' +EOF + end + + def command(region_name) + format_simple_command do + admin.assign(region_name) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/balance_switch.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/balance_switch.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/balance_switch.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/balance_switch.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,42 @@ +# +# +# 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 BalanceSwitch < Command + def help + return <<-EOF +Enable/Disable balancer. Returns previous balancer state. +Examples: + + hbase> balance_switch true + hbase> balance_switch false +EOF + end + + def command(enableDisable) + format_simple_command do + formatter.row([ + admin.balance_switch(enableDisable)? "true" : "false" + ]) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/balancer.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/balancer.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/balancer.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/balancer.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,40 @@ +# +# +# 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 Balancer < Command + def help + return <<-EOF +Trigger the cluster balancer. Returns true if balancer ran and was able to +tell the region servers to unassign all the regions to balance (the re-assignment itself is async). +Otherwise false (Will not run if regions in transition). +EOF + end + + def command() + format_simple_command do + formatter.row([ + admin.balancer()? "true": "false" + ]) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_enabled.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_enabled.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_enabled.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_enabled.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,40 @@ +# +# 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 CatalogjanitorEnabled < Command + def help + return <<-EOF +Query for the CatalogJanitor state (enabled/disabled?) +Examples: + + hbase> catalogjanitor_enabled +EOF + end + + def command() + format_simple_command do + formatter.row([ + admin.catalogjanitor_enabled()? "true" : "false" + ]) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_run.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_run.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_run.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_run.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,37 @@ +# +# 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 CatalogjanitorRun < Command + def help + return <<-EOF +Catalog janitor command to run the (garbage collection) scan from command line. + + hbase> catalogjanitor_run + +EOF + end + def command() + format_simple_command do + admin.catalogjanitor_run() + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_switch.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_switch.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_switch.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/catalogjanitor_switch.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,41 @@ +# +# 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 CatalogjanitorSwitch < Command + def help + return <<-EOF +Enable/Disable CatalogJanitor. Returns previous CatalogJanitor state. +Examples: + + hbase> catalogjanitor_switch true + hbase> catalogjanitor_switch false +EOF + end + + def command(enableDisable) + format_simple_command do + formatter.row([ + admin.catalogjanitor_switch(enableDisable)? "true" : "false" + ]) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,40 @@ +# +# 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 CloneSnapshot < Command + def help + return <<-EOF +Create a new table by cloning the snapshot content. +There're no copies of data involved. +And writing on the newly created table will not influence the snapshot data. + +Examples: + hbase> clone_snapshot 'snapshotName', 'tableName' +EOF + end + + def command(snapshot_name, table) + format_simple_command do + admin.clone_snapshot(snapshot_name, table) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/close_region.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/close_region.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/close_region.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/close_region.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,59 @@ +# +# +# 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 CloseRegion < Command + def help + return <<-EOF +Close a single region. Ask the master to close a region out on the cluster +or if 'SERVER_NAME' is supplied, ask the designated hosting regionserver to +close the region directly. Closing a region, the master expects 'REGIONNAME' +to be a fully qualified region name. When asking the hosting regionserver to +directly close a region, you pass the regions' encoded name only. A region +name looks like this: + + TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396. + +The trailing period is part of the regionserver name. A region's encoded name +is the hash at the end of a region name; e.g. 527db22f95c8a9e0116f0cc13c680396 +(without the period). A 'SERVER_NAME' is its host, port plus startcode. For +example: host187.example.com,60020,1289493121758 (find servername in master ui +or when you do detailed status in shell). This command will end up running +close on the region hosting regionserver. The close is done without the +master's involvement (It will not know of the close). Once closed, region will +stay closed. Use assign to reopen/reassign. Use unassign or move to assign +the region elsewhere on cluster. Use with caution. For experts only. +Examples: + + hbase> close_region 'REGIONNAME' + hbase> close_region 'REGIONNAME', 'SERVER_NAME' + hbase> close_region 'ENCODED_REGIONNAME' + hbase> close_region 'ENCODED_REGIONNAME', 'SERVER_NAME' +EOF + end + + def command(region_name, server = nil) + format_simple_command do + admin.close_region(region_name, server) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/compact.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/compact.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/compact.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/compact.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,47 @@ +# +# +# 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 Compact < Command + def help + return <<-EOF + Compact all regions in passed table or pass a region row + to compact an individual region. You can also compact a single column + family within a region. + Examples: + Compact all regions in a table: + hbase> compact 't1' + Compact an entire region: + hbase> compact 'r1' + Compact only a column family within a region: + hbase> compact 'r1', 'c1' + Compact a column family within a table: + hbase> compact 't1', 'c1' + EOF + end + + def command(table_or_region_name, family = nil) + format_simple_command do + admin.compact(table_or_region_name, family) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/count.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/count.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/count.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/count.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,76 @@ +# +# +# 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 Count < Command + def help + return <<-EOF +Count the number of rows in a table. Return value is the number of rows. +This operation may take a LONG time (Run '$HADOOP_HOME/bin/hadoop jar +hbase.jar rowcount' to run a counting mapreduce job). Current count is shown +every 1000 rows by default. Count interval may be optionally specified. Scan +caching is enabled on count scans by default. Default cache size is 10 rows. +If your rows are small in size, you may want to increase this +parameter. Examples: + + hbase> count 't1' + hbase> count 't1', INTERVAL => 100000 + hbase> count 't1', CACHE => 1000 + hbase> count 't1', INTERVAL => 10, CACHE => 1000 + +The same commands also can be run on a table reference. Suppose you had a reference +t to table 't1', the corresponding commands would be: + + hbase> t.count + hbase> t.count INTERVAL => 100000 + hbase> t.count CACHE => 1000 + hbase> t.count INTERVAL => 10, CACHE => 1000 +EOF + end + + def command(table, params = {}) + count(table(table), params) + end + + def count(table, params = {}) + # If the second parameter is an integer, then it is the old command syntax + params = { 'INTERVAL' => params } if params.kind_of?(Fixnum) + + # Merge params with defaults + params = { + 'INTERVAL' => 1000, + 'CACHE' => 10 + }.merge(params) + + # Call the counter method + now = Time.now + formatter.header + count = table._count_internal(params['INTERVAL'].to_i, params['CACHE'].to_i) do |cnt, row| + formatter.row([ "Current count: #{cnt}, row: #{row}" ]) + end + formatter.footer(now, count) + return count + end + end + end +end + +#Add the method table.count that calls count.count +::Hbase::Table.add_shell_command("count") Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/create.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/create.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/create.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/create.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,68 @@ +# +# +# 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 Create < Command + def help + return <<-EOF +Creates a table. Pass a table name, and a set of column family +specifications (at least one), and, optionally, table configuration. +Column specification can be a simple string (name), or a dictionary +(dictionaries are described below in main help output), necessarily +including NAME attribute. +Examples: + + hbase> create 't1', {NAME => 'f1', VERSIONS => 5} + hbase> create 't1', {NAME => 'f1'}, {NAME => 'f2'}, {NAME => 'f3'} + hbase> # The above in shorthand would be the following: + hbase> create 't1', 'f1', 'f2', 'f3' + hbase> create 't1', {NAME => 'f1', VERSIONS => 1, TTL => 2592000, BLOCKCACHE => true} + hbase> create 't1', {NAME => 'f1', CONFIGURATION => {'hbase.hstore.blockingStoreFiles' => '10'}} + +Table configuration options can be put at the end. +Examples: + + hbase> create 't1', 'f1', SPLITS => ['10', '20', '30', '40'] + hbase> create 't1', 'f1', SPLITS_FILE => 'splits.txt', OWNER => 'johndoe' + hbase> create 't1', {NAME => 'f1', VERSIONS => 5}, METADATA => { 'mykey' => 'myvalue' } + hbase> # Optionally pre-split the table into NUMREGIONS, using + hbase> # SPLITALGO ("HexStringSplit", "UniformSplit" or classname) + hbase> create 't1', 'f1', {NUMREGIONS => 15, SPLITALGO => 'HexStringSplit'} + hbase> create 't1', 'f1', {NUMREGIONS => 15, SPLITALGO => 'HexStringSplit', CONFIGURATION => {'hbase.hregion.scan.loadColumnFamiliesOnDemand' => 'true'}} + +You can also keep around a reference to the created table: + + hbase> t1 = create 't1', 'f1' + +Which gives you a reference to the table named 't1', on which you can then +call methods. +EOF + end + + def command(table, *args) + format_simple_command do + ret = admin.create(table, *args) + end + #and then return the table you just created + table(table) + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/create_namespace.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/create_namespace.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/create_namespace.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/create_namespace.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,41 @@ +# +# +# 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 CreateNamespace < Command + def help + return <<-EOF +Create namespace; pass namespace name, +and optionally a dictionary of namespace configuration. +Examples: + + hbase> create_namespace 'ns1' + hbase> create_namespace 'ns1', {'PROERTY_NAME'=>'PROPERTY_VALUE'} +EOF + end + + def command(namespace, *args) + format_simple_command do + admin.create_namespace(namespace, *args) + end + end + end + end +end Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/delete.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/delete.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/delete.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/delete.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,54 @@ +# +# +# 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 Delete < Command + def help + return <<-EOF +Put a delete cell value at specified table/row/column and optionally +timestamp coordinates. Deletes must match the deleted cell's +coordinates exactly. When scanning, a delete cell suppresses older +versions. To delete a cell from 't1' at row 'r1' under column 'c1' +marked with the time 'ts1', do: + + hbase> delete 't1', 'r1', 'c1', ts1 + +The same command can also be run on a table reference. Suppose you had a reference +t to table 't1', the corresponding command would be: + + hbase> t.delete 'r1', 'c1', ts1 +EOF + end + + def command(table, row, column, timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP) + delete(table(table), row, column, timestamp) + end + + def delete(table, row, column, timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP) + format_simple_command do + table._delete_internal(row, column, timestamp) + end + end + end + end +end + +#Add the method table.delete that calls delete.delete +::Hbase::Table.add_shell_command("delete") Added: hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/delete_snapshot.rb URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/delete_snapshot.rb?rev=1525642&view=auto ============================================================================== --- hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/delete_snapshot.rb (added) +++ hbase/branches/0.96/hbase-shell/src/main/ruby/shell/commands/delete_snapshot.rb Mon Sep 23 16:45:42 2013 @@ -0,0 +1,37 @@ +# +# 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 DeleteSnapshot < Command + def help + return <<-EOF +Delete a specified snapshot. Examples: + + hbase> delete_snapshot 'snapshotName', +EOF + end + + def command(snapshot_name) + format_simple_command do + admin.delete_snapshot(snapshot_name) + end + end + end + end +end