You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by sm...@apache.org on 2015/03/16 17:48:48 UTC

[15/19] incubator-slider git commit: SLIDER-801 make client install command get SSL certs from AM when SSL is enabled

SLIDER-801 make client install command get SSL certs from AM when SSL is enabled


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

Branch: refs/heads/feature/package_simplification_II
Commit: 42f1fbdc7011a08a70a41d3c7b6f2d0cc4229203
Parents: d9b2652
Author: Billie Rinaldi <bi...@gmail.com>
Authored: Fri Mar 13 10:24:22 2015 -0700
Committer: Billie Rinaldi <bi...@gmail.com>
Committed: Fri Mar 13 10:36:46 2015 -0700

----------------------------------------------------------------------
 .../accumulo/clientInstallConfig-default.json   |   3 +-
 .../accumulo/package/scripts/accumulo_client.py |  48 +++++-
 .../accumulo/package/scripts/client_params.py   |  13 +-
 app-packages/accumulo/pom.xml                   |   3 +-
 .../funtest/accumulo/AccumuloScriptIT.groovy    |   9 +-
 .../funtest/accumulo/AccumuloScriptSSLIT.groovy | 147 +++++++++++++++++++
 .../resources/clientInstallConfig-ssl-test.json |   7 +
 .../resources/clientInstallConfig-test.json     |   8 -
 .../src/test/resources/test_shell_cmd_file      |  20 +++
 9 files changed, 240 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/clientInstallConfig-default.json
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/clientInstallConfig-default.json b/app-packages/accumulo/clientInstallConfig-default.json
index 02176dc..db01ac4 100644
--- a/app-packages/accumulo/clientInstallConfig-default.json
+++ b/app-packages/accumulo/clientInstallConfig-default.json
@@ -1,7 +1,6 @@
 {
   "schema":"http://example.org/specification/v2.0.0",
   "global":{
-    "client_root": "{app_install_dir}/accumulo-${accumulo.version}",
-    "slider_home_dir": "${slider.dir}"
+    "client_root": "{app_install_dir}/accumulo-${accumulo.version}"
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/package/scripts/accumulo_client.py
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/package/scripts/accumulo_client.py b/app-packages/accumulo/package/scripts/accumulo_client.py
index 2da02dc..31ef684 100644
--- a/app-packages/accumulo/package/scripts/accumulo_client.py
+++ b/app-packages/accumulo/package/scripts/accumulo_client.py
@@ -21,12 +21,19 @@ limitations under the License.
 from resource_management import *
 
 class AccumuloClient(Script):
+  def check_provider_contains(self, provider, alias):
+    try:
+      Execute( format("hadoop credential list -provider {provider} | "
+                      "grep -i {alias}"))
+    except:
+      raise Fail(format("{provider} did not contain {alias}, try running "
+                        "'hadoop credential create {alias} -provider "
+                        "{provider}' or configure SSL certs manually"))
+
   def install(self, env):
     import client_params
     env.set_params(client_params)
     self.install_packages(env)
-    Directory(client_params.conf_dir,
-              content=format("{conf_dir}/templates"))
     jarname = "SliderAccumuloUtils.jar"
     File(format("{client_root}/lib/{jarname}"),
          mode=0644,
@@ -44,8 +51,45 @@ class AccumuloClient(Script):
                    mode=0755
     )
     if client_params.app_name:
+      Logger.info("Creating configs for app %s" % client_params.app_name)
+      Directory(client_params.conf_dir,
+                content=format("{conf_dir}/templates"))
       Execute( format("{bin_dir}/accumulo-slider "
                       "--appconf {client_root}/conf --app {app_name} getconf "))
+      configs = {}
+      with open(format("{client_root}/conf/client.conf"),"r") as fp:
+        content = fp.readlines()
+        for line in content:
+          index = line.find("=")
+          if index > 0:
+            configs[line[0:index]] = line[index+1:]
+      if 'instance.rpc.ssl.enabled' in configs and configs['instance.rpc.ssl.enabled']=='true':
+        Logger.info("Configuring client SSL")
+        self.check_provider_contains(client_params.credential_provider,
+                                     client_params.keystore_alias)
+        self.check_provider_contains(client_params.credential_provider,
+                                     client_params.truststore_alias)
+        configs['general.security.credential.provider.paths'] = client_params.credential_provider
+        configs['rpc.javax.net.ssl.keyStore'] = client_params.keystore_path
+        configs['rpc.javax.net.ssl.keyStoreType'] = client_params.store_type
+        configs['rpc.javax.net.ssl.trustStore'] = client_params.truststore_path
+        configs['rpc.javax.net.ssl.trustStoreType'] = client_params.store_type
+        PropertiesFile(format("{client_root}/conf/client.conf"),
+                       properties = configs
+        )
+        Execute( format("SLIDER_CONF_DIR={slider_conf_dir} "
+                        "{slider_home_dir}/bin/slider client --getcertstore "
+                        "--keystore {keystore_path} "
+                        "--name {app_name} --alias {keystore_alias} "
+                        "--provider {credential_provider}"))
+        Execute( format("SLIDER_CONF_DIR={slider_conf_dir} "
+                        "{slider_home_dir}/bin/slider client --getcertstore "
+                        "--truststore {truststore_path} "
+                        "--name {app_name} --alias {truststore_alias} "
+                        "--provider {credential_provider}"))
+    else:
+      Logger.info("No app name provided, leaving client install unconfigured")
+
 
   def configure(self, env):
     pass

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/package/scripts/client_params.py
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/package/scripts/client_params.py b/app-packages/accumulo/package/scripts/client_params.py
index 1fc5219..70d3d9a 100644
--- a/app-packages/accumulo/package/scripts/client_params.py
+++ b/app-packages/accumulo/package/scripts/client_params.py
@@ -23,7 +23,6 @@ import os
 
 config = Script.get_config()
 
-app_install_dir = config['configurations']['global']['app_install_dir']
 client_root = config['configurations']['global']['client_root']
 bin_dir = os.path.join(client_root, 'bin')
 conf_dir = os.path.join(client_root, 'conf')
@@ -32,8 +31,18 @@ app_name = None
 if 'app_name' in config['configurations']['global']:
   app_name = config['configurations']['global']['app_name']
 
-slider_home_dir = config['configurations']['global']['slider_home_dir']
+if os.environ.has_key('SLIDER_HOME'):
+  slider_home_dir = os.environ.get('SLIDER_HOME')
+else:
+  slider_home_dir = config['configurations']['global']['slider_home_dir']
 if os.environ.has_key('SLIDER_CONF_DIR'):
   slider_conf_dir = os.environ.get('SLIDER_CONF_DIR')
 else:
   slider_conf_dir = os.path.join(slider_home_dir, 'conf')
+
+keystore_path = format("{client_root}/conf/keystore.p12")
+keystore_alias = "rpc.javax.net.ssl.keyStorePassword"
+truststore_path = format("{client_root}/conf/truststore.p12")
+truststore_alias = "rpc.javax.net.ssl.trustStorePassword"
+store_type = "PKCS12"
+credential_provider = config['configurations']['global']['general.security.credential.provider.paths']

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/pom.xml b/app-packages/accumulo/pom.xml
index a04a2b0..022b499 100644
--- a/app-packages/accumulo/pom.xml
+++ b/app-packages/accumulo/pom.xml
@@ -48,8 +48,6 @@
     <app.java.home>${java.home}</app.java.home>
     <app.user>yarn</app.user>
     <app.user.group>hadoop</app.user.group>
-    <!-- default slider home for clientInstallConfig -->
-    <slider.dir>${slider.bin.dir}</slider.dir>
     <!-- these are for accumulo processes -->
     <accumulo.keytab />
     <accumulo.principal />
@@ -243,6 +241,7 @@
         <includes>
           <include>appConfig-default.json</include>
           <include>appConfig-ssl-default.json</include>
+          <include>clientInstallConfig-default.json</include>
         </includes>
       </resource>
     </resources>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptIT.groovy b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptIT.groovy
index 4fea600..be18609 100644
--- a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptIT.groovy
+++ b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptIT.groovy
@@ -31,11 +31,12 @@ import org.junit.BeforeClass
 class AccumuloScriptIT extends AccumuloBasicIT {
   public static final String CLIENT_INSTALL_DIR = sysprop("test.client.install.dir")
   public static final String CLIENT_HOME_DIR = sysprop("test.client.home.dir")
-  public static final String CLIENT_INSTALL_CONF = sysprop("test.app.resources.dir") + "/clientInstallConfig-test.json"
+  public static final String CLIENT_INSTALL_CONF = sysprop("test.app.resources.dir") + "/clientInstallConfig-default.json"
 
   public static final File ACCUMULO_SLIDER_SCRIPT = new File(CLIENT_HOME_DIR + "/bin", "accumulo-slider").canonicalFile
   public static final File ACCUMULO_SCRIPT = new File(CLIENT_HOME_DIR + "/bin", "accumulo").canonicalFile
   public static final File ACCUMULO_TOOL_SCRIPT = new File(CLIENT_HOME_DIR + "/bin", "tool.sh").canonicalFile
+  public static final File ACCUMULO_CLIENT_CONF = new File(CLIENT_HOME_DIR + "/conf", "client.conf").canonicalFile
 
   @Override
   public String getClusterName() {
@@ -77,10 +78,14 @@ class AccumuloScriptIT extends AccumuloBasicIT {
         ACTION_CLIENT, ARG_INSTALL,
         ARG_PACKAGE, TEST_APP_PKG_DIR+"/"+TEST_APP_PKG_FILE,
         ARG_DEST, CLIENT_INSTALL_DIR,
-        ARG_CONFIG, CLIENT_INSTALL_CONF
+        ARG_CONFIG, CLIENT_INSTALL_CONF,
+        ARG_NAME, clusterName
       ])
     logShell(shell)
 
+    assert ACCUMULO_CLIENT_CONF.exists(), "client.conf did not exist after " +
+      "client install command"
+
     accumulo_slider("--app $clusterName quicklinks")
 
     accumulo("shell -u $USER -p $PASSWORD -e tables")

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptSSLIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptSSLIT.groovy b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptSSLIT.groovy
new file mode 100644
index 0000000..08d39f2
--- /dev/null
+++ b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloScriptSSLIT.groovy
@@ -0,0 +1,147 @@
+/*
+ * 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.slider.funtest.accumulo
+
+import groovy.util.logging.Slf4j
+import org.apache.accumulo.core.conf.Property
+import org.apache.accumulo.examples.simple.helloworld.InsertWithBatchWriter
+import org.apache.accumulo.examples.simple.helloworld.ReadData
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.registry.client.api.RegistryConstants
+import org.apache.hadoop.security.ProviderUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.security.alias.CredentialProvider
+import org.apache.hadoop.security.alias.CredentialProviderFactory
+import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
+import org.apache.slider.funtest.framework.AccumuloSliderShell
+import org.apache.slider.funtest.framework.FuntestProperties
+import org.apache.slider.funtest.framework.SliderShell
+import org.codehaus.jettison.json.JSONObject
+import org.junit.Before
+import org.junit.BeforeClass
+
+import java.nio.charset.Charset
+import java.nio.file.Files
+
+@Slf4j
+class AccumuloScriptSSLIT extends AccumuloSSLTestBase {
+  public static final String CLIENT_INSTALL_DIR = sysprop("test.client.install.dir")
+  public static final String CLIENT_HOME_DIR = sysprop("test.client.home.dir")
+  public static final String CLIENT_INSTALL_CONF = sysprop("test.app.resources.dir") + "/clientInstallConfig-ssl-test.json"
+
+  public static final File ACCUMULO_SCRIPT = new File(CLIENT_HOME_DIR + "/bin", "accumulo").canonicalFile
+  public static final File ACCUMULO_CLIENT_CONF = new File(CLIENT_HOME_DIR + "/conf", "client.conf").canonicalFile
+
+  protected String jks
+  protected Path jksPath
+
+  @Override
+  public String getClusterName() {
+    return "test_script_ssl";
+  }
+
+  @Override
+  public String getDescription() {
+    return "Test accumulo-slider client script with ssl $clusterName"
+  }
+
+  @BeforeClass
+  public static void setShell() {
+    AccumuloSliderShell.setEnv("SLIDER_HOME", SLIDER_TAR_DIR)
+    AccumuloSliderShell.setEnv("SLIDER_CONF_DIR", SLIDER_CONF_DIR)
+  }
+
+  @Before
+  public void readProviderLocation() {
+    byte[] encoded = Files.readAllBytes(new File(CLIENT_INSTALL_CONF).toPath());
+    JSONObject config = new JSONObject(new String(encoded, Charset.defaultCharset()))
+    JSONObject global = config.getJSONObject("global")
+    jks = global.get("general.security.credential.provider.paths")
+    jks = jks.replace("{app_name}", getClusterName())
+    jks = jks.replace("{app_user}", UserGroupInformation.getCurrentUser().getShortUserName())
+    jksPath = ProviderUtils.unnestUri(new URI(jks))
+  }
+
+  public static AccumuloSliderShell accumulo(String cmd) {
+    AccumuloSliderShell.scriptFile = ACCUMULO_SCRIPT
+    return AccumuloSliderShell.run(0, cmd)
+  }
+
+  public void createClientKeyStore() {
+    Configuration conf = loadSliderConf()
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, jks)
+    CredentialProvider provider =
+      CredentialProviderFactory.getProviders(conf).get(0)
+    provider.createCredentialEntry(Property.RPC_SSL_KEYSTORE_PASSWORD
+      .toString(), KEY_PASS.toCharArray())
+    provider.createCredentialEntry(Property.RPC_SSL_TRUSTSTORE_PASSWORD
+      .toString(), TRUST_PASS.toCharArray())
+    provider.flush()
+    assert clusterFS.exists(jksPath), "jks $jks not created"
+    log.info("Created credential provider $jks for test")
+  }
+
+  @Override
+  public void clusterLoadOperations(ClusterDescription cd, SliderClient sliderClient) {
+    if (clusterFS.exists(jksPath)) {
+      clusterFS.delete(jksPath, false)
+    }
+
+    // this throws an error because the store passwords haven't been set up
+    SliderShell shell = slider(EXIT_EXCEPTION_THROWN,
+      [
+        ACTION_CLIENT, ARG_INSTALL,
+        ARG_PACKAGE, TEST_APP_PKG_DIR+"/"+TEST_APP_PKG_FILE,
+        ARG_DEST, CLIENT_INSTALL_DIR,
+        ARG_CONFIG, CLIENT_INSTALL_CONF,
+        ARG_NAME, getClusterName()
+      ])
+    logShell(shell)
+
+    // set up the store passwords
+    createClientKeyStore()
+
+    // now install the client
+    shell = slider(EXIT_SUCCESS,
+      [
+        ACTION_CLIENT, ARG_INSTALL,
+        ARG_PACKAGE, TEST_APP_PKG_DIR+"/"+TEST_APP_PKG_FILE,
+        ARG_DEST, CLIENT_INSTALL_DIR,
+        ARG_CONFIG, CLIENT_INSTALL_CONF,
+        ARG_NAME, getClusterName()
+      ])
+    logShell(shell)
+
+    assert ACCUMULO_CLIENT_CONF.exists(), "client.conf did not exist after " +
+      "client install command"
+
+    accumulo("shell -u $USER -p $PASSWORD -e tables")
+    accumulo("shell -u $USER -p $PASSWORD -f " + sysprop("test.app.resources.dir") + "/test_shell_cmd_file")
+
+    String zookeepers = SLIDER_CONFIG.get(
+      RegistryConstants.KEY_REGISTRY_ZK_QUORUM,
+      FuntestProperties.DEFAULT_SLIDER_ZK_HOSTS)
+    String instance = tree.global.get("site.client.instance.name")
+    accumulo("shell -u $USER -p $PASSWORD -e \"createtable test2\"")
+    accumulo(InsertWithBatchWriter.class.getName() + " -i $instance -z " +
+      "$zookeepers -u $USER -p $PASSWORD -t test2")
+    accumulo(ReadData.class.getName() + " -i $instance -z $zookeepers -u " +
+      "$USER -p $PASSWORD -t test2 --startKey row_0 --endKey row_101")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/src/test/resources/clientInstallConfig-ssl-test.json
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/resources/clientInstallConfig-ssl-test.json b/app-packages/accumulo/src/test/resources/clientInstallConfig-ssl-test.json
new file mode 100644
index 0000000..df0d9c7
--- /dev/null
+++ b/app-packages/accumulo/src/test/resources/clientInstallConfig-ssl-test.json
@@ -0,0 +1,7 @@
+{
+  "schema":"http://example.org/specification/v2.0.0",
+  "global":{
+    "client_root": "{app_install_dir}/accumulo-${accumulo.version}",
+    "general.security.credential.provider.paths": "jceks://hdfs/user/{app_user}/accumulo-{app_name}-client.jceks"
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/src/test/resources/clientInstallConfig-test.json
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/resources/clientInstallConfig-test.json b/app-packages/accumulo/src/test/resources/clientInstallConfig-test.json
deleted file mode 100644
index c8431ce..0000000
--- a/app-packages/accumulo/src/test/resources/clientInstallConfig-test.json
+++ /dev/null
@@ -1,8 +0,0 @@
-{
-  "schema":"http://example.org/specification/v2.0.0",
-  "global":{
-    "client_root": "{app_install_dir}/accumulo-${accumulo.version}",
-    "slider_home_dir": "${slider.dir}",
-    "app_name": "test_script"
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/42f1fbdc/app-packages/accumulo/src/test/resources/test_shell_cmd_file
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/resources/test_shell_cmd_file b/app-packages/accumulo/src/test/resources/test_shell_cmd_file
new file mode 100644
index 0000000..4c7f756
--- /dev/null
+++ b/app-packages/accumulo/src/test/resources/test_shell_cmd_file
@@ -0,0 +1,20 @@
+# 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.
+createtable test1
+insert a b c d
+scan
+flush -w
+scan
+deletetable -f test1