You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@knox.apache.org by lm...@apache.org on 2014/01/22 22:54:50 UTC

git commit: KNOX-105 - CLI for managing security artifacts: aliases, certificates, master secret

Updated Branches:
  refs/heads/master 5c0b0f6b5 -> a9e7deaee


KNOX-105 - CLI for managing security artifacts: aliases, certificates, master secret

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

Branch: refs/heads/master
Commit: a9e7deaeef46104faf10146aed9f5084b3275d84
Parents: 5c0b0f6
Author: Larry McCay <lm...@hortonworks.com>
Authored: Wed Jan 22 16:54:13 2014 -0500
Committer: Larry McCay <lm...@hortonworks.com>
Committed: Wed Jan 22 16:54:13 2014 -0500

----------------------------------------------------------------------
 build.xml                                       |   1 +
 gateway-release/home/bin/gateway.sh             |   2 +-
 gateway-release/home/bin/knoxcli.sh             |  72 +++
 .../home/conf/knoxcli-log4j.properties          |  34 ++
 gateway-release/pom.xml                         |   4 +
 gateway-release/src/assembly.xml                |   7 +
 .../apache/hadoop/gateway/GatewayMessages.java  |   3 +
 .../gateway/services/CLIGatewayServices.java    | 131 +++++
 .../security/impl/DefaultAliasService.java      |  39 ++
 .../security/impl/DefaultKeystoreService.java   |  46 +-
 .../security/impl/DefaultMasterService.java     |   2 -
 .../org/apache/hadoop/gateway/util/KnoxCLI.java | 477 +++++++++++++++++++
 .../services/security/CryptoServiceTest.java    |  11 +
 .../apache/hadoop/gateway/util/KnoxCLITest.java | 143 ++++++
 .../gateway/services/GatewayServices.java       |   1 +
 .../gateway/services/security/AliasService.java |   5 +
 .../services/security/KeystoreService.java      |   5 +-
 .../security/impl/BaseKeystoreService.java      |  12 +
 knox-cli-launcher/pom.xml                       |  68 +++
 .../src/main/resources/META-INF/launcher.cfg    |  18 +
 pom.xml                                         |   6 +
 21 files changed, 1069 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 9af59ed..d35a956 100644
--- a/build.xml
+++ b/build.xml
@@ -310,6 +310,7 @@
         <untar src="target/${gateway-version}/${gateway-artifact}-${gateway-version}.tar.gz" dest="${install.dir}" compression="gzip"/>
         <chmod file="${install.dir}/${gateway-artifact}-${gateway-version}/bin/gateway.sh" perm="544"/>
         <chmod file="${install.dir}/${gateway-artifact}-${gateway-version}/bin/ldap.sh" perm="544"/>
+        <chmod file="${install.dir}/${gateway-artifact}-${gateway-version}/bin/knoxcli.sh" perm="544"/>
     </target>
 
     <target name="install-test-home" depends="_install-test-home-unix,_install-test-home-windows" description="Install binary release into install directory.">

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-release/home/bin/gateway.sh
----------------------------------------------------------------------
diff --git a/gateway-release/home/bin/gateway.sh b/gateway-release/home/bin/gateway.sh
index ae7d388..04351c1 100755
--- a/gateway-release/home/bin/gateway.sh
+++ b/gateway-release/home/bin/gateway.sh
@@ -112,7 +112,7 @@ function appStart {
    
    rm -f $APP_PID_FILE
 
-   nohup $JAVA_CMD $APP_MEM_OPTS $APP_DBC_OPTS $APP_LOG_OPTS -jar $APP_JAR >>$APP_OUT_FILE 2>>$APP_ERR_FILE & printf $!>$APP_PID_FILE || exit 1
+   nohup $JAVA_CMD $APP_MEM_OPTS $APP_DBG_OPTS $APP_LOG_OPTS -jar $APP_JAR >>$APP_OUT_FILE 2>>$APP_ERR_FILE & printf $!>$APP_PID_FILE || exit 1
 
    getPID
    for ((i=0; i<APP_START_WAIT_TIME*10; i++)); do

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-release/home/bin/knoxcli.sh
----------------------------------------------------------------------
diff --git a/gateway-release/home/bin/knoxcli.sh b/gateway-release/home/bin/knoxcli.sh
new file mode 100755
index 0000000..cdb006c
--- /dev/null
+++ b/gateway-release/home/bin/knoxcli.sh
@@ -0,0 +1,72 @@
+#!/bin/sh
+
+#
+#  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.
+#
+
+# The app's label
+APP_LABEL=KnoxCLI
+
+# The app's name
+APP_NAME=knoxcli
+
+# Start/stop script location
+APP_BIN_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+
+# The app's jar name
+APP_JAR="$APP_BIN_DIR/knoxcli.jar"
+
+# The apps home dir
+APP_HOME_DIR=`dirname $APP_BIN_DIR`
+
+# The apps home dir
+APP_CONF_DIR="$APP_HOME_DIR/conf"
+
+# The app's log dir
+APP_LOG_DIR="$APP_HOME_DIR/logs"
+
+# The app's logging options
+APP_LOG_OPTS=""
+
+# The app's memory options
+APP_MEM_OPTS=""
+
+# The app's debugging options
+APP_DBG_OPTS=""
+
+# Name of LOG/OUT/ERR file
+APP_OUT_FILE="$APP_LOG_DIR/$APP_NAME.out"
+APP_ERR_FILE="$APP_LOG_DIR/$APP_NAME.err"
+
+# Java command
+JAVA_CMD=java
+
+function main {
+   printf "Starting $APP_LABEL \n"
+   #printf "$@"
+   
+   exec $JAVA_CMD $APP_MEM_OPTS $APP_DBG_OPTS $APP_LOG_OPTS -jar $APP_JAR "$@" || exit 1
+
+   return 0
+}
+
+function printHelp {
+   $JAVA_CMD -jar $APP_JAR -help
+   return 0
+}
+
+#Starting main
+main $@

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-release/home/conf/knoxcli-log4j.properties
----------------------------------------------------------------------
diff --git a/gateway-release/home/conf/knoxcli-log4j.properties b/gateway-release/home/conf/knoxcli-log4j.properties
new file mode 100644
index 0000000..c7fab24
--- /dev/null
+++ b/gateway-release/home/conf/knoxcli-log4j.properties
@@ -0,0 +1,34 @@
+# 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.
+
+app.log.dir=${launcher.dir}/../logs
+app.log.file=${launcher.name}.log
+
+log4j.rootLogger=ERROR, drfa
+
+log4j.logger.org.apache.hadoop.gateway=INFO
+#log4j.logger.org.apache.hadoop.gateway=DEBUG
+
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
+
+log4j.appender.drfa=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.drfa.File=${app.log.dir}/${app.log.file}
+log4j.appender.drfa.DatePattern=.yyyy-MM-dd
+log4j.appender.drfa.layout=org.apache.log4j.PatternLayout
+log4j.appender.drfa.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-release/pom.xml
----------------------------------------------------------------------
diff --git a/gateway-release/pom.xml b/gateway-release/pom.xml
index 3465533..eb6eafc 100644
--- a/gateway-release/pom.xml
+++ b/gateway-release/pom.xml
@@ -107,6 +107,10 @@
         </dependency>
         <dependency>
             <groupId>${gateway-group}</groupId>
+            <artifactId>knox-cli-launcher</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>${gateway-group}</groupId>
             <artifactId>gateway-server-launcher</artifactId>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-release/src/assembly.xml
----------------------------------------------------------------------
diff --git a/gateway-release/src/assembly.xml b/gateway-release/src/assembly.xml
index 9db3121..56d58d2 100644
--- a/gateway-release/src/assembly.xml
+++ b/gateway-release/src/assembly.xml
@@ -89,6 +89,13 @@
         </dependencySet>
         <dependencySet>
             <outputDirectory>bin</outputDirectory>
+            <outputFileNameMapping>knoxcli.jar</outputFileNameMapping>
+            <includes>
+                <include>${gateway-group}:knox-cli-launcher</include>
+            </includes>
+        </dependencySet>
+        <dependencySet>
+            <outputDirectory>bin</outputDirectory>
             <outputFileNameMapping>gateway.jar</outputFileNameMapping>
             <includes>
                 <include>${gateway-group}:gateway-server-launcher</include>

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-server/src/main/java/org/apache/hadoop/gateway/GatewayMessages.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/GatewayMessages.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/GatewayMessages.java
index 851ed23..31d46d5 100644
--- a/gateway-server/src/main/java/org/apache/hadoop/gateway/GatewayMessages.java
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/GatewayMessages.java
@@ -291,4 +291,7 @@ public interface GatewayMessages {
   @Message( level = MessageLevel.DEBUG, text = "Key passphrase not found in credential store - using master secret." )
   void assumingKeyPassphraseIsMaster();
 
+  @Message( level = MessageLevel.ERROR, text = "Failed to remove alias for cluster: {0} {1}." )
+  void failedToRemoveCredentialForCluster(String clusterName,
+      KeystoreServiceException e);
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-server/src/main/java/org/apache/hadoop/gateway/services/CLIGatewayServices.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/CLIGatewayServices.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/CLIGatewayServices.java
new file mode 100644
index 0000000..4e634dd
--- /dev/null
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/CLIGatewayServices.java
@@ -0,0 +1,131 @@
+/**
+ * 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.gateway.services;
+
+import org.apache.hadoop.gateway.GatewayMessages;
+import org.apache.hadoop.gateway.config.GatewayConfig;
+import org.apache.hadoop.gateway.deploy.DeploymentContext;
+import org.apache.hadoop.gateway.descriptor.FilterParamDescriptor;
+import org.apache.hadoop.gateway.descriptor.ResourceDescriptor;
+import org.apache.hadoop.gateway.i18n.messages.MessagesFactory;
+import org.apache.hadoop.gateway.services.security.impl.DefaultAliasService;
+import org.apache.hadoop.gateway.services.security.impl.DefaultCryptoService;
+import org.apache.hadoop.gateway.services.security.impl.DefaultKeystoreService;
+import org.apache.hadoop.gateway.services.security.impl.DefaultMasterService;
+import org.apache.hadoop.gateway.topology.Provider;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class CLIGatewayServices implements GatewayServices {
+
+  private static GatewayMessages log = MessagesFactory.get( GatewayMessages.class );
+
+  private Map<String,Service> services = new HashMap<String, Service>();
+  private DefaultMasterService ms = null;
+  private DefaultKeystoreService ks = null;
+
+  public CLIGatewayServices() {
+    super();
+  }
+
+  public void init(GatewayConfig config, Map<String,String> options) throws ServiceLifecycleException {
+    ms = new DefaultMasterService();
+    ms.init(config, options);
+
+    ks = new DefaultKeystoreService();
+    ks.setMasterService(ms);
+    ks.init(config, options);
+    services.put(KEYSTORE_SERVICE, ks);
+    
+    DefaultAliasService alias = new DefaultAliasService();
+    alias.setKeystoreService(ks);
+    alias.init(config, options);
+    services.put(ALIAS_SERVICE, alias);
+
+    DefaultCryptoService crypto = new DefaultCryptoService();
+    crypto.setKeystoreService(ks);
+    crypto.setAliasService(alias);
+    crypto.init(config, options);
+    services.put(CRYPTO_SERVICE, crypto);
+  }
+  
+  public void start() throws ServiceLifecycleException {
+    ms.start();
+
+    ks.start();
+
+    DefaultAliasService alias = (DefaultAliasService) services.get(ALIAS_SERVICE);
+    alias.start();
+  }
+
+  public void stop() throws ServiceLifecycleException {
+    ms.stop();
+
+    ks.stop();
+
+    DefaultAliasService alias = (DefaultAliasService) services.get(ALIAS_SERVICE);
+    alias.stop();
+  }
+  
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.gateway.GatewayServices#getServiceNames()
+   */
+  @Override
+  public Collection<String> getServiceNames() {
+    return services.keySet();
+  }
+  
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.gateway.GatewayServices#getService(java.lang.String)
+   */
+  @Override
+  public <T> T getService(String serviceName) {
+    return (T)services.get( serviceName );
+  }
+
+  @Override
+  public String getRole() {
+    return "Services";
+  }
+
+  @Override
+  public String getName() {
+    return "GatewayServices";
+  }
+
+  @Override
+  public void initializeContribution(DeploymentContext context) {
+  }
+
+  @Override
+  public void contributeProvider(DeploymentContext context, Provider provider) {
+  }
+
+  @Override
+  public void contributeFilter(DeploymentContext context, Provider provider,
+      org.apache.hadoop.gateway.topology.Service service,
+      ResourceDescriptor resource, List<FilterParamDescriptor> params) {
+  }
+
+  @Override
+  public void finalizeContribution(DeploymentContext context) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultAliasService.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultAliasService.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultAliasService.java
index c151c53..a58dc6e 100644
--- a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultAliasService.java
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultAliasService.java
@@ -17,8 +17,12 @@
  */
 package org.apache.hadoop.gateway.services.security.impl;
 
+import java.security.KeyStore;
 import java.security.KeyStoreException;
 import java.security.cert.Certificate;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
@@ -119,6 +123,11 @@ public class DefaultAliasService implements AliasService {
   }
 
   @Override
+  public void removeAliasForCluster(String clusterName, String alias) {
+    keystoreService.removeCredentialForCluster(clusterName, alias);
+  }
+
+  @Override
   public char[] getPasswordFromAliasForGateway(String alias) {
     return getPasswordFromAliasForCluster("__gateway", alias);
   }
@@ -144,4 +153,34 @@ public class DefaultAliasService implements AliasService {
     }
     return cert;
   }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.gateway.services.security.AliasService#getAliasesForCluster(java.lang.String)
+   */
+  @Override
+  public List<String> getAliasesForCluster(String clusterName) {
+    ArrayList<String> list = new ArrayList<String>();
+    KeyStore keyStore;
+    try {
+      keyStore = keystoreService.getCredentialStoreForCluster(clusterName);
+      if (keyStore != null) {
+        String alias = null;
+        try {
+          Enumeration<String> e = keyStore.aliases();
+          while (e.hasMoreElements()) {
+             alias = e.nextElement();
+             // only include the metadata key names in the list of names
+             if (!alias.contains("@")) {
+                 list.add(alias);
+             }
+          }
+        } catch (KeyStoreException e) {
+          LOG.failedToGetCredentialForCluster(clusterName, e);
+        }
+      }
+    } catch (KeystoreServiceException kse) {
+      LOG.failedToGetCredentialForCluster(clusterName, kse);
+    }
+    return list;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultKeystoreService.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultKeystoreService.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultKeystoreService.java
index 781375b..207b5ad 100644
--- a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultKeystoreService.java
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultKeystoreService.java
@@ -43,15 +43,12 @@ import java.security.cert.X509Certificate;
 import java.text.MessageFormat;
 import java.util.Map;
 
-
 public class DefaultKeystoreService extends BaseKeystoreService implements KeystoreService, Service {
 
   private static final String dnTemplate = "CN={0},OU=Test,O=Hadoop,L=Test,ST=Test,C=US";
-  private static final String TEST_CERT_DN = "CN=hadoop.gateway,OU=Test,O=Hadoop,L=Test,ST=Test,C=US";
   private static final String CREDENTIALS_SUFFIX = "-credentials.jceks";
   private static final String GATEWAY_KEYSTORE = "gateway.jks";
   private static final String CERT_GEN_MODE = "hadoop.gateway.cert.gen.mode";
-  private static final String CERT_GEN_MODE_HADOOP_GATEWAY = "hadoop.gateway";
   private static final String CERT_GEN_MODE_LOCALHOST = "localhost";
   private static final String CERT_GEN_MODE_HOSTNAME = "hostname";
   private static GatewayMessages LOG = MessagesFactory.get( GatewayMessages.class );
@@ -91,27 +88,27 @@ public class DefaultKeystoreService extends BaseKeystoreService implements Keyst
   
   @Override
   public void addSelfSignedCertForGateway(String alias, char[] passphrase) {
+    addSelfSignedCertForGateway(alias, passphrase, null);
+  }
+
+  @Override
+  public void addSelfSignedCertForGateway(String alias, char[] passphrase, String hostname) {
+
     KeyPairGenerator keyPairGenerator;
     try {
       keyPairGenerator = KeyPairGenerator.getInstance("RSA");
       keyPairGenerator.initialize(1024);  
       KeyPair KPair = keyPairGenerator.generateKeyPair();
-      String certGenMode = System.getProperty(CERT_GEN_MODE, CERT_GEN_MODE_LOCALHOST);
-      X509Certificate cert = null;
-      if (certGenMode.equals(CERT_GEN_MODE_HADOOP_GATEWAY)) {
-        String dn = buildDistinguishedName("hadoop.gateway");
-        cert = generateCertificate(dn, KPair, 365, "SHA1withRSA");
+      if (hostname == null) {
+        hostname = System.getProperty(CERT_GEN_MODE, CERT_GEN_MODE_LOCALHOST);
       }
-      else if(certGenMode.equals(CERT_GEN_MODE_LOCALHOST)) {
-        String dn = buildDistinguishedName("localhost");
-        cert = generateCertificate(dn, KPair, 365, "SHA1withRSA");
-      }
-      else if(certGenMode.equals(CERT_GEN_MODE_HOSTNAME)) {
+      X509Certificate cert = null;
+      if(hostname.equals(CERT_GEN_MODE_HOSTNAME)) {
         String dn = buildDistinguishedName(InetAddress.getLocalHost().getHostName());
         cert = generateCertificate(dn, KPair, 365, "SHA1withRSA");
       }
       else {
-        String dn = buildDistinguishedName("localhost");
+        String dn = buildDistinguishedName(hostname);
         cert = generateCertificate(dn, KPair, 365, "SHA1withRSA");
       }
 
@@ -235,4 +232,25 @@ public class DefaultKeystoreService extends BaseKeystoreService implements Keyst
     }
     return credential;
   }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.gateway.services.security.KeystoreService#removeCredentialForCluster(java.lang.String, java.lang.String, java.security.KeyStore)
+   */
+  @Override
+  public void removeCredentialForCluster(String clusterName, String alias) {
+    KeyStore ks = getCredentialStoreForCluster(clusterName);
+    removeCredential(alias, ks);
+    final File  keyStoreFile = new File( keyStoreDir + clusterName + CREDENTIALS_SUFFIX  );
+    try {
+      writeKeystoreToFile(ks, keyStoreFile);
+    } catch (KeyStoreException e) {
+      LOG.failedToAddCredentialForCluster( clusterName, e );
+    } catch (NoSuchAlgorithmException e) {
+      LOG.failedToAddCredentialForCluster( clusterName, e );
+    } catch (CertificateException e) {
+      LOG.failedToAddCredentialForCluster( clusterName, e );
+    } catch (IOException e) {
+      LOG.failedToAddCredentialForCluster( clusterName, e );
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultMasterService.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultMasterService.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultMasterService.java
index 87067bc..47368d0 100644
--- a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultMasterService.java
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/security/impl/DefaultMasterService.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.gateway.services.security.MasterService;
 import org.apache.hadoop.gateway.services.Service;
 import org.apache.hadoop.gateway.GatewayMessages;
 
-import java.io.File;
 import java.util.Map;
 
 public class DefaultMasterService extends CMFMasterService implements MasterService, Service {
@@ -47,7 +46,6 @@ public class DefaultMasterService extends CMFMasterService implements MasterServ
       setupMasterSecret(securityDir, filename, persisting);
     }
   }
-  
 
   @Override
   public void start() throws ServiceLifecycleException {

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-server/src/main/java/org/apache/hadoop/gateway/util/KnoxCLI.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/util/KnoxCLI.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/util/KnoxCLI.java
new file mode 100644
index 0000000..257b939
--- /dev/null
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/util/KnoxCLI.java
@@ -0,0 +1,477 @@
+/**
+ * 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.gateway.util;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.gateway.GatewayCommandLine;
+import org.apache.hadoop.gateway.config.GatewayConfig;
+import org.apache.hadoop.gateway.config.impl.GatewayConfigImpl;
+import org.apache.hadoop.gateway.services.CLIGatewayServices;
+import org.apache.hadoop.gateway.services.GatewayServices;
+import org.apache.hadoop.gateway.services.Service;
+import org.apache.hadoop.gateway.services.ServiceLifecycleException;
+import org.apache.hadoop.gateway.services.security.AliasService;
+import org.apache.hadoop.gateway.services.security.KeystoreService;
+import org.apache.hadoop.gateway.services.security.KeystoreServiceException;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.PropertyConfigurator;
+
+/**
+ *
+ */
+public class KnoxCLI extends Configured implements Tool {
+
+  private static final String USAGE_PREFIX = "KnoxCLI {cmd} [options]";
+  final static private String COMMANDS =
+      "   [--help]\n" +
+      "   [" + CertCreateCommand.USAGE + "]\n" +
+      "   [" + MasterCreateCommand.USAGE + "]\n" +
+      "   [" + AliasCreateCommand.USAGE + "]\n" +
+      "   [" + AliasDeleteCommand.USAGE + "]\n" +
+      "   [" + AliasListCommand.USAGE + "]\n";
+
+  /** allows stdout to be captured if necessary */
+  public PrintStream out = System.out;
+  /** allows stderr to be captured if necessary */
+  public PrintStream err = System.err;
+  
+  private static GatewayServices services = new CLIGatewayServices();
+  private Command command;
+  private String value = null;
+  private String cluster = null;
+  private String generate = "false";
+  private String hostname = null;
+  
+  // for testing only
+  private String master = null;
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+   */
+  @Override
+  public int run(String[] args) throws Exception {
+    int exitCode = 0;
+    try {
+      exitCode = init(args);
+      if (exitCode != 0) {
+        return exitCode;
+      }
+      if (command.validate()) {
+          initializeServices(command instanceof MasterCreateCommand);
+          command.execute();
+      } else {
+        exitCode = -1;
+      }
+    } catch (Exception e) {
+      e.printStackTrace(err);
+      return -1;
+    }
+    return exitCode;
+  }
+
+  private void initializeServices(boolean persisting) throws ServiceLifecycleException {
+    GatewayConfig config = new GatewayConfigImpl();
+    Map<String,String> options = new HashMap<String,String>();
+    options.put(GatewayCommandLine.PERSIST_LONG, Boolean.toString(persisting));
+    if (master != null) {
+      options.put("master", master);
+    }
+    services.init(config, options);
+  }
+
+  /**
+   * Parse the command line arguments and initialize the data
+   * <pre>
+   * % knox master-create keyName [--size size] [--generate]
+   *    [--provider providerPath]
+   * % knox create-alias alias [--value v]
+   * % knox list-alias [-provider providerPath]
+   * % knox delete=alias keyName [--provider providerPath] [-i]
+   * % knox create-cert keyName [--provider providerPath] [-i]
+   * </pre>
+   * @param args
+   * @return
+   * @throws IOException
+   */
+  private int init(String[] args) throws IOException {
+    for (int i = 0; i < args.length; i++) { // parse command line
+      if (args[i].equals("create-master")) {
+        command = new MasterCreateCommand();
+        if ((args.length > i + 1) && args[i + 1].equals("--help")) {
+          printKnoxShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("delete-alias")) {
+        String alias = args[++i];
+        command = new AliasDeleteCommand(alias);
+        if (alias.equals("--help")) {
+          printKnoxShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("create-alias")) {
+        String alias = args[++i];
+        command = new AliasCreateCommand(alias);
+        if (alias.equals("--help")) {
+          printKnoxShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("create-cert")) {
+        command = new CertCreateCommand();
+        if ((args.length > i + 1) && args[i + 1].equals("--help")) {
+          printKnoxShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("list-alias")) {
+        command = new AliasListCommand();
+      } else if (args[i].equals("--value")) {
+        this.value = args[++i];
+      } else if (args[i].equals("--cluster")) {
+        this.cluster = args[++i];
+      } else if (args[i].equals("--generate")) {
+        this.generate = "true";
+      } else if (args[i].equals("--hostname")) {
+        this.hostname = args[++i];
+      } else if (args[i].equals("--master")) {
+        // testing only
+        this.master = args[++i];
+      } else if (args[i].equals("--help")) {
+        printKnoxShellUsage();
+        return -1;
+      } else {
+        printKnoxShellUsage();
+        ToolRunner.printGenericCommandUsage(System.err);
+        return -1;
+      }
+    }
+    return 0;
+  }
+
+  private void printKnoxShellUsage() {
+    out.println(USAGE_PREFIX + COMMANDS);
+    if (command != null) {
+      out.println(command.getUsage());
+    }
+    else {
+      out.println("=========================================================" +
+          "======");
+      out.println(MasterCreateCommand.USAGE + ":\n\n" + MasterCreateCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(CertCreateCommand.USAGE + ":\n\n" + CertCreateCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(AliasCreateCommand.USAGE + ":\n\n" + AliasCreateCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(AliasDeleteCommand.USAGE + ":\n\n" + AliasDeleteCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(AliasListCommand.USAGE + ":\n\n" + AliasListCommand.DESC);
+    }
+  }
+
+  private abstract class Command {
+    protected Service provider = null;
+
+    public boolean validate() {
+      return true;
+    }
+
+    protected Service getService(String serviceName) {
+      Service service = null;
+
+      return service;
+    }
+
+    public abstract void execute() throws Exception;
+
+    public abstract String getUsage();
+
+    protected AliasService getAliasService() {
+      AliasService as = (AliasService) 
+           services.getService(GatewayServices.ALIAS_SERVICE);
+      return as;
+    }
+
+    protected KeystoreService getKeystoreService() {
+      KeystoreService ks = (KeystoreService) 
+           services.getService(GatewayServices.KEYSTORE_SERVICE);
+      return ks;
+    }
+  }
+  
+  /**
+  *
+  */
+ private class AliasListCommand extends Command {
+
+  public static final String USAGE = "list-alias [--cluster c]";
+  public static final String DESC = "The list-alias command lists all of the aliases\n" +
+  		                               "for the given hadoop --cluster. The default\n" +
+  		                               "--cluster being the gateway itself.";
+
+  /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#execute()
+    */
+   @Override
+   public void execute() throws Exception {
+     AliasService as = getAliasService();
+
+     if (cluster == null) {
+       cluster = "__gateway";
+     }
+     out.println("Listing aliases for: " + cluster);
+     List<String> aliases = as.getAliasesForCluster(cluster);
+     for (String alias : aliases) {
+       out.println(alias);
+     }
+     out.println("\n" + aliases.size() + " items.");
+   }
+
+   /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#getUsage()
+    */
+   @Override
+   public String getUsage() {
+     return USAGE + ":\n\n" + DESC;
+   }
+ }
+
+ /**
+  *
+  */
+ public class CertCreateCommand extends Command {
+
+  public static final String USAGE = "create-cert [--hostname h]";
+  public static final String DESC = "The create-cert command creates and populates\n" +
+  		                               "a gateway.jks keystore with a self-signed certificate\n" +
+  		                               "to be used as the gateway identity. It also adds an alias\n" +
+  		                               "to the __gateway-credentials.jceks credential store for the\n" +
+  		                               "key passphrase.";
+  private static final String GATEWAY_CREDENTIAL_STORE_NAME = "__gateway";
+  private static final String GATEWAY_IDENTITY_PASSPHRASE = "gateway-identity-passphrase";
+
+  /**
+    * 
+    */
+   public CertCreateCommand() {
+     // TODO Auto-generated constructor stub
+   }
+
+   /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#execute()
+    */
+   @Override
+   public void execute() throws Exception {
+     KeystoreService ks = getKeystoreService();
+     
+     AliasService as = getAliasService();
+     
+     if (ks != null) {
+       try {
+         if (!ks.isCredentialStoreForClusterAvailable(GATEWAY_CREDENTIAL_STORE_NAME)) {
+//           log.creatingCredentialStoreForGateway();
+           ks.createCredentialStoreForCluster(GATEWAY_CREDENTIAL_STORE_NAME);
+         }
+         else {
+//           log.credentialStoreForGatewayFoundNotCreating();
+         }
+         as.generateAliasForCluster(GATEWAY_CREDENTIAL_STORE_NAME, GATEWAY_IDENTITY_PASSPHRASE);
+       } catch (KeystoreServiceException e) {
+         throw new ServiceLifecycleException("Keystore was not loaded properly - the provided (or persisted) master secret may not match the password for the keystore.", e);
+       }
+  
+       try {
+         if (!ks.isKeystoreForGatewayAvailable()) {
+//           log.creatingKeyStoreForGateway();
+           ks.createKeystoreForGateway();
+         }
+         else {
+//           log.keyStoreForGatewayFoundNotCreating();
+         }
+         char[] passphrase = as.getPasswordFromAliasForCluster(GATEWAY_CREDENTIAL_STORE_NAME, GATEWAY_IDENTITY_PASSPHRASE);
+         ks.addSelfSignedCertForGateway("gateway-identity", passphrase, hostname);
+//         logAndValidateCertificate();
+         out.println("gateway-identity has been successfully created.");
+       } catch (KeystoreServiceException e) {
+         throw new ServiceLifecycleException("Keystore was not loaded properly - the provided (or persisted) master secret may not match the password for the keystore.", e);
+       }
+     }
+   }
+
+  /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#getUsage()
+    */
+   @Override
+   public String getUsage() {
+     return USAGE + ":\n\n" + DESC;
+   }
+
+ }
+
+ /**
+  *
+  */
+ public class AliasCreateCommand extends Command {
+
+  public static final String USAGE = "create-alias aliasname [--value value]" +
+  		                              " [--cluster c] [--generate]";
+  public static final String DESC = "The create-alias command will create an alias\n" +
+  		                               "and secret pair within the credential store for the\n" +
+  		                               "indicated --cluster otherwise within the gateway\n" +
+  		                               "credential store. The actual secret may be specified via\n" +
+  		                               "the --value option or --generate will create a random secret\n" +
+  		                               "for you.";
+  
+  private String name = null; 
+
+  /**
+    * @param alias
+    */
+   public AliasCreateCommand(String alias) {
+     name = alias;
+   }
+
+   /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#execute()
+    */
+   @Override
+   public void execute() throws Exception {
+     AliasService as = getAliasService();
+     if (cluster == null) {
+       cluster = "__gateway";
+     }
+     if (value != null) {
+       as.addAliasForCluster(cluster, name, value);
+       out.println(name + " has been successfully created.");
+     }
+     else {
+       if (generate.equals("true")) {
+         as.generateAliasForCluster(cluster, name);
+         out.println(name + " has been successfully generated.");
+       }
+       else {
+         throw new IllegalArgumentException("No value has been set. " +
+         		"Consider setting --generate or --value.");
+       }
+     }
+   }
+
+   /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#getUsage()
+    */
+   @Override
+   public String getUsage() {
+     return USAGE + ":\n\n" + DESC;
+   }
+
+ }
+
+ /**
+  *
+  */
+ public class AliasDeleteCommand extends Command {
+  public static final String USAGE = "delete-alias aliasname [--cluster c]";
+  public static final String DESC = "The delete-alias command removes the\n" +
+  		                               "indicated alias from the --cluster specific\n" +
+  		                               "credential store or the gateway credential store.";
+  
+  private String name = null;
+
+  /**
+    * @param alias
+    */
+   public AliasDeleteCommand(String alias) {
+     name = alias;
+   }
+
+   /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#execute()
+    */
+   @Override
+   public void execute() throws Exception {
+     AliasService as = getAliasService();
+     if (as != null) {
+       if (cluster == null) {
+         cluster = "__gateway";
+       }
+       as.removeAliasForCluster(cluster, name);
+       out.println(name + " has been successfully deleted.");
+     }
+   }
+
+   /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#getUsage()
+    */
+   @Override
+   public String getUsage() {
+     return USAGE + ":\n\n" + DESC;
+   }
+
+ }
+
+ /**
+  *
+  */
+ public class MasterCreateCommand extends Command {
+  public static final String USAGE = "create-master";
+  public static final String DESC = "The create-master command persists the\n" +
+  		                               "master secret in a file located at:\n" +
+  		                               "{GATEWAY_HOME}/data/security/master. It\n" +
+  		                               "will prompt the user for the secret to persist.";
+
+  /**
+    * @param keyName
+    */
+   public MasterCreateCommand() {
+   }
+
+   /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#execute()
+    */
+   @Override
+   public void execute() throws Exception {
+     out.println("Master secret has been persisted to disk.");
+   }
+
+   /* (non-Javadoc)
+    * @see org.apache.hadoop.gateway.util.KnoxCLI.Command#getUsage()
+    */
+   @Override
+   public String getUsage() {
+     return USAGE + ":\n\n" + DESC;
+   }
+ }
+
+ /**
+  * @param args
+  * @throws Exception 
+  */
+  public static void main(String[] args) throws Exception {
+    PropertyConfigurator.configure( System.getProperty( "log4j.configuration" ) );
+    int res = ToolRunner.run(new GatewayConfigImpl(), new KnoxCLI(), args);
+    System.exit(res);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-server/src/test/java/org/apache/hadoop/gateway/services/security/CryptoServiceTest.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/test/java/org/apache/hadoop/gateway/services/security/CryptoServiceTest.java b/gateway-server/src/test/java/org/apache/hadoop/gateway/services/security/CryptoServiceTest.java
index af6ba43..e284591 100644
--- a/gateway-server/src/test/java/org/apache/hadoop/gateway/services/security/CryptoServiceTest.java
+++ b/gateway-server/src/test/java/org/apache/hadoop/gateway/services/security/CryptoServiceTest.java
@@ -28,6 +28,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.security.cert.Certificate;
+import java.util.List;
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
@@ -92,6 +93,16 @@ public class CryptoServiceTest {
         // TODO Auto-generated method stub
         return null;
       }
+
+      @Override
+      public void removeAliasForCluster(String clusterName, String alias) {
+      }
+
+      @Override
+      public List<String> getAliasesForCluster(String clusterName) {
+        // TODO Auto-generated method stub
+        return null;
+      }
     };
     cs = new DefaultCryptoService().setAliasService(as);
   }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-server/src/test/java/org/apache/hadoop/gateway/util/KnoxCLITest.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/test/java/org/apache/hadoop/gateway/util/KnoxCLITest.java b/gateway-server/src/test/java/org/apache/hadoop/gateway/util/KnoxCLITest.java
new file mode 100644
index 0000000..fe19cdd
--- /dev/null
+++ b/gateway-server/src/test/java/org/apache/hadoop/gateway/util/KnoxCLITest.java
@@ -0,0 +1,143 @@
+/**
+ * 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.gateway.util;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import org.apache.hadoop.gateway.config.impl.GatewayConfigImpl;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * @author larry
+ *
+ */
+public class KnoxCLITest {
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private final ByteArrayOutputStream errContent = new ByteArrayOutputStream();
+
+  @Before
+  public void setup() throws Exception {
+    System.setOut(new PrintStream(outContent));
+    System.setErr(new PrintStream(errContent));
+  }
+  
+  @Test
+  public void testSuccessfulAlaisLifecycle() throws Exception {
+    outContent.reset();
+    String[] args1 = {"create-alias", "alias1", "--value", "testvalue1", "--master", "master"};
+    int rc = 0;
+    KnoxCLI cli = new KnoxCLI();
+    cli.setConf(new GatewayConfigImpl());
+    rc = cli.run(args1);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString(), outContent.toString().contains("alias1 has been successfully " +
+        "created."));
+
+    outContent.reset();
+    String[] args2 = {"list-alias", "--master", 
+        "master"};
+    rc = cli.run(args2);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString(), outContent.toString().contains("alias1"));
+
+    outContent.reset();
+    String[] args4 = {"delete-alias", "alias1", "--master", 
+      "master"};
+    rc = cli.run(args4);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString(), outContent.toString().contains("alias1 has been successfully " +
+        "deleted."));
+
+    outContent.reset();
+    rc = cli.run(args2);
+    assertEquals(0, rc);
+    assertFalse(outContent.toString(), outContent.toString().contains("alias1"));
+  }
+  
+  @Test
+  public void testGatewayAndClusterStores() throws Exception {
+    outContent.reset();
+    String[] gwCreateArgs = {"create-alias", "alias1", "--value", "testvalue1", "--master", "master"};
+    int rc = 0;
+    KnoxCLI cli = new KnoxCLI();
+    cli.setConf(new GatewayConfigImpl());
+    rc = cli.run(gwCreateArgs);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString(), outContent.toString().contains("alias1 has been successfully " +
+        "created."));
+
+    outContent.reset();
+    String[] clusterCreateArgs = {"create-alias", "alias2", "--value", "testvalue1", "--cluster", "test", 
+        "--master", "master"};
+    cli = new KnoxCLI();
+    cli.setConf(new GatewayConfigImpl());
+    rc = cli.run(clusterCreateArgs);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString(), outContent.toString().contains("alias2 has been successfully " +
+        "created."));
+
+    outContent.reset();
+    String[] args2 = {"list-alias", "--master", "master"};
+    cli = new KnoxCLI();
+    rc = cli.run(args2);
+    assertEquals(0, rc);
+    assertFalse(outContent.toString(), outContent.toString().contains("alias2"));
+    assertTrue(outContent.toString(), outContent.toString().contains("alias1"));
+
+    outContent.reset();
+    String[] args1 = {"list-alias", "--cluster", "test", "--master", "master"};
+    cli = new KnoxCLI();
+    rc = cli.run(args1);
+    assertEquals(0, rc);
+    assertFalse(outContent.toString(), outContent.toString().contains("alias1"));
+    assertTrue(outContent.toString(), outContent.toString().contains("alias2"));
+
+    outContent.reset();
+    String[] args4 = {"delete-alias", "alias1", "--master", "master"};
+    cli = new KnoxCLI();
+    rc = cli.run(args4);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString(), outContent.toString().contains("alias1 has been successfully " +
+        "deleted."));
+    
+    outContent.reset();
+    String[] args5 = {"delete-alias", "alias2", "--cluster", "test", "--master", "master"};
+    cli = new KnoxCLI();
+    rc = cli.run(args5);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString(), outContent.toString().contains("alias2 has been successfully " +
+        "deleted."));
+  }
+  
+  @Test
+  public void testCreateSelfSignedCert() throws Exception {
+    outContent.reset();
+    String[] gwCreateArgs = {"create-cert", "--hostname", "hostname1", "--master", "master"};
+    int rc = 0;
+    KnoxCLI cli = new KnoxCLI();
+    cli.setConf(new GatewayConfigImpl());
+    rc = cli.run(gwCreateArgs);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString(), outContent.toString().contains("gateway-identity has been successfully " +
+        "created."));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/GatewayServices.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/GatewayServices.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/GatewayServices.java
index 94ef9a8..2de2d66 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/GatewayServices.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/GatewayServices.java
@@ -30,6 +30,7 @@ public interface GatewayServices extends Service, ProviderDeploymentContributor
   public static final String SSL_SERVICE = "SSLService";
   public static final String CRYPTO_SERVICE = "CryptoService";
   public static final String ALIAS_SERVICE = "AliasService";
+  public static final String KEYSTORE_SERVICE = "KeystoreService";
   public static final String TOKEN_SERVICE = "TokenService";
   public static final String SERVICE_REGISTRY_SERVICE = "ServiceRegistryService";
   public static final String HOST_MAPPING_SERVICE = "HostMappingService";

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/AliasService.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/AliasService.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/AliasService.java
index a34cf29..70b36a7 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/AliasService.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/AliasService.java
@@ -18,13 +18,18 @@
 package org.apache.hadoop.gateway.services.security;
 
 import java.security.cert.Certificate;
+import java.util.List;
 
 import org.apache.hadoop.gateway.services.Service;
 
 public interface AliasService extends Service {
 
+  public abstract List<String> getAliasesForCluster(String clusterName);
+
   public abstract void addAliasForCluster(String clusterName, String alias, String value);
 
+  public abstract void removeAliasForCluster(String clusterName, String alias);
+
   public abstract char[] getPasswordFromAliasForCluster(String clusterName, String alias);
 
   public abstract char[] getPasswordFromAliasForCluster(String clusterName, String alias, boolean generate);

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/KeystoreService.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/KeystoreService.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/KeystoreService.java
index 9dab8ed..345ba44 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/KeystoreService.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/KeystoreService.java
@@ -26,6 +26,8 @@ public interface KeystoreService {
 
   public void addSelfSignedCertForGateway(String alias, char[] passphrase) throws KeystoreServiceException;
   
+  void addSelfSignedCertForGateway(String alias, char[] passphrase, String hostname);
+
   public KeyStore getKeystoreForGateway() throws KeystoreServiceException;
   
   public Key getKeyForGateway(String alias, char[] passphrase) throws KeystoreServiceException;
@@ -40,6 +42,7 @@ public interface KeystoreService {
 
   public void addCredentialForCluster(String clusterName, String alias, String key) throws KeystoreServiceException;
 
-  public char[] getCredentialForCluster(String clusterName, String alias) throws KeystoreServiceException;
+  public void removeCredentialForCluster(String clusterName, String alias);
 
+  public char[] getCredentialForCluster(String clusterName, String alias) throws KeystoreServiceException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/impl/BaseKeystoreService.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/impl/BaseKeystoreService.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/impl/BaseKeystoreService.java
index 2eafc2d..3abcb8e 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/impl/BaseKeystoreService.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/services/security/impl/BaseKeystoreService.java
@@ -218,6 +218,18 @@ public class BaseKeystoreService {
     }
   }
 
+  public void removeCredential(String alias, KeyStore ks) {
+    if (ks != null) {
+      try {
+        if (ks.containsAlias(alias)) {
+          ks.deleteEntry(alias);
+        }
+      } catch (KeyStoreException e) {
+        LOG.failedToAddCredential(e);
+      }
+    }
+  }
+
   protected char[] getCredential(String alias, char[] credential, KeyStore ks) {
     if (ks != null) {
       try {

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/knox-cli-launcher/pom.xml
----------------------------------------------------------------------
diff --git a/knox-cli-launcher/pom.xml b/knox-cli-launcher/pom.xml
new file mode 100644
index 0000000..eea51cc
--- /dev/null
+++ b/knox-cli-launcher/pom.xml
@@ -0,0 +1,68 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>gateway</artifactId>
+        <groupId>org.apache.hadoop</groupId>
+        <version>0.4.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>knox-cli-launcher</artifactId>
+    <name>knox-cli-launcher</name>
+    <description>Launcher for the knox server CLI.</description>
+
+    <dependencies>
+        <dependency>
+            <groupId>${gateway-group}</groupId>
+            <artifactId>gateway-util-launcher</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <inherited>false</inherited>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <version>2.4</version>
+                <executions>
+                    <execution>
+                        <id>knox-launcher</id>
+                        <phase>package</phase>
+                        <goals><goal>single</goal></goals>
+                        <configuration>
+                            <appendAssemblyId>false</appendAssemblyId>
+                            <descriptorRefs>
+                                <descriptorRef>jar-with-dependencies</descriptorRef>
+                            </descriptorRefs>
+                            <archive>
+                                <manifest>
+                                    <mainClass>org.apache.hadoop.gateway.launcher.Launcher</mainClass>
+                                </manifest>
+                            </archive>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/knox-cli-launcher/src/main/resources/META-INF/launcher.cfg
----------------------------------------------------------------------
diff --git a/knox-cli-launcher/src/main/resources/META-INF/launcher.cfg b/knox-cli-launcher/src/main/resources/META-INF/launcher.cfg
new file mode 100644
index 0000000..1a64b71
--- /dev/null
+++ b/knox-cli-launcher/src/main/resources/META-INF/launcher.cfg
@@ -0,0 +1,18 @@
+# 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.
+main.class = org.apache.hadoop.gateway.util.KnoxCLI
+class.path = ../lib/*.jar;../dep/*.jar;../ext;../ext/*.jar
+GATEWAY_HOME=${launcher.dir}/..
+log4j.configuration=${GATEWAY_HOME}/conf/${launcher.name}-log4j.properties
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/a9e7deae/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2c762f0..7b600dc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -61,6 +61,7 @@
         <module>gateway-service-tgs</module>
         <module>gateway-shell</module>
         <module>gateway-shell-launcher</module>
+        <module>knox-cli-launcher</module>
         <module>hadoop-examples</module>
         <module>gateway-release</module>
         <module>gateway-test</module>
@@ -452,6 +453,11 @@
             </dependency>
             <dependency>
                 <groupId>${gateway-group}</groupId>
+                <artifactId>knox-cli-launcher</artifactId>
+                <version>${gateway-version}</version>
+            </dependency>
+            <dependency>
+                <groupId>${gateway-group}</groupId>
                 <artifactId>gateway-shell</artifactId>
                 <version>${gateway-version}</version>
             </dependency>