You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by ve...@apache.org on 2015/05/13 23:28:27 UTC

[46/50] [abbrv] incubator-atlas git commit: securing the metadata client

securing the metadata client


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

Branch: refs/remotes/origin/master
Commit: 438688121370628ea0879bae748048867e44c05c
Parents: bb3022b
Author: Jon Maron <jm...@hortonworks.com>
Authored: Fri May 8 18:07:13 2015 -0400
Committer: Jon Maron <jm...@hortonworks.com>
Committed: Fri May 8 18:07:13 2015 -0400

----------------------------------------------------------------------
 addons/hive-bridge/pom.xml                      |  34 ++-
 .../hive/hook/BaseSSLAndKerberosTest.java       | 141 ++++++++++
 .../hook/NegativeSSLAndKerberosHiveHookIT.java  | 159 ++++++++++++
 .../hive/hook/SSLAndKerberosHiveHookIT.java     | 257 +++++++++++++++++++
 .../metadata/hive/hook/SSLHiveHookIT.java       | 253 ++++++++++++++++++
 client/pom.xml                                  |  65 +++++
 .../hadoop/metadata/MetadataServiceClient.java  |  31 ++-
 .../apache/hadoop/metadata/PropertiesUtil.java  |  60 +++++
 .../metadata/security/SecureClientUtils.java    | 195 ++++++++++++++
 .../metadata/security/SecurityProperties.java   |  35 +++
 .../metadata/security/BaseSecurityTest.java     | 128 +++++++++
 pom.xml                                         |   4 +-
 repository/pom.xml                              |   5 +
 .../apache/hadoop/metadata/PropertiesUtil.java  |  50 ----
 webapp/pom.xml                                  |   9 +
 .../util/CredentialProviderUtility.java         |   6 +-
 .../filters/MetadataAuthenticationFilter.java   |   7 +-
 .../web/listeners/GuiceServletConfig.java       |  12 +-
 .../web/service/SecureEmbeddedServer.java       |  13 +-
 .../metadata/CredentialProviderUtilityIT.java   |  26 +-
 .../hadoop/metadata/web/BaseSecurityTest.java   | 128 ---------
 .../MetadataAuthenticationKerberosFilterIT.java |  11 +-
 .../MetadataAuthenticationSimpleFilterIT.java   |   9 +-
 .../web/listeners/LoginProcessorIT.java         |  13 +-
 .../web/service/SecureEmbeddedServerIT.java     |   4 +-
 .../web/service/SecureEmbeddedServerITBase.java |  14 +-
 26 files changed, 1431 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/addons/hive-bridge/pom.xml
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/pom.xml b/addons/hive-bridge/pom.xml
index 03d2dcf..fa3df78 100755
--- a/addons/hive-bridge/pom.xml
+++ b/addons/hive-bridge/pom.xml
@@ -35,19 +35,22 @@
     <properties>
         <hive.version>1.1.0</hive.version>
         <calcite.version>0.9.2-incubating</calcite.version>
-        <hadoop.version>2.5.0</hadoop.version>
+        <hadoop.version>2.6.0</hadoop.version>
     </properties>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.hadoop.metadata</groupId>
             <artifactId>metadata-client</artifactId>
+            <version>${version}</version>
             <exclusions>
                 <exclusion>
                     <groupId>com.google.guava</groupId>
                     <artifactId>guava</artifactId>
                 </exclusion>
             </exclusions>
+            <scope>runtime</scope>
+            <type>test-jar</type>
         </dependency>
 
         <dependency>
@@ -55,6 +58,13 @@
             <artifactId>metadata-typesystem</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minikdc</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>test</scope>
+        </dependency>
+
         <!-- Logging -->
         <dependency>
             <groupId>org.slf4j</groupId>
@@ -92,7 +102,12 @@
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
             <version>${hadoop.version}</version>
-            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-annotations</artifactId>
+            <version>${hadoop.version}</version>
         </dependency>
 
         <dependency>
@@ -104,9 +119,14 @@
             <groupId>org.apache.hadoop.metadata</groupId>
             <artifactId>metadata-webapp</artifactId>
             <version>${project.version}</version>
-            <type>war</type>
-            <scope>test</scope>
+            <classifier>classes</classifier>
         </dependency>
+
+        <dependency>
+            <groupId>org.mortbay.jetty</groupId>
+            <artifactId>jetty</artifactId>
+        </dependency>
+
     </dependencies>
 
     <build>
@@ -254,6 +274,12 @@
                     <skip>false</skip>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <inherited>true</inherited>
+                <extensions>true</extensions>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/BaseSSLAndKerberosTest.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/BaseSSLAndKerberosTest.java b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/BaseSSLAndKerberosTest.java
new file mode 100644
index 0000000..75c23bc
--- /dev/null
+++ b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/BaseSSLAndKerberosTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.hadoop.metadata.hive.hook;
+
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.metadata.hive.bridge.HiveMetaStoreBridge;
+import org.apache.hadoop.metadata.security.BaseSecurityTest;
+import org.apache.hadoop.metadata.web.service.SecureEmbeddedServer;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.mortbay.jetty.Server;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.apache.hadoop.metadata.security.SecurityProperties.KEYSTORE_PASSWORD_KEY;
+import static org.apache.hadoop.metadata.security.SecurityProperties.SERVER_CERT_PASSWORD_KEY;
+import static org.apache.hadoop.metadata.security.SecurityProperties.TRUSTSTORE_PASSWORD_KEY;
+
+/**
+ *
+ */
+public class BaseSSLAndKerberosTest extends BaseSecurityTest {
+    public static final String TESTUSER = "testuser";
+    public static final String TESTPASS = "testpass";
+    protected static final String DGI_URL = "https://localhost:21443/";
+    protected Path jksPath;
+    protected String providerUrl;
+    protected File httpKeytabFile;
+    private File userKeytabFile;
+
+    class TestSecureEmbeddedServer extends SecureEmbeddedServer {
+
+        public TestSecureEmbeddedServer(int port, String path) throws IOException {
+            super(port, path);
+        }
+
+        public Server getServer() {
+            return server;
+        }
+
+        @Override
+        public PropertiesConfiguration getConfiguration() {
+            return super.getConfiguration();
+        }
+    }
+
+    protected void setupCredentials() throws Exception {
+        Configuration conf = new Configuration(false);
+
+        File file = new File(jksPath.toUri().getPath());
+        file.delete();
+        conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, providerUrl);
+
+        CredentialProvider provider =
+                CredentialProviderFactory.getProviders(conf).get(0);
+
+        // create new aliases
+        try {
+
+            char[] storepass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+            provider.createCredentialEntry(
+                    KEYSTORE_PASSWORD_KEY, storepass);
+
+            char[] trustpass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+            provider.createCredentialEntry(
+                    TRUSTSTORE_PASSWORD_KEY, trustpass);
+
+            char[] trustpass2 = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+            provider.createCredentialEntry(
+                    "ssl.client.truststore.password", trustpass2);
+
+            char[] certpass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+            provider.createCredentialEntry(
+                    SERVER_CERT_PASSWORD_KEY, certpass);
+
+            // write out so that it can be found in checks
+            provider.flush();
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    public void setupKDCAndPrincipals() throws Exception {
+        // set up the KDC
+        File kdcWorkDir = startKDC();
+
+        userKeytabFile = createKeytab(kdc, kdcWorkDir, "dgi", "dgi.keytab");
+        httpKeytabFile = createKeytab(kdc, kdcWorkDir, "HTTP", "spnego.service.keytab");
+
+        // create a test user principal
+        kdc.createPrincipal(TESTUSER, TESTPASS);
+
+        StringBuilder jaas = new StringBuilder(1024);
+        jaas.append("TestUser {\n" +
+                "    com.sun.security.auth.module.Krb5LoginModule required\nuseTicketCache=true;\n" +
+                "};\n");
+        jaas.append(createJAASEntry("Client", "dgi", userKeytabFile));
+        jaas.append(createJAASEntry("Server", "HTTP", httpKeytabFile));
+
+        File jaasFile = new File(kdcWorkDir, "jaas.txt");
+        FileUtils.write(jaasFile, jaas.toString());
+        bindJVMtoJAASFile(jaasFile);
+    }
+
+    protected String getWarPath() {
+        return String.format("/../../webapp/target/metadata-webapp-%s",
+                System.getProperty("project.version", "0.1-incubating-SNAPSHOT"));
+    }
+
+    protected HiveConf getHiveConf() {
+        HiveConf hiveConf = new HiveConf(this.getClass());
+        hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, "");
+        hiveConf.setVar(HiveConf.ConfVars.POSTEXECHOOKS, HiveHook.class.getName());
+        hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+        hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, System.getProperty("user.dir") + "/target/metastore");
+        hiveConf.set(HiveMetaStoreBridge.DGI_URL_PROPERTY, DGI_URL);
+        hiveConf.set("javax.jdo.option.ConnectionURL", "jdbc:derby:./target/metastore_db;create=true");
+        hiveConf.set("hive.hook.dgi.synchronous", "true");
+        return hiveConf;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/NegativeSSLAndKerberosHiveHookIT.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/NegativeSSLAndKerberosHiveHookIT.java b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/NegativeSSLAndKerberosHiveHookIT.java
new file mode 100755
index 0000000..af073f5
--- /dev/null
+++ b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/NegativeSSLAndKerberosHiveHookIT.java
@@ -0,0 +1,159 @@
+/**
+ * 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.hadoop.metadata.hive.hook;
+
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.metadata.PropertiesUtil;
+import org.apache.hadoop.metadata.security.SecurityProperties;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.ssl.SSLHostnameVerifier;
+import org.mortbay.jetty.webapp.WebAppContext;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.net.URL;
+import java.nio.file.Files;
+
+import static org.apache.hadoop.metadata.security.SecurityProperties.*;
+
+/**
+ * Perform all the necessary setup steps for client and server comm over SSL/Kerberos, but then don't estalish a
+ * kerberos user for the invocation.  Need a separate use case since the Jersey layer cached the URL connection handler,
+ * which indirectly caches the kerberos delegation token.
+ */
+public class NegativeSSLAndKerberosHiveHookIT extends BaseSSLAndKerberosTest {
+
+    private Driver driver;
+    private SessionState ss;
+    private TestSecureEmbeddedServer secureEmbeddedServer;
+    private String originalConf;
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        //Set-up hive session
+        HiveConf conf = getHiveConf();
+        driver = new Driver(conf);
+        ss = new SessionState(conf, System.getProperty("user.name"));
+        ss = SessionState.start(ss);
+        SessionState.setCurrentSessionState(ss);
+
+        jksPath = new Path(Files.createTempDirectory("tempproviders").toString(), "test.jks");
+        providerUrl = JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+        String persistDir = null;
+        URL resource = NegativeSSLAndKerberosHiveHookIT.class.getResource("/");
+        if (resource != null) {
+            persistDir = resource.toURI().getPath();
+        }
+        // delete prior ssl-client.xml file
+        resource = NegativeSSLAndKerberosHiveHookIT.class.getResource("/" + SecurityProperties.SSL_CLIENT_PROPERTIES);
+        if (resource != null) {
+            File sslClientFile = new File(persistDir, SecurityProperties.SSL_CLIENT_PROPERTIES);
+            if (sslClientFile != null && sslClientFile.exists()) {
+                sslClientFile.delete();
+            }
+        }
+        setupKDCAndPrincipals();
+        setupCredentials();
+
+        // client will actually only leverage subset of these properties
+        final PropertiesConfiguration configuration = new PropertiesConfiguration();
+        configuration.setProperty(TLS_ENABLED, true);
+        configuration.setProperty(TRUSTSTORE_FILE_KEY, "../../webapp/target/metadata.keystore");
+        configuration.setProperty(KEYSTORE_FILE_KEY, "../../webapp/target/metadata.keystore");
+        configuration.setProperty(CERT_STORES_CREDENTIAL_PROVIDER_PATH, providerUrl);
+        configuration.setProperty("metadata.http.authentication.type", "kerberos");
+        configuration.setProperty(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, SSLHostnameVerifier.DEFAULT_AND_LOCALHOST.toString());
+
+        configuration.save(new FileWriter(persistDir + File.separator + "client.properties"));
+
+        String confLocation = System.getProperty("metadata.conf");
+        URL url;
+        if (confLocation == null) {
+            url = PropertiesUtil.class.getResource("/application.properties");
+        } else {
+            url = new File(confLocation, "application.properties").toURI().toURL();
+        }
+        configuration.load(url);
+        configuration.setProperty(TLS_ENABLED, true);
+        configuration.setProperty("metadata.http.authentication.enabled", "true");
+        configuration.setProperty("metadata.http.authentication.kerberos.principal", "HTTP/localhost@" + kdc.getRealm());
+        configuration.setProperty("metadata.http.authentication.kerberos.keytab", httpKeytabFile.getAbsolutePath());
+        configuration.setProperty("metadata.http.authentication.kerberos.name.rules",
+                "RULE:[1:$1@$0](.*@EXAMPLE.COM)s/@.*//\nDEFAULT");
+
+        configuration.save(new FileWriter(persistDir + File.separator + "application.properties"));
+
+        secureEmbeddedServer = new TestSecureEmbeddedServer(21443, "webapp/target/metadata-governance") {
+            @Override
+            public PropertiesConfiguration getConfiguration() {
+                return configuration;
+            }
+        };
+        WebAppContext webapp = new WebAppContext();
+        webapp.setContextPath("/");
+        webapp.setWar(System.getProperty("user.dir") + getWarPath());
+        secureEmbeddedServer.getServer().setHandler(webapp);
+
+        // save original setting
+        originalConf = System.getProperty("metadata.conf");
+        System.setProperty("metadata.conf", persistDir);
+        secureEmbeddedServer.getServer().start();
+
+    }
+
+    @AfterClass
+    public void tearDown() throws Exception {
+        if (secureEmbeddedServer != null) {
+            secureEmbeddedServer.getServer().stop();
+        }
+
+        if (kdc != null) {
+            kdc.stop();
+        }
+
+        if (originalConf != null) {
+            System.setProperty("metadata.conf", originalConf);
+        }
+    }
+
+    private void runCommand(final String cmd) throws Exception {
+        ss.setCommandType(null);
+        driver.run(cmd);
+        Assert.assertNotNull(driver.getErrorMsg());
+        Assert.assertTrue(driver.getErrorMsg().contains("Mechanism level: Failed to find any Kerberos tgt"));
+    }
+
+    @Test
+    public void testUnsecuredCreateDatabase() throws Exception {
+        String dbName = "db" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create database " + dbName);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/SSLAndKerberosHiveHookIT.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/SSLAndKerberosHiveHookIT.java b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/SSLAndKerberosHiveHookIT.java
new file mode 100755
index 0000000..0b28e65
--- /dev/null
+++ b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/SSLAndKerberosHiveHookIT.java
@@ -0,0 +1,257 @@
+/**
+ * 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.hadoop.metadata.hive.hook;
+
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.metadata.MetadataException;
+import org.apache.hadoop.metadata.MetadataServiceClient;
+import org.apache.hadoop.metadata.PropertiesUtil;
+import org.apache.hadoop.metadata.hive.model.HiveDataTypes;
+import org.apache.hadoop.metadata.security.SecurityProperties;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.ssl.SSLHostnameVerifier;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONObject;
+import org.mortbay.jetty.webapp.WebAppContext;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import javax.security.auth.Subject;
+import javax.security.auth.callback.*;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.net.URL;
+import java.nio.file.Files;
+import java.security.PrivilegedExceptionAction;
+
+import static org.apache.hadoop.metadata.security.SecurityProperties.*;
+
+public class SSLAndKerberosHiveHookIT extends BaseSSLAndKerberosTest {
+    public static final String TEST_USER_JAAS_SECTION = "TestUser";
+    public static final String TESTUSER = "testuser";
+    public static final String TESTPASS = "testpass";
+
+    private static final String DGI_URL = "https://localhost:21443/";
+    private Driver driver;
+    private MetadataServiceClient dgiCLient;
+    private SessionState ss;
+    private TestSecureEmbeddedServer secureEmbeddedServer;
+    private Subject subject;
+    private String originalConf;
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        //Set-up hive session
+        HiveConf conf = getHiveConf();
+        driver = new Driver(conf);
+        ss = new SessionState(conf, System.getProperty("user.name"));
+        ss = SessionState.start(ss);
+        SessionState.setCurrentSessionState(ss);
+
+        jksPath = new Path(Files.createTempDirectory("tempproviders").toString(), "test.jks");
+        providerUrl = JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+        String persistDir = null;
+        URL resource = SSLAndKerberosHiveHookIT.class.getResource("/");
+        if (resource != null) {
+            persistDir = resource.toURI().getPath();
+        }
+        // delete prior ssl-client.xml file
+        resource = SSLAndKerberosHiveHookIT.class.getResource("/" + SecurityProperties.SSL_CLIENT_PROPERTIES);
+        if (resource != null) {
+            File sslClientFile = new File(persistDir, SecurityProperties.SSL_CLIENT_PROPERTIES);
+            if (sslClientFile != null && sslClientFile.exists()) {
+                sslClientFile.delete();
+            }
+        }
+        setupKDCAndPrincipals();
+        setupCredentials();
+
+        // client will actually only leverage subset of these properties
+        final PropertiesConfiguration configuration = new PropertiesConfiguration();
+        configuration.setProperty(TLS_ENABLED, true);
+        configuration.setProperty(TRUSTSTORE_FILE_KEY, "../../webapp/target/metadata.keystore");
+        configuration.setProperty(KEYSTORE_FILE_KEY, "../../webapp/target/metadata.keystore");
+        configuration.setProperty(CERT_STORES_CREDENTIAL_PROVIDER_PATH, providerUrl);
+        configuration.setProperty("metadata.http.authentication.type", "kerberos");
+        configuration.setProperty(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, SSLHostnameVerifier.DEFAULT_AND_LOCALHOST.toString());
+
+        configuration.save(new FileWriter(persistDir + File.separator + "client.properties"));
+
+        String confLocation = System.getProperty("metadata.conf");
+        URL url;
+        if (confLocation == null) {
+            url = PropertiesUtil.class.getResource("/application.properties");
+        } else {
+            url = new File(confLocation, "application.properties").toURI().toURL();
+        }
+        configuration.load(url);
+        configuration.setProperty(TLS_ENABLED, true);
+        configuration.setProperty("metadata.http.authentication.enabled", "true");
+        configuration.setProperty("metadata.http.authentication.kerberos.principal", "HTTP/localhost@" + kdc.getRealm());
+        configuration.setProperty("metadata.http.authentication.kerberos.keytab", httpKeytabFile.getAbsolutePath());
+        configuration.setProperty("metadata.http.authentication.kerberos.name.rules",
+                "RULE:[1:$1@$0](.*@EXAMPLE.COM)s/@.*//\nDEFAULT");
+
+        configuration.save(new FileWriter(persistDir + File.separator + "application.properties"));
+
+        dgiCLient = new MetadataServiceClient(DGI_URL) {
+            @Override
+            protected PropertiesConfiguration getClientProperties() throws MetadataException {
+                return configuration;
+            }
+        };
+
+        secureEmbeddedServer = new TestSecureEmbeddedServer(21443, "webapp/target/metadata-governance") {
+            @Override
+            public PropertiesConfiguration getConfiguration() {
+                return configuration;
+            }
+        };
+        WebAppContext webapp = new WebAppContext();
+        webapp.setContextPath("/");
+        webapp.setWar(System.getProperty("user.dir") + getWarPath());
+        secureEmbeddedServer.getServer().setHandler(webapp);
+
+        // save original setting
+        originalConf = System.getProperty("metadata.conf");
+        System.setProperty("metadata.conf", persistDir);
+        secureEmbeddedServer.getServer().start();
+
+        subject = loginTestUser();
+    }
+
+    @AfterClass
+    public void tearDown() throws Exception {
+        if (secureEmbeddedServer != null) {
+            secureEmbeddedServer.getServer().stop();
+        }
+
+        if (kdc != null) {
+            kdc.stop();
+        }
+
+        if (originalConf != null) {
+            System.setProperty("metadata.conf", originalConf);
+        }
+    }
+
+    protected Subject loginTestUser() throws LoginException, IOException {
+        LoginContext lc = new LoginContext(TEST_USER_JAAS_SECTION, new CallbackHandler() {
+
+            @Override
+            public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+                for (int i = 0; i < callbacks.length; i++) {
+                    if (callbacks[i] instanceof PasswordCallback) {
+                        PasswordCallback passwordCallback = (PasswordCallback) callbacks[i];
+                        passwordCallback.setPassword(TESTPASS.toCharArray());
+                    }
+                    if (callbacks[i] instanceof NameCallback) {
+                        NameCallback nameCallback = (NameCallback) callbacks[i];
+                        nameCallback.setName(TESTUSER);
+                    }
+                }
+            }
+        });
+        // attempt authentication
+        lc.login();
+        return lc.getSubject();
+    }
+
+    private void runCommand(final String cmd) throws Exception {
+        ss.setCommandType(null);
+        Subject.doAs(subject, new PrivilegedExceptionAction<Object>() {
+            @Override
+            public Object run() throws Exception {
+                driver.run(cmd);
+
+                return null;
+            }
+        });
+    }
+
+    @Test
+    public void testCreateDatabase() throws Exception {
+        String dbName = "db" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create database " + dbName);
+
+        assertDatabaseIsRegistered(dbName);
+    }
+
+    @Test
+    public void testCreateTable() throws Exception {
+        String dbName = "db" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create database " + dbName);
+
+        String tableName = "table" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create table " + dbName + "." + tableName + "(id int, name string)");
+        assertTableIsRegistered(tableName);
+
+        tableName = "table" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create table " + tableName + "(id int, name string)");
+        assertTableIsRegistered(tableName);
+
+        //Create table where database doesn't exist, will create database instance as well
+        assertDatabaseIsRegistered("default");
+    }
+
+    @Test
+    public void testCTAS() throws Exception {
+        String tableName = "table" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create table " + tableName + "(id int, name string)");
+
+        String newTableName = "table" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        String query = "create table " + newTableName + " as select * from " + tableName;
+        runCommand(query);
+
+        assertTableIsRegistered(newTableName);
+        assertInstanceIsRegistered(HiveDataTypes.HIVE_PROCESS.getName(), "queryText", query);
+    }
+
+    private void assertTableIsRegistered(String tableName) throws Exception {
+        assertInstanceIsRegistered(HiveDataTypes.HIVE_TABLE.getName(), "tableName", tableName);
+    }
+
+    private void assertDatabaseIsRegistered(String dbName) throws Exception {
+        assertInstanceIsRegistered(HiveDataTypes.HIVE_DB.getName(), "name", dbName);
+    }
+
+    private void assertInstanceIsRegistered(final String typeName, final String colName, final String colValue) throws Exception {
+        Subject.doAs(subject, new PrivilegedExceptionAction<Object>() {
+            @Override
+            public Object run() throws Exception {
+                JSONArray results = dgiCLient.rawSearch(typeName, colName, colValue);
+                Assert.assertEquals(results.length(), 1);
+
+                return null;
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/SSLHiveHookIT.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/SSLHiveHookIT.java b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/SSLHiveHookIT.java
new file mode 100755
index 0000000..8ef9de7
--- /dev/null
+++ b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/SSLHiveHookIT.java
@@ -0,0 +1,253 @@
+/**
+ * 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.hadoop.metadata.hive.hook;
+
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.metadata.MetadataException;
+import org.apache.hadoop.metadata.MetadataServiceClient;
+import org.apache.hadoop.metadata.hive.bridge.HiveMetaStoreBridge;
+import org.apache.hadoop.metadata.hive.model.HiveDataTypes;
+import org.apache.hadoop.metadata.security.SecurityProperties;
+import org.apache.hadoop.metadata.web.service.SecureEmbeddedServer;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.ssl.SSLHostnameVerifier;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONObject;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.webapp.WebAppContext;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.net.URL;
+import java.nio.file.Files;
+
+import static org.apache.hadoop.metadata.security.SecurityProperties.*;
+
+public class SSLHiveHookIT {
+    private static final String DGI_URL = "https://localhost:21443/";
+    private Driver driver;
+    private MetadataServiceClient dgiCLient;
+    private SessionState ss;
+    private Path jksPath;
+    private String providerUrl;
+    private TestSecureEmbeddedServer secureEmbeddedServer;
+
+    class TestSecureEmbeddedServer extends SecureEmbeddedServer {
+
+        public TestSecureEmbeddedServer(int port, String path) throws IOException {
+            super(port, path);
+        }
+
+        public Server getServer () { return server; }
+
+        @Override
+        public PropertiesConfiguration getConfiguration() {
+            return super.getConfiguration();
+        }
+    }
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        //Set-up hive session
+        HiveConf conf = getHiveConf();
+        driver = new Driver(conf);
+        ss = new SessionState(conf, System.getProperty("user.name"));
+        ss = SessionState.start(ss);
+        SessionState.setCurrentSessionState(ss);
+
+        jksPath = new Path(Files.createTempDirectory("tempproviders").toString(), "test.jks");
+        providerUrl = JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+        String persistDir = null;
+        URL resource = SSLHiveHookIT.class.getResource("/");
+        if (resource != null) {
+            persistDir = resource.toURI().getPath();
+        }
+        // delete prior ssl-client.xml file
+        resource = SSLHiveHookIT.class.getResource("/" + SecurityProperties.SSL_CLIENT_PROPERTIES);
+        if (resource != null) {
+            File sslClientFile = new File(persistDir, SecurityProperties.SSL_CLIENT_PROPERTIES);
+            if (sslClientFile != null && sslClientFile.exists()) {
+                sslClientFile.delete();
+            }
+        }
+        setupCredentials();
+
+        final PropertiesConfiguration configuration = new PropertiesConfiguration();
+        configuration.setProperty(TLS_ENABLED, true);
+        configuration.setProperty(TRUSTSTORE_FILE_KEY, "../../webapp/target/metadata.keystore");
+        configuration.setProperty(KEYSTORE_FILE_KEY, "../../webapp/target/metadata.keystore");
+        configuration.setProperty(CERT_STORES_CREDENTIAL_PROVIDER_PATH, providerUrl);
+        configuration.setProperty(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, SSLHostnameVerifier.DEFAULT_AND_LOCALHOST.toString());
+
+        configuration.save(new FileWriter(persistDir + File.separator + "client.properties"));
+
+        dgiCLient = new MetadataServiceClient(DGI_URL) {
+            @Override
+            protected PropertiesConfiguration getClientProperties() throws MetadataException {
+                return configuration;
+            }
+        };
+
+        secureEmbeddedServer = new TestSecureEmbeddedServer(21443, "webapp/target/metadata-governance") {
+            @Override
+            public PropertiesConfiguration getConfiguration() {
+                return configuration;
+            }
+        };
+        WebAppContext webapp = new WebAppContext();
+        webapp.setContextPath("/");
+        webapp.setWar(System.getProperty("user.dir") + getWarPath());
+        secureEmbeddedServer.getServer().setHandler(webapp);
+
+        secureEmbeddedServer.getServer().start();
+
+    }
+
+    @AfterClass
+    public void tearDown() throws Exception {
+        if (secureEmbeddedServer != null) {
+            secureEmbeddedServer.getServer().stop();
+        }
+    }
+
+    protected void setupCredentials() throws Exception {
+        Configuration conf = new Configuration(false);
+
+        File file = new File(jksPath.toUri().getPath());
+        file.delete();
+        conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, providerUrl);
+
+        CredentialProvider provider =
+                CredentialProviderFactory.getProviders(conf).get(0);
+
+        // create new aliases
+        try {
+
+            char[] storepass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+            provider.createCredentialEntry(
+                    KEYSTORE_PASSWORD_KEY, storepass);
+
+            char[] trustpass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+            provider.createCredentialEntry(
+                    TRUSTSTORE_PASSWORD_KEY, trustpass);
+
+            char[] trustpass2 = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+            provider.createCredentialEntry(
+                    "ssl.client.truststore.password", trustpass2);
+
+            char[] certpass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+            provider.createCredentialEntry(
+                    SERVER_CERT_PASSWORD_KEY, certpass);
+
+            // write out so that it can be found in checks
+            provider.flush();
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    protected String getWarPath() {
+        return String.format("/../../webapp/target/metadata-webapp-%s",
+                System.getProperty("project.version", "0.1-incubating-SNAPSHOT"));
+    }
+
+    private HiveConf getHiveConf() {
+        HiveConf hiveConf = new HiveConf(this.getClass());
+        hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, "");
+        hiveConf.setVar(HiveConf.ConfVars.POSTEXECHOOKS, HiveHook.class.getName());
+        hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+        hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, System.getProperty("user.dir") + "/target/metastore");
+        hiveConf.set(HiveMetaStoreBridge.DGI_URL_PROPERTY, DGI_URL);
+        hiveConf.set("javax.jdo.option.ConnectionURL", "jdbc:derby:./target/metastore_db;create=true");
+        hiveConf.set("hive.hook.dgi.synchronous", "true");
+        return hiveConf;
+    }
+
+    private void runCommand(String cmd) throws Exception {
+        ss.setCommandType(null);
+        driver.run(cmd);
+    }
+
+    @Test
+    public void testCreateDatabase() throws Exception {
+        String dbName = "db" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create database " + dbName);
+
+        assertDatabaseIsRegistered(dbName);
+    }
+
+    @Test
+    public void testCreateTable() throws Exception {
+        String dbName = "db" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create database " + dbName);
+
+        String tableName = "table" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create table " + dbName + "." + tableName + "(id int, name string)");
+        assertTableIsRegistered(tableName);
+
+        tableName = "table" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create table " + tableName + "(id int, name string)");
+        assertTableIsRegistered(tableName);
+
+        //Create table where database doesn't exist, will create database instance as well
+        assertDatabaseIsRegistered("default");
+    }
+
+    @Test
+    public void testCTAS() throws Exception {
+        String tableName = "table" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        runCommand("create table " + tableName + "(id int, name string)");
+
+        String newTableName = "table" + RandomStringUtils.randomAlphanumeric(5).toLowerCase();
+        String query = "create table " + newTableName + " as select * from " + tableName;
+        runCommand(query);
+
+        assertTableIsRegistered(newTableName);
+        assertInstanceIsRegistered(HiveDataTypes.HIVE_PROCESS.getName(), "queryText", query);
+    }
+
+    private void assertTableIsRegistered(String tableName) throws Exception {
+        assertInstanceIsRegistered(HiveDataTypes.HIVE_TABLE.getName(), "tableName", tableName);
+    }
+
+    private void assertDatabaseIsRegistered(String dbName) throws Exception {
+        assertInstanceIsRegistered(HiveDataTypes.HIVE_DB.getName(), "name", dbName);
+    }
+
+    private void assertInstanceIsRegistered(String typeName, String colName, String colValue) throws Exception{
+        JSONArray results = dgiCLient.rawSearch(typeName, colName, colValue);
+        Assert.assertEquals(results.length(), 1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/client/pom.xml
----------------------------------------------------------------------
diff --git a/client/pom.xml b/client/pom.xml
index cdfbecc..6d75915 100755
--- a/client/pom.xml
+++ b/client/pom.xml
@@ -38,13 +38,78 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minikdc</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.directory.jdbm</groupId>
+            <artifactId>apacheds-jdbm1</artifactId>
+            <version>2.0.0-M2</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>com.sun.jersey</groupId>
             <artifactId>jersey-client</artifactId>
         </dependency>
 
         <dependency>
+            <groupId>commons-configuration</groupId>
+            <artifactId>commons-configuration</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${hadoop.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-annotations</artifactId>
+            <version>${hadoop.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>org.testng</groupId>
             <artifactId>testng</artifactId>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.2</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <inherited>true</inherited>
+                <extensions>true</extensions>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/client/src/main/java/org/apache/hadoop/metadata/MetadataServiceClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/hadoop/metadata/MetadataServiceClient.java b/client/src/main/java/org/apache/hadoop/metadata/MetadataServiceClient.java
index 9379aa5..5cb4584 100755
--- a/client/src/main/java/org/apache/hadoop/metadata/MetadataServiceClient.java
+++ b/client/src/main/java/org/apache/hadoop/metadata/MetadataServiceClient.java
@@ -22,13 +22,16 @@ import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.hadoop.metadata.security.SecureClientUtils;
 import org.apache.hadoop.metadata.typesystem.ITypedReferenceableInstance;
-import org.apache.hadoop.metadata.typesystem.Referenceable;
-import org.apache.hadoop.metadata.typesystem.json.InstanceSerialization;
 import org.apache.hadoop.metadata.typesystem.json.Serialization;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.ws.rs.HttpMethod;
 import javax.ws.rs.core.MediaType;
@@ -37,25 +40,45 @@ import javax.ws.rs.core.UriBuilder;
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.apache.hadoop.metadata.security.SecurityProperties.TLS_ENABLED;
+
 /**
  * Client for metadata.
  */
 public class MetadataServiceClient {
+    private static final Logger LOG = LoggerFactory.getLogger(MetadataServiceClient.class);
     public static final String REQUEST_ID = "requestId";
     public static final String RESULTS = "results";
     public static final String TOTAL_SIZE = "totalSize";
 
 
-    private final WebResource service;
+    private WebResource service;
 
     public MetadataServiceClient(String baseUrl) {
         DefaultClientConfig config = new DefaultClientConfig();
-        Client client = Client.create(config);
+        PropertiesConfiguration clientConfig = null;
+        try {
+            clientConfig = getClientProperties();
+            if (clientConfig.getBoolean(TLS_ENABLED)  || clientConfig.getString("metadata.http.authentication.type") != null) {
+                // create an SSL properties configuration if one doesn't exist.  SSLFactory expects a file, so forced to create a
+                // configuration object, persist it, then subsequently pass in an empty configuration to SSLFactory
+                SecureClientUtils.persistSSLClientConfiguration(clientConfig);
+            }
+        } catch (Exception e) {
+            LOG.info("Error processing client configuration.", e);
+        }
+        URLConnectionClientHandler handler = SecureClientUtils.getClientConnectionHandler(config, clientConfig);
+
+        Client client = new Client(handler, config);
         client.resource(UriBuilder.fromUri(baseUrl).build());
 
         service = client.resource(UriBuilder.fromUri(baseUrl).build());
     }
 
+    protected PropertiesConfiguration getClientProperties() throws MetadataException {
+        return PropertiesUtil.getClientProperties();
+    }
+
     static enum API {
         //Type operations
         CREATE_TYPE("api/metadata/types/submit", HttpMethod.POST),

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/client/src/main/java/org/apache/hadoop/metadata/PropertiesUtil.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/hadoop/metadata/PropertiesUtil.java b/client/src/main/java/org/apache/hadoop/metadata/PropertiesUtil.java
new file mode 100644
index 0000000..7e69377
--- /dev/null
+++ b/client/src/main/java/org/apache/hadoop/metadata/PropertiesUtil.java
@@ -0,0 +1,60 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hadoop.metadata;
+
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+public class PropertiesUtil {
+    private static final Logger LOG = LoggerFactory.getLogger(PropertiesUtil.class);
+
+    private static final String APPLICATION_PROPERTIES = "application.properties";
+    public static final String CLIENT_PROPERTIES = "client.properties";
+
+    public static final PropertiesConfiguration getApplicationProperties() throws MetadataException {
+        return getPropertiesConfiguration(APPLICATION_PROPERTIES);
+    }
+
+    public static final PropertiesConfiguration getClientProperties() throws MetadataException {
+        return getPropertiesConfiguration(CLIENT_PROPERTIES);
+    }
+
+    private static PropertiesConfiguration getPropertiesConfiguration(String name) throws MetadataException {
+        String confLocation = System.getProperty("metadata.conf");
+        URL url;
+        try {
+            if (confLocation == null) {
+                url = PropertiesUtil.class.getResource("/" + name);
+            } else {
+                url = new File(confLocation, name).toURI().toURL();
+            }
+            LOG.info("Loading {} from {}", name, url);
+            return new PropertiesConfiguration(url);
+        } catch (Exception e) {
+            throw new MetadataException("Failed to load application properties", e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/client/src/main/java/org/apache/hadoop/metadata/security/SecureClientUtils.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/hadoop/metadata/security/SecureClientUtils.java b/client/src/main/java/org/apache/hadoop/metadata/security/SecureClientUtils.java
new file mode 100644
index 0000000..7755b82
--- /dev/null
+++ b/client/src/main/java/org/apache/hadoop/metadata/security/SecureClientUtils.java
@@ -0,0 +1,195 @@
+/*
+ * 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.hadoop.metadata.security;
+
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metadata.MetadataException;
+import org.apache.hadoop.metadata.PropertiesUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.Authenticator;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.GeneralSecurityException;
+import java.security.PrivilegedExceptionAction;
+
+import static org.apache.hadoop.metadata.security.SecurityProperties.*;
+
+/**
+ *
+ */
+public class SecureClientUtils {
+
+    public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
+    private static final Logger LOG = LoggerFactory.getLogger(SecureClientUtils.class);
+
+
+    public static URLConnectionClientHandler getClientConnectionHandler(DefaultClientConfig config,
+                                                                        PropertiesConfiguration clientConfig) {
+        config.getProperties().put(
+                URLConnectionClientHandler.PROPERTY_HTTP_URL_CONNECTION_SET_METHOD_WORKAROUND,
+                true);
+        Configuration conf = new Configuration(false);
+        conf.addResource(conf.get(SSLFactory.SSL_CLIENT_CONF_KEY, "ssl-client.xml"));
+        String authType = "simple";
+        if (clientConfig != null) {
+            authType = clientConfig.getString("metadata.http.authentication.type", "simple");
+        }
+        UserGroupInformation.setConfiguration(conf);
+        final ConnectionConfigurator connConfigurator = newConnConfigurator(conf);
+        Authenticator authenticator = new PseudoDelegationTokenAuthenticator();
+        if (!authType.equals("simple")) {
+            authenticator = new KerberosDelegationTokenAuthenticator();
+        }
+        authenticator.setConnectionConfigurator(connConfigurator);
+        final DelegationTokenAuthenticator finalAuthenticator = (DelegationTokenAuthenticator) authenticator;
+        final DelegationTokenAuthenticatedURL.Token token = new DelegationTokenAuthenticatedURL.Token();
+        HttpURLConnectionFactory httpURLConnectionFactory = new HttpURLConnectionFactory() {
+            @Override
+            public HttpURLConnection getHttpURLConnection(final URL url) throws IOException {
+                try {
+                    return new DelegationTokenAuthenticatedURL(finalAuthenticator, connConfigurator)
+                            .openConnection(url, token, null);
+                } catch (Exception e) {
+                    throw new IOException(e);
+                }
+            }
+        };
+
+        return new URLConnectionClientHandler(httpURLConnectionFactory);
+    }
+
+    private final static ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR =
+            new ConnectionConfigurator() {
+                @Override
+                public HttpURLConnection configure(HttpURLConnection conn)
+                        throws IOException {
+                    setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
+                    return conn;
+                }
+            };
+
+    private static ConnectionConfigurator newConnConfigurator(Configuration conf) {
+        try {
+            return newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
+        } catch (Exception e) {
+            LOG.debug("Cannot load customized ssl related configuration. " +
+                    "Fallback to system-generic settings.", e);
+            return DEFAULT_TIMEOUT_CONN_CONFIGURATOR;
+        }
+    }
+
+    private static ConnectionConfigurator newSslConnConfigurator(final int timeout,
+                                                          Configuration conf) throws IOException, GeneralSecurityException {
+        final SSLFactory factory;
+        final SSLSocketFactory sf;
+        final HostnameVerifier hv;
+
+        factory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
+        factory.init();
+        sf = factory.createSSLSocketFactory();
+        hv = factory.getHostnameVerifier();
+
+        return new ConnectionConfigurator() {
+            @Override
+            public HttpURLConnection configure(HttpURLConnection conn)
+                    throws IOException {
+                if (conn instanceof HttpsURLConnection) {
+                    HttpsURLConnection c = (HttpsURLConnection) conn;
+                    c.setSSLSocketFactory(sf);
+                    c.setHostnameVerifier(hv);
+                }
+                setTimeouts(conn, timeout);
+                return conn;
+            }
+        };
+    }
+
+    private static void setTimeouts(URLConnection connection, int socketTimeout) {
+        connection.setConnectTimeout(socketTimeout);
+        connection.setReadTimeout(socketTimeout);
+    }
+
+    private static File getSSLClientFile() throws MetadataException {
+        String confLocation = System.getProperty("metadata.conf");
+        File sslDir;
+        try {
+            if (confLocation == null) {
+                String persistDir = null;
+                URL resource = PropertiesUtil.class.getResource("/");
+                if (resource != null) {
+                    persistDir = resource.toURI().getPath();
+                }
+                assert persistDir != null;
+                sslDir = new File(persistDir);
+            } else {
+                sslDir = new File(confLocation);
+            }
+            LOG.info("ssl-client.xml will be created in {}", sslDir);
+        } catch (Exception e) {
+            throw new MetadataException("Failed to find client configuration directory", e);
+        }
+        return new File(sslDir, SecurityProperties.SSL_CLIENT_PROPERTIES);
+    }
+
+    public static void persistSSLClientConfiguration(PropertiesConfiguration clientConfig) throws MetadataException, IOException {
+        //trust settings
+        Configuration configuration = new Configuration(false);
+        File sslClientFile = getSSLClientFile();
+        if (!sslClientFile.exists()) {
+            configuration.set("ssl.client.truststore.type", "jks");
+            configuration.set("ssl.client.truststore.location", clientConfig.getString(TRUSTSTORE_FILE_KEY));
+            if (clientConfig.getBoolean(CLIENT_AUTH_KEY, false)) {
+                // need to get client key properties
+                configuration.set("ssl.client.keystore.location", clientConfig.getString(KEYSTORE_FILE_KEY));
+                configuration.set("ssl.client.keystore.type", "jks");
+            }
+            // add the configured credential provider
+            configuration.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+                    clientConfig.getString(CERT_STORES_CREDENTIAL_PROVIDER_PATH));
+            String hostnameVerifier = clientConfig.getString(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY);
+            if (hostnameVerifier != null) {
+                configuration.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, hostnameVerifier);
+            }
+
+            configuration.writeXml(new FileWriter(sslClientFile));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/client/src/main/java/org/apache/hadoop/metadata/security/SecurityProperties.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/hadoop/metadata/security/SecurityProperties.java b/client/src/main/java/org/apache/hadoop/metadata/security/SecurityProperties.java
new file mode 100644
index 0000000..8cfff8e
--- /dev/null
+++ b/client/src/main/java/org/apache/hadoop/metadata/security/SecurityProperties.java
@@ -0,0 +1,35 @@
+/*
+ * 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.hadoop.metadata.security;
+
+/**
+ *
+ */
+public interface SecurityProperties {
+    public static final String TLS_ENABLED = "metadata.enableTLS";
+    public static final String KEYSTORE_FILE_KEY = "keystore.file";
+    public static final String DEFAULT_KEYSTORE_FILE_LOCATION = "target/metadata.keystore";
+    public static final String KEYSTORE_PASSWORD_KEY = "keystore.password";
+    public static final String TRUSTSTORE_FILE_KEY = "truststore.file";
+    public static final String DEFATULT_TRUSTORE_FILE_LOCATION = "target/metadata.keystore";
+    public static final String TRUSTSTORE_PASSWORD_KEY = "truststore.password";
+    public static final String SERVER_CERT_PASSWORD_KEY = "password";
+    public static final String CLIENT_AUTH_KEY = "client.auth.enabled";
+    public static final String CERT_STORES_CREDENTIAL_PROVIDER_PATH = "cert.stores.credential.provider.path";
+    String SSL_CLIENT_PROPERTIES = "ssl-client.xml";
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/client/src/test/java/org/apache/hadoop/metadata/security/BaseSecurityTest.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/hadoop/metadata/security/BaseSecurityTest.java b/client/src/test/java/org/apache/hadoop/metadata/security/BaseSecurityTest.java
new file mode 100644
index 0000000..ac099e8
--- /dev/null
+++ b/client/src/test/java/org/apache/hadoop/metadata/security/BaseSecurityTest.java
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop.metadata.security;
+
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.zookeeper.Environment;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.webapp.WebAppContext;
+import org.testng.Assert;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.Writer;
+import java.nio.file.Files;
+import java.util.Locale;
+import java.util.Properties;
+
+/**
+ *
+ */
+public class BaseSecurityTest {
+    private static final String JAAS_ENTRY =
+            "%s { \n"
+                    + " %s required\n"
+                    // kerberos module
+                    + " keyTab=\"%s\"\n"
+                    + " debug=true\n"
+                    + " principal=\"%s\"\n"
+                    + " useKeyTab=true\n"
+                    + " useTicketCache=false\n"
+                    + " doNotPrompt=true\n"
+                    + " storeKey=true;\n"
+                    + "}; \n";
+    protected MiniKdc kdc;
+
+    protected String getWarPath() {
+        return String.format("/target/metadata-webapp-%s.war",
+                System.getProperty("release.version", "0.1-incubating-SNAPSHOT"));
+    }
+
+    protected void generateTestProperties(Properties props) throws ConfigurationException, IOException {
+        PropertiesConfiguration config = new PropertiesConfiguration(System.getProperty("user.dir") +
+                "/../src/conf/application.properties");
+        for (String propName : props.stringPropertyNames()) {
+            config.setProperty(propName, props.getProperty(propName));
+        }
+        File file = new File(System.getProperty("user.dir"), "application.properties");
+        file.deleteOnExit();
+        Writer fileWriter = new FileWriter(file);
+        config.save(fileWriter);
+    }
+
+    protected void startEmbeddedServer(Server server) throws Exception {
+        WebAppContext webapp = new WebAppContext();
+        webapp.setContextPath("/");
+        webapp.setWar(System.getProperty("user.dir") + getWarPath());
+        server.setHandler(webapp);
+
+        server.start();
+    }
+
+    protected File startKDC() throws Exception {
+        File target = Files.createTempDirectory("sectest").toFile();
+        File kdcWorkDir = new File(target, "kdc");
+        Properties kdcConf = MiniKdc.createConf();
+        kdcConf.setProperty(MiniKdc.DEBUG, "true");
+        kdc = new MiniKdc(kdcConf, kdcWorkDir);
+        kdc.start();
+
+        Assert.assertNotNull(kdc.getRealm());
+        return kdcWorkDir;
+    }
+
+    public String createJAASEntry(
+            String context,
+            String principal,
+            File keytab) {
+        String keytabpath = keytab.getAbsolutePath();
+        // fix up for windows; no-op on unix
+        keytabpath =  keytabpath.replace('\\', '/');
+        return String.format(
+                Locale.ENGLISH,
+                JAAS_ENTRY,
+                context,
+                getKerberosAuthModuleForJVM(),
+                keytabpath,
+                principal);
+    }
+
+    protected String getKerberosAuthModuleForJVM() {
+        if (System.getProperty("java.vendor").contains("IBM")) {
+            return "com.ibm.security.auth.module.Krb5LoginModule";
+        } else {
+            return "com.sun.security.auth.module.Krb5LoginModule";
+        }
+    }
+
+    protected void bindJVMtoJAASFile(File jaasFile) {
+        String path = jaasFile.getAbsolutePath();
+        System.setProperty(Environment.JAAS_CONF_KEY, path);
+    }
+
+    protected File createKeytab(MiniKdc kdc, File kdcWorkDir, String principal, String filename) throws Exception {
+        File keytab = new File(kdcWorkDir, filename);
+        kdc.createPrincipal(keytab,
+                principal,
+                principal + "/localhost",
+                principal + "/127.0.0.1");
+        return keytab;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0ac2c2a..91a8430 100755
--- a/pom.xml
+++ b/pom.xml
@@ -823,7 +823,9 @@
                     <!--<skipTests>true</skipTests>-->
                     <forkMode>always</forkMode>
                     <redirectTestOutputToFile>true</redirectTestOutputToFile>
-                    <argLine>-Djava.awt.headless=true</argLine>
+                    <argLine>-Djava.awt.headless=true -Dproject.version=${project.version}
+                        -Dhadoop.tmp.dir=${project.build.directory}/tmp-hadoop-${user.name}
+                        -Xmx1024m -XX:MaxPermSize=512m</argLine>
                 </configuration>
                 <dependencies>
                     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/repository/pom.xml
----------------------------------------------------------------------
diff --git a/repository/pom.xml b/repository/pom.xml
index 7003ad2..c626962 100755
--- a/repository/pom.xml
+++ b/repository/pom.xml
@@ -40,6 +40,11 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.hadoop.metadata</groupId>
+            <artifactId>metadata-client</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-log4j12</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/repository/src/main/java/org/apache/hadoop/metadata/PropertiesUtil.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/hadoop/metadata/PropertiesUtil.java b/repository/src/main/java/org/apache/hadoop/metadata/PropertiesUtil.java
deleted file mode 100644
index d30c4cc..0000000
--- a/repository/src/main/java/org/apache/hadoop/metadata/PropertiesUtil.java
+++ /dev/null
@@ -1,50 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.hadoop.metadata;
-
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.PropertiesConfiguration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.net.MalformedURLException;
-import java.net.URL;
-
-public class PropertiesUtil {
-    private static final Logger LOG = LoggerFactory.getLogger(PropertiesUtil.class);
-
-    private static final String APPLICATION_PROPERTIES = "application.properties";
-
-    public static final PropertiesConfiguration getApplicationProperties() throws MetadataException {
-        String confLocation = System.getProperty("metadata.conf");
-        URL url;
-        try {
-            if (confLocation == null) {
-                url = PropertiesUtil.class.getResource("/" + APPLICATION_PROPERTIES);
-            } else {
-                url = new File(confLocation, APPLICATION_PROPERTIES).toURI().toURL();
-            }
-            LOG.info("Loading {} from {}", APPLICATION_PROPERTIES, url);
-            return new PropertiesConfiguration(url);
-        } catch (Exception e) {
-            throw new MetadataException("Failed to load application properties", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/webapp/pom.xml
----------------------------------------------------------------------
diff --git a/webapp/pom.xml b/webapp/pom.xml
index 6a66bea..2e56689 100755
--- a/webapp/pom.xml
+++ b/webapp/pom.xml
@@ -48,9 +48,17 @@
             <artifactId>metadata-repository</artifactId>
         </dependency>
 
+        <!--<dependency>-->
+            <!--<groupId>org.apache.hadoop.metadata</groupId>-->
+            <!--<artifactId>metadata-client</artifactId>-->
+        <!--</dependency>-->
+
         <dependency>
             <groupId>org.apache.hadoop.metadata</groupId>
             <artifactId>metadata-client</artifactId>
+            <version>${version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
         </dependency>
 
         <dependency>
@@ -214,6 +222,7 @@
                 <artifactId>maven-war-plugin</artifactId>
                 <version>2.4</version>
                 <configuration>
+                    <attachClasses>true</attachClasses>
                     <webResources>
                         <resource>
                             <directory>../dashboard/v3</directory>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/webapp/src/main/java/org/apache/hadoop/metadata/util/CredentialProviderUtility.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/hadoop/metadata/util/CredentialProviderUtility.java b/webapp/src/main/java/org/apache/hadoop/metadata/util/CredentialProviderUtility.java
index 158e191..a91db75 100755
--- a/webapp/src/main/java/org/apache/hadoop/metadata/util/CredentialProviderUtility.java
+++ b/webapp/src/main/java/org/apache/hadoop/metadata/util/CredentialProviderUtility.java
@@ -26,13 +26,15 @@ import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
 import java.io.*;
 import java.util.Arrays;
 
+import static org.apache.hadoop.metadata.security.SecurityProperties.*;
+
 /**
  * A utility class for generating a credential provider containing the entries required for supporting the SSL implementation
  * of the DGC server.
  */
 public class CredentialProviderUtility {
-    private static final String[] KEYS = new String[] {SecureEmbeddedServer.KEYSTORE_PASSWORD_KEY,
-            SecureEmbeddedServer.TRUSTSTORE_PASSWORD_KEY, SecureEmbeddedServer.SERVER_CERT_PASSWORD_KEY};
+    private static final String[] KEYS = new String[] {KEYSTORE_PASSWORD_KEY,
+            TRUSTSTORE_PASSWORD_KEY, SERVER_CERT_PASSWORD_KEY};
 
     public static abstract class TextDevice {
         public abstract void printf(String fmt, Object... params);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/webapp/src/main/java/org/apache/hadoop/metadata/web/filters/MetadataAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/hadoop/metadata/web/filters/MetadataAuthenticationFilter.java b/webapp/src/main/java/org/apache/hadoop/metadata/web/filters/MetadataAuthenticationFilter.java
index cd79f63..3874d25 100644
--- a/webapp/src/main/java/org/apache/hadoop/metadata/web/filters/MetadataAuthenticationFilter.java
+++ b/webapp/src/main/java/org/apache/hadoop/metadata/web/filters/MetadataAuthenticationFilter.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.metadata.web.filters;
 import com.google.inject.Singleton;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.hadoop.metadata.PropertiesUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
@@ -49,8 +50,8 @@ public class MetadataAuthenticationFilter extends AuthenticationFilter {
     protected Properties getConfiguration(String configPrefix, FilterConfig filterConfig) throws ServletException {
         PropertiesConfiguration configuration;
         try {
-            configuration = new PropertiesConfiguration("application.properties");
-        } catch (ConfigurationException e) {
+            configuration = PropertiesUtil.getApplicationProperties();
+        } catch (Exception e) {
             throw new ServletException(e);
         }
 
@@ -95,6 +96,8 @@ public class MetadataAuthenticationFilter extends AuthenticationFilter {
             config.put(KerberosAuthenticationHandler.PRINCIPAL, principal);
         }
 
+        LOG.info("AuthenticationFilterConfig: {}", config);
+
         return config;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/webapp/src/main/java/org/apache/hadoop/metadata/web/listeners/GuiceServletConfig.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/hadoop/metadata/web/listeners/GuiceServletConfig.java b/webapp/src/main/java/org/apache/hadoop/metadata/web/listeners/GuiceServletConfig.java
index 92947d2..b7eeda4 100755
--- a/webapp/src/main/java/org/apache/hadoop/metadata/web/listeners/GuiceServletConfig.java
+++ b/webapp/src/main/java/org/apache/hadoop/metadata/web/listeners/GuiceServletConfig.java
@@ -27,6 +27,7 @@ import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.hadoop.metadata.MetadataException;
+import org.apache.hadoop.metadata.PropertiesUtil;
 import org.apache.hadoop.metadata.RepositoryMetadataModule;
 import org.apache.hadoop.metadata.repository.typestore.ITypeStore;
 import org.apache.hadoop.metadata.typesystem.TypesDef;
@@ -80,10 +81,13 @@ public class GuiceServletConfig extends GuiceServletContextListener {
                         }
 
                         private void configureAuthenticationFilter() throws ConfigurationException {
-                            PropertiesConfiguration configuration =
-                                    new PropertiesConfiguration("application.properties");
-                            if (Boolean.valueOf(configuration.getString(HTTP_AUTHENTICATION_ENABLED))) {
-                                filter("/*").through(MetadataAuthenticationFilter.class);
+                            try {
+                                PropertiesConfiguration configuration = PropertiesUtil.getApplicationProperties();
+                                if (Boolean.valueOf(configuration.getString(HTTP_AUTHENTICATION_ENABLED))) {
+                                    filter("/*").through(MetadataAuthenticationFilter.class);
+                                }
+                            } catch (MetadataException e) {
+                                LOG.warn("Error loading configuration and initializing authentication filter", e);
                             }
                         }
                     });

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/webapp/src/main/java/org/apache/hadoop/metadata/web/service/SecureEmbeddedServer.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/hadoop/metadata/web/service/SecureEmbeddedServer.java b/webapp/src/main/java/org/apache/hadoop/metadata/web/service/SecureEmbeddedServer.java
index 865bc4f..6e71366 100755
--- a/webapp/src/main/java/org/apache/hadoop/metadata/web/service/SecureEmbeddedServer.java
+++ b/webapp/src/main/java/org/apache/hadoop/metadata/web/service/SecureEmbeddedServer.java
@@ -28,23 +28,16 @@ import org.mortbay.jetty.security.SslSocketConnector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 import java.io.IOException;
 
+import static org.apache.hadoop.metadata.security.SecurityProperties.*;
+
 /**
  * This is a jetty server which requires client auth via certificates.
  */
 public class SecureEmbeddedServer extends EmbeddedServer {
 
-    public static final String KEYSTORE_FILE_KEY = "keystore.file";
-    public static final String DEFAULT_KEYSTORE_FILE_LOCATION = "target/metadata.keystore";
-    public static final String KEYSTORE_PASSWORD_KEY = "keystore.password";
-    public static final String TRUSTSTORE_FILE_KEY = "truststore.file";
-    public static final String DEFATULT_TRUSTORE_FILE_LOCATION = "target/metadata.keystore";
-    public static final String TRUSTSTORE_PASSWORD_KEY = "truststore.password";
-    public static final String SERVER_CERT_PASSWORD_KEY = "password";
-    public static final String CLIENT_AUTH_KEY = "client.auth.enabled";
-    public static final String CERT_STORES_CREDENTIAL_PROVIDER_PATH = "cert.stores.credential.provider.path";
-
     private static final Logger LOG = LoggerFactory.getLogger(SecureEmbeddedServer.class);
 
     public SecureEmbeddedServer(int port, String path) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/43868812/webapp/src/test/java/org/apache/hadoop/metadata/CredentialProviderUtilityIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/hadoop/metadata/CredentialProviderUtilityIT.java b/webapp/src/test/java/org/apache/hadoop/metadata/CredentialProviderUtilityIT.java
index 518615f..1ae733f 100755
--- a/webapp/src/test/java/org/apache/hadoop/metadata/CredentialProviderUtilityIT.java
+++ b/webapp/src/test/java/org/apache/hadoop/metadata/CredentialProviderUtilityIT.java
@@ -31,6 +31,8 @@ import java.io.IOException;
 import java.nio.file.Files;
 import java.util.*;
 
+import static org.apache.hadoop.metadata.security.SecurityProperties.*;
+
 /**
  *
  */
@@ -75,11 +77,11 @@ public class CredentialProviderUtilityIT {
                 CredentialProviderFactory.getProviders(conf).get(0);
 
         CredentialProvider.CredentialEntry entry =
-                provider.getCredentialEntry(SecureEmbeddedServer.KEYSTORE_PASSWORD_KEY);
+                provider.getCredentialEntry(KEYSTORE_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
-        entry = provider.getCredentialEntry(SecureEmbeddedServer.TRUSTSTORE_PASSWORD_KEY);
+        entry = provider.getCredentialEntry(TRUSTSTORE_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
-        entry = provider.getCredentialEntry(SecureEmbeddedServer.SERVER_CERT_PASSWORD_KEY);
+        entry = provider.getCredentialEntry(SERVER_CERT_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
     }
 
@@ -138,11 +140,11 @@ public class CredentialProviderUtilityIT {
                 CredentialProviderFactory.getProviders(conf).get(0);
 
         CredentialProvider.CredentialEntry entry =
-                provider.getCredentialEntry(SecureEmbeddedServer.KEYSTORE_PASSWORD_KEY);
+                provider.getCredentialEntry(KEYSTORE_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
-        entry = provider.getCredentialEntry(SecureEmbeddedServer.TRUSTSTORE_PASSWORD_KEY);
+        entry = provider.getCredentialEntry(TRUSTSTORE_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
-        entry = provider.getCredentialEntry(SecureEmbeddedServer.SERVER_CERT_PASSWORD_KEY);
+        entry = provider.getCredentialEntry(SERVER_CERT_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
     }
 
@@ -192,11 +194,11 @@ public class CredentialProviderUtilityIT {
                 CredentialProviderFactory.getProviders(conf).get(0);
 
         CredentialProvider.CredentialEntry entry =
-                provider.getCredentialEntry(SecureEmbeddedServer.KEYSTORE_PASSWORD_KEY);
+                provider.getCredentialEntry(KEYSTORE_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
-        entry = provider.getCredentialEntry(SecureEmbeddedServer.TRUSTSTORE_PASSWORD_KEY);
+        entry = provider.getCredentialEntry(TRUSTSTORE_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
-        entry = provider.getCredentialEntry(SecureEmbeddedServer.SERVER_CERT_PASSWORD_KEY);
+        entry = provider.getCredentialEntry(SERVER_CERT_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry);
     }
 
@@ -260,11 +262,11 @@ public class CredentialProviderUtilityIT {
 
         char[] newpass = "newpass".toCharArray();
         CredentialProvider.CredentialEntry entry =
-                provider.getCredentialEntry(SecureEmbeddedServer.KEYSTORE_PASSWORD_KEY);
+                provider.getCredentialEntry(KEYSTORE_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry, newpass);
-        entry = provider.getCredentialEntry(SecureEmbeddedServer.TRUSTSTORE_PASSWORD_KEY);
+        entry = provider.getCredentialEntry(TRUSTSTORE_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry, newpass);
-        entry = provider.getCredentialEntry(SecureEmbeddedServer.SERVER_CERT_PASSWORD_KEY);
+        entry = provider.getCredentialEntry(SERVER_CERT_PASSWORD_KEY);
         assertCredentialEntryCorrect(entry, newpass);
     }
 }