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/02/25 20:44:20 UTC

[1/2] sqoop git commit: SQOOP-2843: Sqoop2: Enable SSL/TLS for API calls

Repository: sqoop
Updated Branches:
  refs/heads/sqoop2 9aec8f965 -> 96f3d9c19


http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/test/java/org/apache/sqoop/integration/serverproperties/ConnectorClasspathIsolationTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/serverproperties/ConnectorClasspathIsolationTest.java b/test/src/test/java/org/apache/sqoop/integration/serverproperties/ConnectorClasspathIsolationTest.java
new file mode 100644
index 0000000..1829257
--- /dev/null
+++ b/test/src/test/java/org/apache/sqoop/integration/serverproperties/ConnectorClasspathIsolationTest.java
@@ -0,0 +1,190 @@
+/**
+ * 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.sqoop.integration.serverproperties;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.core.ConfigurationConstants;
+import org.apache.sqoop.model.MDriverConfig;
+import org.apache.sqoop.model.MJob;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.test.infrastructure.Infrastructure;
+import org.apache.sqoop.test.infrastructure.SqoopTestCase;
+import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.KdcInfrastructureProvider;
+import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
+import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
+import org.apache.sqoop.test.utils.ConnectorUtils;
+import org.apache.sqoop.test.utils.HdfsUtils;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Test(groups = "no-real-cluster")
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class})
+public class ConnectorClasspathIsolationTest extends SqoopTestCase {
+
+  private static final String TEST_FROM_CONNECTOR_JAR_NAME = "test-from-connector.jar";
+  private static final String TEST_TO_CONNECTOR_JAR_NAME = "test-to-connector.jar";
+  private static final String TEST_FROM_DEPENDENCY_JAR_NAME = "test-from-dependency.jar";
+  private static final String TEST_TO_DEPENDENCY_JAR_NAME = "test-to-dependency.jar";
+
+  private static final String[] FROM_CONNECTOR_SOURCE_FILES = {
+    "TestConnectorClasspathIsolation/from/TestFromConnector.java",
+    "TestConnectorClasspathIsolation/from/TestExtractor.java",
+    "TestConnectorClasspathIsolation/from/TestFromDestroyer.java",
+    "TestConnectorClasspathIsolation/from/TestFromInitializer.java",
+    "TestConnectorClasspathIsolation/from/TestFromJobConfiguration.java",
+    "TestConnectorClasspathIsolation/from/TestPartition.java",
+    "TestConnectorClasspathIsolation/from/TestPartitioner.java",
+    "TestConnectorClasspathIsolation/from/TestFromLinkConfiguration.java"
+  };
+
+  private static final String[] FROM_CONNECTOR_DEPENDENCY_SOURCE_FILES = {
+    "TestConnectorClasspathIsolation/from/TestClasspathIsolation.java"
+  };
+
+  private static final String[] FROM_CONNECTOR_PROPERTY_FILES = {
+    "TestConnectorClasspathIsolation/from/sqoopconnector.properties"
+  };
+
+  private static final String[] TO_CONNECTOR_SOURCE_FILES = {
+    "TestConnectorClasspathIsolation/to/TestToConnector.java",
+    "TestConnectorClasspathIsolation/to/TestLoader.java",
+    "TestConnectorClasspathIsolation/to/TestToDestroyer.java",
+    "TestConnectorClasspathIsolation/to/TestToInitializer.java",
+    "TestConnectorClasspathIsolation/to/TestToJobConfiguration.java",
+    "TestConnectorClasspathIsolation/to/TestToLinkConfiguration.java"
+  };
+
+  private static final String[] TO_CONNECTOR_DEPENDENCY_SOURCE_FILES = {
+    "TestConnectorClasspathIsolation/to/TestClasspathIsolation.java"
+  };
+
+  private static final String[] TO_CONNECTOR_PROPERTY_FILES = {
+    "TestConnectorClasspathIsolation/to/sqoopconnector.properties"
+  };
+
+  private ClassLoader classLoader;
+  private SqoopMiniCluster sqoopMiniCluster;
+
+  public static class DerbySqoopMiniCluster extends JettySqoopMiniCluster {
+
+    private String extraClasspath;
+
+    public DerbySqoopMiniCluster(String temporaryPath, Configuration configuration, String extraClasspath) throws Exception {
+      super(temporaryPath, configuration);
+      this.extraClasspath = extraClasspath;
+    }
+
+    @Override
+    protected Map<String, String> getClasspathConfiguration() {
+      Map<String, String> properties = new HashMap<>();
+
+      if (extraClasspath != null) {
+        properties.put(ConfigurationConstants.CLASSPATH, extraClasspath);
+      }
+
+      return properties;
+    }
+  }
+
+  public void startSqoopMiniCluster(String extraClasspath) throws Exception {
+    // And use them for new Derby repo instance
+    sqoopMiniCluster = new DerbySqoopMiniCluster(HdfsUtils.joinPathFragments(super.getTemporaryPath(), getTestName()), getHadoopConf(), extraClasspath);
+    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
+    if (kdcProvider != null) {
+      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
+    }
+
+    // Start server
+    sqoopMiniCluster.start();
+
+    // Initialize Sqoop Client API
+    initSqoopClient(sqoopMiniCluster.getServerUrl());
+  }
+
+  @BeforeMethod
+  public void captureClasspath() {
+    classLoader = Thread.currentThread().getContextClassLoader();
+  }
+
+  @AfterMethod
+  public void restoreClasspath(){
+    Thread.currentThread().setContextClassLoader(classLoader);
+  }
+
+  @Test
+  public void testConnectorClasspathIsolation() throws Exception {
+    Map<String, String> fromConnectorJarMap = ConnectorUtils.compileTestConnectorAndDependency(
+        FROM_CONNECTOR_SOURCE_FILES,
+        FROM_CONNECTOR_DEPENDENCY_SOURCE_FILES,
+        FROM_CONNECTOR_PROPERTY_FILES,
+        TEST_FROM_CONNECTOR_JAR_NAME,
+        TEST_FROM_DEPENDENCY_JAR_NAME,
+        true);
+    Map<String, String> toConnectorJarMap = ConnectorUtils.compileTestConnectorAndDependency(
+        TO_CONNECTOR_SOURCE_FILES,
+        TO_CONNECTOR_DEPENDENCY_SOURCE_FILES,
+        TO_CONNECTOR_PROPERTY_FILES,
+        TEST_TO_CONNECTOR_JAR_NAME,
+        TEST_TO_DEPENDENCY_JAR_NAME,
+        true);
+    startSqoopMiniCluster(
+        StringUtils.join(Arrays.asList(fromConnectorJarMap.get(TEST_FROM_CONNECTOR_JAR_NAME), toConnectorJarMap.get(TEST_TO_CONNECTOR_JAR_NAME)), ":"));
+
+    MJob job = prepareJob();
+
+    prepareDriverConfig(job);
+
+    saveJob(job);
+
+    executeJob(job);
+
+    stopSqoop();
+    ConnectorUtils.deleteJars(fromConnectorJarMap);
+  }
+
+  private MJob prepareJob() {
+    MLink rdbmsConnection = getClient().createLink("test-from-connector");
+    saveLink(rdbmsConnection);
+
+    MLink testConnection = getClient().createLink("test-to-connector");
+    saveLink(testConnection);
+
+    MJob job = getClient().createJob(rdbmsConnection.getName(), testConnection.getName());
+
+    return job;
+  }
+
+  private MDriverConfig prepareDriverConfig(MJob job) {
+    MDriverConfig driverConfig = job.getDriverConfig();
+    driverConfig.getIntegerInput("throttlingConfig.numExtractors").setValue(3);
+
+    return driverConfig;
+  }
+
+  private void stopSqoop() throws Exception {
+    sqoopMiniCluster.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/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
new file mode 100644
index 0000000..17503f3
--- /dev/null
+++ b/test/src/test/java/org/apache/sqoop/integration/serverproperties/SslTest.java
@@ -0,0 +1,164 @@
+/**
+ * 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.sqoop.integration.serverproperties;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.sqoop.security.SecurityConstants;
+import org.apache.sqoop.test.infrastructure.Infrastructure;
+import org.apache.sqoop.test.infrastructure.SqoopTestCase;
+import org.apache.sqoop.test.infrastructure.providers.DatabaseInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.KdcInfrastructureProvider;
+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.BeforeMethod;
+import org.testng.annotations.Test;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSession;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.core.MediaType;
+import java.io.File;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.security.cert.Certificate;
+import java.security.cert.X509Certificate;
+import java.util.Map;
+
+import static org.testng.Assert.assertEquals;
+
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
+@Test(groups = {"no-real-cluster"})
+public class SslTest extends SqoopTestCase {
+
+  private SqoopMiniCluster sqoopMiniCluster;
+  private SSLContext defaultSslContext;
+  private HostnameVerifier defaultHostNameVerifier;
+
+  public static class SslSqoopMiniCluster extends JettySqoopMiniCluster {
+
+    private String keyStoreFilePath;
+    private String keyStorePassword;
+    private String keyManagerPassword;
+
+    public SslSqoopMiniCluster(String temporaryPath, Configuration configuration, String keyStoreFilePath, String keyStorePassword, String keyManagerPassword) throws Exception {
+      super(temporaryPath, configuration);
+      this.keyStoreFilePath = keyStoreFilePath;
+      this.keyStorePassword = keyStorePassword;
+      this.keyManagerPassword = keyManagerPassword;
+    }
+
+    @Override
+    protected Map<String, String> getSecurityConfiguration() {
+      Map<String, String> properties = super.getSecurityConfiguration();
+
+      properties.put(SecurityConstants.TLS_ENABLED, String.valueOf(true));
+      properties.put(SecurityConstants.TLS_PROTOCOL, "TLSv1.2");
+      properties.put(SecurityConstants.KEYSTORE_LOCATION, keyStoreFilePath);
+      properties.put(SecurityConstants.KEYSTORE_PASSWORD, keyStorePassword);
+      properties.put(SecurityConstants.KEYMANAGER_PASSWORD, keyManagerPassword);
+
+      return properties;
+    }
+  }
+
+  @BeforeMethod
+  public void backupSslContext() throws Exception {
+    defaultSslContext = SSLContext.getDefault();
+    defaultHostNameVerifier = HttpsURLConnection.getDefaultHostnameVerifier();
+  }
+
+  @AfterMethod
+  public void restoreSslContext() {
+    SSLContext.setDefault(defaultSslContext);
+    HttpsURLConnection.setDefaultHostnameVerifier(defaultHostNameVerifier);
+  }
+
+  @AfterMethod
+  public void stopCluster() throws Exception {
+    sqoopMiniCluster.stop();
+  }
+
+  @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);
+
+    sqoopMiniCluster =
+      new SslSqoopMiniCluster(HdfsUtils.joinPathFragments(getTemporaryPath(), getTestName()), getHadoopConf(), sslKeystoresDir + SecurityUtils.SERVER_KEYSTORE, SecurityUtils.SERVER_KEY_STORE_PASSWORD, SecurityUtils.SERVER_KEY_PASSWORD);
+
+    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
+    if (kdcProvider != null) {
+      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
+    }
+
+    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.start();
+
+    SSLContext.setDefault(sslContextFactory.getSslContext());
+
+    initSqoopClient(sqoopMiniCluster.getServerUrl());
+
+    // Make a request and check the cert
+    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);
+
+    assertEquals(conn.getResponseCode(), 200);
+
+    HttpsURLConnection secured = (HttpsURLConnection) conn;
+    Certificate actualCertificate = secured.getServerCertificates()[0];
+    assertEquals(actualCertificate, serverCertificate);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/test/resources/connector-loading-tests-suite.xml
----------------------------------------------------------------------
diff --git a/test/src/test/resources/connector-loading-tests-suite.xml b/test/src/test/resources/connector-loading-tests-suite.xml
deleted file mode 100644
index c03fb4f..0000000
--- a/test/src/test/resources/connector-loading-tests-suite.xml
+++ /dev/null
@@ -1,34 +0,0 @@
-<?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.
--->
-
-<!DOCTYPE suite SYSTEM "http://testng.org/testng-1.0.dtd" >
-
-<suite name="ConnectorLoadingTests" verbose="2" parallel="false">
-
-    <listeners>
-        <listener class-name="org.apache.sqoop.test.testng.SqoopTestListener" />
-        <listener class-name="org.apache.sqoop.test.testng.ReconfigureLogListener" />
-    </listeners>
-
-    <test name="ConnectorLoadingTests">
-        <packages>
-            <package name="org.apache.sqoop.integration.connectorloading"/>
-        </packages>
-    </test>
-
-</suite>

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/test/resources/server-properties-tests-suite.xml
----------------------------------------------------------------------
diff --git a/test/src/test/resources/server-properties-tests-suite.xml b/test/src/test/resources/server-properties-tests-suite.xml
new file mode 100644
index 0000000..2743996
--- /dev/null
+++ b/test/src/test/resources/server-properties-tests-suite.xml
@@ -0,0 +1,34 @@
+<?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.
+-->
+
+<!DOCTYPE suite SYSTEM "http://testng.org/testng-1.0.dtd" >
+
+<suite name="ServerPropertiesTests" verbose="2" parallel="false">
+
+    <listeners>
+        <listener class-name="org.apache.sqoop.test.testng.SqoopTestListener" />
+        <listener class-name="org.apache.sqoop.test.testng.ReconfigureLogListener" />
+    </listeners>
+
+    <test name="ServerPropertiesTests">
+        <packages>
+            <package name="org.apache.sqoop.integration.serverproperties"/>
+        </packages>
+    </test>
+
+</suite>


[2/2] sqoop git commit: SQOOP-2843: Sqoop2: Enable SSL/TLS for API calls

Posted by ja...@apache.org.
SQOOP-2843: Sqoop2: Enable SSL/TLS for API calls

(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/96f3d9c1
Tree: http://git-wip-us.apache.org/repos/asf/sqoop/tree/96f3d9c1
Diff: http://git-wip-us.apache.org/repos/asf/sqoop/diff/96f3d9c1

Branch: refs/heads/sqoop2
Commit: 96f3d9c195c78c1f315a61f7c84aa095a041f72b
Parents: 9aec8f9
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Thu Feb 25 11:42:54 2016 -0800
Committer: Jarek Jarcec Cecho <ja...@apache.org>
Committed: Thu Feb 25 11:42:54 2016 -0800

----------------------------------------------------------------------
 .../sqoop/security/SecurityConstants.java       |  43 +++
 dist/src/main/conf/sqoop.properties             |   9 +
 .../sqoop/docs/generator/DocPreprocessor.java   |   2 +
 .../plugins/GenerateConnectorPages.java         |  65 +++++
 .../apache/sqoop/server/SqoopJettyServer.java   |  49 +++-
 .../apache/sqoop/server/common/ServerError.java |   3 +
 .../apache/sqoop/shell/ShellEnvironment.java    |  13 +
 .../sqoop/shell/ShowConnectorFunction.java      |   3 +-
 .../org/apache/sqoop/shell/ShowJobFunction.java |   6 +-
 .../apache/sqoop/shell/ShowLinkFunction.java    |   8 +-
 .../sqoop/shell/ShowPrincipalFunction.java      |   4 +-
 .../sqoop/shell/ShowPrivilegeFunction.java      |   4 +-
 .../apache/sqoop/shell/ShowRoleFunction.java    |   4 +-
 .../sqoop/shell/ShowSubmissionFunction.java     |   3 +-
 .../sqoop/shell/utils/TableDisplayer.java       |  41 ++-
 test/pom.xml                                    |   6 +-
 .../apache/sqoop/test/kdc/MiniKdcRunner.java    | 247 +----------------
 .../apache/sqoop/test/utils/SecurityUtils.java  | 269 +++++++++++++++++++
 .../org/apache/sqoop/test/utils/SqoopUtils.java |  25 +-
 .../BlacklistedConnectorTest.java               |  82 ------
 .../connectorloading/ClasspathTest.java         | 202 --------------
 .../ConnectorClasspathIsolationTest.java        | 190 -------------
 .../BlacklistedConnectorTest.java               |  82 ++++++
 .../serverproperties/ClasspathTest.java         | 202 ++++++++++++++
 .../ConnectorClasspathIsolationTest.java        | 190 +++++++++++++
 .../integration/serverproperties/SslTest.java   | 164 +++++++++++
 .../resources/connector-loading-tests-suite.xml |  34 ---
 .../resources/server-properties-tests-suite.xml |  34 +++
 28 files changed, 1199 insertions(+), 785 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/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 6f32e04..91a1b8b 100644
--- a/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
+++ b/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
@@ -37,6 +37,13 @@ public final class SecurityConstants {
     PREFIX_SECURITY_CONFIG + "authentication.";
 
   /**
+   * All tls related configuration is prefixed with this:
+   * <tt>org.apache.sqoop.security.tls.</tt>
+   */
+  public static final String PREFIX_TLS_CONFIG =
+    PREFIX_SECURITY_CONFIG + "tls.";
+
+  /**
    * The config specifies the sqoop authentication type (SIMPLE, KERBEROS).
    * The default type is SIMPLE
    * <tt>org.apache.sqoop.security.authentication.type</tt>.
@@ -158,6 +165,42 @@ public final class SecurityConstants {
           PREFIX_AUTHORIZATION_CONFIG + "server_name";
 
   /**
+   * The config specifies the whether the http server should use TLS.
+   * <tt>org.apache.sqoop.security.tls.enabled</tt>.
+   */
+  public static final String TLS_ENABLED =
+          PREFIX_TLS_CONFIG + "enabled";
+
+  /**
+   * The config specifies the tls protocol version
+   * <tt>org.apache.sqoop.security.tls.protocol</tt>.
+   */
+  public static final String TLS_PROTOCOL =
+    PREFIX_TLS_CONFIG + "protocol";
+
+  /**
+   * The config specifies the location of the JKS formatted keystore
+   * <tt>org.apache.sqoop.security.tls.keystore</tt>.
+   */
+  public static final String KEYSTORE_LOCATION =
+          PREFIX_TLS_CONFIG + "keystore";
+
+  /**
+   * The config specifies the password of the JKS formatted keystore
+   * <tt>org.apache.sqoop.security.tls.keystorepassword</tt>.
+   */
+  public static final String KEYSTORE_PASSWORD =
+          PREFIX_TLS_CONFIG + "keystore_password";
+
+  /**
+   * The config specifies the password for the private key in the JKS formatted
+   * keystore
+   * <tt>org.apache.sqoop.security.tls.keymanagerpassword</tt>.
+   */
+  public static final String KEYMANAGER_PASSWORD =
+    PREFIX_TLS_CONFIG + "keymanager_password";
+
+  /**
    * 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/96f3d9c1/dist/src/main/conf/sqoop.properties
----------------------------------------------------------------------
diff --git a/dist/src/main/conf/sqoop.properties b/dist/src/main/conf/sqoop.properties
index 620146d..767d3f2 100755
--- a/dist/src/main/conf/sqoop.properties
+++ b/dist/src/main/conf/sqoop.properties
@@ -180,6 +180,15 @@ org.apache.sqoop.execution.engine=org.apache.sqoop.execution.mapreduce.Mapreduce
 #org.apache.sqoop.security.authorization.authentication_provider=org.apache.sqoop.security.authorization.DefaultAuthenticationProvider
 #org.apache.sqoop.security.authorization.server_name=SqoopServer1
 
+#
+# SSL/TLS configuration
+#
+#org.apache.sqoop.security.tls.enabled=false
+#org.apache.sqoop.security.tls.protocol="TLSv1.2"
+#org.apache.sqoop.security.tls.keystore=
+#org.apache.sqoop.security.tls.keystore_password=
+
+
 # External connectors load path
 # "/path/to/external/connectors/": Add all the connector JARs in the specified folder
 #

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/docs/src/main/java/org/apache/sqoop/docs/generator/DocPreprocessor.java
----------------------------------------------------------------------
diff --git a/docs/src/main/java/org/apache/sqoop/docs/generator/DocPreprocessor.java b/docs/src/main/java/org/apache/sqoop/docs/generator/DocPreprocessor.java
index 5ea3aa0..cb79654 100644
--- a/docs/src/main/java/org/apache/sqoop/docs/generator/DocPreprocessor.java
+++ b/docs/src/main/java/org/apache/sqoop/docs/generator/DocPreprocessor.java
@@ -20,6 +20,7 @@ package org.apache.sqoop.docs.generator;
 import org.apache.log4j.Logger;
 import org.apache.sqoop.docs.generator.plugins.AbstractPlugin;
 import org.apache.sqoop.docs.generator.plugins.CopySourceToDestination;
+import org.apache.sqoop.docs.generator.plugins.GenerateConnectorPages;
 import org.apache.sqoop.utils.ClassUtils;
 
 import java.util.LinkedList;
@@ -36,6 +37,7 @@ public class DocPreprocessor {
   static {
     plugins = new LinkedList<>();
     plugins.add(CopySourceToDestination.class);
+    plugins.add(GenerateConnectorPages.class);
   }
 
   public static void main(String []args) {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/docs/src/main/java/org/apache/sqoop/docs/generator/plugins/GenerateConnectorPages.java
----------------------------------------------------------------------
diff --git a/docs/src/main/java/org/apache/sqoop/docs/generator/plugins/GenerateConnectorPages.java b/docs/src/main/java/org/apache/sqoop/docs/generator/plugins/GenerateConnectorPages.java
new file mode 100644
index 0000000..119b76d
--- /dev/null
+++ b/docs/src/main/java/org/apache/sqoop/docs/generator/plugins/GenerateConnectorPages.java
@@ -0,0 +1,65 @@
+/**
+ * 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.sqoop.docs.generator.plugins;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.sqoop.connector.ConnectorHandler;
+import org.apache.sqoop.connector.ConnectorManagerUtils;
+import org.apache.sqoop.connector.spi.SqoopConnector;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.URL;
+import java.util.Collections;
+import java.util.List;
+
+public class GenerateConnectorPages extends AbstractPlugin {
+
+  private static final Logger LOG = Logger.getLogger(GenerateConnectorPages.class);
+
+  @Override
+  public void run() {
+    try {
+      runWithException();
+    } catch (IOException e) {
+      throw new RuntimeException("Can't generate connector documentation", e);
+    }
+  }
+
+  public void runWithException() throws IOException {
+    List<URL> connectorUrls = ConnectorManagerUtils.getConnectorConfigs(Collections.<String>emptySet());
+    for (URL url : connectorUrls) {
+      SqoopConnector connector = new ConnectorHandler(url).getSqoopConnector();
+      LOG.info("Loaded " +  connector.getConnectorName() + " from  " + url.toString());
+
+      String connectorName = connector.getConnectorName();
+
+      File outputFile = FileUtils.getFile(getDestination(), "user", "connectors", connectorName + ".rst");
+      PrintWriter writer = new PrintWriter(outputFile);
+
+      // Writing headline
+      writer.println(connectorName);
+      writer.println(StringUtils.repeat("=", connectorName.length()));
+
+      writer.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/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 2c4cb7a..60368af 100644
--- a/server/src/main/java/org/apache/sqoop/server/SqoopJettyServer.java
+++ b/server/src/main/java/org/apache/sqoop/server/SqoopJettyServer.java
@@ -19,12 +19,21 @@
 package org.apache.sqoop.server;
 
 import org.apache.log4j.Logger;
+import org.apache.sqoop.common.MapContext;
+import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.core.SqoopConfiguration;
 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.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.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;
 
@@ -58,7 +67,45 @@ public class SqoopJettyServer {
     webServer = new Server(threadPool);
 
     // Connector configs
-    ServerConnector connector = new ServerConnector(webServer);
+    ServerConnector connector;
+
+    MapContext configurationContext = SqoopConfiguration.getInstance().getContext();
+
+    if (configurationContext.getBoolean(SecurityConstants.TLS_ENABLED, false)) {
+      String keyStorePath = configurationContext.getString(SecurityConstants.KEYSTORE_LOCATION);
+      if (keyStorePath == null) {
+        throw new SqoopException(ServerError.SERVER_0007);
+      }
+
+      SslContextFactory sslContextFactory = new SslContextFactory();
+      sslContextFactory.setKeyStorePath(keyStorePath);
+
+      String protocol = configurationContext.getString(SecurityConstants.TLS_PROTOCOL);
+      if (protocol != null && protocol.length() > 0) {
+        sslContextFactory.setProtocol(protocol.trim());
+      }
+
+      String keyStorePassword = configurationContext.getString(SecurityConstants.KEYSTORE_PASSWORD);
+      if (keyStorePassword != null && keyStorePassword.length() > 0) {
+        sslContextFactory.setKeyStorePassword(keyStorePassword);
+      }
+
+      String keyManagerPassword = configurationContext.getString(SecurityConstants.KEYMANAGER_PASSWORD);
+      if (keyManagerPassword != null && keyManagerPassword.length() > 0) {
+        sslContextFactory.setKeyManagerPassword(keyManagerPassword);
+      }
+
+      HttpConfiguration https = new HttpConfiguration();
+      https.addCustomizer(new SecureRequestCustomizer());
+
+      connector = new ServerConnector(webServer,
+        new SslConnectionFactory(sslContextFactory, "http/1.1"),
+        new HttpConnectionFactory(https));
+    } else {
+      connector = new ServerConnector(webServer);
+    }
+
+
     connector.setPort(sqoopJettyContext.getPort());
     webServer.addConnector(connector);
     webServer.setHandler(createServletContextHandler());

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/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 1b021cf..a644e9f 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
@@ -38,6 +38,9 @@ public enum ServerError implements ErrorCode {
 
   /** Entity requested doesn't exist*/
   SERVER_0006("Entity requested doesn't exist"),
+
+  /** TLS enabled but keystore location not set*/
+  SERVER_0007("TLS enabled but keystore location not set"),
   ;
 
   private final String message;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/ShellEnvironment.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/ShellEnvironment.java b/shell/src/main/java/org/apache/sqoop/shell/ShellEnvironment.java
index 36d0712..b7f1129 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/ShellEnvironment.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/ShellEnvironment.java
@@ -21,12 +21,14 @@ import org.apache.sqoop.client.SqoopClient;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.shell.core.ShellError;
 import org.apache.sqoop.shell.core.Constants;
+import org.apache.sqoop.shell.utils.TableDisplayer;
 import org.codehaus.groovy.tools.shell.IO;
 
 import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.text.MessageFormat;
+import java.util.List;
 import java.util.Locale;
 import java.util.ResourceBundle;
 
@@ -57,6 +59,13 @@ public final class ShellEnvironment {
   private static boolean interactive = false;
   private static long pollTimeout = DEFAULT_POLL_TIMEOUT;
 
+  private static TableDisplayer tableDisplayer = new TableDisplayer(new TableDisplayer.TableDisplayerWriter() {
+    @Override
+    public void append(String text) {
+      print(text);
+    }
+  });
+
   static ResourceBundle resource = ResourceBundle.getBundle(Constants.RESOURCE_NAME, Locale.getDefault());
   static SqoopClient client = new SqoopClient(getServerUrl());
   static IO io;
@@ -226,6 +235,10 @@ public final class ShellEnvironment {
     io.out.printf(format, args);
   }
 
+  public static void displayTable(List<String> headers, List<String> ...columns) {
+    tableDisplayer.display(headers, columns);
+  }
+
   // for tests only
   public static void cleanup() {
     serverHost = DEFAULT_SERVER_HOST;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/ShowConnectorFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/ShowConnectorFunction.java b/shell/src/main/java/org/apache/sqoop/shell/ShowConnectorFunction.java
index d973499..c90fe94 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/ShowConnectorFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/ShowConnectorFunction.java
@@ -27,7 +27,6 @@ import org.apache.sqoop.client.ClientError;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.shell.core.Constants;
-import org.apache.sqoop.shell.utils.TableDisplayer;
 import org.apache.sqoop.validation.Status;
 
 import static org.apache.sqoop.shell.ShellEnvironment.*;
@@ -83,7 +82,7 @@ public class ShowConnectorFunction extends SqoopFunction {
       supportedDirections.add(connector.getSupportedDirections().toString());
     }
 
-    TableDisplayer.display(header, uniqueNames, versions, classes, supportedDirections);
+    displayTable(header, uniqueNames, versions, classes, supportedDirections);
   }
 
   private void showConnectors() {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/ShowJobFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/ShowJobFunction.java b/shell/src/main/java/org/apache/sqoop/shell/ShowJobFunction.java
index ebbfe82..139e756 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/ShowJobFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/ShowJobFunction.java
@@ -19,12 +19,8 @@ package org.apache.sqoop.shell;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.OptionBuilder;
-import org.apache.sqoop.common.Direction;
-import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MJob;
-import org.apache.sqoop.model.MLink;
 import org.apache.sqoop.shell.core.Constants;
-import org.apache.sqoop.shell.utils.TableDisplayer;
 import org.apache.sqoop.validation.Status;
 
 import java.text.DateFormat;
@@ -97,7 +93,7 @@ public class ShowJobFunction extends SqoopFunction {
       availabilities.add(String.valueOf(job.getEnabled()));
     }
 
-    TableDisplayer.display(header, ids, names, fromConnectors, toConnectors, availabilities);
+    displayTable(header, ids, names, fromConnectors, toConnectors, availabilities);
   }
 
   private void showJobs(String jArg) {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/ShowLinkFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/ShowLinkFunction.java b/shell/src/main/java/org/apache/sqoop/shell/ShowLinkFunction.java
index 25bd1db..4976c98 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/ShowLinkFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/ShowLinkFunction.java
@@ -19,19 +19,13 @@ package org.apache.sqoop.shell;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.OptionBuilder;
-import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MLink;
 import org.apache.sqoop.shell.core.Constants;
-import org.apache.sqoop.shell.utils.TableDisplayer;
 import org.apache.sqoop.validation.Status;
 
 import java.text.DateFormat;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
 
 import static org.apache.sqoop.shell.ShellEnvironment.*;
 import static org.apache.sqoop.shell.utils.ConfigDisplayer.*;
@@ -86,7 +80,7 @@ public class ShowLinkFunction extends SqoopFunction {
       availabilities.add(String.valueOf(link.getEnabled()));
     }
 
-    TableDisplayer.display(header, names, connectorNames, availabilities);
+    displayTable(header, names, connectorNames, availabilities);
   }
 
   private void showLinks() {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/ShowPrincipalFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/ShowPrincipalFunction.java b/shell/src/main/java/org/apache/sqoop/shell/ShowPrincipalFunction.java
index a450aaf..45abf14 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/ShowPrincipalFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/ShowPrincipalFunction.java
@@ -22,13 +22,13 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.sqoop.model.MPrincipal;
 import org.apache.sqoop.model.MRole;
 import org.apache.sqoop.shell.core.Constants;
-import org.apache.sqoop.shell.utils.TableDisplayer;
 import org.apache.sqoop.validation.Status;
 
 import java.util.LinkedList;
 import java.util.List;
 
 import static org.apache.sqoop.shell.ShellEnvironment.client;
+import static org.apache.sqoop.shell.ShellEnvironment.displayTable;
 import static org.apache.sqoop.shell.ShellEnvironment.resourceString;
 
 @SuppressWarnings("serial")
@@ -70,6 +70,6 @@ public class ShowPrincipalFunction extends SqoopFunction {
       types.add(principal.getType());
     }
 
-    TableDisplayer.display(header, names, types);
+    displayTable(header, names, types);
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/ShowPrivilegeFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/ShowPrivilegeFunction.java b/shell/src/main/java/org/apache/sqoop/shell/ShowPrivilegeFunction.java
index 2cf6972..efd4b19 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/ShowPrivilegeFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/ShowPrivilegeFunction.java
@@ -25,13 +25,13 @@ import org.apache.sqoop.model.MPrivilege;
 import org.apache.sqoop.model.MResource;
 import org.apache.sqoop.shell.core.Constants;
 import org.apache.sqoop.shell.core.ShellError;
-import org.apache.sqoop.shell.utils.TableDisplayer;
 import org.apache.sqoop.validation.Status;
 
 import java.util.LinkedList;
 import java.util.List;
 
 import static org.apache.sqoop.shell.ShellEnvironment.client;
+import static org.apache.sqoop.shell.ShellEnvironment.displayTable;
 import static org.apache.sqoop.shell.ShellEnvironment.resourceString;
 
 @SuppressWarnings("serial")
@@ -108,6 +108,6 @@ public class ShowPrivilegeFunction extends SqoopFunction {
       withGrant.add(Boolean.toString(privilege.isWith_grant_option()));
     }
 
-    TableDisplayer.display(header, actions, resourceNames, resourceTypes, withGrant);
+    displayTable(header, actions, resourceNames, resourceTypes, withGrant);
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/ShowRoleFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/ShowRoleFunction.java b/shell/src/main/java/org/apache/sqoop/shell/ShowRoleFunction.java
index 6b61921..91ceff3 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/ShowRoleFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/ShowRoleFunction.java
@@ -24,13 +24,13 @@ import org.apache.sqoop.model.MPrincipal;
 import org.apache.sqoop.model.MRole;
 import org.apache.sqoop.shell.core.Constants;
 import org.apache.sqoop.shell.core.ShellError;
-import org.apache.sqoop.shell.utils.TableDisplayer;
 import org.apache.sqoop.validation.Status;
 
 import java.util.LinkedList;
 import java.util.List;
 
 import static org.apache.sqoop.shell.ShellEnvironment.client;
+import static org.apache.sqoop.shell.ShellEnvironment.displayTable;
 import static org.apache.sqoop.shell.ShellEnvironment.resourceString;
 
 @SuppressWarnings("serial")
@@ -83,6 +83,6 @@ public class ShowRoleFunction extends SqoopFunction {
       names.add(role.getName());
     }
 
-    TableDisplayer.display(header, names);
+    displayTable(header, names);
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/ShowSubmissionFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/ShowSubmissionFunction.java b/shell/src/main/java/org/apache/sqoop/shell/ShowSubmissionFunction.java
index 8989913..c7f371f 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/ShowSubmissionFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/ShowSubmissionFunction.java
@@ -25,7 +25,6 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.sqoop.model.MSubmission;
 import org.apache.sqoop.shell.core.Constants;
 import org.apache.sqoop.shell.utils.SubmissionDisplayer;
-import org.apache.sqoop.shell.utils.TableDisplayer;
 import org.apache.sqoop.validation.Status;
 
 import static org.apache.sqoop.shell.ShellEnvironment.*;
@@ -91,7 +90,7 @@ public class ShowSubmissionFunction extends SqoopFunction {
       dates.add(submission.getLastUpdateDate().toString());
     }
 
-    TableDisplayer.display(header, jnames, eids, status, dates);
+    displayTable(header, jnames, eids, status, dates);
   }
 
   private void showSubmissions(String jArg) {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/shell/src/main/java/org/apache/sqoop/shell/utils/TableDisplayer.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/utils/TableDisplayer.java b/shell/src/main/java/org/apache/sqoop/shell/utils/TableDisplayer.java
index 51030d0..ba3698e 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/utils/TableDisplayer.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/utils/TableDisplayer.java
@@ -22,23 +22,43 @@ import org.apache.commons.lang.StringUtils;
 import java.util.LinkedList;
 import java.util.List;
 
-import static org.apache.sqoop.shell.ShellEnvironment.*;
-
 /**
  * Display table based data
  */
 public class TableDisplayer {
 
   /**
+   * Interface that this displayer will use to write out formatted text.
+   */
+  public interface TableDisplayerWriter {
+    /**
+     * Print out addition formatted text to the output
+     *
+     * @param text
+     */
+    void append(String text);
+  }
+
+  public TableDisplayer(TableDisplayerWriter writer) {
+    this.writer = writer;
+  }
+
+  /**
+   * Writer instance that we should use to write something out
+   */
+  private TableDisplayerWriter writer;
+
+  /**
    * Display given columns in nice table structure to given IO object.
    *
    * @param headers List of headers
    * @param columns Array of columns
    */
-  public static void display(List<String> headers, List<String> ...columns) {
+  public void display(List<String> headers, List<String> ...columns) {
     assert headers != null;
     assert columns != null;
     assert headers.size() == columns.length;
+    assert writer != null;
 
     // Count of columns
     int columnCount = headers.size();
@@ -85,7 +105,7 @@ public class TableDisplayer {
    *
    * @param widths List of widths of each column
    */
-  private static void drawLine(List<Integer> widths) {
+  private void drawLine(List<Integer> widths) {
     int last = widths.size() - 1;
     print("+-");
     for(int i = 0; i < widths.size(); i++) {
@@ -102,7 +122,7 @@ public class TableDisplayer {
    * @param column All column values
    * @return Maximal
    */
-  private static int getMaximalWidth(String header, List<String> column) {
+  private int getMaximalWidth(String header, List<String> column) {
     assert header != null;
     assert column != null;
 
@@ -123,7 +143,7 @@ public class TableDisplayer {
    * @param columns Array with all column values
    * @return
    */
-  private static int getMaximalRows(List<String>... columns) {
+  private int getMaximalRows(List<String>... columns) {
     int max = 0;
 
     for(List<String> column : columns) {
@@ -135,7 +155,12 @@ public class TableDisplayer {
     return max;
   }
 
-  private TableDisplayer() {
-    // Instantiation is prohibited
+  private void print(String text) {
+    writer.append(text);
   }
+
+  private void println() {
+    writer.append("\n");
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 1e88b34..4bac683 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -274,17 +274,17 @@ limitations under the License.
             </configuration>
           </execution>
           <execution>
-            <id>connector-loading-test</id>
+            <id>server-properties-test</id>
             <goals>
               <goal>test</goal>
             </goals>
             <phase>integration-test</phase>
             <configuration>
               <suiteXmlFiles>
-                <suiteXmlFile>src/test/resources/connector-loading-tests-suite.xml</suiteXmlFile>
+                <suiteXmlFile>src/test/resources/server-properties-tests-suite.xml</suiteXmlFile>
               </suiteXmlFiles>
               <properties>
-                <suitename>connector-loading-tests</suitename>
+                <suitename>server-properties-tests</suitename>
               </properties>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/main/java/org/apache/sqoop/test/kdc/MiniKdcRunner.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/kdc/MiniKdcRunner.java b/test/src/main/java/org/apache/sqoop/test/kdc/MiniKdcRunner.java
index 83f960b..bca4412 100644
--- a/test/src/main/java/org/apache/sqoop/test/kdc/MiniKdcRunner.java
+++ b/test/src/main/java/org/apache/sqoop/test/kdc/MiniKdcRunner.java
@@ -18,31 +18,12 @@
 package org.apache.sqoop.test.kdc;
 
 import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
 import java.lang.reflect.Constructor;
-import java.math.BigInteger;
 import java.net.URL;
-import java.security.GeneralSecurityException;
-import java.security.InvalidKeyException;
-import java.security.Key;
-import java.security.KeyPair;
-import java.security.KeyPairGenerator;
-import java.security.KeyStore;
-import java.security.NoSuchAlgorithmException;
-import java.security.NoSuchProviderException;
 import java.security.Principal;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
-import java.security.SecureRandom;
-import java.security.SignatureException;
-import java.security.cert.Certificate;
-import java.security.cert.CertificateEncodingException;
-import java.security.cert.X509Certificate;
 import java.util.Collection;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -53,19 +34,16 @@ import java.util.concurrent.Callable;
 import javax.security.auth.Subject;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
-import javax.security.auth.x500.X500Principal;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.minikdc.MiniKdc;
-import org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory;
-import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
 import org.apache.sqoop.client.SqoopClient;
 import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.test.utils.HdfsUtils;
+import org.apache.sqoop.test.utils.SecurityUtils;
 import org.apache.sqoop.test.utils.SqoopUtils;
-import org.bouncycastle.x509.X509V1CertificateGenerator;
 
 /**
  * Represents a Minikdc setup. Minikdc should be only used together with
@@ -107,17 +85,17 @@ public class MiniKdcRunner extends KdcRunner {
     config.set("dfs.datanode.kerberos.principal", hadoopPrincipal);
     config.set("dfs.datanode.keytab.file", hadoopKeytabFile);
     String sslKeystoresDir = getTemporaryPath() + "/ssl-keystore";
-    String sslConfDir = getClasspathDir(MiniKdcRunner.class);
+    String sslConfDir = SqoopUtils.getClasspathDir(MiniKdcRunner.class);
     FileUtils.deleteDirectory(new File(sslKeystoresDir));
     FileUtils.forceMkdir(new File(sslKeystoresDir));
-    setupSSLConfig(sslKeystoresDir, sslConfDir, config, false, true);
-    config.set("dfs.https.server.keystore.resource", getSSLConfigFileName("ssl-server"));
+    SecurityUtils.setupSSLConfig(sslKeystoresDir, sslConfDir, config, false, true);
+    config.set("dfs.https.server.keystore.resource", SecurityUtils.getSSLConfigFileName("ssl-server"));
     // Configurations used by both NameNode and DataNode
     config.set("dfs.block.access.token.enable", "true");
     config.set("dfs.http.policy", "HTTPS_ONLY");
     // Configurations used by DFSClient
     config.set("dfs.data.transfer.protection", "privacy");
-    config.set("dfs.client.https.keystore.resource", getSSLConfigFileName("ssl-client"));
+    config.set("dfs.client.https.keystore.resource", SecurityUtils.getSSLConfigFileName("ssl-client"));
 
     // YARN related configurations
     config.set("yarn.resourcemanager.principal", hadoopPrincipal);
@@ -331,219 +309,4 @@ public class MiniKdcRunner extends KdcRunner {
       }
     }
   }
-
-  @SuppressWarnings("rawtypes")
-  private static String getClasspathDir(Class klass) throws Exception {
-    String file = klass.getName();
-    file = file.replace('.', '/') + ".class";
-    URL url = Thread.currentThread().getContextClassLoader().getResource(file);
-    String baseDir = url.toURI().getPath();
-    baseDir = baseDir.substring(0, baseDir.length() - file.length() - 1);
-    return baseDir;
-  }
-
-  /**
-   * Performs complete setup of SSL configuration. This includes keys, certs,
-   * keystores, truststores, the server SSL configuration file,
-   * the client SSL configuration file.
-   *
-   * @param keystoresDir String directory to save keystores
-   * @param sslConfDir String directory to save SSL configuration files
-   * @param conf Configuration
-   * @param useClientCert boolean true to make the client present a cert in the
-   * SSL handshake
-   * @param trustStore boolean true to create truststore, false not to create it
-   */
-  private void setupSSLConfig(String keystoresDir, String sslConfDir,
-      Configuration conf, boolean useClientCert, boolean trustStore)
-          throws Exception {
-    String clientKS = keystoresDir + "/clientKS.jks";
-    String clientPassword = "clientP";
-    String serverKS = keystoresDir + "/serverKS.jks";
-    String serverPassword = "serverP";
-    String trustKS = null;
-    String trustPassword = "trustP";
-
-    File sslClientConfFile = new File(sslConfDir, getSSLConfigFileName("ssl-client"));
-    File sslServerConfFile = new File(sslConfDir, getSSLConfigFileName("ssl-server"));
-
-    Map<String, X509Certificate> certs = new HashMap<String, X509Certificate>();
-
-    if (useClientCert) {
-      KeyPair cKP = generateKeyPair("RSA");
-      X509Certificate cCert = generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA");
-      createKeyStore(clientKS, clientPassword, "client", cKP.getPrivate(), cCert);
-      certs.put("client", cCert);
-    }
-
-    KeyPair sKP = generateKeyPair("RSA");
-    X509Certificate sCert = generateCertificate("CN=localhost, O=server", sKP, 30, "SHA1withRSA");
-    createKeyStore(serverKS, serverPassword, "server", sKP.getPrivate(), sCert);
-    certs.put("server", sCert);
-
-    if (trustStore) {
-      trustKS = keystoresDir + "/trustKS.jks";
-      createTrustStore(trustKS, trustPassword, certs);
-    }
-
-    Configuration clientSSLConf = createSSLConfig(
-        SSLFactory.Mode.CLIENT, clientKS, clientPassword, clientPassword, trustKS);
-    Configuration serverSSLConf = createSSLConfig(
-        SSLFactory.Mode.SERVER, serverKS, serverPassword, serverPassword, trustKS);
-
-    saveConfig(sslClientConfFile, clientSSLConf);
-    saveConfig(sslServerConfFile, serverSSLConf);
-
-    conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "ALLOW_ALL");
-    conf.set(SSLFactory.SSL_CLIENT_CONF_KEY, sslClientConfFile.getName());
-    conf.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile.getName());
-    conf.setBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY, useClientCert);
-  }
-
-  /**
-   * Returns an SSL configuration file name.  Under parallel test
-   * execution, this file name is parameterized by a unique ID to ensure that
-   * concurrent tests don't collide on an SSL configuration file.
-   *
-   * @param base the base of the file name
-   * @return SSL configuration file name for base
-   */
-  private static String getSSLConfigFileName(String base) {
-    String testUniqueForkId = System.getProperty("test.unique.fork.id");
-    String fileSuffix = testUniqueForkId != null ? "-" + testUniqueForkId : "";
-    return base + fileSuffix + ".xml";
-  }
-
-  /**
-   * Creates SSL configuration.
-   *
-   * @param mode SSLFactory.Mode mode to configure
-   * @param keystore String keystore file
-   * @param password String store password, or null to avoid setting store
-   *   password
-   * @param keyPassword String key password, or null to avoid setting key
-   *   password
-   * @param trustKS String truststore file
-   * @return Configuration for SSL
-   */
-  private static Configuration createSSLConfig(SSLFactory.Mode mode,
-      String keystore, String password, String keyPassword, String trustKS) {
-    String trustPassword = "trustP";
-
-    Configuration sslConf = new Configuration(false);
-    if (keystore != null) {
-      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
-          FileBasedKeyStoresFactory.SSL_KEYSTORE_LOCATION_TPL_KEY), keystore);
-    }
-    if (password != null) {
-      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
-          FileBasedKeyStoresFactory.SSL_KEYSTORE_PASSWORD_TPL_KEY), password);
-    }
-    if (keyPassword != null) {
-      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
-          FileBasedKeyStoresFactory.SSL_KEYSTORE_KEYPASSWORD_TPL_KEY),
-          keyPassword);
-    }
-    if (trustKS != null) {
-      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
-          FileBasedKeyStoresFactory.SSL_TRUSTSTORE_LOCATION_TPL_KEY), trustKS);
-    }
-    sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
-        FileBasedKeyStoresFactory.SSL_TRUSTSTORE_PASSWORD_TPL_KEY),
-        trustPassword);
-    sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
-        FileBasedKeyStoresFactory.SSL_TRUSTSTORE_RELOAD_INTERVAL_TPL_KEY), "1000");
-
-    return sslConf;
-  }
-
-  private static KeyPair generateKeyPair(String algorithm)
-      throws NoSuchAlgorithmException {
-    KeyPairGenerator keyGen = KeyPairGenerator.getInstance(algorithm);
-    keyGen.initialize(1024);
-    return keyGen.genKeyPair();
-  }
-
-  /**
-   * Create a self-signed X.509 Certificate.
-   *
-   * @param dn the X.509 Distinguished Name, eg "CN=Test, L=London, C=GB"
-   * @param pair the KeyPair
-   * @param days how many days from now the Certificate is valid for
-   * @param algorithm the signing algorithm, eg "SHA1withRSA"
-   * @return the self-signed certificate
-   */
-  private static X509Certificate generateCertificate(String dn, KeyPair pair, int days, String algorithm)
-      throws CertificateEncodingException,
-             InvalidKeyException,
-             IllegalStateException,
-             NoSuchProviderException, NoSuchAlgorithmException, SignatureException{
-    Date from = new Date();
-    Date to = new Date(from.getTime() + days * 86400000l);
-    BigInteger sn = new BigInteger(64, new SecureRandom());
-    KeyPair keyPair = pair;
-    X509V1CertificateGenerator certGen = new X509V1CertificateGenerator();
-    X500Principal dnName = new X500Principal(dn);
-
-    certGen.setSerialNumber(sn);
-    certGen.setIssuerDN(dnName);
-    certGen.setNotBefore(from);
-    certGen.setNotAfter(to);
-    certGen.setSubjectDN(dnName);
-    certGen.setPublicKey(keyPair.getPublic());
-    certGen.setSignatureAlgorithm(algorithm);
-
-    X509Certificate cert = certGen.generate(pair.getPrivate());
-    return cert;
-  }
-
-  private static void createKeyStore(String filename,
-      String password, String alias,
-      Key privateKey, Certificate cert)
-      throws GeneralSecurityException, IOException {
-    KeyStore ks = KeyStore.getInstance("JKS");
-    ks.load(null, null); // initialize
-    ks.setKeyEntry(alias, privateKey, password.toCharArray(),
-        new Certificate[]{cert});
-    saveKeyStore(ks, filename, password);
-  }
-
-  private static <T extends Certificate> void createTrustStore(
-      String filename, String password, Map<String, T> certs)
-      throws GeneralSecurityException, IOException {
-    KeyStore ks = KeyStore.getInstance("JKS");
-    ks.load(null, null); // initialize
-    for (Map.Entry<String, T> cert : certs.entrySet()) {
-      ks.setCertificateEntry(cert.getKey(), cert.getValue());
-    }
-    saveKeyStore(ks, filename, password);
-  }
-
-  private static void saveKeyStore(KeyStore ks, String filename,
-      String password)
-      throws GeneralSecurityException, IOException {
-    FileOutputStream out = new FileOutputStream(filename);
-    try {
-      ks.store(out, password.toCharArray());
-    } finally {
-      out.close();
-    }
-  }
-
-  /**
-   * Saves configuration to a file.
-   *
-   * @param file File to save
-   * @param conf Configuration contents to write to file
-   * @throws IOException if there is an I/O error saving the file
-   */
-  private static void saveConfig(File file, Configuration conf)
-      throws IOException {
-    Writer writer = new OutputStreamWriter(new FileOutputStream(file), "UTF-8");
-    try {
-      conf.writeXml(writer);
-    } finally {
-      writer.close();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/main/java/org/apache/sqoop/test/utils/SecurityUtils.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/utils/SecurityUtils.java b/test/src/main/java/org/apache/sqoop/test/utils/SecurityUtils.java
new file mode 100644
index 0000000..35b9b7d
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/utils/SecurityUtils.java
@@ -0,0 +1,269 @@
+/**
+ * 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.sqoop.test.utils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.bouncycastle.x509.X509V1CertificateGenerator;
+
+import javax.security.auth.x500.X500Principal;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.math.BigInteger;
+import java.security.GeneralSecurityException;
+import java.security.InvalidKeyException;
+import java.security.Key;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.SecureRandom;
+import java.security.SignatureException;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateEncodingException;
+import java.security.cert.X509Certificate;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+public class SecurityUtils {
+
+  public static final String CLIENT_KEYSTORE = "clientKS.jks";
+  public static final String SERVER_KEYSTORE = "serverKS.jks";
+  public static final String TRUSTSTORE = "trustKS.jks";
+
+  public static final String CLIENT_KEY_PASSWORD = "client_key";
+  public static final String CLIENT_KEY_STORE_PASSWORD = "client_keystore";
+  public static final String SERVER_KEY_PASSWORD = "server_key";
+  public static final String SERVER_KEY_STORE_PASSWORD = "server_keystore";
+
+  /**
+   * Performs complete setup of SSL configuration. This includes keys, certs,
+   * keystores, truststores, the server SSL configuration file,
+   * the client SSL configuration file.
+   *
+   * @param keystoresDir String directory to save keystores
+   * @param sslConfDir String directory to save SSL configuration files
+   * @param conf Configuration
+   * @param useClientCert boolean true to make the client present a cert in the
+   * SSL handshake
+   * @param trustStore boolean true to create truststore, false not to create it
+   */
+  public static X509Certificate setupSSLConfig(String keystoresDir, String sslConfDir,
+                                    Configuration conf, boolean useClientCert, boolean trustStore)
+    throws Exception {
+    String clientKeyStorePath = keystoresDir + "/" + CLIENT_KEYSTORE;
+    String serverKeyStorePath = keystoresDir + "/" + SERVER_KEYSTORE;
+    String serverPassword = "serverP";
+    String trustKS = null;
+    String trustPassword = "trustP";
+
+    File sslClientConfFile = new File(sslConfDir, getSSLConfigFileName("ssl-client"));
+    File sslServerConfFile = new File(sslConfDir, getSSLConfigFileName("ssl-server"));
+
+    Map<String, X509Certificate> certs = new HashMap<String, X509Certificate>();
+
+    String hostname = SqoopUtils.getLocalHostName();
+
+    if (useClientCert) {
+      KeyPair cKP = generateKeyPair("RSA");
+      X509Certificate cCert = generateCertificate("CN=" + hostname + ", O=client", cKP, 30, "SHA1withRSA");
+      createKeyStore(clientKeyStorePath, CLIENT_KEY_PASSWORD, CLIENT_KEY_STORE_PASSWORD, "client", cKP.getPrivate(), cCert);
+      certs.put("client", cCert);
+    }
+
+    KeyPair sKP = generateKeyPair("RSA");
+    X509Certificate sCert = generateCertificate("CN=" + hostname + ", O=server", sKP, 30, "SHA1withRSA");
+    createKeyStore(serverKeyStorePath, SERVER_KEY_PASSWORD, SERVER_KEY_STORE_PASSWORD, "server", sKP.getPrivate(), sCert);
+    certs.put("server", sCert);
+
+    if (trustStore) {
+      trustKS = keystoresDir + TRUSTSTORE;
+      createTrustStore(trustKS, trustPassword, certs);
+    }
+
+    Configuration clientSSLConf = createSSLConfig(
+      SSLFactory.Mode.CLIENT, clientKeyStorePath, CLIENT_KEY_STORE_PASSWORD, CLIENT_KEY_PASSWORD, trustKS);
+    Configuration serverSSLConf = createSSLConfig(
+      SSLFactory.Mode.SERVER, serverKeyStorePath, SERVER_KEY_STORE_PASSWORD, SERVER_KEY_PASSWORD, trustKS);
+
+    saveConfig(sslClientConfFile, clientSSLConf);
+    saveConfig(sslServerConfFile, serverSSLConf);
+
+    conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "ALLOW_ALL");
+    conf.set(SSLFactory.SSL_CLIENT_CONF_KEY, sslClientConfFile.getName());
+    conf.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile.getName());
+    conf.setBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY, useClientCert);
+
+    return sCert;
+  }
+
+  /**
+   * Returns an SSL configuration file name.  Under parallel test
+   * execution, this file name is parameterized by a unique ID to ensure that
+   * concurrent tests don't collide on an SSL configuration file.
+   *
+   * @param base the base of the file name
+   * @return SSL configuration file name for base
+   */
+  public static String getSSLConfigFileName(String base) {
+    String testUniqueForkId = System.getProperty("test.unique.fork.id");
+    String fileSuffix = testUniqueForkId != null ? "-" + testUniqueForkId : "";
+    return base + fileSuffix + ".xml";
+  }
+
+  /**
+   * Creates SSL configuration.
+   *
+   * @param mode SSLFactory.Mode mode to configure
+   * @param keystore String keystore file
+   * @param keyStorePassword String store password, or null to avoid setting store
+   *   password
+   * @param keyPassword String key password, or null to avoid setting key
+   *   password
+   * @param trustKS String truststore file
+   * @return Configuration for SSL
+   */
+  public static Configuration createSSLConfig(SSLFactory.Mode mode,
+                                              String keystore, String keyStorePassword,
+                                              String keyPassword, String trustKS) {
+    String trustPassword = "trustP";
+
+    Configuration sslConf = new Configuration(false);
+    if (keystore != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_KEYSTORE_LOCATION_TPL_KEY), keystore);
+    }
+
+    if (keyStorePassword != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_KEYSTORE_PASSWORD_TPL_KEY), keyStorePassword);
+    }
+    if (keyPassword != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_KEYSTORE_KEYPASSWORD_TPL_KEY),
+        keyPassword);
+    }
+    if (trustKS != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_TRUSTSTORE_LOCATION_TPL_KEY), trustKS);
+    }
+    sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+      FileBasedKeyStoresFactory.SSL_TRUSTSTORE_PASSWORD_TPL_KEY),
+      trustPassword);
+    sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+      FileBasedKeyStoresFactory.SSL_TRUSTSTORE_RELOAD_INTERVAL_TPL_KEY), "1000");
+
+    return sslConf;
+  }
+
+  public static KeyPair generateKeyPair(String algorithm)
+    throws NoSuchAlgorithmException {
+    KeyPairGenerator keyGen = KeyPairGenerator.getInstance(algorithm);
+    keyGen.initialize(1024);
+    return keyGen.genKeyPair();
+  }
+
+  /**
+   * Create a self-signed X.509 Certificate.
+   *
+   * @param dn the X.509 Distinguished Name, eg "CN=Test, L=London, C=GB"
+   * @param pair the KeyPair
+   * @param days how many days from now the Certificate is valid for
+   * @param algorithm the signing algorithm, eg "SHA1withRSA"
+   * @return the self-signed certificate
+   */
+  public static X509Certificate generateCertificate(String dn, KeyPair pair, int days, String algorithm)
+    throws CertificateEncodingException,
+    InvalidKeyException,
+    IllegalStateException,
+    NoSuchProviderException, NoSuchAlgorithmException, SignatureException {
+    Date from = new Date();
+    Date to = new Date(from.getTime() + days * 86400000l);
+    BigInteger sn = new BigInteger(64, new SecureRandom());
+    KeyPair keyPair = pair;
+    X509V1CertificateGenerator certGen = new X509V1CertificateGenerator();
+    X500Principal dnName = new X500Principal(dn);
+
+    certGen.setSerialNumber(sn);
+    certGen.setIssuerDN(dnName);
+    certGen.setNotBefore(from);
+    certGen.setNotAfter(to);
+    certGen.setSubjectDN(dnName);
+    certGen.setPublicKey(keyPair.getPublic());
+    certGen.setSignatureAlgorithm(algorithm);
+
+    X509Certificate cert = certGen.generate(pair.getPrivate());
+    return cert;
+  }
+
+  public static void createKeyStore(String filename,
+                                    String keyPassword, String keyStorePassword,
+                                    String alias, Key privateKey, Certificate cert)
+    throws GeneralSecurityException, IOException {
+    KeyStore ks = KeyStore.getInstance("JKS");
+    ks.load(null, null); // initialize
+    ks.setKeyEntry(alias, privateKey, keyPassword.toCharArray(),
+      new Certificate[]{cert});
+    saveKeyStore(ks, filename, keyStorePassword);
+  }
+
+  public static <T extends Certificate> void createTrustStore(
+    String filename, String password, Map<String, T> certs)
+    throws GeneralSecurityException, IOException {
+    KeyStore ks = KeyStore.getInstance("JKS");
+    ks.load(null, null); // initialize
+    for (Map.Entry<String, T> cert : certs.entrySet()) {
+      ks.setCertificateEntry(cert.getKey(), cert.getValue());
+    }
+    saveKeyStore(ks, filename, password);
+  }
+
+  public static void saveKeyStore(KeyStore ks, String filename,
+                                   String password)
+    throws GeneralSecurityException, IOException {
+    FileOutputStream out = new FileOutputStream(filename);
+    try {
+      ks.store(out, password.toCharArray());
+    } finally {
+      out.close();
+    }
+  }
+
+  /**
+   * Saves configuration to a file.
+   *
+   * @param file File to save
+   * @param conf Configuration contents to write to file
+   * @throws IOException if there is an I/O error saving the file
+   */
+  public static void saveConfig(File file, Configuration conf)
+    throws IOException {
+    Writer writer = new OutputStreamWriter(new FileOutputStream(file), "UTF-8");
+    try {
+      conf.writeXml(writer);
+    } finally {
+      writer.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/main/java/org/apache/sqoop/test/utils/SqoopUtils.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/utils/SqoopUtils.java b/test/src/main/java/org/apache/sqoop/test/utils/SqoopUtils.java
index 6614b19..3e0566f 100644
--- a/test/src/main/java/org/apache/sqoop/test/utils/SqoopUtils.java
+++ b/test/src/main/java/org/apache/sqoop/test/utils/SqoopUtils.java
@@ -18,6 +18,7 @@
 package org.apache.sqoop.test.utils;
 
 import java.net.InetAddress;
+import java.net.URL;
 import java.net.UnknownHostException;
 import java.util.Locale;
 import java.util.Random;
@@ -44,7 +45,7 @@ public class SqoopUtils {
     object.setName(prefix + rand.nextLong());
   }
 
-  //Retrieve the FQDN of the current host
+  // Retrieve the FQDN of the current host
   public static String getLocalHostName() {
     String fqdn;
     try {
@@ -54,4 +55,26 @@ public class SqoopUtils {
     }
     return fqdn;
   }
+
+  // Retrieve the IP address of the current host
+  public static String getLocalIpAddress() {
+    String address;
+    try {
+      address = InetAddress.getLocalHost().getHostAddress();
+    } catch (UnknownHostException e1) {
+      address = "127.0.0.1";
+    }
+    return address;
+  }
+
+
+  @SuppressWarnings("rawtypes")
+  public static String getClasspathDir(Class klass) throws Exception {
+    String file = klass.getName();
+    file = file.replace('.', '/') + ".class";
+    URL url = Thread.currentThread().getContextClassLoader().getResource(file);
+    String baseDir = url.toURI().getPath();
+    baseDir = baseDir.substring(0, baseDir.length() - file.length() - 1);
+    return baseDir;
+  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/test/java/org/apache/sqoop/integration/connectorloading/BlacklistedConnectorTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/connectorloading/BlacklistedConnectorTest.java b/test/src/test/java/org/apache/sqoop/integration/connectorloading/BlacklistedConnectorTest.java
deleted file mode 100644
index 7f0575b..0000000
--- a/test/src/test/java/org/apache/sqoop/integration/connectorloading/BlacklistedConnectorTest.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     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.sqoop.integration.connectorloading;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.core.ConfigurationConstants;
-import org.apache.sqoop.test.infrastructure.Infrastructure;
-import org.apache.sqoop.test.infrastructure.SqoopTestCase;
-import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
-import org.apache.sqoop.test.infrastructure.providers.KdcInfrastructureProvider;
-import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
-import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
-import org.apache.sqoop.test.utils.HdfsUtils;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-@Test(groups = "no-real-cluster")
-@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class})
-public class BlacklistedConnectorTest extends SqoopTestCase {
-  private SqoopMiniCluster sqoopMiniCluster;
-
-  public static class DerbySqoopMiniCluster extends JettySqoopMiniCluster {
-    public DerbySqoopMiniCluster(String temporaryPath, Configuration configuration) throws Exception {
-      super(temporaryPath, configuration);
-    }
-
-    @Override
-    protected Map<String, String> getBlacklistedConnectorConfiguration() {
-      Map<String, String> properties = new HashMap<>();
-
-      properties.put(ConfigurationConstants.BLACKLISTED_CONNECTORS, "fake-connector:generic-jdbc-connector");
-      return properties;
-    }
-  }
-
-  @BeforeMethod(dependsOnMethods = { "init" })
-  public void startSqoopMiniCluster() throws Exception {
-    // And use them for new Derby repo instance
-    sqoopMiniCluster = new DerbySqoopMiniCluster(HdfsUtils.joinPathFragments(super
-        .getTemporaryPath(), getTestName()), getHadoopConf());
-    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
-    if (kdcProvider != null) {
-      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
-    }
-
-    // Start server
-    sqoopMiniCluster.start();
-
-    // Initialize Sqoop Client API
-    initSqoopClient(sqoopMiniCluster.getServerUrl());
-  }
-
-  @Test(expectedExceptions = {SqoopException.class})
-  public void testCreateLinkWithNonexistantConnector() throws Exception {
-    getClient().createLink("generic-jdbc-connector");
-  }
-
-  @AfterMethod
-  public void stopCluster() throws Exception {
-    sqoopMiniCluster.stop();
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java b/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java
deleted file mode 100644
index 87f0eb1..0000000
--- a/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     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.sqoop.integration.connectorloading;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.sqoop.core.ConfigurationConstants;
-import org.apache.sqoop.model.MDriverConfig;
-import org.apache.sqoop.model.MJob;
-import org.apache.sqoop.model.MLink;
-import org.apache.sqoop.test.infrastructure.Infrastructure;
-import org.apache.sqoop.test.infrastructure.SqoopTestCase;
-import org.apache.sqoop.test.infrastructure.providers.DatabaseInfrastructureProvider;
-import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
-import org.apache.sqoop.test.infrastructure.providers.KdcInfrastructureProvider;
-import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
-import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
-import org.apache.sqoop.test.utils.ConnectorUtils;
-import org.apache.sqoop.test.utils.HdfsUtils;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-@Test(groups = "no-real-cluster")
-@Infrastructure(dependencies = {KdcInfrastructureProvider.class, DatabaseInfrastructureProvider.class, HadoopInfrastructureProvider.class})
-public class ClasspathTest extends SqoopTestCase {
-
-  private static final String TEST_CONNECTOR_JAR_NAME = "test-connector.jar";
-  private static final String TEST_DEPENDENCY_JAR_NAME = "test-dependency.jar";
-
-  private static final String[] CONNECTOR_SOURCE_FILES = {
-    "TestConnector/TestConnector.java",
-    "TestConnector/TestLinkConfiguration.java",
-    "TestConnector/TestLoader.java",
-    "TestConnector/TestToDestroyer.java",
-    "TestConnector/TestToInitializer.java",
-    "TestConnector/TestToJobConfiguration.java"
-  };
-
-  private static final String[] CONNECTOR_DEPENDENCY_SOURCE_FILES = {
-    "TestConnector/TestDependency.java"
-  };
-
-  private static final String[] CONNECTOR_PROPERTY_FILES = {
-    "TestConnector/sqoopconnector.properties"
-  };
-
-  private ClassLoader classLoader;
-  private SqoopMiniCluster sqoopMiniCluster;
-
-  public static class DerbySqoopMiniCluster extends JettySqoopMiniCluster {
-
-    private String extraClasspath;
-    private String jobExtraClasspath;
-
-    public DerbySqoopMiniCluster(String temporaryPath, Configuration configuration, String extraClasspath, String jobExtraClasspath) throws Exception {
-      super(temporaryPath, configuration);
-      this.extraClasspath = extraClasspath;
-      this.jobExtraClasspath = jobExtraClasspath;
-    }
-
-    @Override
-    protected Map<String, String> getClasspathConfiguration() {
-      Map<String, String> properties = new HashMap<>();
-
-      if (extraClasspath != null) {
-        properties.put(ConfigurationConstants.CLASSPATH, extraClasspath);
-      }
-      if (jobExtraClasspath != null) {
-        properties.put(ConfigurationConstants.JOB_CLASSPATH, jobExtraClasspath);
-      }
-
-
-      return properties;
-    }
-  }
-
-  public void startSqoopMiniCluster(String extraClasspath, String jobExtraClasspath) throws Exception {
-    // And use them for new Derby repo instance
-    sqoopMiniCluster = new DerbySqoopMiniCluster(HdfsUtils.joinPathFragments
-        (super.getTemporaryPath(), getTestName()), getHadoopConf(), extraClasspath, jobExtraClasspath);
-    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
-    if (kdcProvider != null) {
-      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
-    }
-
-    // Start server
-    sqoopMiniCluster.start();
-
-    // Initialize Sqoop Client API
-    initSqoopClient(sqoopMiniCluster.getServerUrl());
-  }
-
-  @BeforeMethod
-  public void captureClasspath() {
-    classLoader = Thread.currentThread().getContextClassLoader();
-  }
-
-  @AfterMethod
-  public void restoreClasspath(){
-    Thread.currentThread().setContextClassLoader(classLoader);
-  }
-
-  @Test
-  public void testClasspathSqoopProperties() throws Exception {
-    Map<String, String> jarMap = ConnectorUtils.compileTestConnectorAndDependency(
-            CONNECTOR_SOURCE_FILES,
-            CONNECTOR_DEPENDENCY_SOURCE_FILES,
-            CONNECTOR_PROPERTY_FILES,
-            TEST_CONNECTOR_JAR_NAME,
-            TEST_DEPENDENCY_JAR_NAME,
-            false);
-    startSqoopMiniCluster(jarMap.get(TEST_CONNECTOR_JAR_NAME), jarMap.get
-      (TEST_DEPENDENCY_JAR_NAME));
-    createAndLoadTableCities();
-
-    MJob job = prepareJob();
-
-    prepareDriverConfig(job);
-
-    saveJob(job);
-
-    executeJob(job);
-
-    stopSqoop();
-    ConnectorUtils.deleteJars(jarMap);
-  }
-
-  @Test
-  public void testClasspathDriverInput() throws Exception{
-    Map<String, String> jarMap = ConnectorUtils.compileTestConnectorAndDependency(
-        CONNECTOR_SOURCE_FILES,
-        CONNECTOR_DEPENDENCY_SOURCE_FILES,
-        CONNECTOR_PROPERTY_FILES,
-        TEST_CONNECTOR_JAR_NAME,
-        TEST_DEPENDENCY_JAR_NAME,
-        false);
-    startSqoopMiniCluster(jarMap.get(TEST_CONNECTOR_JAR_NAME), null);
-    createAndLoadTableCities();
-
-    MJob job = prepareJob();
-
-    MDriverConfig driverConfig = prepareDriverConfig(job);
-
-    List<String> extraJars = new ArrayList<>();
-    extraJars.add("file:" + jarMap.get(TEST_DEPENDENCY_JAR_NAME));
-    driverConfig.getListInput("jarConfig.extraJars").setValue(extraJars);
-
-    saveJob(job);
-
-    executeJob(job);
-
-    stopSqoop();
-    ConnectorUtils.deleteJars(jarMap);
-  }
-
-  private MJob prepareJob() {
-    MLink rdbmsConnection = getClient().createLink("generic-jdbc-connector");
-    fillRdbmsLinkConfig(rdbmsConnection);
-    saveLink(rdbmsConnection);
-
-    MLink testConnection = getClient().createLink("test-connector");
-    saveLink(testConnection);
-
-    MJob job = getClient().createJob(rdbmsConnection.getName(), testConnection.getName());
-
-    fillRdbmsFromConfig(job, "id");
-
-    return job;
-  }
-
-  private MDriverConfig prepareDriverConfig(MJob job) {
-    MDriverConfig driverConfig = job.getDriverConfig();
-    driverConfig.getIntegerInput("throttlingConfig.numExtractors").setValue(3);
-
-    return driverConfig;
-  }
-
-  private void stopSqoop() throws Exception {
-    sqoopMiniCluster.stop();
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/test/java/org/apache/sqoop/integration/connectorloading/ConnectorClasspathIsolationTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/connectorloading/ConnectorClasspathIsolationTest.java b/test/src/test/java/org/apache/sqoop/integration/connectorloading/ConnectorClasspathIsolationTest.java
deleted file mode 100644
index a82a4da..0000000
--- a/test/src/test/java/org/apache/sqoop/integration/connectorloading/ConnectorClasspathIsolationTest.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     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.sqoop.integration.connectorloading;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.sqoop.core.ConfigurationConstants;
-import org.apache.sqoop.model.MDriverConfig;
-import org.apache.sqoop.model.MJob;
-import org.apache.sqoop.model.MLink;
-import org.apache.sqoop.test.infrastructure.Infrastructure;
-import org.apache.sqoop.test.infrastructure.SqoopTestCase;
-import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
-import org.apache.sqoop.test.infrastructure.providers.KdcInfrastructureProvider;
-import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
-import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
-import org.apache.sqoop.test.utils.ConnectorUtils;
-import org.apache.sqoop.test.utils.HdfsUtils;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-@Test(groups = "no-real-cluster")
-@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class})
-public class ConnectorClasspathIsolationTest extends SqoopTestCase {
-
-  private static final String TEST_FROM_CONNECTOR_JAR_NAME = "test-from-connector.jar";
-  private static final String TEST_TO_CONNECTOR_JAR_NAME = "test-to-connector.jar";
-  private static final String TEST_FROM_DEPENDENCY_JAR_NAME = "test-from-dependency.jar";
-  private static final String TEST_TO_DEPENDENCY_JAR_NAME = "test-to-dependency.jar";
-
-  private static final String[] FROM_CONNECTOR_SOURCE_FILES = {
-    "TestConnectorClasspathIsolation/from/TestFromConnector.java",
-    "TestConnectorClasspathIsolation/from/TestExtractor.java",
-    "TestConnectorClasspathIsolation/from/TestFromDestroyer.java",
-    "TestConnectorClasspathIsolation/from/TestFromInitializer.java",
-    "TestConnectorClasspathIsolation/from/TestFromJobConfiguration.java",
-    "TestConnectorClasspathIsolation/from/TestPartition.java",
-    "TestConnectorClasspathIsolation/from/TestPartitioner.java",
-    "TestConnectorClasspathIsolation/from/TestFromLinkConfiguration.java"
-  };
-
-  private static final String[] FROM_CONNECTOR_DEPENDENCY_SOURCE_FILES = {
-    "TestConnectorClasspathIsolation/from/TestClasspathIsolation.java"
-  };
-
-  private static final String[] FROM_CONNECTOR_PROPERTY_FILES = {
-    "TestConnectorClasspathIsolation/from/sqoopconnector.properties"
-  };
-
-  private static final String[] TO_CONNECTOR_SOURCE_FILES = {
-    "TestConnectorClasspathIsolation/to/TestToConnector.java",
-    "TestConnectorClasspathIsolation/to/TestLoader.java",
-    "TestConnectorClasspathIsolation/to/TestToDestroyer.java",
-    "TestConnectorClasspathIsolation/to/TestToInitializer.java",
-    "TestConnectorClasspathIsolation/to/TestToJobConfiguration.java",
-    "TestConnectorClasspathIsolation/to/TestToLinkConfiguration.java"
-  };
-
-  private static final String[] TO_CONNECTOR_DEPENDENCY_SOURCE_FILES = {
-    "TestConnectorClasspathIsolation/to/TestClasspathIsolation.java"
-  };
-
-  private static final String[] TO_CONNECTOR_PROPERTY_FILES = {
-    "TestConnectorClasspathIsolation/to/sqoopconnector.properties"
-  };
-
-  private ClassLoader classLoader;
-  private SqoopMiniCluster sqoopMiniCluster;
-
-  public static class DerbySqoopMiniCluster extends JettySqoopMiniCluster {
-
-    private String extraClasspath;
-
-    public DerbySqoopMiniCluster(String temporaryPath, Configuration configuration, String extraClasspath) throws Exception {
-      super(temporaryPath, configuration);
-      this.extraClasspath = extraClasspath;
-    }
-
-    @Override
-    protected Map<String, String> getClasspathConfiguration() {
-      Map<String, String> properties = new HashMap<>();
-
-      if (extraClasspath != null) {
-        properties.put(ConfigurationConstants.CLASSPATH, extraClasspath);
-      }
-
-      return properties;
-    }
-  }
-
-  public void startSqoopMiniCluster(String extraClasspath) throws Exception {
-    // And use them for new Derby repo instance
-    sqoopMiniCluster = new DerbySqoopMiniCluster(HdfsUtils.joinPathFragments(super.getTemporaryPath(), getTestName()), getHadoopConf(), extraClasspath);
-    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
-    if (kdcProvider != null) {
-      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
-    }
-
-    // Start server
-    sqoopMiniCluster.start();
-
-    // Initialize Sqoop Client API
-    initSqoopClient(sqoopMiniCluster.getServerUrl());
-  }
-
-  @BeforeMethod
-  public void captureClasspath() {
-    classLoader = Thread.currentThread().getContextClassLoader();
-  }
-
-  @AfterMethod
-  public void restoreClasspath(){
-    Thread.currentThread().setContextClassLoader(classLoader);
-  }
-
-  @Test
-  public void testConnectorClasspathIsolation() throws Exception {
-    Map<String, String> fromConnectorJarMap = ConnectorUtils.compileTestConnectorAndDependency(
-        FROM_CONNECTOR_SOURCE_FILES,
-        FROM_CONNECTOR_DEPENDENCY_SOURCE_FILES,
-        FROM_CONNECTOR_PROPERTY_FILES,
-        TEST_FROM_CONNECTOR_JAR_NAME,
-        TEST_FROM_DEPENDENCY_JAR_NAME,
-        true);
-    Map<String, String> toConnectorJarMap = ConnectorUtils.compileTestConnectorAndDependency(
-        TO_CONNECTOR_SOURCE_FILES,
-        TO_CONNECTOR_DEPENDENCY_SOURCE_FILES,
-        TO_CONNECTOR_PROPERTY_FILES,
-        TEST_TO_CONNECTOR_JAR_NAME,
-        TEST_TO_DEPENDENCY_JAR_NAME,
-        true);
-    startSqoopMiniCluster(
-        StringUtils.join(Arrays.asList(fromConnectorJarMap.get(TEST_FROM_CONNECTOR_JAR_NAME), toConnectorJarMap.get(TEST_TO_CONNECTOR_JAR_NAME)), ":"));
-
-    MJob job = prepareJob();
-
-    prepareDriverConfig(job);
-
-    saveJob(job);
-
-    executeJob(job);
-
-    stopSqoop();
-    ConnectorUtils.deleteJars(fromConnectorJarMap);
-  }
-
-  private MJob prepareJob() {
-    MLink rdbmsConnection = getClient().createLink("test-from-connector");
-    saveLink(rdbmsConnection);
-
-    MLink testConnection = getClient().createLink("test-to-connector");
-    saveLink(testConnection);
-
-    MJob job = getClient().createJob(rdbmsConnection.getName(), testConnection.getName());
-
-    return job;
-  }
-
-  private MDriverConfig prepareDriverConfig(MJob job) {
-    MDriverConfig driverConfig = job.getDriverConfig();
-    driverConfig.getIntegerInput("throttlingConfig.numExtractors").setValue(3);
-
-    return driverConfig;
-  }
-
-  private void stopSqoop() throws Exception {
-    sqoopMiniCluster.stop();
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/test/java/org/apache/sqoop/integration/serverproperties/BlacklistedConnectorTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/serverproperties/BlacklistedConnectorTest.java b/test/src/test/java/org/apache/sqoop/integration/serverproperties/BlacklistedConnectorTest.java
new file mode 100644
index 0000000..438e182
--- /dev/null
+++ b/test/src/test/java/org/apache/sqoop/integration/serverproperties/BlacklistedConnectorTest.java
@@ -0,0 +1,82 @@
+/**
+ * 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.sqoop.integration.serverproperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.core.ConfigurationConstants;
+import org.apache.sqoop.test.infrastructure.Infrastructure;
+import org.apache.sqoop.test.infrastructure.SqoopTestCase;
+import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.KdcInfrastructureProvider;
+import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
+import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
+import org.apache.sqoop.test.utils.HdfsUtils;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+@Test(groups = "no-real-cluster")
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class})
+public class BlacklistedConnectorTest extends SqoopTestCase {
+  private SqoopMiniCluster sqoopMiniCluster;
+
+  public static class DerbySqoopMiniCluster extends JettySqoopMiniCluster {
+    public DerbySqoopMiniCluster(String temporaryPath, Configuration configuration) throws Exception {
+      super(temporaryPath, configuration);
+    }
+
+    @Override
+    protected Map<String, String> getBlacklistedConnectorConfiguration() {
+      Map<String, String> properties = new HashMap<>();
+
+      properties.put(ConfigurationConstants.BLACKLISTED_CONNECTORS, "fake-connector:generic-jdbc-connector");
+      return properties;
+    }
+  }
+
+  @BeforeMethod(dependsOnMethods = { "init" })
+  public void startSqoopMiniCluster() throws Exception {
+    // And use them for new Derby repo instance
+    sqoopMiniCluster = new DerbySqoopMiniCluster(HdfsUtils.joinPathFragments(super
+        .getTemporaryPath(), getTestName()), getHadoopConf());
+    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
+    if (kdcProvider != null) {
+      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
+    }
+
+    // Start server
+    sqoopMiniCluster.start();
+
+    // Initialize Sqoop Client API
+    initSqoopClient(sqoopMiniCluster.getServerUrl());
+  }
+
+  @Test(expectedExceptions = {SqoopException.class})
+  public void testCreateLinkWithNonexistantConnector() throws Exception {
+    getClient().createLink("generic-jdbc-connector");
+  }
+
+  @AfterMethod
+  public void stopCluster() throws Exception {
+    sqoopMiniCluster.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/96f3d9c1/test/src/test/java/org/apache/sqoop/integration/serverproperties/ClasspathTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/serverproperties/ClasspathTest.java b/test/src/test/java/org/apache/sqoop/integration/serverproperties/ClasspathTest.java
new file mode 100644
index 0000000..1a33230
--- /dev/null
+++ b/test/src/test/java/org/apache/sqoop/integration/serverproperties/ClasspathTest.java
@@ -0,0 +1,202 @@
+/**
+ * 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.sqoop.integration.serverproperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.core.ConfigurationConstants;
+import org.apache.sqoop.model.MDriverConfig;
+import org.apache.sqoop.model.MJob;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.test.infrastructure.Infrastructure;
+import org.apache.sqoop.test.infrastructure.SqoopTestCase;
+import org.apache.sqoop.test.infrastructure.providers.DatabaseInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.KdcInfrastructureProvider;
+import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
+import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
+import org.apache.sqoop.test.utils.ConnectorUtils;
+import org.apache.sqoop.test.utils.HdfsUtils;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+@Test(groups = "no-real-cluster")
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, DatabaseInfrastructureProvider.class, HadoopInfrastructureProvider.class})
+public class ClasspathTest extends SqoopTestCase {
+
+  private static final String TEST_CONNECTOR_JAR_NAME = "test-connector.jar";
+  private static final String TEST_DEPENDENCY_JAR_NAME = "test-dependency.jar";
+
+  private static final String[] CONNECTOR_SOURCE_FILES = {
+    "TestConnector/TestConnector.java",
+    "TestConnector/TestLinkConfiguration.java",
+    "TestConnector/TestLoader.java",
+    "TestConnector/TestToDestroyer.java",
+    "TestConnector/TestToInitializer.java",
+    "TestConnector/TestToJobConfiguration.java"
+  };
+
+  private static final String[] CONNECTOR_DEPENDENCY_SOURCE_FILES = {
+    "TestConnector/TestDependency.java"
+  };
+
+  private static final String[] CONNECTOR_PROPERTY_FILES = {
+    "TestConnector/sqoopconnector.properties"
+  };
+
+  private ClassLoader classLoader;
+  private SqoopMiniCluster sqoopMiniCluster;
+
+  public static class DerbySqoopMiniCluster extends JettySqoopMiniCluster {
+
+    private String extraClasspath;
+    private String jobExtraClasspath;
+
+    public DerbySqoopMiniCluster(String temporaryPath, Configuration configuration, String extraClasspath, String jobExtraClasspath) throws Exception {
+      super(temporaryPath, configuration);
+      this.extraClasspath = extraClasspath;
+      this.jobExtraClasspath = jobExtraClasspath;
+    }
+
+    @Override
+    protected Map<String, String> getClasspathConfiguration() {
+      Map<String, String> properties = new HashMap<>();
+
+      if (extraClasspath != null) {
+        properties.put(ConfigurationConstants.CLASSPATH, extraClasspath);
+      }
+      if (jobExtraClasspath != null) {
+        properties.put(ConfigurationConstants.JOB_CLASSPATH, jobExtraClasspath);
+      }
+
+
+      return properties;
+    }
+  }
+
+  public void startSqoopMiniCluster(String extraClasspath, String jobExtraClasspath) throws Exception {
+    // And use them for new Derby repo instance
+    sqoopMiniCluster = new DerbySqoopMiniCluster(HdfsUtils.joinPathFragments
+        (super.getTemporaryPath(), getTestName()), getHadoopConf(), extraClasspath, jobExtraClasspath);
+    KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
+    if (kdcProvider != null) {
+      sqoopMiniCluster.setKdc(kdcProvider.getInstance());
+    }
+
+    // Start server
+    sqoopMiniCluster.start();
+
+    // Initialize Sqoop Client API
+    initSqoopClient(sqoopMiniCluster.getServerUrl());
+  }
+
+  @BeforeMethod
+  public void captureClasspath() {
+    classLoader = Thread.currentThread().getContextClassLoader();
+  }
+
+  @AfterMethod
+  public void restoreClasspath(){
+    Thread.currentThread().setContextClassLoader(classLoader);
+  }
+
+  @Test
+  public void testClasspathSqoopProperties() throws Exception {
+    Map<String, String> jarMap = ConnectorUtils.compileTestConnectorAndDependency(
+            CONNECTOR_SOURCE_FILES,
+            CONNECTOR_DEPENDENCY_SOURCE_FILES,
+            CONNECTOR_PROPERTY_FILES,
+            TEST_CONNECTOR_JAR_NAME,
+            TEST_DEPENDENCY_JAR_NAME,
+            false);
+    startSqoopMiniCluster(jarMap.get(TEST_CONNECTOR_JAR_NAME), jarMap.get
+      (TEST_DEPENDENCY_JAR_NAME));
+    createAndLoadTableCities();
+
+    MJob job = prepareJob();
+
+    prepareDriverConfig(job);
+
+    saveJob(job);
+
+    executeJob(job);
+
+    stopSqoop();
+    ConnectorUtils.deleteJars(jarMap);
+  }
+
+  @Test
+  public void testClasspathDriverInput() throws Exception{
+    Map<String, String> jarMap = ConnectorUtils.compileTestConnectorAndDependency(
+        CONNECTOR_SOURCE_FILES,
+        CONNECTOR_DEPENDENCY_SOURCE_FILES,
+        CONNECTOR_PROPERTY_FILES,
+        TEST_CONNECTOR_JAR_NAME,
+        TEST_DEPENDENCY_JAR_NAME,
+        false);
+    startSqoopMiniCluster(jarMap.get(TEST_CONNECTOR_JAR_NAME), null);
+    createAndLoadTableCities();
+
+    MJob job = prepareJob();
+
+    MDriverConfig driverConfig = prepareDriverConfig(job);
+
+    List<String> extraJars = new ArrayList<>();
+    extraJars.add("file:" + jarMap.get(TEST_DEPENDENCY_JAR_NAME));
+    driverConfig.getListInput("jarConfig.extraJars").setValue(extraJars);
+
+    saveJob(job);
+
+    executeJob(job);
+
+    stopSqoop();
+    ConnectorUtils.deleteJars(jarMap);
+  }
+
+  private MJob prepareJob() {
+    MLink rdbmsConnection = getClient().createLink("generic-jdbc-connector");
+    fillRdbmsLinkConfig(rdbmsConnection);
+    saveLink(rdbmsConnection);
+
+    MLink testConnection = getClient().createLink("test-connector");
+    saveLink(testConnection);
+
+    MJob job = getClient().createJob(rdbmsConnection.getName(), testConnection.getName());
+
+    fillRdbmsFromConfig(job, "id");
+
+    return job;
+  }
+
+  private MDriverConfig prepareDriverConfig(MJob job) {
+    MDriverConfig driverConfig = job.getDriverConfig();
+    driverConfig.getIntegerInput("throttlingConfig.numExtractors").setValue(3);
+
+    return driverConfig;
+  }
+
+  private void stopSqoop() throws Exception {
+    sqoopMiniCluster.stop();
+  }
+}