You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ja...@apache.org on 2016/03/04 18:23:32 UTC

sqoop git commit: SQOOP-2845: Sqoop2: Derive keystore password from a script passed to configuration

Repository: sqoop
Updated Branches:
  refs/heads/sqoop2 506104b0c -> 7bf9eacfc


SQOOP-2845: Sqoop2: Derive keystore password from a script passed to configuration

(Abraham Fine via Jarek Jarcec Cecho)


Project: http://git-wip-us.apache.org/repos/asf/sqoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/sqoop/commit/7bf9eacf
Tree: http://git-wip-us.apache.org/repos/asf/sqoop/tree/7bf9eacf
Diff: http://git-wip-us.apache.org/repos/asf/sqoop/diff/7bf9eacf

Branch: refs/heads/sqoop2
Commit: 7bf9eacfcbfda56dd02d979e7eed1b5e79a48081
Parents: 506104b
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Fri Mar 4 09:23:02 2016 -0800
Committer: Jarek Jarcec Cecho <ja...@apache.org>
Committed: Fri Mar 4 09:23:02 2016 -0800

----------------------------------------------------------------------
 .../sqoop/security/SecurityConstants.java       |  18 ++-
 .../apache/sqoop/server/SqoopJettyServer.java   |  60 +++++++-
 .../apache/sqoop/server/common/ServerError.java |   3 +
 .../test/infrastructure/SqoopTestCase.java      |   2 +-
 .../integration/serverproperties/SslTest.java   | 138 ++++++++++++++-----
 5 files changed, 179 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/7bf9eacf/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java b/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
index 91a1b8b..0241c86 100644
--- a/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
+++ b/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
@@ -187,12 +187,20 @@ public final class SecurityConstants {
 
   /**
    * The config specifies the password of the JKS formatted keystore
-   * <tt>org.apache.sqoop.security.tls.keystorepassword</tt>.
+   * <tt>org.apache.sqoop.security.tls.keystore_password</tt>.
    */
   public static final String KEYSTORE_PASSWORD =
           PREFIX_TLS_CONFIG + "keystore_password";
 
   /**
+   * The config specifies a script that outputs the password of the JKS formatted keystore
+   * to standard out
+   * <tt>org.apache.sqoop.security.tls.keystore_password_generator</tt>.
+   */
+  public static final String KEYSTORE_PASSWORD_GENERATOR =
+    PREFIX_TLS_CONFIG + "keystore_password_generator";
+
+  /**
    * The config specifies the password for the private key in the JKS formatted
    * keystore
    * <tt>org.apache.sqoop.security.tls.keymanagerpassword</tt>.
@@ -201,6 +209,14 @@ public final class SecurityConstants {
     PREFIX_TLS_CONFIG + "keymanager_password";
 
   /**
+   * The config specifies a script that outputs the password for the
+   * private key in the JKS formatted keystore to standard out
+   * <tt>org.apache.sqoop.security.tls.keymanager_password_generator</tt>.
+   */
+  public static final String KEYMANAGER_PASSWORD_GENERATOR =
+    PREFIX_TLS_CONFIG + "keymanager_password_generator";
+
+  /**
    * The config specifies the token kind in delegation token.
    */
   public static final String TOKEN_KIND = "sqoop_token_kind";

http://git-wip-us.apache.org/repos/asf/sqoop/blob/7bf9eacf/server/src/main/java/org/apache/sqoop/server/SqoopJettyServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/SqoopJettyServer.java b/server/src/main/java/org/apache/sqoop/server/SqoopJettyServer.java
index 60368af..4696a87 100644
--- a/server/src/main/java/org/apache/sqoop/server/SqoopJettyServer.java
+++ b/server/src/main/java/org/apache/sqoop/server/SqoopJettyServer.java
@@ -18,6 +18,7 @@
 
 package org.apache.sqoop.server;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 import org.apache.sqoop.common.MapContext;
 import org.apache.sqoop.common.SqoopException;
@@ -26,18 +27,28 @@ import org.apache.sqoop.core.SqoopServer;
 import org.apache.sqoop.filter.SqoopAuthenticationFilter;
 import org.apache.sqoop.security.SecurityConstants;
 import org.apache.sqoop.server.common.ServerError;
-import org.apache.sqoop.server.v1.*;
+import org.apache.sqoop.server.v1.AuthorizationServlet;
+import org.apache.sqoop.server.v1.ConfigurableServlet;
+import org.apache.sqoop.server.v1.ConnectorServlet;
+import org.apache.sqoop.server.v1.DriverServlet;
+import org.apache.sqoop.server.v1.JobServlet;
+import org.apache.sqoop.server.v1.LinkServlet;
+import org.apache.sqoop.server.v1.SubmissionsServlet;
 import org.eclipse.jetty.server.HttpConfiguration;
 import org.eclipse.jetty.server.HttpConnectionFactory;
 import org.eclipse.jetty.server.SecureRequestCustomizer;
 import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.server.SslConnectionFactory;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.eclipse.jetty.util.thread.ExecutorThreadPool;
-import org.eclipse.jetty.server.ServerConnector;
 
 import javax.servlet.DispatcherType;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
 import java.util.EnumSet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
@@ -86,13 +97,37 @@ public class SqoopJettyServer {
       }
 
       String keyStorePassword = configurationContext.getString(SecurityConstants.KEYSTORE_PASSWORD);
-      if (keyStorePassword != null && keyStorePassword.length() > 0) {
+      String keyStorePasswordGenerator = configurationContext.getString(SecurityConstants.KEYSTORE_PASSWORD_GENERATOR);
+      if (StringUtils.isNotBlank(keyStorePassword)) {
+        if (StringUtils.isNotBlank(keyStorePasswordGenerator)) {
+          LOG.warn(SecurityConstants.KEYSTORE_PASSWORD + " and " + SecurityConstants.KEYSTORE_PASSWORD_GENERATOR
+            + "are both set, using " + SecurityConstants.KEYSTORE_PASSWORD);
+        }
         sslContextFactory.setKeyStorePassword(keyStorePassword);
+      } else if (StringUtils.isNotBlank(keyStorePasswordGenerator)) {
+        try {
+          String passwordFromGenerator = readPasswordFromGenerator(keyStorePasswordGenerator);
+          sslContextFactory.setKeyStorePassword(passwordFromGenerator);
+        } catch (IOException exception) {
+          throw new SqoopException(ServerError.SERVER_0008, "failed to execute generator: " + SecurityConstants.KEYSTORE_PASSWORD_GENERATOR, exception);
+        }
       }
 
       String keyManagerPassword = configurationContext.getString(SecurityConstants.KEYMANAGER_PASSWORD);
-      if (keyManagerPassword != null && keyManagerPassword.length() > 0) {
+      String keyManagerPasswordGenerator = configurationContext.getString(SecurityConstants.KEYMANAGER_PASSWORD_GENERATOR);
+      if (StringUtils.isNotBlank(keyManagerPassword)) {
         sslContextFactory.setKeyManagerPassword(keyManagerPassword);
+        if (StringUtils.isNotBlank(keyManagerPasswordGenerator)) {
+          LOG.warn(SecurityConstants.KEYMANAGER_PASSWORD + " and " + SecurityConstants.KEYMANAGER_PASSWORD_GENERATOR
+            + "are both set, using " + SecurityConstants.KEYMANAGER_PASSWORD);
+        }
+      } else if (StringUtils.isNotBlank(keyManagerPasswordGenerator)) {
+        try {
+          String passwordFromGenerator = readPasswordFromGenerator(keyManagerPasswordGenerator);
+          sslContextFactory.setKeyManagerPassword(passwordFromGenerator);
+        } catch (IOException exception) {
+          throw new SqoopException(ServerError.SERVER_0008, "failed to execute generator: " + SecurityConstants.KEYMANAGER_PASSWORD_GENERATOR, exception);
+        }
       }
 
       HttpConfiguration https = new HttpConfiguration();
@@ -111,6 +146,21 @@ public class SqoopJettyServer {
     webServer.setHandler(createServletContextHandler());
   }
 
+  private String readPasswordFromGenerator(String generatorCommand) throws IOException {
+    ProcessBuilder processBuilder = new ProcessBuilder("/bin/sh", "-c", generatorCommand);
+    Process process = processBuilder.start();
+    String output;
+    try (
+      InputStreamReader inputStreamReader = new InputStreamReader(process.getInputStream(), Charset.forName("UTF-8"));
+      BufferedReader bufferedReader = new BufferedReader(inputStreamReader);
+    ) {
+      output =  bufferedReader.readLine();
+    } catch(IOException exception) {
+      throw exception;
+    }
+    return output;
+  }
+
   public synchronized void startServer() {
     try {
       webServer.start();
@@ -162,7 +212,7 @@ public class SqoopJettyServer {
     return context;
   }
 
-  public static void main(String[] args) {
+  public static void main(String[] args){
     SqoopJettyServer sqoopJettyServer = new SqoopJettyServer();
     sqoopJettyServer.startServer();
     sqoopJettyServer.joinServerThread();

http://git-wip-us.apache.org/repos/asf/sqoop/blob/7bf9eacf/server/src/main/java/org/apache/sqoop/server/common/ServerError.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/common/ServerError.java b/server/src/main/java/org/apache/sqoop/server/common/ServerError.java
index a644e9f..e820951 100644
--- a/server/src/main/java/org/apache/sqoop/server/common/ServerError.java
+++ b/server/src/main/java/org/apache/sqoop/server/common/ServerError.java
@@ -41,6 +41,9 @@ public enum ServerError implements ErrorCode {
 
   /** TLS enabled but keystore location not set*/
   SERVER_0007("TLS enabled but keystore location not set"),
+
+  /** Execution of password generator failed*/
+  SERVER_0008("Execution of password generator failed"),
   ;
 
   private final String message;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/7bf9eacf/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java b/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
index 9e1c683..5602023 100644
--- a/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
+++ b/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
@@ -115,7 +115,7 @@ public class SqoopTestCase implements ITest {
 
   private SqoopClient client;
 
-  private DelegationTokenAuthenticatedURL.Token authToken = new DelegationTokenAuthenticatedURL.Token();
+  protected DelegationTokenAuthenticatedURL.Token authToken = new DelegationTokenAuthenticatedURL.Token();
 
   protected FileSystem hdfsClient;
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/7bf9eacf/test/src/test/java/org/apache/sqoop/integration/serverproperties/SslTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/serverproperties/SslTest.java b/test/src/test/java/org/apache/sqoop/integration/serverproperties/SslTest.java
index 17503f3..c981db2 100644
--- a/test/src/test/java/org/apache/sqoop/integration/serverproperties/SslTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/serverproperties/SslTest.java
@@ -31,10 +31,11 @@ import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
 import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
 import org.apache.sqoop.test.utils.HdfsUtils;
 import org.apache.sqoop.test.utils.SecurityUtils;
-import org.apache.sqoop.test.utils.SqoopUtils;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.testng.annotations.AfterMethod;
+import org.testng.annotations.AfterSuite;
 import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.BeforeSuite;
 import org.testng.annotations.Test;
 
 import javax.net.ssl.HostnameVerifier;
@@ -53,25 +54,36 @@ import java.util.Map;
 
 import static org.testng.Assert.assertEquals;
 
-@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
+@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class, KdcInfrastructureProvider.class})
 @Test(groups = {"no-real-cluster"})
 public class SslTest extends SqoopTestCase {
 
   private SqoopMiniCluster sqoopMiniCluster;
   private SSLContext defaultSslContext;
   private HostnameVerifier defaultHostNameVerifier;
+  private X509Certificate serverCertificate;
+  private String sslKeystoreDir;
+
+  private final String KEYSTORE_DIR = "ssltestkeystores/";
 
   public static class SslSqoopMiniCluster extends JettySqoopMiniCluster {
 
     private String keyStoreFilePath;
     private String keyStorePassword;
+    private String keyStorePasswordGenerator;
     private String keyManagerPassword;
+    private String keyManagerPasswordGenerator;
 
-    public SslSqoopMiniCluster(String temporaryPath, Configuration configuration, String keyStoreFilePath, String keyStorePassword, String keyManagerPassword) throws Exception {
+    public SslSqoopMiniCluster(String temporaryPath, Configuration configuration,
+                               String keyStoreFilePath, String keyStorePassword,
+                               String keyStorePasswordGenerator, String keyManagerPassword,
+                               String keyManagerPasswordGenerator) throws Exception {
       super(temporaryPath, configuration);
       this.keyStoreFilePath = keyStoreFilePath;
       this.keyStorePassword = keyStorePassword;
+      this.keyStorePasswordGenerator = keyStorePasswordGenerator;
       this.keyManagerPassword = keyManagerPassword;
+      this.keyManagerPasswordGenerator = keyManagerPasswordGenerator;
     }
 
     @Override
@@ -82,74 +94,127 @@ public class SslTest extends SqoopTestCase {
       properties.put(SecurityConstants.TLS_PROTOCOL, "TLSv1.2");
       properties.put(SecurityConstants.KEYSTORE_LOCATION, keyStoreFilePath);
       properties.put(SecurityConstants.KEYSTORE_PASSWORD, keyStorePassword);
+      properties.put(SecurityConstants.KEYSTORE_PASSWORD_GENERATOR, keyStorePasswordGenerator);
       properties.put(SecurityConstants.KEYMANAGER_PASSWORD, keyManagerPassword);
+      properties.put(SecurityConstants.KEYMANAGER_PASSWORD_GENERATOR, keyManagerPasswordGenerator);
 
       return properties;
     }
   }
 
+  @BeforeSuite
+  public void createCertificates() throws Exception {
+    sslKeystoreDir = getTemporaryPath() + KEYSTORE_DIR;
+    serverCertificate = setupKeystore(sslKeystoreDir);
+  }
+
   @BeforeMethod
-  public void backupSslContext() throws Exception {
+  public void backupState() throws Exception {
+    authToken = new DelegationTokenAuthenticatedURL.Token();
     defaultSslContext = SSLContext.getDefault();
     defaultHostNameVerifier = HttpsURLConnection.getDefaultHostnameVerifier();
   }
 
+  @BeforeMethod(dependsOnMethods = { "backupState" })
+  public void bypassHostnameVerification() throws Exception {
+    HttpsURLConnection.setDefaultHostnameVerifier(new HostnameVerifier() {
+      public boolean verify(String hostname, SSLSession session) {
+        try {
+          if (hostname.equals((new URL(sqoopMiniCluster.getServerUrl())).getHost())) {
+            return true;
+          }
+        } catch (MalformedURLException e) {
+          return false;
+        }
+        return false;
+      }
+    });
+  }
+
+  private void prepareKDC() {
+    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
+    if (kdcProvider != null) {
+      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
+    }
+  }
+
+  private X509Certificate setupKeystore(String sslKeystoreDir) throws Exception {
+    String sslConfDir = sslKeystoreDir;
+    FileUtils.deleteDirectory(new File(sslKeystoreDir));
+    FileUtils.forceMkdir(new File(sslKeystoreDir));
+    return SecurityUtils.setupSSLConfig(
+      sslKeystoreDir, sslConfDir, new Configuration(), false, true);
+  }
+
+  @AfterSuite
+  public void deleteCertificates() throws Exception {
+    FileUtils.deleteDirectory(new File(sslKeystoreDir));
+  }
+
   @AfterMethod
-  public void restoreSslContext() {
+  public void restoreState() {
     SSLContext.setDefault(defaultSslContext);
     HttpsURLConnection.setDefaultHostnameVerifier(defaultHostNameVerifier);
   }
 
   @AfterMethod
   public void stopCluster() throws Exception {
-    sqoopMiniCluster.stop();
+    try {
+      sqoopMiniCluster.stop();
+    } catch (Exception e) {
+      throw e;
+    }
+
   }
 
   @Test
-  public void testSslInUse() throws Exception {
-    String sslKeystoresDir = getTemporaryPath() + "ssl-keystore/";
-    String sslConfDir = SqoopUtils.getClasspathDir(SslTest.class);
-    FileUtils.deleteDirectory(new File(sslKeystoresDir));
-    FileUtils.forceMkdir(new File(sslKeystoresDir));
-    X509Certificate serverCertificate = SecurityUtils.setupSSLConfig(
-      sslKeystoresDir, sslConfDir, new Configuration(), false, true);
-
+  public void testSslInUseWithPassword() throws Exception {
     sqoopMiniCluster =
-      new SslSqoopMiniCluster(HdfsUtils.joinPathFragments(getTemporaryPath(), getTestName()), getHadoopConf(), sslKeystoresDir + SecurityUtils.SERVER_KEYSTORE, SecurityUtils.SERVER_KEY_STORE_PASSWORD, SecurityUtils.SERVER_KEY_PASSWORD);
+      new SslSqoopMiniCluster(HdfsUtils.joinPathFragments(getTemporaryPath(), getTestName()),
+        getHadoopConf(), sslKeystoreDir + SecurityUtils.SERVER_KEYSTORE,
+        SecurityUtils.SERVER_KEY_STORE_PASSWORD, "",
+        SecurityUtils.SERVER_KEY_PASSWORD, "");
 
-    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
-    if (kdcProvider != null) {
-      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
-    }
+    prepareKDC();
 
     sqoopMiniCluster.start();
 
-    // Bypass hostname verification
-    HttpsURLConnection.setDefaultHostnameVerifier(new HostnameVerifier() {
-      public boolean verify(String hostname, SSLSession session) {
-        try {
-          if (hostname.equals((new URL(sqoopMiniCluster.getServerUrl())).getHost())) {
-            return true;
-          }
-        } catch (MalformedURLException e) {
-          return false;
-        }
-        return false;
-      }
-    });
-
     SslContextFactory sslContextFactory = new SslContextFactory();
-    sslContextFactory.setKeyStorePath(sslKeystoresDir + SecurityUtils.TRUSTSTORE);
+    sslContextFactory.setKeyStorePath(sslKeystoreDir + SecurityUtils.TRUSTSTORE);
+    sslContextFactory.start();
+    SSLContext.setDefault(sslContextFactory.getSslContext());
+
+    initSqoopClient(sqoopMiniCluster.getServerUrl());
+
+    verifySsl(serverCertificate);
+  }
+
+  @Test
+  public void testSslInUseWithPasswordGenerator() throws Exception {
+    sqoopMiniCluster =
+      new SslSqoopMiniCluster(HdfsUtils.joinPathFragments(getTemporaryPath(), getTestName()),
+        getHadoopConf(), sslKeystoreDir + SecurityUtils.SERVER_KEYSTORE,
+        "", "echo " + SecurityUtils.SERVER_KEY_STORE_PASSWORD,
+        "", "echo " + SecurityUtils.SERVER_KEY_PASSWORD );
 
+    prepareKDC();
+
+    sqoopMiniCluster.start();
+
+    SslContextFactory sslContextFactory = new SslContextFactory();
+    sslContextFactory.setKeyStorePath(sslKeystoreDir + SecurityUtils.TRUSTSTORE);
     sslContextFactory.start();
 
     SSLContext.setDefault(sslContextFactory.getSslContext());
 
     initSqoopClient(sqoopMiniCluster.getServerUrl());
+    verifySsl(serverCertificate);
+  }
 
-    // Make a request and check the cert
+  private void verifySsl(X509Certificate serverCertificate) throws Exception {
     URL url = new URL(sqoopMiniCluster.getServerUrl() + "version?" +
       PseudoAuthenticator.USER_NAME + "=" + System.getProperty("user.name"));
+
     HttpURLConnection conn = new DelegationTokenAuthenticatedURL().openConnection(url, getAuthToken());
     conn.setRequestMethod(HttpMethod.GET);
     conn.setRequestProperty("Accept", MediaType.APPLICATION_JSON);
@@ -157,7 +222,10 @@ public class SslTest extends SqoopTestCase {
     assertEquals(conn.getResponseCode(), 200);
 
     HttpsURLConnection secured = (HttpsURLConnection) conn;
+
     Certificate actualCertificate = secured.getServerCertificates()[0];
+
+    secured.disconnect();
     assertEquals(actualCertificate, serverCertificate);
   }