You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by sh...@apache.org on 2016/05/24 06:39:21 UTC

incubator-atlas git commit: ATLAS-809 JAAS configuration needed for Kafka interaction via Atlas config file (abhayk via shwethags)

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 74eafd3e6 -> 74d9b3fb0


ATLAS-809 JAAS configuration needed for Kafka interaction via Atlas config file (abhayk via shwethags)


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

Branch: refs/heads/master
Commit: 74d9b3fb0ea6ced1468c57c11ef3885c1107f86b
Parents: 74eafd3
Author: Shwetha GS <ss...@hortonworks.com>
Authored: Tue May 24 12:09:10 2016 +0530
Committer: Shwetha GS <ss...@hortonworks.com>
Committed: Tue May 24 12:09:10 2016 +0530

----------------------------------------------------------------------
 common/pom.xml                                  |  10 +
 .../org/apache/atlas/ApplicationProperties.java |   2 +
 .../security/InMemoryJAASConfiguration.java     | 353 +++++++++++++++++++
 .../security/InMemoryJAASConfigurationTest.java |  82 +++++
 common/src/test/resources/atlas-jaas.properties |  57 +++
 distro/src/conf/atlas-application.properties    |  10 +
 docs/src/site/twiki/security.twiki              |  78 +++-
 pom.xml                                         |   7 +
 release-log.txt                                 |   1 +
 .../main/resources/atlas-application.properties |  11 +
 10 files changed, 598 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index ba1210a..8239f26 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -52,6 +52,16 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>commons-collections</groupId>
+            <artifactId>commons-collections</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-all</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/common/src/main/java/org/apache/atlas/ApplicationProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/atlas/ApplicationProperties.java b/common/src/main/java/org/apache/atlas/ApplicationProperties.java
index 877d7db..48cb7a6 100644
--- a/common/src/main/java/org/apache/atlas/ApplicationProperties.java
+++ b/common/src/main/java/org/apache/atlas/ApplicationProperties.java
@@ -17,6 +17,7 @@
  */
 package org.apache.atlas;
 
+import org.apache.atlas.security.InMemoryJAASConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
@@ -56,6 +57,7 @@ public final class ApplicationProperties extends PropertiesConfiguration {
             synchronized (ApplicationProperties.class) {
                 if (instance == null) {
                     instance = get(APPLICATION_PROPERTIES);
+                    InMemoryJAASConfiguration.init(instance);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/common/src/main/java/org/apache/atlas/security/InMemoryJAASConfiguration.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/atlas/security/InMemoryJAASConfiguration.java b/common/src/main/java/org/apache/atlas/security/InMemoryJAASConfiguration.java
new file mode 100644
index 0000000..fb32ff5
--- /dev/null
+++ b/common/src/main/java/org/apache/atlas/security/InMemoryJAASConfiguration.java
@@ -0,0 +1,353 @@
+/**
+ * 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.atlas.security;
+
+import org.apache.atlas.AtlasException;
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.configuration.ConfigurationConverter;
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.SortedSet;
+import java.util.StringTokenizer;
+import java.util.TreeSet;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+
+/**
+ * InMemoryJAASConfiguration
+ *
+ * An utility class - which has a static method init to load all JAAS configuration from Application
+ * properties file (eg: atlas.properties) and set it as part of the default lookup configuration for
+ * all JAAS configuration lookup.
+ *
+ * Example settings in jaas-application.properties:
+ *
+ * atlas.jaas.KafkaClient.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+ * atlas.jaas.KafkaClient.loginModuleControlFlag = required
+ * atlas.jaas.KafkaClient.option.useKeyTab = true
+ * atlas.jaas.KafkaClient.option.storeKey = true
+ * atlas.jaas.KafkaClient.option.serviceName = kafka
+ * atlas.jaas.KafkaClient.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+ * atlas.jaas.KafkaClient.option.principal = kafka-client-1@EXAMPLE.COM
+
+ * atlas.jaas.MyClient.0.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+ * atlas.jaas.MyClient.0.loginModuleControlFlag = required
+ * atlas.jaas.MyClient.0.option.useKeyTab = true
+ * atlas.jaas.MyClient.0.option.storeKey = true
+ * atlas.jaas.MyClient.0.option.serviceName = kafka
+ * atlas.jaas.MyClient.0.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+ * atlas.jaas.MyClient.0.option.principal = kafka-client-1@EXAMPLE.COM
+ * atlas.jaas.MyClient.1.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+ * atlas.jaas.MyClient.1.loginModuleControlFlag = optional
+ * atlas.jaas.MyClient.1.option.useKeyTab = true
+ * atlas.jaas.MyClient.1.option.storeKey = true
+ * atlas.jaas.MyClient.1.option.serviceName = kafka
+ * atlas.jaas.MyClient.1.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+ * atlas.jaas.MyClient.1.option.principal = kafka-client-1@EXAMPLE.COM
+
+ * This will set the JAAS configuration - equivalent to the jaas.conf file entries:
+ *  KafkaClient {
+ *      com.sun.security.auth.module.Krb5LoginModule required
+ *          useKeyTab=true
+ *          storeKey=true
+ *          serviceName=kafka
+ *          keyTab="/etc/security/keytabs/kafka_client.keytab"
+ *          principal="kafka-client-1@EXAMPLE.COM";
+ *  };
+ *  MyClient {
+ *      com.sun.security.auth.module.Krb5LoginModule required
+ *          useKeyTab=true
+ *          storeKey=true
+ *          serviceName=kafka keyTab="/etc/security/keytabs/kafka_client.keytab"
+ *          principal="kafka-client-1@EXAMPLE.COM";
+ *  };
+ *  MyClient {
+ *      com.sun.security.auth.module.Krb5LoginModule optional
+ *          useKeyTab=true
+ *          storeKey=true
+ *          serviceName=kafka
+ *          keyTab="/etc/security/keytabs/kafka_client.keytab"
+ *          principal="kafka-client-1@EXAMPLE.COM";
+ *  };
+ *
+ *  Here is the syntax for atlas.properties to add JAAS configuration:
+ *
+ *  The property name has to begin with   'atlas.jaas.' +  clientId (in case of Kafka client,
+ *  it expects the clientId to be  KafkaClient).
+ *  The following property must be there to specify the JAAS loginModule name
+ *          'atlas.jaas.' +  clientId  + '.loginModuleName'
+ *  The following optional property should be set to specify the loginModuleControlFlag
+ *          'atlas.jaas.' + clientId + '.loginModuleControlFlag'
+ *          Default value :  required ,  Possible values:  required, optional, sufficient, requisite
+ *  Then you can add additional optional parameters as options for the configuration using the following
+ *  syntax:
+ *          'atlas.jaas.' + clientId + '.option.' + <optionName>  = <optionValue>
+ *
+ *  The current setup will lookup JAAS configration from the atlas-application.properties first, if not available,
+ *  it will delegate to the original configuration
+ *
+ */
+
+
+public final class InMemoryJAASConfiguration extends Configuration {
+
+    private static final Logger LOG = LoggerFactory.getLogger(InMemoryJAASConfiguration.class);
+
+    private static final String JAAS_CONFIG_PREFIX_PARAM = "atlas.jaas.";
+    private static final String JAAS_CONFIG_LOGIN_MODULE_NAME_PARAM = "loginModuleName";
+    private static final String JAAS_CONFIG_LOGIN_MODULE_CONTROL_FLAG_PARAM = "loginModuleControlFlag";
+    private static final String JAAS_CONFIG_LOGIN_OPTIONS_PREFIX = "option";
+    private static final String JAAS_PRINCIPAL_PROP = "principal";
+
+    private Configuration parent = null;
+    private Map<String, List<AppConfigurationEntry>> applicationConfigEntryMap = new HashMap<>();
+
+    public static void init(String propFile) throws AtlasException {
+        LOG.debug("==> InMemoryJAASConfiguration.init( {} )", propFile);
+
+        InputStream in = null;
+
+        try {
+            Properties properties = new Properties();
+            in = ClassLoader.getSystemResourceAsStream(propFile);
+            if (in == null) {
+                if (!propFile.startsWith("/")) {
+                    in = ClassLoader.getSystemResourceAsStream("/" + propFile);
+                }
+                if (in == null) {
+                    in = new FileInputStream(new File(propFile));
+                }
+            }
+            properties.load(in);
+            init(properties);
+        } catch (IOException e) {
+            throw new AtlasException("Failed to load JAAS application properties", e);
+        } finally {
+            if (in != null) {
+                try {
+                    in.close();
+                } catch (Exception exception) {
+                    // Ignore
+                }
+            }
+        }
+
+        LOG.debug("<== InMemoryJAASConfiguration.init( {} )", propFile);
+    }
+
+    public static void init(org.apache.commons.configuration.Configuration atlasConfiguration) throws AtlasException {
+        LOG.debug("==> InMemoryJAASConfiguration.init()");
+
+        if (atlasConfiguration != null && !atlasConfiguration.isEmpty()) {
+            Properties properties = ConfigurationConverter.getProperties(atlasConfiguration);
+            init(properties);
+        } else {
+            throw new AtlasException("Failed to load JAAS application properties: configuration NULL or empty!");
+        }
+
+        LOG.debug("<== InMemoryJAASConfiguration.init()");
+    }
+
+    public static void init(Properties properties) throws AtlasException {
+        LOG.debug("==> InMemoryJAASConfiguration.init()");
+
+        if (properties != null && MapUtils.isNotEmpty(properties)) {
+            InMemoryJAASConfiguration conf = new InMemoryJAASConfiguration(properties);
+            Configuration.setConfiguration(conf);
+        } else {
+            throw new AtlasException("Failed to load JAAS application properties: properties NULL or empty!");
+        }
+
+        LOG.debug("<== InMemoryJAASConfiguration.init()");
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+        LOG.trace("==> InMemoryJAASConfiguration.getAppConfigurationEntry( {} )", name);
+
+        AppConfigurationEntry[] ret = null;
+        List<AppConfigurationEntry> retList = applicationConfigEntryMap.get(name);
+        if (retList == null || retList.size() == 0) {
+            if (parent != null) {
+                ret = parent.getAppConfigurationEntry(name);
+            }
+        } else {
+            int sz = retList.size();
+            ret = new AppConfigurationEntry[sz];
+            ret = retList.toArray(ret);
+        }
+
+        LOG.trace("==> InMemoryJAASConfiguration.getAppConfigurationEntry( {} ) : {}", name, ArrayUtils.toString(ret));
+
+        return ret;
+    }
+
+    private InMemoryJAASConfiguration(Properties prop) {
+        parent = Configuration.getConfiguration();
+        initialize(prop);
+    }
+
+    private void initialize(Properties properties) {
+        LOG.debug("==> InMemoryJAASConfiguration.initialize()");
+
+        int prefixLen = JAAS_CONFIG_PREFIX_PARAM.length();
+
+        Map<String, SortedSet<Integer>> jaasClients = new HashMap<>();
+        for(String key : properties.stringPropertyNames()) {
+            if (key.startsWith(JAAS_CONFIG_PREFIX_PARAM)) {
+                String jaasKey  = key.substring(prefixLen);
+                StringTokenizer tokenizer = new StringTokenizer(jaasKey, ".");
+                int tokenCount =tokenizer.countTokens();
+                if (tokenCount > 0) {
+                    String clientId = tokenizer.nextToken();
+                    SortedSet<Integer> indexList = jaasClients.get(clientId);
+                    if (indexList == null) {
+                        indexList = new TreeSet<Integer>();
+                        jaasClients.put(clientId, indexList);
+                    }
+                    String indexStr = tokenizer.nextToken();
+
+                    int indexId =  isNumeric(indexStr) ? Integer.parseInt(indexStr)  : -1;
+
+                    Integer clientIdIndex = Integer.valueOf(indexId);
+
+                    if (!indexList.contains(clientIdIndex)) {
+                        indexList.add(clientIdIndex);
+                    }
+
+                }
+            }
+        }
+        for(String jaasClient : jaasClients.keySet()) {
+
+            for(Integer index :  jaasClients.get(jaasClient)) {
+
+                String keyPrefix = JAAS_CONFIG_PREFIX_PARAM + jaasClient + ".";
+
+                if (index > -1) {
+                    keyPrefix = keyPrefix  + String.valueOf(index) + ".";
+                }
+
+                String keyParam = keyPrefix + JAAS_CONFIG_LOGIN_MODULE_NAME_PARAM;
+                String loginModuleName = properties.getProperty(keyParam);
+
+                if (loginModuleName == null) {
+                    LOG.error("Unable to add JAAS configuration for "
+                            + "client [" + jaasClient + "] as it is missing param [" + keyParam + "]."
+                            + " Skipping JAAS config for [" + jaasClient + "]");
+                    continue;
+                } else {
+                    loginModuleName = loginModuleName.trim();
+                }
+
+                keyParam = keyPrefix + JAAS_CONFIG_LOGIN_MODULE_CONTROL_FLAG_PARAM;
+                String  controlFlag =    properties.getProperty(keyParam);
+
+                AppConfigurationEntry.LoginModuleControlFlag loginControlFlag = null;
+                if (controlFlag != null) {
+                    controlFlag = controlFlag.trim().toLowerCase();
+                    if (controlFlag.equals("optional")) {
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.OPTIONAL;
+                    } else if (controlFlag.equals("requisite")) {
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUISITE;
+                    } else if (controlFlag.equals("sufficient")) {
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.SUFFICIENT;
+                    } else if (controlFlag.equals("required")) {
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUIRED;
+                    } else {
+                        String validValues = "optional|requisite|sufficient|required";
+                        LOG.warn("Unknown JAAS configuration value for (" + keyParam
+                                + ") = [" + controlFlag + "], valid value are [" + validValues
+                                + "] using the default value, REQUIRED");
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUIRED;
+                    }
+                } else {
+                    LOG.warn("Unable to find JAAS configuration ("
+                            + keyParam + "); using the default value, REQUIRED");
+                    loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUIRED;
+                }
+
+
+                Map<String, String> options = new HashMap<>();
+                String optionPrefix =  keyPrefix + JAAS_CONFIG_LOGIN_OPTIONS_PREFIX + ".";
+                int optionPrefixLen = optionPrefix.length();
+                for(String key : properties.stringPropertyNames()) {
+                    if (key.startsWith(optionPrefix)) {
+                        String optionKey = key.substring(optionPrefixLen);
+                        String optionVal = properties.getProperty(key);
+                        if (optionVal != null) {
+                            optionVal = optionVal.trim();
+
+                            try {
+                                if (optionKey.equalsIgnoreCase(JAAS_PRINCIPAL_PROP)) {
+                                    optionVal = SecurityUtil.getServerPrincipal(optionVal, (String) null);
+                                }
+                            } catch (IOException e) {
+                                LOG.warn("Failed to build serverPrincipal. Using provided value:["
+                                        + optionVal + "]");
+                            }
+                        }
+                        options.put(optionKey, optionVal);
+                    }
+                }
+
+                AppConfigurationEntry entry = new AppConfigurationEntry(loginModuleName, loginControlFlag, options);
+
+                if (LOG.isDebugEnabled()) {
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("Adding client: [").append(jaasClient).append("{").append(index).append("}]\n");
+                    sb.append("\tloginModule: [").append(loginModuleName).append("]\n");
+                    sb.append("\tcontrolFlag: [").append(loginControlFlag).append("]\n");
+                    for (String key : options.keySet()) {
+                        String val = options.get(key);
+                        sb.append("\tOptions:  [").append(key).append("] => [").append(val).append("]\n");
+                    }
+                    LOG.debug(sb.toString());
+                }
+
+                List<AppConfigurationEntry> retList =  applicationConfigEntryMap.get(jaasClient);
+                if (retList == null) {
+                    retList = new ArrayList<AppConfigurationEntry>();
+                    applicationConfigEntryMap.put(jaasClient, retList);
+                }
+
+                retList.add(entry);
+            }
+        }
+
+        LOG.debug("<== InMemoryJAASConfiguration.initialize()");
+    }
+
+    private static boolean isNumeric(String str) {
+        return str.matches("-?\\d+(\\.\\d+)?");  //match a number with optional '-' and decimal.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/common/src/test/java/org/apache/atlas/security/InMemoryJAASConfigurationTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/atlas/security/InMemoryJAASConfigurationTest.java b/common/src/test/java/org/apache/atlas/security/InMemoryJAASConfigurationTest.java
new file mode 100644
index 0000000..5f1f914
--- /dev/null
+++ b/common/src/test/java/org/apache/atlas/security/InMemoryJAASConfigurationTest.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.atlas.security;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+import org.apache.hadoop.util.StringUtils;
+
+public class InMemoryJAASConfigurationTest extends TestCase {
+
+    private static final String ATLAS_JAAS_PROP_FILE = "atlas-jaas.properties";
+
+    protected void setUp() throws Exception {
+        super.setUp();
+        try {
+            InMemoryJAASConfiguration.init(ATLAS_JAAS_PROP_FILE);
+        } catch(Throwable t) {
+            fail("InMemoryJAASConfiguration.init() is not expected to throw Exception:" +  t);
+        }
+    }
+
+    protected void tearDown() throws Exception {
+        super.tearDown();
+    }
+
+    public void testGetAppConfigurationEntryStringForKafkaClient() {
+        AppConfigurationEntry[] entries =
+                Configuration.getConfiguration().getAppConfigurationEntry("KafkaClient");
+        Assert.assertNotNull(entries);
+        Assert.assertEquals(1, entries.length);
+        String principal = (String) entries[0].getOptions().get("principal");
+        Assert.assertNotNull(principal);
+        String[] components = principal.split("[/@]");
+        Assert.assertEquals(3, components.length);
+        Assert.assertEquals(false, StringUtils.equalsIgnoreCase(components[1], "_HOST"));
+
+    }
+
+    public void testGetAppConfigurationEntryStringForMyClient() {
+        AppConfigurationEntry[] entries =
+                Configuration.getConfiguration().getAppConfigurationEntry("myClient");
+        Assert.assertNotNull(entries);
+        Assert.assertEquals(2, entries.length);
+        String principal = (String) entries[0].getOptions().get("principal");
+        Assert.assertNotNull(principal);
+        String[] components = principal.split("[/@]");
+        Assert.assertEquals(3, components.length);
+        Assert.assertEquals(true, StringUtils.equalsIgnoreCase(components[1], "abcd"));
+
+        principal = (String) entries[1].getOptions().get("principal");
+        Assert.assertNotNull(principal);
+        components = principal.split("[/@]");
+        Assert.assertEquals(2, components.length);
+    }
+
+    public void testGetAppConfigurationEntryStringForUnknownClient() {
+        AppConfigurationEntry[] entries =
+                Configuration.getConfiguration().getAppConfigurationEntry("UnknownClient");
+        Assert.assertNull(entries);
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/common/src/test/resources/atlas-jaas.properties
----------------------------------------------------------------------
diff --git a/common/src/test/resources/atlas-jaas.properties b/common/src/test/resources/atlas-jaas.properties
new file mode 100644
index 0000000..90a5682
--- /dev/null
+++ b/common/src/test/resources/atlas-jaas.properties
@@ -0,0 +1,57 @@
+#
+# 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.
+#
+
+#########  Notification Configs  #########
+atlas.notification.embedded=true
+
+atlas.kafka.zookeeper.connect=localhost:19026
+atlas.kafka.bootstrap.servers=localhost:19027
+atlas.kafka.data=${sys:atlas.data}/kafka
+atlas.kafka.zookeeper.session.timeout.ms=4000
+atlas.kafka.zookeeper.sync.time.ms=20
+atlas.kafka.consumer.timeout.ms=100
+atlas.kafka.auto.commit.interval.ms=100
+atlas.kafka.hook.group.id=atlas
+atlas.kafka.entities.group.id=atlas_entities
+atlas.kafka.auto.commit.enable=false
+
+######## JAAS configs ##################
+
+atlas.jaas.KafkaClient.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+atlas.jaas.KafkaClient.loginModuleControlFlag = required
+atlas.jaas.KafkaClient.option.useKeyTab = true
+atlas.jaas.KafkaClient.option.storeKey = true
+atlas.jaas.KafkaClient.option.serviceName = kafka
+atlas.jaas.KafkaClient.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+atlas.jaas.KafkaClient.option.principal = kafka-client-1/_HOST@EXAMPLE.COM
+
+atlas.jaas.myClient.0.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+atlas.jaas.myClient.0.loginModuleControlFlag = required
+atlas.jaas.myClient.0.option.useKeyTab = true
+atlas.jaas.myClient.0.option.storeKey = true
+atlas.jaas.myClient.0.option.serviceName = kafka
+atlas.jaas.myClient.0.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+atlas.jaas.myClient.0.option.principal = kafka-client-1/abcd@EXAMPLE.COM
+
+atlas.jaas.myClient.1.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+atlas.jaas.myClient.1.loginModuleControlFlag = optional
+atlas.jaas.myClient.1.option.useKeyTab = true
+atlas.jaas.myClient.1.option.storeKey = true
+atlas.jaas.myClient.1.option.serviceName = kafka
+atlas.jaas.myClient.1.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+atlas.jaas.myClient.1.option.principal = kafka-client-1@EXAMPLE.COM
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/distro/src/conf/atlas-application.properties
----------------------------------------------------------------------
diff --git a/distro/src/conf/atlas-application.properties b/distro/src/conf/atlas-application.properties
index d1deae4..26f84e9 100755
--- a/distro/src/conf/atlas-application.properties
+++ b/distro/src/conf/atlas-application.properties
@@ -89,6 +89,16 @@ atlas.http.authentication.enabled=false
 # type:  simple or kerberos
 atlas.http.authentication.type=simple
 
+#########  JAAS Configuration ########
+
+#atlas.jaas.KafkaClient.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+#atlas.jaas.KafkaClient.loginModuleControlFlag = required
+#atlas.jaas.KafkaClient.option.useKeyTab = true
+#atlas.jaas.KafkaClient.option.storeKey = true
+#atlas.jaas.KafkaClient.option.serviceName = kafka
+#atlas.jaas.KafkaClient.option.keyTab = /etc/security/keytabs/atlas.service.keytab
+#atlas.jaas.KafkaClient.option.principal = atlas/_HOST@EXAMPLE.COM
+
 #########  Server Properties  #########
 atlas.rest.address=http://localhost:21000
 # If enabled and set to true, this will run setup steps when the server starts

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/docs/src/site/twiki/security.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/security.twiki b/docs/src/site/twiki/security.twiki
index 88e35d2..1dcee2a 100755
--- a/docs/src/site/twiki/security.twiki
+++ b/docs/src/site/twiki/security.twiki
@@ -47,24 +47,76 @@ Note that when Atlas is configured with HBase as the storage backend in a secure
 
 ---+++ JAAS configuration
 
-When Atlas is configured with HBase as the storage backend in a secure cluster, a JAAS configuration file should be created and specified so that the HBase client can attempt to SASL-authenticate.
+In a secure cluster, some of the components (such as Kafka) that Atlas interacts with, require Atlas to authenticate itself to them using JAAS. The following properties are used to set up appropriate JAAS Configuration.
 
-   * Create Atlas JAAS configuration file (e.g. /etc/atlas/conf/atlas-jaas.conf).
+   * <code>atlas.jaas.<code>client-id<code>.loginModuleName<code> - the authentication method used by the component (for example, com.sun.security.auth.module.Krb5LoginModule)
+   * <code>atlas.jaas.<code>client-id<code>.loginModuleControlFlag<code> (required|requisite|sufficient|optional) [default: required]
+   * <code>atlas.jaas.<code>client-id<code>.option.useKeyTab<code> (true|false)
+   * <code>atlas.jaas.<code>client-id<code>.option.storeKey<code> (true | false)
+   * <code>atlas.jaas.<code>client-id<code>.option.serviceName<code> - service name of server component
+   * <code>atlas.jaas.<code>client-id<code>.option.keyTab<code> = <atlas keytab>
+   * <code>atlas.jaas.<code>client-id<code>.option.principal<code> = <atlas principal>
+
+For example, the following property settings in jaas-application.properties file
 
 <verbatim>
-	Client {
-	   com.sun.security.auth.module.Krb5LoginModule required
-	   useKeyTab=true
-	   useTicketCache=false
-	   storeKey=true
-	   doNotPrompt=false
-	   keyTab="<atlas keytab>"
-	   principal="<atlas principal>";
-	};
+
+atlas.jaas.KafkaClient.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+atlas.jaas.KafkaClient.loginModuleControlFlag = required
+atlas.jaas.KafkaClient.option.useKeyTab = true
+atlas.jaas.KafkaClient.option.storeKey = true
+atlas.jaas.KafkaClient.option.serviceName = kafka
+atlas.jaas.KafkaClient.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+atlas.jaas.KafkaClient.option.principal = kafka-client-1@EXAMPLE.COM
+
+atlas.jaas.MyClient.0.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+atlas.jaas.MyClient.0.loginModuleControlFlag = required
+atlas.jaas.MyClient.0.option.useKeyTab = true
+atlas.jaas.MyClient.0.option.storeKey = true
+atlas.jaas.MyClient.0.option.serviceName = kafka
+atlas.jaas.MyClient.0.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+atlas.jaas.MyClient.0.option.principal = kafka-client-1@EXAMPLE.COM
+
+atlas.jaas.MyClient.1.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+atlas.jaas.MyClient.1.loginModuleControlFlag = optional
+atlas.jaas.MyClient.1.option.useKeyTab = true
+atlas.jaas.MyClient.1.option.storeKey = true
+atlas.jaas.MyClient.1.option.serviceName = kafka
+atlas.jaas.MyClient.1.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+atlas.jaas.MyClient.1.option.principal = kafka-client-1@EXAMPLE.COM
+
+</verbatim>
+
+will set the JAAS configuration that is equivalent to the following jaas.conf file entries.
+
+<verbatim>
+
+KafkaClient {
+	com.sun.security.auth.module.Krb5LoginModule required
+	useKeyTab=true
+	storeKey=true
+	serviceName=kafka
+	keyTab="/etc/security/keytabs/kafka_client.keytab"
+	principal="kafka-client-1@EXAMPLE.COM";
+};
+MyClient {
+	com.sun.security.auth.module.Krb5LoginModule required
+	useKeyTab=true
+	storeKey=true
+	serviceName=kafka keyTab="/etc/security/keytabs/kafka_client.keytab"
+	principal="kafka-client-1@EXAMPLE.COM";
+};
+MyClient {
+	com.sun.security.auth.module.Krb5LoginModule optional
+	useKeyTab=true
+	storeKey=true
+	serviceName=kafka
+	keyTab="/etc/security/keytabs/kafka_client.keytab"
+	principal="kafka-client-1@EXAMPLE.COM";
+};
+
 </verbatim>
 
-   * Update Atlas ATLAS_OPTS to include \u2018java.security.auth.login.config\u2019 set to the above Atlas JAAS configuration file.
-      * For example, <code>-Djava.security.auth.login.config=/etc/atlas/conf/atlas-jaas.conf</code>
 
 ---+++ SPNEGO-based HTTP Authentication
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 98184ae..6013cdc 100755
--- a/pom.xml
+++ b/pom.xml
@@ -390,6 +390,7 @@
         <!-- Needed for hooks -->
         <aopalliance.version>1.0</aopalliance.version>
         <commons-conf.version>1.10</commons-conf.version>
+        <commons-collections.version>3.2.2</commons-collections.version>
         <commons-logging.version>1.1.3</commons-logging.version>
         <javax-inject.version>1</javax-inject.version>
         <jettison.version>1.3.7</jettison.version>
@@ -705,6 +706,12 @@
                 <version>2.4</version>
             </dependency>
 
+            <dependency>
+                <groupId>commons-collections</groupId>
+                <artifactId>commons-collections</artifactId>
+                <version>${commons-collections.version}</version>
+            </dependency>
+
             <!-- utilities -->
             <dependency>
                 <groupId>com.google.inject</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index e443943..5838c69 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -21,6 +21,7 @@ ATLAS-409 Atlas will not import avro tables with schema read from a file (dosset
 ATLAS-379 Create sqoop and falcon metadata addons (venkatnrangan,bvellanki,sowmyaramesh via shwethags)
 
 ALL CHANGES:
+ATLAS-809 JAAS configuration needed for Kafka interaction via Atlas config file (abhayk via shwethags)
 ATLAS-817 Asset details page -- generate schema dynamically based on attributeDefinitions (kevalbhatt18 via yhemanth)
 ATLAS-495 Atlas Ranger Authorization Plugin (nixonrodrigues via shwethags)
 ATLAS-805 Quickstart is failing if run after queries to the business taxonomy API (jspeidel via shwethags)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/74d9b3fb/typesystem/src/main/resources/atlas-application.properties
----------------------------------------------------------------------
diff --git a/typesystem/src/main/resources/atlas-application.properties b/typesystem/src/main/resources/atlas-application.properties
index a8e77bb..3318ff3 100644
--- a/typesystem/src/main/resources/atlas-application.properties
+++ b/typesystem/src/main/resources/atlas-application.properties
@@ -83,6 +83,17 @@ atlas.server.https.port=31443
 hbase.security.authentication=simple
 
 atlas.hook.falcon.synchronous=true
+
+#########  JAAS Configuration ########
+
+atlas.jaas.KafkaClient.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+atlas.jaas.KafkaClient.loginModuleControlFlag = required
+atlas.jaas.KafkaClient.option.useKeyTab = true
+atlas.jaas.KafkaClient.option.storeKey = true
+atlas.jaas.KafkaClient.option.serviceName = kafka
+atlas.jaas.KafkaClient.option.keyTab = /etc/security/keytabs/atlas.service.keytab
+atlas.jaas.KafkaClient.option.principal = atlas/_HOST@EXAMPLE.COM
+
 #########  High Availability Configuration ########
 atlas.server.ha.enabled=false
 #atlas.server.ids=id1