You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/05/22 02:32:32 UTC

[1/2] hbase git commit: HBASE-18075 Support non-latin table names and namespaces

Repository: hbase
Updated Branches:
  refs/heads/master 553d5db35 -> 709f5a198


HBASE-18075 Support non-latin table names and namespaces


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

Branch: refs/heads/master
Commit: 709f5a1980abe65eb85c638dacaaea8502ee0034
Parents: f1544c3
Author: Josh Elser <el...@apache.org>
Authored: Thu May 18 18:38:25 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 21 22:24:12 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/TestHTableDescriptor.java      | 20 ++++-
 .../java/org/apache/hadoop/hbase/TableName.java | 84 ++++++++++++--------
 .../org/apache/hadoop/hbase/io/HFileLink.java   |  3 +-
 3 files changed, 73 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/709f5a19/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
index 9a6d3e3..bcff565 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
@@ -181,10 +182,13 @@ public class TestHTableDescriptor {
   String legalTableNames[] = { "foo", "with-dash_under.dot", "_under_start_ok",
       "with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02"
       , "dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
-      "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02"};
+      "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02",
+      "汉", "汉:字", "_字_", "foo:字", "foo.字", "字.foo"};
+  // Avoiding "zookeeper" in here as it's tough to encode in regex
   String illegalTableNames[] = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
       "-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
-      "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2"};
+      "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2", String.valueOf((char)130),
+      String.valueOf((char)5), String.valueOf((char)65530)};
 
   @Test
   public void testLegalHTableNames() {
@@ -206,6 +210,18 @@ public class TestHTableDescriptor {
   }
 
   @Test
+  public void testIllegalZooKeeperName() {
+    for (String name : Arrays.asList("zookeeper", "ns:zookeeper", "zookeeper:table")) {
+      try {
+        TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(name));
+        fail("invalid tablename " + name + " should have failed");
+      } catch (Exception e) {
+        // expected
+      }
+    }
+  }
+
+  @Test
   public void testLegalHTableNamesRegex() {
     for (String tn : legalTableNames) {
       TableName tName = TableName.valueOf(tn);

http://git-wip-us.apache.org/repos/asf/hbase/blob/709f5a19/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
index c4c15d0..a9b2527 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArraySet;
@@ -66,10 +67,10 @@ public final class TableName implements Comparable<TableName> {
   // in default namespace
   //Allows only letters, digits and '_'
   public static final String VALID_NAMESPACE_REGEX =
-      "(?:[a-zA-Z_0-9]+)";
+      "(?:[_\\p{Digit}\\p{IsAlphabetic}]+)";
   //Allows only letters, digits, '_', '-' and '.'
   public static final String VALID_TABLE_QUALIFIER_REGEX =
-      "(?:[a-zA-Z_0-9][a-zA-Z_0-9-.]*)";
+      "(?:[_\\p{Digit}\\p{IsAlphabetic}][-_.\\p{Digit}\\p{IsAlphabetic}]*)";
   //Concatenation of NAMESPACE_REGEX and TABLE_QUALIFIER_REGEX,
   //with NAMESPACE_DELIM as delimiter
   public static final String VALID_USER_TABLE_REGEX =
@@ -87,6 +88,9 @@ public final class TableName implements Comparable<TableName> {
   public static final String OLD_META_STR = ".META.";
   public static final String OLD_ROOT_STR = "-ROOT-";
 
+  /** One globally disallowed name */
+  public static final String DISALLOWED_TABLE_NAME = "zookeeper";
+
   /**
    * @return True if <code>tn</code> is the hbase:meta table name.
    */
@@ -118,14 +122,14 @@ public final class TableName implements Comparable<TableName> {
    * @return Returns passed <code>tableName</code> param
    * @throws IllegalArgumentException if passed a tableName is null or
    * is made of other than 'word' characters or underscores: i.e.
-   * <code>[a-zA-Z_0-9.-:]</code>. The ':' is used to delimit the namespace
+   * <code>[\p{IsAlphabetic}\p{Digit}.-:]</code>. The ':' is used to delimit the namespace
    * from the table name and can be used for nothing else.
    *
    * Namespace names can only contain 'word' characters
-   * <code>[a-zA-Z_0-9]</code> or '_'
+   * <code>[\p{IsAlphabetic}\p{Digit}]</code> or '_'
    *
    * Qualifier names can only contain 'word' characters
-   * <code>[a-zA-Z_0-9]</code> or '_', '.' or '-'.
+   * <code>[\p{IsAlphabetic}\p{Digit}]</code> or '_', '.' or '-'.
    * The name may not start with '.' or '-'.
    *
    * Valid fully qualified table names:
@@ -161,7 +165,7 @@ public final class TableName implements Comparable<TableName> {
 
   /**
    * Qualifier names can only contain 'word' characters
-   * <code>[a-zA-Z_0-9]</code> or '_', '.' or '-'.
+   * <code>[\p{IsAlphabetic}\p{Digit}]</code> or '_', '.' or '-'.
    * The name may not start with '.' or '-'.
    *
    * @param qualifierName byte array containing the qualifier name
@@ -181,29 +185,37 @@ public final class TableName implements Comparable<TableName> {
     if(end - start < 1) {
       throw new IllegalArgumentException(isSnapshot ? "Snapshot" : "Table" + " qualifier must not be empty");
     }
-
     if (qualifierName[start] == '.' || qualifierName[start] == '-') {
       throw new IllegalArgumentException("Illegal first character <" + qualifierName[start] +
                                          "> at 0. " + (isSnapshot ? "Snapshot" : "User-space table") +
                                          " qualifiers can only start with 'alphanumeric " +
-                                         "characters': i.e. [a-zA-Z_0-9]: " +
+                                         "characters' from any language: " +
                                          Bytes.toString(qualifierName, start, end));
     }
-    for (int i = start; i < end; i++) {
-      if (Character.isLetterOrDigit(qualifierName[i]) ||
-          qualifierName[i] == '_' ||
-          qualifierName[i] == '-' ||
-          qualifierName[i] == '.') {
+    // Treat the bytes as UTF-8
+    String qualifierString = new String(
+        qualifierName, start, (end - start), StandardCharsets.UTF_8);
+    if (qualifierString.equals(DISALLOWED_TABLE_NAME)) {
+      // Per https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
+      // A znode named "zookeeper" is disallowed by zookeeper.
+      throw new IllegalArgumentException("Tables may not be named '" + DISALLOWED_TABLE_NAME + "'");
+    }
+    for (int i = 0; i < qualifierString.length(); i++) {
+      // Treat the string as a char-array as some characters may be multi-byte
+      char c = qualifierString.charAt(i);
+      // Check for letter, digit, underscore, hyphen, or period, and allowed by ZK.
+      // ZooKeeper also has limitations, but Character.isAlphabetic omits those all
+      //   See https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
+      if (Character.isAlphabetic(c) || Character.isDigit(c) || c == '_' || c == '-' || c == '.') {
         continue;
       }
-      throw new IllegalArgumentException("Illegal character code:" + qualifierName[i] +
-                                         ", <" + (char) qualifierName[i] + "> at " + i +
-                                         ". " + (isSnapshot ? "Snapshot" : "User-space table") +
-                                         " qualifiers can only contain " +
-                                         "'alphanumeric characters': i.e. [a-zA-Z_0-9-.]: " +
-                                         Bytes.toString(qualifierName, start, end));
+      throw new IllegalArgumentException("Illegal character code:" + (int) c + ", <" + c + "> at " +
+          i + ". " + (isSnapshot ? "Snapshot" : "User-space table") +
+          " qualifiers may only contain 'alphanumeric characters' and digits: " +
+          qualifierString);
     }
   }
+
   public static void isLegalNamespaceName(byte[] namespaceName) {
     isLegalNamespaceName(namespaceName, 0, namespaceName.length);
   }
@@ -217,14 +229,23 @@ public final class TableName implements Comparable<TableName> {
     if(end - start < 1) {
       throw new IllegalArgumentException("Namespace name must not be empty");
     }
-    for (int i = start; i < end; i++) {
-      if (Character.isLetterOrDigit(namespaceName[i])|| namespaceName[i] == '_') {
+    String nsString = new String(namespaceName, start, (end - start), StandardCharsets.UTF_8);
+    if (nsString.equals(DISALLOWED_TABLE_NAME)) {
+      // Per https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
+      // A znode named "zookeeper" is disallowed by zookeeper.
+      throw new IllegalArgumentException("Tables may not be named '" + DISALLOWED_TABLE_NAME + "'");
+    }
+    for (int i = 0; i < nsString.length(); i++) {
+      // Treat the string as a char-array as some characters may be multi-byte
+      char c = nsString.charAt(i);
+      // ZooKeeper also has limitations, but Character.isAlphabetic omits those all
+      //   See https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
+      if (Character.isAlphabetic(c) || Character.isDigit(c)|| c == '_') {
         continue;
       }
-      throw new IllegalArgumentException("Illegal character <" + namespaceName[i] +
-        "> at " + i + ". Namespaces can only contain " +
-        "'alphanumeric characters': i.e. [a-zA-Z_0-9]: " + Bytes.toString(namespaceName,
-          start, end));
+      throw new IllegalArgumentException("Illegal character <" + c +
+        "> at " + i + ". Namespaces may only contain " +
+        "'alphanumeric characters' from any language and digits: " + nsString);
     }
   }
 
@@ -441,18 +462,19 @@ public final class TableName implements Comparable<TableName> {
       }
     }
 
-    int namespaceDelimIndex = name.indexOf(NAMESPACE_DELIM);
-    byte[] nameB = Bytes.toBytes(name);
+    final int namespaceDelimIndex = name.indexOf(NAMESPACE_DELIM);
 
     if (namespaceDelimIndex < 0) {
       return createTableNameIfNecessary(
           ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
-          ByteBuffer.wrap(nameB));
+          ByteBuffer.wrap(Bytes.toBytes(name)));
     } else {
+      // indexOf is by character, not byte (consider multi-byte characters)
+      String ns = name.substring(0, namespaceDelimIndex);
+      String qualifier = name.substring(namespaceDelimIndex + 1);
       return createTableNameIfNecessary(
-          ByteBuffer.wrap(nameB, 0, namespaceDelimIndex),
-          ByteBuffer.wrap(nameB, namespaceDelimIndex + 1,
-              nameB.length - (namespaceDelimIndex + 1)));
+          ByteBuffer.wrap(Bytes.toBytes(ns)),
+          ByteBuffer.wrap(Bytes.toBytes(qualifier)));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/709f5a19/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
index cdc5be1..96ad7de 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
@@ -64,7 +64,8 @@ public class HFileLink extends FileLink {
    * The HFileLink describe a link to an hfile in a different table/region
    * and the name is in the form: table=region-hfile.
    * <p>
-   * Table name is ([a-zA-Z_0-9][a-zA-Z_0-9.-]*), so '=' is an invalid character for the table name.
+   * Table name is ([\p{IsAlphabetic}\p{Digit}][\p{IsAlphabetic}\p{Digit}.-]*), so '=' is an invalid
+   * character for the table name.
    * Region name is ([a-f0-9]+), so '-' is an invalid character for the region name.
    * HFile is ([0-9a-f]+(?:_SeqId_[0-9]+_)?) covering the plain hfiles (uuid)
    * and the bulk loaded (_SeqId_[0-9]+_) hfiles.


[2/2] hbase git commit: HBASE-18067 Allow default FORMATTER for shell put/get commands

Posted by el...@apache.org.
HBASE-18067 Allow default FORMATTER for shell put/get commands


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

Branch: refs/heads/master
Commit: f1544c34669a69cb3ed163e45274f2fcca18e3eb
Parents: 553d5db
Author: Josh Elser <el...@apache.org>
Authored: Wed May 17 19:19:23 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 21 22:24:12 2017 -0400

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase/table.rb        |  51 +++---
 hbase-shell/src/main/ruby/hbase_constants.rb    |   2 +
 hbase-shell/src/main/ruby/shell/commands/get.rb |   9 +-
 .../src/main/ruby/shell/commands/scan.rb        |   9 +-
 .../src/test/ruby/shell/converter_test.rb       | 157 +++++++++++++++++++
 5 files changed, 206 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f1544c34/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 22bbcfe..946c72c 100644
--- a/hbase-shell/src/main/ruby/hbase/table.rb
+++ b/hbase-shell/src/main/ruby/hbase/table.rb
@@ -347,6 +347,8 @@ EOF
       authorizations = args[AUTHORIZATIONS]
       consistency = args.delete(CONSISTENCY) if args[CONSISTENCY]
       replicaId = args.delete(REGION_REPLICA_ID) if args[REGION_REPLICA_ID]
+      converter = args.delete(FORMATTER) || nil
+      converter_class = args.delete(FORMATTER_CLASS) || 'org.apache.hadoop.hbase.util.Bytes'
       unless args.empty?
         columns = args[COLUMN] || args[COLUMNS]
         if args[VERSIONS]
@@ -419,13 +421,13 @@ EOF
       # Print out results.  Result can be Cell or RowResult.
       res = {}
       result.listCells.each do |c|
-        family = org.apache.hadoop.hbase.util.Bytes::toStringBinary(c.getFamilyArray,
-          c.getFamilyOffset, c.getFamilyLength)
-        qualifier = org.apache.hadoop.hbase.util.Bytes::toStringBinary(c.getQualifierArray,
-          c.getQualifierOffset, c.getQualifierLength)
+        family = convert_bytes_with_position(c.getFamilyArray,
+          c.getFamilyOffset, c.getFamilyLength, converter_class, converter)
+        qualifier = convert_bytes_with_position(c.getQualifierArray,
+          c.getQualifierOffset, c.getQualifierLength, converter_class, converter)
 
         column = "#{family}:#{qualifier}"
-        value = to_string(column, c, maxlength)
+        value = to_string(column, c, maxlength, converter_class, converter)
 
         if block_given?
           yield(column, value)
@@ -544,6 +546,8 @@ EOF
 
       limit = args["LIMIT"] || -1
       maxlength = args.delete("MAXLENGTH") || -1
+      converter = args.delete(FORMATTER) || nil
+      converter_class = args.delete(FORMATTER_CLASS) || 'org.apache.hadoop.hbase.util.Bytes'
       count = 0
       res = {}
 
@@ -555,17 +559,17 @@ EOF
       # Iterate results
       while iter.hasNext
         row = iter.next
-        key = org.apache.hadoop.hbase.util.Bytes::toStringBinary(row.getRow)
+        key = convert_bytes(row.getRow, nil, converter)
         is_stale |= row.isStale
 
         row.listCells.each do |c|
-          family = org.apache.hadoop.hbase.util.Bytes::toStringBinary(c.getFamilyArray,
-            c.getFamilyOffset, c.getFamilyLength)
-          qualifier = org.apache.hadoop.hbase.util.Bytes::toStringBinary(c.getQualifierArray,
-            c.getQualifierOffset, c.getQualifierLength)
+          family = convert_bytes_with_position(c.getFamilyArray,
+            c.getFamilyOffset, c.getFamilyLength, converter_class, converter)
+          qualifier = convert_bytes_with_position(c.getQualifierArray,
+            c.getQualifierOffset, c.getQualifierLength, converter_class, converter)
 
           column = "#{family}:#{qualifier}"
-          cell = to_string(column, c, maxlength)
+          cell = to_string(column, c, maxlength, converter_class, converter)
 
           if block_given?
             yield(key, "column=#{column}, #{cell}")
@@ -693,7 +697,7 @@ EOF
 
     # 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)
+    def to_string(column, kv, maxlength = -1, converter_class = nil, converter=nil)
       if is_meta_table?
         if column == 'info:regioninfo' or column == 'info:splitA' or column == 'info:splitB'
           hri = org.apache.hadoop.hbase.HRegionInfo.parseFromOrNull(kv.getValueArray,
@@ -715,16 +719,16 @@ EOF
       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)}"
+        val = "timestamp=#{kv.getTimestamp}, value=#{convert(column, kv, converter_class, converter)}"
       end
       (maxlength != -1) ? val[0, maxlength] : val
     end
 
-    def convert(column, kv)
+    def convert(column, kv, converter_class='org.apache.hadoop.hbase.util.Bytes', converter='toStringBinary')
       #use org.apache.hadoop.hbase.util.Bytes as the default class
-      klazz_name = 'org.apache.hadoop.hbase.util.Bytes'
+      converter_class = 'org.apache.hadoop.hbase.util.Bytes' unless converter_class
       #use org.apache.hadoop.hbase.util.Bytes::toStringBinary as the default convertor
-      converter = 'toStringBinary'
+      converter = 'toStringBinary' unless converter
       if @converters.has_key?(column)
         # lookup the CONVERTER for certain column - "cf:qualifier"
         matches = /c\((.+)\)\.(.+)/.match(@converters[column])
@@ -737,8 +741,19 @@ EOF
           converter = matches[2]
         end
       end
-      method = eval(klazz_name).method(converter)
-      return method.call(org.apache.hadoop.hbase.CellUtil.cloneValue(kv)) # apply the converter
+      # apply the converter
+      convert_bytes(org.apache.hadoop.hbase.CellUtil.cloneValue(kv), klazz_name, converter)
+    end
+
+    def convert_bytes(bytes, converter_class=nil, converter_method=nil)
+      convert_bytes_with_position(bytes, 0, bytes.length, converter_class, converter_method)
+    end
+
+    def convert_bytes_with_position(bytes, offset, len, converter_class, converter_method)
+      # Avoid nil
+      converter_class = 'org.apache.hadoop.hbase.util.Bytes' unless converter_class
+      converter_method = 'toStringBinary' unless converter_method
+      eval(converter_class).method(converter_method).call(bytes, offset, len)
     end
 
     # if the column spec contains CONVERTER information, to get rid of :CONVERTER info from column pair.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f1544c34/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
index 52819c0..7d6da9f 100644
--- a/hbase-shell/src/main/ruby/hbase_constants.rb
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -84,6 +84,8 @@ module HBaseConstants
   SERVER_NAME = 'SERVER_NAME'
   LOCALITY_THRESHOLD = 'LOCALITY_THRESHOLD'
   RESTORE_ACL = 'RESTORE_ACL'
+  FORMATTER = 'FORMATTER'
+  FORMATTER_CLASS = 'FORMATTER_CLASS'
 
   # Load constants from hbase java API
   def self.promote_constants(constants)

http://git-wip-us.apache.org/repos/asf/hbase/blob/f1544c34/hbase-shell/src/main/ruby/shell/commands/get.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/get.rb b/hbase-shell/src/main/ruby/shell/commands/get.rb
index 8191c22..6b9ad42 100644
--- a/hbase-shell/src/main/ruby/shell/commands/get.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/get.rb
@@ -53,8 +53,13 @@ Example formatting cf:qualifier1 and cf:qualifier2 both as Integers:
   hbase> get 't1', 'r1' {COLUMN => ['cf:qualifier1:toInt',
     'cf:qualifier2:c(org.apache.hadoop.hbase.util.Bytes).toInt'] } 
 
-Note that you can specify a FORMATTER by column only (cf:qualifier).  You cannot specify
-a FORMATTER for all columns of a column family.
+Note that you can specify a FORMATTER by column only (cf:qualifier). You can set a
+formatter for all columns (including, all key parts) using the "FORMATTER"
+and "FORMATTER_CLASS" options. The default "FORMATTER_CLASS" is
+"org.apache.hadoop.hbase.util.Bytes".
+
+  hbase> get 't1', 'r1', {FORMATTER => 'toString'}
+  hbase> get 't1', 'r1', {FORMATTER_CLASS => 'org.apache.hadoop.hbase.util.Bytes', FORMATTER => 'toString'}
     
 The same commands also can be run on a reference to a table (obtained via get_table or
 create_table). Suppose you had a reference t to table 't1', the corresponding commands

http://git-wip-us.apache.org/repos/asf/hbase/blob/f1544c34/hbase-shell/src/main/ruby/shell/commands/scan.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/scan.rb b/hbase-shell/src/main/ruby/shell/commands/scan.rb
index b3cc5c8..dda9899 100644
--- a/hbase-shell/src/main/ruby/shell/commands/scan.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/scan.rb
@@ -83,8 +83,13 @@ Example formatting cf:qualifier1 and cf:qualifier2 both as Integers:
   hbase> scan 't1', {COLUMNS => ['cf:qualifier1:toInt',
     'cf:qualifier2:c(org.apache.hadoop.hbase.util.Bytes).toInt'] } 
 
-Note that you can specify a FORMATTER by column only (cf:qualifier).  You cannot
-specify a FORMATTER for all columns of a column family.
+Note that you can specify a FORMATTER by column only (cf:qualifier). You can set a
+formatter for all columns (including, all key parts) using the "FORMATTER"
+and "FORMATTER_CLASS" options. The default "FORMATTER_CLASS" is
+"org.apache.hadoop.hbase.util.Bytes".
+
+  hbase> scan 't1', {FORMATTER => 'toString'}
+  hbase> scan 't1', {FORMATTER_CLASS => 'org.apache.hadoop.hbase.util.Bytes', FORMATTER => 'toString'}
 
 Scan can also be used directly from a table, by first getting a reference to a
 table, like such:

http://git-wip-us.apache.org/repos/asf/hbase/blob/f1544c34/hbase-shell/src/test/ruby/shell/converter_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/converter_test.rb b/hbase-shell/src/test/ruby/shell/converter_test.rb
new file mode 100644
index 0000000..8b6079b
--- /dev/null
+++ b/hbase-shell/src/test/ruby/shell/converter_test.rb
@@ -0,0 +1,157 @@
+# 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 'hbase_constants'
+require 'shell'
+
+include HBaseConstants
+
+module Hbase
+  class ConverterTest < Test::Unit::TestCase
+    include TestHelpers
+
+    non_ascii_text = '⻆⻇'
+    non_ascii_row = '⻄'
+    non_ascii_family = 'ㄹ'
+    non_ascii_qualifier = '⻅'
+    non_ascii_column = "#{non_ascii_family}:#{non_ascii_qualifier}"
+    hex_text = '\xE2\xBB\x86\xE2\xBB\x87'
+    hex_row = '\xE2\xBB\x84'
+    hex_family = '\xE3\x84\xB9'
+    hex_qualifier = '\xE2\xBB\x85'
+    hex_column = "#{hex_family}:#{hex_qualifier}"
+
+    def setup
+      setup_hbase
+    end
+
+    def teardown
+      shutdown
+    end
+
+    define_test 'Test scan for non-ascii data' do
+      table_name = 'scan-test'
+      create_test_table(table_name)
+      # Write a record
+      command(:put, table_name, 'r1', 'x:a', non_ascii_text)
+      output = capture_stdout{ command(:scan, table_name) }
+      # Encoded value not there by default
+      assert(!output.include?(non_ascii_text))
+      # Hex-encoded value is there by default (manually converted)
+      assert(output.include?(hex_text))
+
+      # Use the formatter method
+      output = capture_stdout{ command(:scan, table_name, {'FORMATTER'=>'toString'}) }
+      # Should have chinese characters
+      assert(output.include?(non_ascii_text))
+      # Should not have hex-encoded string
+      assert(!output.include?(hex_text))
+
+      # Use the formatter method + class
+      output = capture_stdout{ command(:scan, table_name, {'FORMATTER'=>'toString', 'FORMATTER_CLASS' => 'org.apache.hadoop.hbase.util.Bytes'}) }
+      # Should have chinese characters
+      assert(output.include?(non_ascii_text))
+      # Should not have hex-encoded string
+      assert(!output.include?(hex_text))
+
+      command(:disable, table_name)
+      command(:drop, table_name)
+      command(:create, table_name, non_ascii_family)
+
+      command(:put, table_name, non_ascii_row, non_ascii_column, non_ascii_text)
+      output = capture_stdout{ command(:scan, table_name) }
+      # By default, get hex-encoded data
+      assert(!output.include?(non_ascii_text))
+      assert(!output.include?(non_ascii_row))
+      assert(!output.include?(non_ascii_column))
+      assert(output.include?(hex_text))
+      assert(output.include?(hex_row))
+      assert(output.include?(hex_column))
+
+      # Use the formatter method
+      output = capture_stdout{ command(:scan, table_name, {'FORMATTER'=>'toString'}) }
+      # By default, get hex-encoded data
+      assert(output.include?(non_ascii_text))
+      assert(output.include?(non_ascii_row))
+      assert(output.include?(non_ascii_column))
+      assert(!output.include?(hex_text))
+      assert(!output.include?(hex_row))
+      assert(!output.include?(hex_column))
+
+      # Use the formatter method + class
+      output = capture_stdout{ command(:scan, table_name, {'FORMATTER'=>'toString', 'FORMATTER_CLASS' => 'org.apache.hadoop.hbase.util.Bytes'}) }
+      # By default, get hex-encoded data
+      assert(output.include?(non_ascii_text))
+      assert(output.include?(non_ascii_row))
+      assert(output.include?(non_ascii_column))
+      assert(!output.include?(hex_text))
+      assert(!output.include?(hex_row))
+      assert(!output.include?(hex_column))
+    end
+
+    define_test 'Test get for non-ascii data' do
+      table_name = 'get-test'
+      create_test_table(table_name)
+      # Write a record
+      command(:put, table_name, 'r1', 'x:a', non_ascii_text)
+      output = capture_stdout{ command(:get, table_name, 'r1') }
+      # Encoded value not there by default
+      assert(!output.include?(non_ascii_text))
+      # Hex-encoded value is there by default (manually converted)
+      assert(output.include?(hex_text))
+
+      # use the formatter method
+      output = capture_stdout{ command(:get, table_name, 'r1', {'FORMATTER'=>'toString'}) }
+      # Should have chinese characters
+      assert(output.include?(non_ascii_text))
+      # Should not have hex-encoded string
+      assert(!output.include?(hex_text))
+
+      # use the formatter method + class
+      output = capture_stdout{ command(:get, table_name, 'r1', {'FORMATTER'=>'toString', 'FORMATTER_CLASS' => 'org.apache.hadoop.hbase.util.Bytes'}) }
+      # Should have chinese characters
+      assert(output.include?(non_ascii_text))
+      # Should not have hex-encoded string
+      assert(!output.include?(hex_text))
+
+      command(:disable, table_name)
+      command(:drop, table_name)
+      command(:create, table_name, non_ascii_family)
+
+      # use no formatter (expect hex)
+      command(:put, table_name, non_ascii_row, non_ascii_column, non_ascii_text)
+      output = capture_stdout{ command(:get, table_name, non_ascii_row) }
+      assert(!output.include?(non_ascii_text))
+      assert(!output.include?(non_ascii_column))
+      assert(output.include?(hex_text))
+      assert(output.include?(hex_column))
+
+      # use the formatter method
+      output = capture_stdout{ command(:get, table_name, non_ascii_row, {'FORMATTER'=>'toString'}) }
+      assert(output.include?(non_ascii_text))
+      assert(output.include?(non_ascii_column))
+      assert(!output.include?(hex_text))
+      assert(!output.include?(hex_column))
+
+      # use the formatter method + class
+      output = capture_stdout{ command(:get, table_name, non_ascii_row, {'FORMATTER'=>'toString', 'FORMATTER_CLASS' => 'org.apache.hadoop.hbase.util.Bytes'}) }
+      assert(output.include?(non_ascii_text))
+      assert(output.include?(non_ascii_column))
+      assert(!output.include?(hex_text))
+      assert(!output.include?(hex_column))
+    end
+  end
+end
\ No newline at end of file