You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by jm...@apache.org on 2015/05/22 00:24:22 UTC

incubator-slider git commit: SLIDER-888 synchronize access to key provider during cert store creation

Repository: incubator-slider
Updated Branches:
  refs/heads/develop 6263573dc -> dae41ee06
Updated Tags:  refs/tags/HDP-2.2.3.0 [created] a40cdecc3
  refs/tags/HDP-2.2.4.0 [created] a40cdecc3
  refs/tags/HDP-2.2.4.2 [created] a40cdecc3
  refs/tags/HDP-2.2.5.0 [created] a40cdecc3


SLIDER-888 synchronize access to key provider during cert store creation


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

Branch: refs/heads/develop
Commit: dae41ee066fc297b43c0b0b78b81fa0d1bcac8f5
Parents: 6263573
Author: Jon Maron <jm...@hortonworks.com>
Authored: Thu May 21 18:23:59 2015 -0400
Committer: Jon Maron <jm...@hortonworks.com>
Committed: Thu May 21 18:23:59 2015 -0400

----------------------------------------------------------------------
 .../AbstractSecurityStoreGenerator.java         |  23 ++-
 .../TestMultiThreadedStoreGeneration.java       | 156 +++++++++++++++++++
 2 files changed, 172 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/dae41ee0/slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java b/slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java
index 04daaf4..11d3aa1 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java
@@ -17,6 +17,7 @@
 package org.apache.slider.server.services.security;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.slider.common.SliderKeys;
 import org.apache.slider.core.conf.MapOperations;
@@ -49,6 +50,7 @@ public abstract class AbstractSecurityStoreGenerator implements
     if (password == null) {
       // need to leverage credential provider
       String alias = getAlias(compOps);
+      LOG.debug("Alias {} found for role {}", alias, role);
       if (alias == null) {
         throw new SliderException("No store password or credential provider "
                                   + "alias found");
@@ -57,13 +59,20 @@ public abstract class AbstractSecurityStoreGenerator implements
         LOG.info("Credentials can not be retrieved for store generation since "
                  + "no CP paths are configured");
       }
-      for (Map.Entry<String, List<String>> cred : credentials.entrySet()) {
-        String provider = cred.getKey();
-        Configuration c = new Configuration();
-        c.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, provider);
-        char[] credential = c.getPassword(alias);
-        if (credential != null) {
-          return String.valueOf(credential);
+      synchronized (this) {
+        for (Map.Entry<String, List<String>> cred : credentials.entrySet()) {
+          String provider = cred.getKey();
+          Configuration c = new Configuration();
+          c.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, provider);
+          LOG.debug("Configured provider {}", provider);
+          CredentialProvider cp =
+              CredentialProviderFactory.getProviders(c).get(0);
+          LOG.debug("Aliases: {}", cp.getAliases());
+          char[] credential = c.getPassword(alias);
+          if (credential != null) {
+            LOG.info("Credential found for role {}", role);
+            return String.valueOf(credential);
+          }
         }
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/dae41ee0/slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java b/slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java
new file mode 100644
index 0000000..2e2ffce
--- /dev/null
+++ b/slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java
@@ -0,0 +1,156 @@
+/*
+ * 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.server.services.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ *
+ */
+public class TestMultiThreadedStoreGeneration {
+
+  public static final int NUM_THREADS = 30;
+  @Rule
+  public TemporaryFolder workDir = new TemporaryFolder();;
+
+  private void setupCredentials(AggregateConf instanceDefinition,
+                                String keyAlias, String trustAlias)
+      throws Exception {
+    Configuration conf = new Configuration();
+    final Path jksPath = new Path(SecurityUtils.getSecurityDir(), "test.jks");
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+    File file = new File(SecurityUtils.getSecurityDir(), "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    instanceDefinition.getAppConf().credentials.put(ourUrl, new ArrayList<String>());
+
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+
+    // create new aliases
+    try {
+
+      if (keyAlias != null) {
+        char[] storepass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            keyAlias, storepass);
+      }
+
+      if (trustAlias != null) {
+        char[] trustpass = {'t', 'r', 'u', 's', 't', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            trustAlias, trustpass);
+      }
+
+      // write out so that it can be found in checks
+      provider.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+
+  @Test
+  public void testMultiThreadedStoreGeneration() throws Exception {
+
+    CertificateManager certMan = new CertificateManager();
+    MapOperations compOperations = new MapOperations();
+    File secDir = new File(workDir.getRoot(), SliderKeys.SECURITY_DIR);
+    File keystoreFile = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    compOperations.put(SliderXmlConfKeys.KEY_KEYSTORE_LOCATION,
+                       keystoreFile.getAbsolutePath());
+    certMan.initialize(compOperations, "cahost", null, null);
+
+    final CountDownLatch latch = new CountDownLatch(1);
+    final List<SecurityStore> stores = new ArrayList<>();
+    List<Thread> threads = new ArrayList<>();
+    final AggregateConf instanceDefinition = new AggregateConf();
+
+    setupCredentials(instanceDefinition,
+                     SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_DEFAULT, null);
+    final MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+
+    for (int i=0; i<NUM_THREADS; ++i) {
+      final int finalI = i;
+      Runnable runner = new Runnable() {
+        public void run() {
+          System.out.println ("----> In run");
+          try {
+            latch.await();
+            SecurityStore[] stores1 = StoresGenerator.generateSecurityStores(
+                "testhost",
+                "container" + finalI,
+                "component" + finalI,
+                instanceDefinition,
+                compOps);
+            System.out.println ("----> stores1" + stores1);
+            List<SecurityStore>
+                securityStores =
+                Arrays.asList(stores1);
+            stores.addAll(securityStores);
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          } catch (SliderException e) {
+            e.printStackTrace();
+          } catch (IOException e) {
+            e.printStackTrace();
+          } catch (Exception e) {
+            e.printStackTrace();
+          }
+        }
+      };
+      Thread thread = new Thread(runner, "TestThread" + i);
+      threads.add(thread);
+      thread.start();
+    }
+    latch.countDown();
+    for (Thread t : threads) {
+      t.join();
+    }
+
+    for (int i=0; i < NUM_THREADS; i++) {
+      assertTrue("keystore " + i + " not generated", stores.get(i).getFile().exists());
+    }
+  }
+
+}