You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2014/07/29 00:19:44 UTC

[3/4] git commit: STORM-347. (Security) authentication should allow for groups not just users. Added Unit test for ShellBaasedGroupsMapping.

STORM-347. (Security) authentication should allow for groups not just
users. Added Unit test for ShellBaasedGroupsMapping.


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

Branch: refs/heads/security
Commit: fb882ca18f124c7f6b65b7edeb2c0a652e254c3a
Parents: 9e13a35
Author: Sriharsha Chintalapani <ma...@harsha.io>
Authored: Mon Jul 28 12:51:13 2014 -0700
Committer: Sriharsha Chintalapani <ma...@harsha.io>
Committed: Mon Jul 28 12:51:13 2014 -0700

----------------------------------------------------------------------
 conf/defaults.yaml                              |  2 +-
 .../security/auth/ShellBasedGroupsMapping.java  | 94 ++++++++++++++++++++
 .../auth/ShellBasedUnixGroupsMapping.java       | 94 --------------------
 .../backtype/storm/security/auth/auth_test.clj  | 11 ++-
 4 files changed, 105 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fb882ca1/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index b0ebb4d..d4283a4 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -38,7 +38,7 @@ storm.cluster.mode: "distributed" # can be distributed or local
 storm.local.mode.zmq: false
 storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin"
 storm.principal.tolocal: "backtype.storm.security.auth.DefaultPrincipalToLocal"
-storm.group.mapping.service: "backtype.storm.security.auth.ShellBasedUnixGroupsMapping"
+storm.group.mapping.service: "backtype.storm.security.auth.ShellBasedGroupsMapping"
 storm.messaging.transport: "backtype.storm.messaging.netty.Context"
 storm.nimbus.retry.times: 5
 storm.nimbus.retry.interval.millis: 2000

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fb882ca1/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java
new file mode 100644
index 0000000..62a4c7e
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java
@@ -0,0 +1,94 @@
+/**
+ * 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.
+ */
+
+package backtype.storm.security.auth;
+
+import java.io.IOException;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.StringTokenizer;
+import backtype.storm.Config;
+import backtype.storm.utils.Utils;
+import backtype.storm.utils.ShellUtils;
+import backtype.storm.utils.TimeCacheMap;
+import backtype.storm.utils.ShellUtils.ExitCodeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ShellBasedGroupsMapping implements
+                                             IGroupMappingServiceProvider {
+
+    public static Logger LOG = LoggerFactory.getLogger(ShellBasedGroupsMapping.class);
+    public TimeCacheMap<String, Set<String>> cachedGroups;
+
+    /**
+     * Invoked once immediately after construction
+     * @param storm_conf Storm configuration
+     */
+    @Override
+    public void prepare(Map storm_conf) {
+        int timeout = Utils.getInt(storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS));
+        cachedGroups = new TimeCacheMap<String, Set<String>>(timeout);
+    }
+
+    /**
+     * Returns list of groups for a user
+     *
+     * @param user get groups for this user
+     * @return list of groups for a given user
+     */
+    @Override
+    public Set<String> getGroups(String user) throws IOException {
+        if(cachedGroups.containsKey(user)) {
+            return cachedGroups.get(user);
+        }
+        Set<String> groups = getUnixGroups(user);
+        if(!groups.isEmpty())
+            cachedGroups.put(user,groups);
+        return groups;
+    }
+
+    /**
+     * Get the current user's group list from Unix by running the command 'groups'
+     * NOTE. For non-existing user it will return EMPTY list
+     * @param user user name
+     * @return the groups set that the <code>user</code> belongs to
+     * @throws IOException if encounter any error when running the command
+     */
+    private static Set<String> getUnixGroups(final String user) throws IOException {
+        String result = "";
+        try {
+            result = ShellUtils.execCommand(ShellUtils.getGroupsForUserCommand(user));
+        } catch (ExitCodeException e) {
+            // if we didn't get the group - just return empty list;
+            LOG.warn("got exception trying to get groups for user " + user, e);
+            return new HashSet<String>();
+        }
+
+        StringTokenizer tokenizer =
+            new StringTokenizer(result, ShellUtils.TOKEN_SEPARATOR_REGEX);
+        Set<String> groups = new HashSet<String>();
+        while (tokenizer.hasMoreTokens()) {
+            groups.add(tokenizer.nextToken());
+        }
+        return groups;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fb882ca1/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java
deleted file mode 100644
index 438b938..0000000
--- a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package backtype.storm.security.auth;
-
-import java.io.IOException;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.StringTokenizer;
-import backtype.storm.Config;
-import backtype.storm.utils.Utils;
-import backtype.storm.utils.ShellUtils;
-import backtype.storm.utils.TimeCacheMap;
-import backtype.storm.utils.ShellUtils.ExitCodeException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class ShellBasedUnixGroupsMapping implements
-                                             IGroupMappingServiceProvider {
-
-    public static Logger LOG = LoggerFactory.getLogger(ShellBasedUnixGroupsMapping.class);
-    public TimeCacheMap<String, Set<String>> cachedGroups;
-
-    /**
-     * Invoked once immediately after construction
-     * @param storm_conf Storm configuration
-     */
-    @Override
-    public void prepare(Map storm_conf) {
-        int timeout = Utils.getInt(storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS));
-        cachedGroups = new TimeCacheMap<String, Set<String>>(timeout);
-    }
-
-    /**
-     * Returns list of groups for a user
-     *
-     * @param user get groups for this user
-     * @return list of groups for a given user
-     */
-    @Override
-    public Set<String> getGroups(String user) throws IOException {
-        if(cachedGroups.containsKey(user)) {
-            return cachedGroups.get(user);
-        }
-        Set<String> groups = getUnixGroups(user);
-        if(!groups.isEmpty())
-            cachedGroups.put(user,groups);
-        return groups;
-    }
-
-    /**
-     * Get the current user's group list from Unix by running the command 'groups'
-     * NOTE. For non-existing user it will return EMPTY list
-     * @param user user name
-     * @return the groups set that the <code>user</code> belongs to
-     * @throws IOException if encounter any error when running the command
-     */
-    private static Set<String> getUnixGroups(final String user) throws IOException {
-        String result = "";
-        try {
-            result = ShellUtils.execCommand(ShellUtils.getGroupsForUserCommand(user));
-        } catch (ExitCodeException e) {
-            // if we didn't get the group - just return empty list;
-            LOG.warn("got exception trying to get groups for user " + user, e);
-            return new HashSet<String>();
-        }
-
-        StringTokenizer tokenizer =
-            new StringTokenizer(result, ShellUtils.TOKEN_SEPARATOR_REGEX);
-        Set<String> groups = new HashSet<String>();
-        while (tokenizer.hasMoreTokens()) {
-            groups.add(tokenizer.nextToken());
-        }
-        return groups;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fb882ca1/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
index 6fdd485..12411e7 100644
--- a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
@@ -26,7 +26,7 @@
   (:import [backtype.storm.generated AuthorizationException])
   (:import [backtype.storm.utils NimbusClient])
   (:import [backtype.storm.security.auth.authorizer SimpleWhitelistAuthorizer SimpleACLAuthorizer])
-  (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient 
+  (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient ShellBasedGroupsMapping 
             ReqContext SimpleTransportPlugin KerberosPrincipalToLocal ThriftConnectionType])
   (:use [backtype.storm bootstrap util])
   (:use [backtype.storm.daemon common])
@@ -278,6 +278,15 @@
   (is (= false (.permit authorizer (ReqContext. supervisor-user) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]})))
 ))
 
+(deftest shell-based-groups-mapping-test
+  (let [cluster-conf (merge (read-storm-config))
+        groups (ShellBasedGroupsMapping. )
+        user-name (System/getProperty "user.name")]
+    (.prepare groups cluster-conf)
+    (>= 0 (.size (.getGroups groups user-name)))
+    (>= 0 (.size (.getGroups groups "userDoesNotExist")))
+    (>= 0 (.size (.getGroups groups nil)))))
+
 (deftest simple-acl-same-user-auth-test
   (let [cluster-conf (merge (read-storm-config)
                        {NIMBUS-ADMINS ["admin"]