You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by ka...@apache.org on 2016/06/30 13:37:03 UTC

[1/3] storm git commit: STORM-1233: Port AuthUtilsTest to java

Repository: storm
Updated Branches:
  refs/heads/master 878f5eb36 -> 743bafbaf


STORM-1233: Port AuthUtilsTest to java


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

Branch: refs/heads/master
Commit: 19a7f3640ca2607f970546c02dfad19401994b47
Parents: 6390d18
Author: Alessandro Bellina <ab...@yahoo-inc.com>
Authored: Mon Feb 29 07:49:07 2016 -0600
Committer: Alessandro Bellina <ab...@yahoo-inc.com>
Committed: Thu Mar 10 13:54:36 2016 -0600

----------------------------------------------------------------------
 .../apache/storm/security/auth/AuthUtils.java   | 139 ++++++-----
 .../storm/security/auth/AuthUtils_test.clj      |  75 ------
 .../storm/security/auth/AuthUtilsTest.java      | 240 +++++++++++++++++++
 .../storm/security/auth/AuthUtilsTestMock.java  |  82 +++++++
 4 files changed, 398 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/19a7f364/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java
index 72b7d7c..3c6e961 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java
@@ -22,7 +22,6 @@ import org.apache.storm.Config;
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.Subject;
-import javax.xml.bind.DatatypeConverter;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.ObjectInputStream;
@@ -30,7 +29,9 @@ import java.io.ObjectOutputStream;
 import java.security.URIParameter;
 import java.security.MessageDigest;
 
+import org.apache.commons.codec.binary.Hex;
 import org.apache.storm.security.INimbusCredentialPlugin;
+import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.io.File;
@@ -42,7 +43,6 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
-import java.lang.StringBuilder;
 
 public class AuthUtils {
     private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class);
@@ -65,7 +65,7 @@ public class AuthUtils {
         String loginConfigurationFile = (String)storm_conf.get("java.security.auth.login.config");
         if ((loginConfigurationFile != null) && (loginConfigurationFile.length()>0)) {
             File config_file = new File(loginConfigurationFile);
-            if (! config_file.canRead()) {
+            if (!config_file.canRead()) {
                 throw new RuntimeException("File " + loginConfigurationFile +
                         " cannot be read.");
             }
@@ -81,35 +81,74 @@ public class AuthUtils {
     }
 
     /**
-     * Pull a set of keys out of a Configuration.
-     * @param conf The config to pull the key/value pairs out of.
-     * @param conf_entry The app configuration entry name to get stuff from.
-     * @return Return a map of the configs in conf.
+     * Get configurations for a section
+     * @param configuration The config to pull the key/value pairs out of.
+     * @param section The app configuration entry name to get stuff from.
+     * @return Return array of config entries or null if configuration is null
      */
-    public static SortedMap<String, ?> PullConfig(Configuration conf,
-                                            String conf_entry) throws IOException {
-        if(conf == null) {
+    public static AppConfigurationEntry[] getEntries(Configuration configuration, 
+                                                String section) throws IOException {
+        if (configuration == null) {
             return null;
         }
-        AppConfigurationEntry configurationEntries[] = conf.getAppConfigurationEntry(conf_entry);
-        if(configurationEntries == null) {
-            String errorMessage = "Could not find a '" + conf_entry
-                + "' entry in this configuration: Client cannot start.";
+
+        AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(section);
+        if (configurationEntries == null) {
+            String errorMessage = "Could not find a '"+ section + "' entry in this configuration.";
             throw new IOException(errorMessage);
         }
+        return configurationEntries;
+    }
 
+    /**
+     * Pull a set of keys out of a Configuration.
+     * @param configuration The config to pull the key/value pairs out of.
+     * @param section The app configuration entry name to get stuff from.
+     * @return Return a map of the configs in conf.
+     */
+    public static SortedMap<String, ?> pullConfig(Configuration configuration,
+                                            String section) throws IOException {
+        AppConfigurationEntry[] configurationEntries = AuthUtils.getEntries(configuration, section);
+
+        if (configurationEntries == null) {
+            return null;
+        }
+        
         TreeMap<String, Object> results = new TreeMap<>();
 
-        for(AppConfigurationEntry entry: configurationEntries) {
+        for (AppConfigurationEntry entry: configurationEntries) {
             Map<String, ?> options = entry.getOptions();
-            for(String key : options.keySet()) {
+            for (String key : options.keySet()) {
                 results.put(key, options.get(key));
             }
         }
+
         return results;
     }
 
     /**
+     * Pull a the value given section and key from Configuration
+     * @param configuration The config to pull the key/value pairs out of.
+     * @param section The app configuration entry name to get stuff from.
+     * @param key The key to look up inside of the section
+     * @return Return a the String value of the configuration value
+     */
+    public static String get(Configuration configuration, String section, String key) throws IOException {
+        AppConfigurationEntry[] configurationEntries = AuthUtils.getEntries(configuration, section);
+
+        if (configurationEntries == null){
+            return null;
+        }
+
+        for (AppConfigurationEntry entry: configurationEntries) {
+            Object val = entry.getOptions().get(key);
+            if (val != null)
+                return (String)val;
+        }
+        return null;
+    }
+
+    /**
      * Construct a principal to local plugin
      * @param storm_conf storm configuration
      * @return the plugin
@@ -117,12 +156,11 @@ public class AuthUtils {
     public static IPrincipalToLocal GetPrincipalToLocalPlugin(Map storm_conf) {
         IPrincipalToLocal ptol;
         try {
-          String ptol_klassName = (String) storm_conf.get(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN);
-          Class klass = Class.forName(ptol_klassName);
-          ptol = (IPrincipalToLocal)klass.newInstance();
-          ptol.prepare(storm_conf);
+            String ptol_klassName = (String) storm_conf.get(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN);
+            ptol = Utils.newInstance(ptol_klassName);
+            ptol.prepare(storm_conf);
         } catch (Exception e) {
-          throw new RuntimeException(e);
+            throw new RuntimeException(e);
         }
         return ptol;
     }
@@ -136,11 +174,10 @@ public class AuthUtils {
         IGroupMappingServiceProvider gmsp;
         try {
             String gmsp_klassName = (String) storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN);
-            Class klass = Class.forName(gmsp_klassName);
-            gmsp = (IGroupMappingServiceProvider)klass.newInstance();
+            gmsp = Utils.newInstance(gmsp_klassName);
             gmsp.prepare(storm_conf);
         } catch (Exception e) {
-          throw new RuntimeException(e);
+            throw new RuntimeException(e);
         }
         return gmsp;
     }
@@ -156,7 +193,7 @@ public class AuthUtils {
             Collection<String> clazzes = (Collection<String>)conf.get(Config.NIMBUS_CREDENTIAL_RENEWERS);
             if (clazzes != null) {
                 for (String clazz : clazzes) {
-                    ICredentialsRenewer inst = (ICredentialsRenewer)Class.forName(clazz).newInstance();
+                    ICredentialsRenewer inst = Utils.newInstance(clazz);
                     inst.prepare(conf);
                     ret.add(inst);
                 }
@@ -178,7 +215,7 @@ public class AuthUtils {
             Collection<String> clazzes = (Collection<String>)conf.get(Config.NIMBUS_AUTO_CRED_PLUGINS);
             if (clazzes != null) {
                 for (String clazz : clazzes) {
-                    INimbusCredentialPlugin inst = (INimbusCredentialPlugin)Class.forName(clazz).newInstance();
+                    INimbusCredentialPlugin inst = Utils.newInstance(clazz);
                     inst.prepare(conf);
                     ret.add(inst);
                 }
@@ -200,7 +237,7 @@ public class AuthUtils {
             Collection<String> clazzes = (Collection<String>)storm_conf.get(Config.TOPOLOGY_AUTO_CREDENTIALS);
             if (clazzes != null) {
                 for (String clazz : clazzes) {
-                    IAutoCredentials a = (IAutoCredentials)Class.forName(clazz).newInstance();
+                    IAutoCredentials a = Utils.newInstance(clazz);
                     a.prepare(storm_conf);
                     autos.add(a);
                 }
@@ -240,8 +277,8 @@ public class AuthUtils {
      * @param credentials the credentials to pull from
      */
     public static void updateSubject(Subject subject, Collection<IAutoCredentials> autos, Map<String,String> credentials) {
-        if (subject == null) {
-            throw new RuntimeException("The subject cannot be null when updating a subject with credentials");
+        if (subject == null || autos == null) {
+            throw new RuntimeException("The subject or auto credentials cannot be null when updating a subject with credentials");
         }
 
         try {
@@ -257,29 +294,25 @@ public class AuthUtils {
      * Construct a transport plugin per storm configuration
      */
     public static ITransportPlugin GetTransportPlugin(ThriftConnectionType type, Map storm_conf, Configuration login_conf) {
-        ITransportPlugin  transportPlugin;
         try {
             String transport_plugin_klassName = type.getTransportPlugin(storm_conf);
-            Class klass = Class.forName(transport_plugin_klassName);
-            transportPlugin = (ITransportPlugin)klass.newInstance();
+            ITransportPlugin transportPlugin = Utils.newInstance(transport_plugin_klassName);
             transportPlugin.prepare(type, storm_conf, login_conf);
-        } catch(Exception e) {
+            return transportPlugin;
+        } catch (Exception e) {
             throw new RuntimeException(e);
         }
-        return transportPlugin;
     }
 
     private static IHttpCredentialsPlugin GetHttpCredentialsPlugin(Map conf,
             String klassName) {
-        IHttpCredentialsPlugin plugin;
         try {
-            Class klass = Class.forName(klassName);
-            plugin = (IHttpCredentialsPlugin)klass.newInstance();
+            IHttpCredentialsPlugin plugin = Utils.newInstance(klassName);
             plugin.prepare(conf);
-        } catch(Exception e) {
+            return plugin;
+        } catch (Exception e) {
             throw new RuntimeException(e);
         }
-        return plugin;
     }
 
     /**
@@ -304,21 +337,6 @@ public class AuthUtils {
         return AuthUtils.GetHttpCredentialsPlugin(conf, klassName);
     }
 
-    public static String get(Configuration configuration, String section, String key) throws IOException {
-        AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(section);
-        if (configurationEntries == null) {
-            String errorMessage = "Could not find a '"+ section + "' entry in this configuration.";
-            throw new IOException(errorMessage);
-        }
-
-        for(AppConfigurationEntry entry: configurationEntries) {
-            Object val = entry.getOptions().get(key);
-            if (val != null)
-                return (String)val;
-        }
-        return null;
-    }
-
     private static final String USERNAME = "username";
     private static final String PASSWORD = "password";
 
@@ -326,26 +344,21 @@ public class AuthUtils {
         String username = null;
         String password = null;
         try {
-            Map<String, ?> results = AuthUtils.PullConfig(login_config, config_section);
+            Map<String, ?> results = AuthUtils.pullConfig(login_config, config_section);
             username = (String)results.get(USERNAME);
             password = (String)results.get(PASSWORD);
         } catch (Exception e) {
             LOG.error("Failed to pull username/password out of jaas conf", e);
         }
 
-        if(username == null || password == null) {
+        if (username == null || password == null) {
             return null;
         }
 
         try {
             MessageDigest digest = MessageDigest.getInstance("SHA-512");
             byte[] output = digest.digest((username + ":" + password).getBytes());
-
-            StringBuilder builder = new StringBuilder();
-            for(byte b : output) {
-                builder.append(String.format("%02x", b));
-            }
-            return builder.toString();
+            return Hex.encodeHexString(output);
         } catch (java.security.NoSuchAlgorithmException e) {
             LOG.error("Cant run SHA-512 digest. Algorithm not available.", e);
             throw new RuntimeException(e);
@@ -376,7 +389,7 @@ public class AuthUtils {
     }
 
     public static KerberosTicket cloneKerberosTicket(KerberosTicket kerberosTicket) {
-        if(kerberosTicket != null) {
+        if (kerberosTicket != null) {
             try {
                 return (deserializeKerberosTicket(serializeKerberosTicket(kerberosTicket)));
             } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/storm/blob/19a7f364/storm-core/test/clj/org/apache/storm/security/auth/AuthUtils_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/security/auth/AuthUtils_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/AuthUtils_test.clj
deleted file mode 100644
index c14d038..0000000
--- a/storm-core/test/clj/org/apache/storm/security/auth/AuthUtils_test.clj
+++ /dev/null
@@ -1,75 +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.
-(ns org.apache.storm.security.auth.AuthUtils-test
-  (:import [org.apache.storm.security.auth AuthUtils IAutoCredentials])
-  (:import [java.io IOException])
-  (:import [javax.security.auth.login AppConfigurationEntry Configuration])
-  (:import [org.mockito Mockito])
-  (:use [clojure test]))
-
-(deftest test-throws-on-missing-section
-  (is (thrown? IOException
-    (AuthUtils/get (Mockito/mock Configuration) "bogus-section" "")))
-)
-
-(defn- mk-mock-app-config-entry []
-  (let [toRet (Mockito/mock AppConfigurationEntry)]
-    (. (Mockito/when (.getOptions toRet)) thenReturn (hash-map))
-    toRet
-  )
-)
-
-(deftest test-returns-null-if-no-such-section
-  (let [entry (mk-mock-app-config-entry)
-        entries (into-array (.getClass entry) [entry])
-        section "bogus-section"
-        conf (Mockito/mock Configuration)]
-    (. (Mockito/when (. conf getAppConfigurationEntry section ))
-       thenReturn entries)
-    (is (nil? (AuthUtils/get conf section "nonexistent-key")))
-  )
-)
-
-(deftest test-returns-first-value-for-valid-key
-  (let [k "the-key"
-        expected "good-value"
-        empty-entry (mk-mock-app-config-entry)
-        bad-entry (Mockito/mock AppConfigurationEntry)
-        good-entry (Mockito/mock AppConfigurationEntry)
-        conf (Mockito/mock Configuration)]
-    (. (Mockito/when (.getOptions bad-entry)) thenReturn {k "bad-value"})
-    (. (Mockito/when (.getOptions good-entry)) thenReturn {k expected})
-    (let [entries (into-array (.getClass empty-entry)
-                    [empty-entry good-entry bad-entry])
-          section "bogus-section"]
-      (. (Mockito/when (. conf getAppConfigurationEntry section))
-         thenReturn entries)
-      (is (not (nil? (AuthUtils/get conf section k))))
-      (is (= (AuthUtils/get conf section k) expected))
-    )
-  ))
-
-(deftest test-empty-auto-creds
-  (let [result (AuthUtils/GetAutoCredentials {})]
-    (is (.isEmpty result))
-  )
-)
-
-(deftest test-empty-creds-renewers
-  (let [result (AuthUtils/GetCredentialRenewers {})]
-    (is (.isEmpty result))
-  )
-)

http://git-wip-us.apache.org/repos/asf/storm/blob/19a7f364/storm-core/test/jvm/org/apache/storm/security/auth/AuthUtilsTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/security/auth/AuthUtilsTest.java b/storm-core/test/jvm/org/apache/storm/security/auth/AuthUtilsTest.java
new file mode 100644
index 0000000..40a0062
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/security/auth/AuthUtilsTest.java
@@ -0,0 +1,240 @@
+/**
+ * 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 org.apache.storm.security.auth;
+
+import java.io.IOException;
+import java.io.File;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.Subject;
+
+import org.apache.commons.codec.binary.Hex;
+import org.apache.storm.Config;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class AuthUtilsTest {
+
+    @Test(expected = IOException.class)
+    public void getOptionsThrowsOnMissingSectionTest() throws IOException {
+        Configuration mockConfig = Mockito.mock(Configuration.class);
+        AuthUtils.get(mockConfig, "bogus-section", "");
+    }
+
+    @Test
+    public void getNonExistentSectionTest() throws IOException {
+        Map<String, String> optionMap = new HashMap<String, String>();
+        AppConfigurationEntry entry = Mockito.mock(AppConfigurationEntry.class);
+        
+        Mockito.<Map<String, ?>>when(entry.getOptions()).thenReturn(optionMap);
+        String section = "bogus-section";
+        Configuration mockConfig = Mockito.mock(Configuration.class);
+        Mockito.when(mockConfig.getAppConfigurationEntry(section))
+               .thenReturn(new AppConfigurationEntry[] {entry});
+        Assert.assertNull(
+                AuthUtils.get(mockConfig, section, "nonexistent-key"));
+    }
+
+    @Test
+    public void getFirstValueForValidKeyTest() throws IOException {
+        String k = "the-key";
+        String expected = "good-value";
+
+        Map<String, String> optionMap = new HashMap<String, String>();
+        optionMap.put(k, expected);
+
+        Map<String, String> badOptionMap = new HashMap<String, String>();
+        badOptionMap.put(k, "bad-value");
+
+        AppConfigurationEntry emptyEntry = Mockito.mock(AppConfigurationEntry.class);
+        AppConfigurationEntry badEntry = Mockito.mock(AppConfigurationEntry.class);
+        AppConfigurationEntry goodEntry = Mockito.mock(AppConfigurationEntry.class);
+        
+        Mockito.<Map<String, ?>>when(emptyEntry.getOptions()).thenReturn(new HashMap<String, String>());
+        Mockito.<Map<String, ?>>when(badEntry.getOptions()).thenReturn(badOptionMap);
+        Mockito.<Map<String, ?>>when(goodEntry.getOptions()).thenReturn(optionMap);
+
+        String section = "bogus-section";
+        Configuration mockConfig = Mockito.mock(Configuration.class);
+        Mockito.when(mockConfig.getAppConfigurationEntry(section))
+               .thenReturn(new AppConfigurationEntry[] {emptyEntry, goodEntry, badEntry});
+
+        Assert.assertEquals(
+                AuthUtils.get(mockConfig, section, k), expected);
+    }
+
+    @Test
+    public void objGettersReturnNullWithNullConfigTest() throws IOException {
+        Assert.assertNull(AuthUtils.pullConfig(null, "foo"));
+        Assert.assertNull(AuthUtils.get(null, "foo", "bar"));
+
+        Map emptyMap = new HashMap<String, String>();
+        Assert.assertNull(AuthUtils.GetConfiguration(emptyMap));
+    }
+
+    @Test
+    public void getAutoCredentialsTest() {
+        Map emptyMap = new HashMap<String, String>();
+        Map<String, Collection<String>> map = new HashMap<String, Collection<String>>();
+        map.put(Config.TOPOLOGY_AUTO_CREDENTIALS, 
+                Arrays.asList(new String[]{"org.apache.storm.security.auth.AuthUtilsTestMock"}));
+
+        Assert.assertTrue(AuthUtils.GetAutoCredentials(emptyMap).isEmpty());
+        Assert.assertEquals(AuthUtils.GetAutoCredentials(map).size(), 1);
+    }
+
+    @Test
+    public void getNimbusAutoCredPluginTest() {
+        Map emptyMap = new HashMap<String, String>();
+        Map<String, Collection<String>> map = new HashMap<String, Collection<String>>();
+        map.put(Config.NIMBUS_AUTO_CRED_PLUGINS, 
+                Arrays.asList(new String[]{"org.apache.storm.security.auth.AuthUtilsTestMock"}));
+
+        Assert.assertTrue(AuthUtils.getNimbusAutoCredPlugins(emptyMap).isEmpty());
+        Assert.assertEquals(AuthUtils.getNimbusAutoCredPlugins(map).size(), 1);
+    }
+
+    @Test
+    public void GetCredentialRenewersTest() {
+        Map emptyMap = new HashMap<String, String>();
+        Map<String, Collection<String>> map = new HashMap<String, Collection<String>>();
+        map.put(Config.NIMBUS_CREDENTIAL_RENEWERS, 
+                Arrays.asList(new String[]{"org.apache.storm.security.auth.AuthUtilsTestMock"}));
+
+        Assert.assertTrue(AuthUtils.GetCredentialRenewers(emptyMap).isEmpty());
+        Assert.assertEquals(AuthUtils.GetCredentialRenewers(map).size(), 1);
+    }
+
+    @Test
+    public void populateSubjectTest() {
+        AuthUtilsTestMock autoCred = Mockito.mock(AuthUtilsTestMock.class);
+        Subject subject = new Subject();
+        Map<String, String> cred = new HashMap<String, String>();
+        Collection<IAutoCredentials> autos = Arrays.asList(new IAutoCredentials[]{autoCred}); 
+        AuthUtils.populateSubject(subject, autos, cred);
+        Mockito.verify(autoCred, Mockito.times(1)).populateSubject(subject, cred); 
+    }
+
+    @Test
+    public void makeDigestPayloadTest() throws NoSuchAlgorithmException { 
+        String section = "user-pass-section";
+        Map<String, String> optionMap = new HashMap<String, String>();
+        String user = "user";
+        String pass = "pass";
+        optionMap.put("username", user);
+        optionMap.put("password", pass);
+        AppConfigurationEntry entry = Mockito.mock(AppConfigurationEntry.class);
+        
+        Mockito.<Map<String, ?>>when(entry.getOptions()).thenReturn(optionMap);
+        Configuration mockConfig = Mockito.mock(Configuration.class);
+        Mockito.when(mockConfig.getAppConfigurationEntry(section))
+               .thenReturn(new AppConfigurationEntry[] {entry});
+
+        MessageDigest digest = MessageDigest.getInstance("SHA-512");
+        byte[] output = digest.digest((user + ":" + pass).getBytes());
+        String sha = Hex.encodeHexString(output);
+
+        // previous code used this method to generate the string, ensure the two match
+        StringBuilder builder = new StringBuilder();        
+        for(byte b : output) {     
+            builder.append(String.format("%02x", b));      
+        }      
+        String stringFormatMethod = builder.toString();
+
+        Assert.assertEquals(
+            AuthUtils.makeDigestPayload(mockConfig, "user-pass-section"),
+            sha);
+
+        Assert.assertEquals(sha, stringFormatMethod); 
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void invalidConfigResultsInIOException() throws RuntimeException {
+        HashMap<String, String> conf = new HashMap<String, String>();
+        conf.put("java.security.auth.login.config", "__FAKE_FILE__");
+        Assert.assertNotNull(AuthUtils.GetConfiguration(conf));
+    }
+
+    // JUnit ensures that the temporary folder is removed after
+    // the test finishes
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    @Test
+    public void validConfigResultsInNotNullConfigurationTest() throws IOException {
+        File file1 = folder.newFile("mockfile.txt");
+        HashMap<String, String> conf = new HashMap<String, String>();
+        conf.put("java.security.auth.login.config", file1.getAbsolutePath());
+        Assert.assertNotNull(AuthUtils.GetConfiguration(conf));
+    }
+
+    @Test
+    public void uiHttpCredentialsPluginTest(){
+        Map conf = new HashMap<String, String>();
+        conf.put(
+            Config.UI_HTTP_CREDS_PLUGIN, 
+            "org.apache.storm.security.auth.AuthUtilsTestMock");
+        conf.put(
+            Config.DRPC_HTTP_CREDS_PLUGIN, 
+            "org.apache.storm.security.auth.AuthUtilsTestMock");
+        conf.put(
+            Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN, 
+            "org.apache.storm.security.auth.AuthUtilsTestMock");
+        conf.put(
+            Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN, 
+            "org.apache.storm.security.auth.AuthUtilsTestMock");
+
+        Assert.assertTrue(
+            AuthUtils.GetUiHttpCredentialsPlugin(conf).getClass() == AuthUtilsTestMock.class);
+        Assert.assertTrue(
+            AuthUtils.GetDrpcHttpCredentialsPlugin(conf).getClass() == AuthUtilsTestMock.class);
+        Assert.assertTrue(
+            AuthUtils.GetPrincipalToLocalPlugin(conf).getClass() == AuthUtilsTestMock.class);
+        Assert.assertTrue(
+            AuthUtils.GetGroupMappingServiceProviderPlugin(conf).getClass() == AuthUtilsTestMock.class);
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void updateSubjectWithNullThrowsTest() {
+        AuthUtils.updateSubject(null, null, null);
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void updateSubjectWithNullAutosThrowsTest() {
+        AuthUtils.updateSubject(new Subject(), null, null);
+    }
+
+    @Test
+    public void updateSubjectWithNullAutosTest() {
+        AuthUtilsTestMock mock = Mockito.mock(AuthUtilsTestMock.class);
+        Collection<IAutoCredentials> autos = Arrays.asList(new IAutoCredentials[]{mock}); 
+        Subject s = new Subject();
+        AuthUtils.updateSubject(s, autos, null);
+        Mockito.verify(mock, Mockito.times(1)).updateSubject(s, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/19a7f364/storm-core/test/jvm/org/apache/storm/security/auth/AuthUtilsTestMock.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/security/auth/AuthUtilsTestMock.java b/storm-core/test/jvm/org/apache/storm/security/auth/AuthUtilsTestMock.java
new file mode 100644
index 0000000..9bf041d
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/security/auth/AuthUtilsTestMock.java
@@ -0,0 +1,82 @@
+/**
+ * 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 org.apache.storm.security.auth;
+
+import java.io.IOException;
+import java.security.Principal;
+import java.util.Map;
+import java.util.Set;
+
+import javax.security.auth.Subject;
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.storm.security.INimbusCredentialPlugin;
+
+public class AuthUtilsTestMock implements IAutoCredentials, 
+                                          ICredentialsRenewer,
+                                          IHttpCredentialsPlugin, 
+                                          INimbusCredentialPlugin,
+                                          IPrincipalToLocal,
+                                          IGroupMappingServiceProvider {
+
+    // IAutoCredentials
+    // ICredentialsRenewer
+    // IHttpCredentialsPlugin
+    // INimbusCredentialPlugin 
+    // IPrincipalToLocal 
+    // IGroupMappingServiceProvider 
+    public void prepare(Map conf) {}
+
+    // IHttpCredentialsPlugin
+    public ReqContext populateContext(ReqContext ctx, HttpServletRequest req) {
+        return null;
+    }
+
+    // IHttpCredentialsPlugin
+    public String getUserName(HttpServletRequest req){
+        return null;
+    }
+
+    // IPrincipalToLocal 
+    public String toLocal(Principal principal) {
+        return null;
+    }
+
+    // IGroupMappingServiceProvider 
+    public Set<String> getGroups(String user) throws IOException {
+        return null;
+    }
+
+    // ICredentialsRenewer
+    public void renew(Map<String, String> credentials, Map topologyConf) {}
+
+    // IAutoCredentials
+    public void updateSubject(Subject subject, Map<String,String> conf) {}
+
+    // IAutoCredentials
+    public void populateSubject(Subject subject, Map<String,String> conf) {}
+
+    // IAutoCredentials
+    public void populateCredentials(Map<String,String> conf) {}
+
+    // INimbusCredentialPlugin
+    public void populateCredentials(Map<String,String> credentials, Map conf) {}
+
+    // Shutdownable via INimbusCredentailPlugin
+    public void shutdown() {}
+}


[2/3] storm git commit: Merge branch 'STORM-1233_port_AuthUtils_test_to_java' of https://github.com/abellina/storm into STORM-1233

Posted by ka...@apache.org.
Merge branch 'STORM-1233_port_AuthUtils_test_to_java' of https://github.com/abellina/storm into STORM-1233


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

Branch: refs/heads/master
Commit: 356c1c8f52b339ed1ac51b5d91fba1e7ecb7f2da
Parents: 878f5eb 19a7f36
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Jun 30 22:25:54 2016 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Jun 30 22:25:54 2016 +0900

----------------------------------------------------------------------
 .../apache/storm/security/auth/AuthUtils.java   | 139 ++++++-----
 .../storm/security/auth/AuthUtils_test.clj      |  75 ------
 .../storm/security/auth/AuthUtilsTest.java      | 240 +++++++++++++++++++
 .../storm/security/auth/AuthUtilsTestMock.java  |  82 +++++++
 4 files changed, 398 insertions(+), 138 deletions(-)
----------------------------------------------------------------------



[3/3] storm git commit: add STORM-1233 to CHANGELOG

Posted by ka...@apache.org.
add STORM-1233 to CHANGELOG


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

Branch: refs/heads/master
Commit: 743bafbafd9ed85ca242d89986f65619afe3c80e
Parents: 356c1c8
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Jun 30 22:36:28 2016 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Jun 30 22:36:28 2016 +0900

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/743bafba/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index ac8cd28..b272123 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 2.0.0
+ * STORM-1233: Port AuthUtilsTest to java
  * STORM-1920: version of parent pom for storm-kafka-monitor is set 1.0.2-SNAPSHOT in master branch
  * STORM-1896: Remove duplicate code from HDFS spout
  * STORM-1909: Update HDFS spout documentation