You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2015/08/13 00:22:05 UTC

[5/8] hbase git commit: HBASE-14122 Client API for determining if server side supports cell level security

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index a4e5f74..dc66754 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -208,6 +208,15 @@ public class AccessController extends BaseMasterAndRegionObserver
   /** if the ACL table is available, only relevant in the master */
   private volatile boolean aclTabAvailable = false;
 
+  public static boolean isAuthorizationSupported(Configuration conf) {
+    return conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+  }
+
+  public static boolean isCellAuthorizationSupported(Configuration conf) {
+    return isAuthorizationSupported(conf) &&
+        (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS);
+  }
+
   public Region getRegion() {
     return regionEnv != null ? regionEnv.getRegion() : null;
   }
@@ -924,7 +933,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     CompoundConfiguration conf = new CompoundConfiguration();
     conf.add(env.getConfiguration());
 
-    authorizationEnabled = conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+    authorizationEnabled = isAuthorizationSupported(conf);
     if (!authorizationEnabled) {
       LOG.warn("The AccessController has been loaded with authorization checks disabled.");
     }
@@ -932,7 +941,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     shouldCheckExecPermission = conf.getBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY,
       AccessControlConstants.DEFAULT_EXEC_PERMISSION_CHECKS);
 
-    cellFeaturesEnabled = HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS;
+    cellFeaturesEnabled = (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS);
     if (!cellFeaturesEnabled) {
       LOG.info("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
           + " is required to persist cell ACLs. Consider setting " + HFile.FORMAT_VERSION_KEY

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 7d3b82e..d013515 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -148,11 +148,19 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     RESERVED_VIS_TAG_TYPES.add(TagType.STRING_VIS_TAG_TYPE);
   }
 
+  public static boolean isAuthorizationSupported(Configuration conf) {
+    return conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+  }
+
+  public static boolean isCellAuthorizationSupported(Configuration conf) {
+    return isAuthorizationSupported(conf);
+  }
+
   @Override
   public void start(CoprocessorEnvironment env) throws IOException {
     this.conf = env.getConfiguration();
 
-    authorizationEnabled = conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+    authorizationEnabled = isAuthorizationSupported(conf);
     if (!authorizationEnabled) {
       LOG.warn("The VisibilityController has been loaded with authorization checks disabled.");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 5abef21..4bad60a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@@ -316,6 +317,16 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Test
+  public void testSecurityCapabilities() throws Exception {
+    List<SecurityCapability> capabilities = TEST_UTIL.getConnection().getAdmin()
+      .getSecurityCapabilities();
+    assertTrue("AUTHORIZATION capability is missing",
+      capabilities.contains(SecurityCapability.AUTHORIZATION));
+    assertTrue("CELL_AUTHORIZATION capability is missing",
+      capabilities.contains(SecurityCapability.CELL_AUTHORIZATION));
+  }
+
+  @Test
   public void testTableCreate() throws Exception {
     AccessTestAction createTable = new AccessTestAction() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
index 185893a..a15db00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
@@ -116,6 +117,14 @@ public abstract class TestVisibilityLabels {
   }
 
   @Test
+  public void testSecurityCapabilities() throws Exception {
+    List<SecurityCapability> capabilities = TEST_UTIL.getConnection().getAdmin()
+      .getSecurityCapabilities();
+    assertTrue("CELL_VISIBILITY capability is missing",
+      capabilities.contains(SecurityCapability.CELL_VISIBILITY));
+  }
+
+  @Test
   public void testSimpleVisibilityLabels() throws Exception {
     TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
     try (Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "|" + CONFIDENTIAL,

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 94bc684..5090109 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -995,5 +995,10 @@ module Hbase
       @admin.deleteNamespace(namespace_name)
     end
 
+    #----------------------------------------------------------------------------------------------
+    # Get security capabilities
+    def get_security_capabilities
+      @admin.getSecurityCapabilities
+    end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-shell/src/main/ruby/hbase/security.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/security.rb b/hbase-shell/src/main/ruby/hbase/security.rb
index 2aaef02..c7b94e8 100644
--- a/hbase-shell/src/main/ruby/hbase/security.rb
+++ b/hbase-shell/src/main/ruby/hbase/security.rb
@@ -182,10 +182,23 @@ module Hbase
       end
     end
 
-    # Make sure that security tables are available
+    # Make sure that security features 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)
+      caps = []
+      begin
+        # Try the getSecurityCapabilities API where supported.
+        # We only need to look at AUTHORIZATION, the AccessController doesn't support
+        # CELL_AUTHORIZATION without AUTHORIZATION also available.
+        caps = @admin.getSecurityCapabilities
+      rescue
+        # If we are unable to use getSecurityCapabilities, fall back with a check for
+        # deployment of the ACL table
+        raise(ArgumentError, "DISABLED: Security features are not available") unless \
+          exists?(org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
+        return
+      end
+      raise(ArgumentError, "DISABLED: Security features are not available") unless \
+        caps.include? org.apache.hadoop.hbase.client.security.SecurityCapability::AUTHORIZATION
     end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
index a3f8b1a..47df48a 100644
--- a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
+++ b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
@@ -35,7 +35,7 @@ module Hbase
     end
 
     def add_labels(*args)
-      lables_table_available?
+      visibility_feature_available?
       # Normalize args
       if args.kind_of?(Array)
         labels = [ args ].flatten.compact
@@ -63,7 +63,7 @@ module Hbase
     end
 
     def set_auths(user, *args)
-      lables_table_available?
+      visibility_feature_available?
       # Normalize args
       if args.kind_of?(Array)
         auths = [ args ].flatten.compact
@@ -88,7 +88,7 @@ module Hbase
     end
 
     def get_auths(user)
-      lables_table_available?
+      visibility_feature_available?
       begin
         response = VisibilityClient.getAuths(@config, user)
         if response.nil?
@@ -102,7 +102,7 @@ module Hbase
     end
 
     def list_labels(regex = ".*")
-      lables_table_available?
+      visibility_feature_available?
       begin
         response = VisibilityClient.listLabels(@config, regex)
         if response.nil?
@@ -116,7 +116,7 @@ module Hbase
     end
 
     def clear_auths(user, *args)
-      lables_table_available?
+      visibility_feature_available?
       # Normalize args
       if args.kind_of?(Array)
         auths = [ args ].flatten.compact
@@ -141,9 +141,20 @@ module Hbase
     end
 
     # Make sure that lables table is available
-    def lables_table_available?()
-      raise(ArgumentError, "DISABLED: Visibility labels feature is not available") \
-        unless exists?(VisibilityConstants::LABELS_TABLE_NAME)
+    def visibility_feature_available?()
+      caps = []
+      begin
+        # Try the getSecurityCapabilities API where supported.
+        caps = @admin.getSecurityCapabilities
+      rescue
+        # If we are unable to use getSecurityCapabilities, fall back with a check for
+        # deployment of the labels table
+        raise(ArgumentError, "DISABLED: Visibility labels feature is not available") unless \
+          exists?(VisibilityConstants::LABELS_TABLE_NAME)
+        return
+      end
+      raise(ArgumentError, "DISABLED: Visibility labels feature is not available") unless \
+        caps.include? org.apache.hadoop.hbase.client.security.SecurityCapability::CELL_VISIBILITY
     end
 
     # Does table exist?

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index c47aa4b..704eb1c 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -392,6 +392,7 @@ Shell.load_command_group(
   :full_name => 'SECURITY TOOLS',
   :comment => "NOTE: Above commands are only applicable if running with the AccessController coprocessor",
   :commands => %w[
+    list_security_capabilities
     grant
     revoke
     user_permission

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a5b5c79/hbase-shell/src/main/ruby/shell/commands/list_security_capabilities.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_security_capabilities.rb b/hbase-shell/src/main/ruby/shell/commands/list_security_capabilities.rb
new file mode 100644
index 0000000..922ad11
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_security_capabilities.rb
@@ -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 ListSecurityCapabilities < Command
+      def help
+        return <<-EOF
+List supported security capabilities
+
+Example:
+    hbase> list_security_capabilities
+EOF
+      end
+
+      def command()
+        begin
+          list = admin.get_security_capabilities
+          list.each do |s|
+            puts s.getName
+          end
+          return list.map { |s| s.getName() }
+        rescue Exception => e
+          if e.to_s.include? "UnsupportedOperationException"
+            puts "ERROR: Master does not support getSecurityCapabilities"
+            return []
+          end
+          raise e
+        end
+      end
+    end
+  end
+end