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 2015/12/16 20:40:38 UTC

sqoop git commit: SQOOP-2705: Sqoop2: Add kerberos support for SqoopMiniCluster

Repository: sqoop
Updated Branches:
  refs/heads/sqoop2 6e33d6afb -> 5329c1b33


SQOOP-2705: Sqoop2: Add kerberos support for SqoopMiniCluster

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

Branch: refs/heads/sqoop2
Commit: 5329c1b332e0200b1ae39f2ef283fa55762807cf
Parents: 6e33d6a
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Wed Dec 16 20:39:50 2015 +0100
Committer: Jarek Jarcec Cecho <ja...@apache.org>
Committed: Wed Dec 16 20:39:50 2015 +0100

----------------------------------------------------------------------
 pom.xml                                         |  11 +
 test/pom.xml                                    |  11 +
 .../test/infrastructure/SqoopTestCase.java      |  50 +++-
 .../DatabaseInfrastructureProvider.java         |   6 +
 .../providers/HadoopInfrastructureProvider.java |   6 +
 .../providers/InfrastructureProvider.java       |   6 +
 .../providers/KdcInfrastructureProvider.java    |  89 +++++++
 .../providers/SqoopInfrastructureProvider.java  |   8 +-
 .../org/apache/sqoop/test/kdc/KdcRunner.java    |  91 +++++++
 .../apache/sqoop/test/kdc/KdcRunnerFactory.java |  38 +++
 .../apache/sqoop/test/kdc/MiniKdcRunner.java    | 235 +++++++++++++++++++
 .../sqoop/test/kdc/NoKerberosKdcRunner.java     |  66 ++++++
 .../test/minicluster/JettySqoopMiniCluster.java |  17 +-
 .../test/minicluster/SqoopMiniCluster.java      |  23 +-
 .../minicluster/SqoopMiniClusterFactory.java    |   7 +-
 .../sqoop/test/testcases/JettyTestCase.java     |  28 ++-
 .../org/apache/sqoop/test/utils/SqoopUtils.java |  14 ++
 .../server/InformalObjectNameTest.java          |   3 +-
 .../integration/server/ShowJobInOrderTest.java  |   3 +-
 .../SubmissionWithDisabledModelObjectsTest.java |   3 +-
 .../sqoop/integration/server/VersionTest.java   |   5 +-
 .../sqoop/integration/server/rest/RestTest.java |   5 +-
 22 files changed, 696 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e66de4d..460273a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -123,6 +123,7 @@ limitations under the License.
     <jline.version>2.13</jline.version>
     <groovy.version>2.4.0</groovy.version>
     <jansi.version>1.7</jansi.version>
+    <felix.version>2.4.0</felix.version>
   </properties>
 
   <dependencies>
@@ -454,6 +455,11 @@ limitations under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-minikdc</artifactId>
+        <version>${hadoop.2.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-auth</artifactId>
         <version>${hadoop.2.version}</version>
       </dependency>
@@ -792,6 +798,11 @@ limitations under the License.
             <argLine>-Xms512m -Xmx2g -XX:MaxPermSize=1024m -XX:-UseSplitVerifier</argLine>
           </configuration>
         </plugin>
+        <plugin>
+          <groupId>org.apache.felix</groupId>
+          <artifactId>maven-bundle-plugin</artifactId>
+          <version>${felix.version}</version>
+        </plugin>
       </plugins>
     </pluginManagement>
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 4e1e197..5259d80 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -114,6 +114,11 @@ limitations under the License.
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-auth</artifactId>
     </dependency>
 
@@ -254,6 +259,12 @@ limitations under the License.
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.felix</groupId>
+        <artifactId>maven-bundle-plugin</artifactId>
+        <inherited>true</inherited>
+        <extensions>true</extensions>
+      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java b/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
index 47f1e45..fc7e172 100644
--- a/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
+++ b/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
@@ -21,6 +21,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
 import org.apache.sqoop.client.SqoopClient;
 import org.apache.sqoop.client.SubmissionCallback;
 import org.apache.sqoop.common.test.db.DatabaseProvider;
@@ -38,7 +39,9 @@ import org.apache.sqoop.test.data.UbuntuReleases;
 import org.apache.sqoop.test.infrastructure.providers.DatabaseInfrastructureProvider;
 import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
 import org.apache.sqoop.test.infrastructure.providers.InfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.KdcInfrastructureProvider;
 import org.apache.sqoop.test.infrastructure.providers.SqoopInfrastructureProvider;
+import org.apache.sqoop.test.kdc.KdcRunner;
 import org.apache.sqoop.test.utils.HdfsUtils;
 import org.apache.sqoop.test.utils.SqoopUtils;
 import org.apache.sqoop.validation.Status;
@@ -50,6 +53,7 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.BeforeSuite;
 
 import java.lang.reflect.Method;
+import java.net.URL;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -98,6 +102,8 @@ public class SqoopTestCase implements ITest {
 
   private SqoopClient client;
 
+  private DelegationTokenAuthenticatedURL.Token authToken = new DelegationTokenAuthenticatedURL.Token();
+
   @BeforeSuite
   public static void findSuiteName(ITestContext context) {
     suiteName = context.getSuite().getName();
@@ -153,19 +159,29 @@ public class SqoopTestCase implements ITest {
     // Create/start infrastructure providers.
     Configuration conf = new JobConf();
 
-    // Start hadoop first.
+    KdcRunner kdc = null;
+
+    // Start kdc first.
+    if (providers.contains(KdcInfrastructureProvider.class)) {
+      KdcInfrastructureProvider kdcProviderObject = startInfrastructureProvider(KdcInfrastructureProvider.class, conf, null);
+      kdc = kdcProviderObject.getInstance();
+      providers.remove(KdcInfrastructureProvider.class);
+    }
+
+    // Start hadoop secondly.
     if (providers.contains(HadoopInfrastructureProvider.class)) {
-      InfrastructureProvider hadoopProviderObject = startInfrastructureProvider(HadoopInfrastructureProvider.class, conf);
+      InfrastructureProvider hadoopProviderObject = startInfrastructureProvider(HadoopInfrastructureProvider.class, conf, kdc);
 
       // Use the prepared hadoop configuration for the rest of the components.
       if (hadoopProviderObject != null) {
         conf = hadoopProviderObject.getHadoopConfiguration();
       }
+      providers.remove(HadoopInfrastructureProvider.class);
     }
 
     // Start the rest of the providers.
     for (Class<? extends InfrastructureProvider> provider : providers) {
-      startInfrastructureProvider(provider, conf);
+      startInfrastructureProvider(provider, conf, kdc);
     }
   }
 
@@ -177,7 +193,7 @@ public class SqoopTestCase implements ITest {
    * @param <T>
    * @return
    */
-  private static <T extends InfrastructureProvider> T startInfrastructureProvider(Class<T> providerClass, Configuration hadoopConfiguration) {
+  private static <T extends InfrastructureProvider> T startInfrastructureProvider(Class<T> providerClass, Configuration hadoopConfiguration, KdcRunner kdc) {
     T providerObject;
 
     try {
@@ -189,6 +205,7 @@ public class SqoopTestCase implements ITest {
 
     providerObject.setRootPath(HdfsUtils.joinPathFragments(ROOT_PATH, suiteName, providerClass.getCanonicalName()));
     providerObject.setHadoopConfiguration(hadoopConfiguration);
+    providerObject.setKdc(kdc);
     providerObject.start();
 
     // Add for recall later.
@@ -310,19 +327,30 @@ public class SqoopTestCase implements ITest {
         .getServerUrl();
   }
 
+  public SqoopClient getClient() {
+    return client;
+  }
+
+  public DelegationTokenAuthenticatedURL.Token getAuthToken() {
+    return authToken;
+  }
+
   /**
    * Create a sqoop client
-   * @return SqoopClient
    */
-  public SqoopClient getClient() {
-    if (client == null) {
-      String serverUrl = getSqoopServerUrl();
+  @BeforeMethod
+  public void initSqoopClient() throws Exception {
+    String serverUrl = getSqoopServerUrl();
+
+    if (serverUrl != null) {
+      client = new SqoopClient(serverUrl);
 
-      if (serverUrl != null) {
-        client = new SqoopClient(serverUrl);
+      KdcInfrastructureProvider kdcProvider = getInfrastructureProvider(KdcInfrastructureProvider.class);
+      if (kdcProvider != null) {
+        kdcProvider.getInstance().authenticateWithSqoopServer(client);
+        kdcProvider.getInstance().authenticateWithSqoopServer(new URL(serverUrl), authToken);
       }
     }
-    return client;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/DatabaseInfrastructureProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/DatabaseInfrastructureProvider.java b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/DatabaseInfrastructureProvider.java
index e4481c4..a52542e 100644
--- a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/DatabaseInfrastructureProvider.java
+++ b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/DatabaseInfrastructureProvider.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
 import org.apache.sqoop.common.test.db.DatabaseProvider;
 import org.apache.sqoop.common.test.db.DatabaseProviderFactory;
+import org.apache.sqoop.test.kdc.KdcRunner;
 
 /**
  * Database infrastructure provider.
@@ -69,6 +70,11 @@ public class DatabaseInfrastructureProvider extends InfrastructureProvider {
     return null;
   }
 
+  @Override
+  public void setKdc(KdcRunner kdc) {
+    // No-op.
+  }
+
   public DatabaseProvider getInstance() {
     return instance;
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/HadoopInfrastructureProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/HadoopInfrastructureProvider.java b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/HadoopInfrastructureProvider.java
index f6ba23c..62f93fe 100644
--- a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/HadoopInfrastructureProvider.java
+++ b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/HadoopInfrastructureProvider.java
@@ -22,6 +22,7 @@ import org.apache.log4j.Logger;
 import org.apache.sqoop.test.hadoop.HadoopMiniClusterRunner;
 import org.apache.sqoop.test.hadoop.HadoopRunner;
 import org.apache.sqoop.test.hadoop.HadoopRunnerFactory;
+import org.apache.sqoop.test.kdc.KdcRunner;
 
 /**
  * Hadoop infrastructure provider.
@@ -84,4 +85,9 @@ public class HadoopInfrastructureProvider extends InfrastructureProvider {
   public HadoopRunner getInstance() {
     return instance;
   }
+
+  @Override
+  public void setKdc(KdcRunner kdc) {
+    // Do nothing for the time being. Need to handle this when we support kerberos enabled MiniCluster.
+  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/InfrastructureProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/InfrastructureProvider.java b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/InfrastructureProvider.java
index d904b3e..3f8f687 100644
--- a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/InfrastructureProvider.java
+++ b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/InfrastructureProvider.java
@@ -18,6 +18,7 @@
 package org.apache.sqoop.test.infrastructure.providers;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.test.kdc.KdcRunner;
 
 /**
  * Infrastructure classes enable the development of integration tests.
@@ -62,4 +63,9 @@ public abstract class InfrastructureProvider {
    * @return root path for component.
    */
   abstract public String getRootPath();
+
+  /**
+   * Set the KdcRunner to be used by this infrastructure component.
+   */
+  abstract public void setKdc(KdcRunner kdc);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/KdcInfrastructureProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/KdcInfrastructureProvider.java b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/KdcInfrastructureProvider.java
new file mode 100644
index 0000000..f337560
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/KdcInfrastructureProvider.java
@@ -0,0 +1,89 @@
+/**
+ * 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.infrastructure.providers;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
+import org.apache.sqoop.test.kdc.KdcRunner;
+import org.apache.sqoop.test.kdc.KdcRunnerFactory;
+import org.apache.sqoop.test.kdc.MiniKdcRunner;
+
+/**
+ * Kdc infrastructure provider.
+ */
+public class KdcInfrastructureProvider extends InfrastructureProvider {
+  private static final Logger LOG = Logger.getLogger(KdcInfrastructureProvider.class);
+
+  private KdcRunner instance;
+  private Configuration conf;
+
+  public KdcInfrastructureProvider() {
+    try {
+      instance = KdcRunnerFactory.getKdc(System.getProperties(), MiniKdcRunner.class);
+    } catch (Exception e) {
+      LOG.error("Error fetching Kdc runner.", e);
+    }
+  }
+
+  @Override
+  public void start() {
+    try {
+      instance.start();
+    } catch (Exception e) {
+      LOG.error("Could not start kdc.", e);
+    }
+  }
+
+  @Override
+  public void stop() {
+    try {
+      instance.stop();
+    } catch (Exception e) {
+      LOG.error("Could not stop kdc.", e);
+    }
+  }
+
+  public KdcRunner getInstance() {
+    return instance;
+  }
+
+  @Override
+  public void setHadoopConfiguration(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getHadoopConfiguration() {
+    return conf;
+  }
+
+  @Override
+  public void setRootPath(String path) {
+    instance.setTemporaryPath(path);
+  }
+
+  @Override
+  public String getRootPath() {
+    return instance.getTemporaryPath();
+  }
+
+  @Override
+  public void setKdc(KdcRunner kdc) {
+    // Do nothing as KdcRunner is created by this class.
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/SqoopInfrastructureProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/SqoopInfrastructureProvider.java b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/SqoopInfrastructureProvider.java
index 2c8af9c..4d51ed6 100644
--- a/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/SqoopInfrastructureProvider.java
+++ b/test/src/main/java/org/apache/sqoop/test/infrastructure/providers/SqoopInfrastructureProvider.java
@@ -19,6 +19,7 @@ package org.apache.sqoop.test.infrastructure.providers;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
+import org.apache.sqoop.test.kdc.KdcRunner;
 import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
 import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
 import org.apache.sqoop.test.minicluster.SqoopMiniClusterFactory;
@@ -32,13 +33,14 @@ public class SqoopInfrastructureProvider extends InfrastructureProvider {
   private SqoopMiniCluster instance;
   private String rootPath;
   private Configuration hadoopConf;
+  private KdcRunner kdc;
 
   public SqoopInfrastructureProvider() {}
 
   @Override
   public void start() {
     try {
-      instance = SqoopMiniClusterFactory.getSqoopMiniCluster(System.getProperties(), JettySqoopMiniCluster.class, rootPath, hadoopConf);
+      instance = SqoopMiniClusterFactory.getSqoopMiniCluster(System.getProperties(), JettySqoopMiniCluster.class, rootPath, hadoopConf, kdc);
       instance.start();
     } catch (Exception e) {
       LOG.error("Could not start Sqoop mini cluster.", e);
@@ -74,6 +76,10 @@ public class SqoopInfrastructureProvider extends InfrastructureProvider {
     return rootPath;
   }
 
+  public void setKdc(KdcRunner kdc) {
+    this.kdc = kdc;
+  }
+
   public SqoopMiniCluster getInstance() {
     return instance;
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/kdc/KdcRunner.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/kdc/KdcRunner.java b/test/src/main/java/org/apache/sqoop/test/kdc/KdcRunner.java
new file mode 100644
index 0000000..aa5c6fc
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/kdc/KdcRunner.java
@@ -0,0 +1,91 @@
+/**
+ * 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.kdc;
+
+import java.net.URL;
+
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.sqoop.client.SqoopClient;
+
+/**
+ * Kdc runner for testing purpose.
+ *
+ * Runner provides methods for bootstrapping and using Kdc. This
+ * abstract implementation is agnostic about in what mode Kdc is running.
+ * Each mode will have it's own concrete implementation (for example
+ * MiniKdc or Real existing kdc).
+ */
+public abstract class KdcRunner {
+
+  /**
+   * Temporary path that can be used as a root for other directories of kdc.
+   */
+  private String temporaryPath;
+
+  /**
+   * Start kdc.
+   *
+   * @throws Exception
+   */
+  public abstract void start() throws Exception;
+
+  /**
+   * Stop kdc.
+   *
+   * @throws Exception
+   */
+  public abstract void stop() throws Exception;
+
+  /**
+   * Trigger client to do kerberos authentication with sqoop server, a delegation token will
+   * be generated and subsequent requests don't need to do kerberos authentication any more.
+   */
+  public abstract void authenticateWithSqoopServer(final SqoopClient client) throws Exception;
+
+  /**
+   * Trigger client to do kerberos authentication with sqoop server, a delegation token will
+   * be generated and subsequent requests which uses this token don't need to do kerberos
+   * authentication any more.
+   */
+  public abstract void authenticateWithSqoopServer(final URL url,
+      final DelegationTokenAuthenticatedURL.Token authToken) throws Exception;
+
+  public abstract boolean isKerberosEnabled();
+
+  public abstract String getSpnegoPrincipal();
+
+  public abstract String getSqoopServerKeytabFile();
+
+  /**
+   * Get temporary path.
+   *
+   * @return
+   */
+  public String getTemporaryPath() {
+    return temporaryPath;
+  }
+
+  /**
+   * Set temporary path.
+   *
+   * @param temporaryPath
+   */
+  public void setTemporaryPath(String temporaryPath) {
+    this.temporaryPath = temporaryPath;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/kdc/KdcRunnerFactory.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/kdc/KdcRunnerFactory.java b/test/src/main/java/org/apache/sqoop/test/kdc/KdcRunnerFactory.java
new file mode 100644
index 0000000..36ca51c
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/kdc/KdcRunnerFactory.java
@@ -0,0 +1,38 @@
+/**
+ * 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.kdc;
+
+import java.util.Properties;
+
+/**
+ * Create KdcRunner.
+ */
+public class KdcRunnerFactory {
+
+  public static final String KDC_CLASS_PROPERTY = "sqoop.kdc.runner.class";
+
+  public static KdcRunner getKdc(Properties properties, Class<? extends KdcRunner> defaultClusterClass) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
+    String className = properties.getProperty(KDC_CLASS_PROPERTY);
+    if(className == null) {
+      return defaultClusterClass.newInstance();
+    }
+
+    Class<?> klass = Class.forName(className);
+    return (KdcRunner)klass.newInstance();
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/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
new file mode 100644
index 0000000..299fd9f
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/kdc/MiniKdcRunner.java
@@ -0,0 +1,235 @@
+/**
+ * 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.kdc;
+
+import java.io.File;
+import java.net.URL;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Callable;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.minikdc.MiniKdc;
+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.SqoopUtils;
+
+/**
+ * Represents a Minikdc setup. Minikdc should be only used together with
+ * mini clusters such as JettySqoopMiniCluster, HadoopMiniClusterRunner,
+ * InternalHiveServerRunner, InternalMetastoreServerRunner, etc.
+ * It should not be used with real cluster.
+ */
+@edu.umd.cs.findbugs.annotations.SuppressWarnings({"SIC_INNER_SHOULD_BE_STATIC_ANON"})
+public class MiniKdcRunner extends KdcRunner {
+
+  private MiniKdc miniKdc;
+
+  private String sqoopClientPrincipal;
+  private String sqoopClientKeytabFile;
+
+  private String spnegoPrincipal;
+  private String sqoopServerKeytabFile;
+
+  @Override
+  public void start() throws Exception {
+    Properties kdcConf = MiniKdc.createConf();
+    File baseDir = new File(getTemporaryPath(), "minikdc");
+    FileUtils.deleteDirectory(baseDir);
+    FileUtils.forceMkdir(baseDir);
+    miniKdc = new MiniKdc(kdcConf, baseDir);
+    miniKdc.start();
+
+    createPrincipals();
+  }
+
+  @Override
+  public void stop() throws Exception {
+    miniKdc.stop();
+  }
+
+  public MiniKdc getMiniKdc() {
+    return miniKdc;
+  }
+
+  @Override
+  public String getSpnegoPrincipal() {
+    return spnegoPrincipal;
+  }
+
+  @Override
+  public String getSqoopServerKeytabFile() {
+    return sqoopServerKeytabFile;
+  }
+
+  private static class KerberosConfiguration extends Configuration {
+    private String principal;
+    private String keytabFile;
+
+    public KerberosConfiguration(String principal, String keytabFile) {
+      this.principal = principal;
+      this.keytabFile = keytabFile;
+    }
+
+    private String getKrb5LoginModuleName() {
+      return System.getProperty("java.vendor").contains("IBM")
+          ? "com.ibm.security.auth.module.Krb5LoginModule"
+          : "com.sun.security.auth.module.Krb5LoginModule";
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      Map<String, String> options = new HashMap<String, String>();
+      if (System.getProperty("java.vendor").contains("IBM")) {
+        options.put("useKeytab", keytabFile.startsWith("file://") ? keytabFile : "file://" +  keytabFile);
+        options.put("principal", principal);
+        options.put("refreshKrb5Config", "true");
+        options.put("credsType", "both");
+      } else {
+        options.put("keyTab", keytabFile);
+        options.put("principal", principal);
+        options.put("useKeyTab", "true");
+        options.put("storeKey", "true");
+        options.put("doNotPrompt", "true");
+        options.put("useTicketCache", "true");
+        options.put("renewTGT", "true");
+        options.put("refreshKrb5Config", "true");
+        options.put("isInitiator", "true");
+      }
+      String ticketCache = System.getenv("KRB5CCNAME");
+      if (ticketCache != null) {
+        if (System.getProperty("java.vendor").contains("IBM")) {
+          // IBM JAVA only respect system property and not env variable
+          // The first value searched when "useDefaultCcache" is used.
+          System.setProperty("KRB5CCNAME", ticketCache);
+          options.put("useDefaultCcache", "true");
+          options.put("renewTGT", "true");
+        } else {
+          options.put("ticketCache", ticketCache);
+        }
+      }
+      options.put("debug", "true");
+
+      return new AppConfigurationEntry[]{
+        new AppConfigurationEntry(getKrb5LoginModuleName(),
+                                  AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                                  options),};
+    }
+  }
+
+  @Override
+  public void authenticateWithSqoopServer(final SqoopClient client) throws Exception {
+    doAsSqoopClient(new Callable<Collection<MConnector>>() {
+      @Override
+      public Collection<MConnector> call() {
+        return client.getConnectors();
+      }
+    });
+  }
+
+  @Override
+  public void authenticateWithSqoopServer(final URL url, final DelegationTokenAuthenticatedURL.Token authToken) throws Exception {
+    doAsSqoopClient(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        new DelegationTokenAuthenticatedURL().openConnection(url, authToken);
+        return null;
+      }
+    });
+  }
+
+  @Override
+  public boolean isKerberosEnabled() {
+    return true;
+  }
+
+  private void createPrincipals() throws Exception {
+    createPrincipalsForSqoopClient();
+    createPrincipalsForSqoopServer();
+  }
+
+  private void createPrincipalsForSqoopClient() throws Exception {
+    String keytabDir = HdfsUtils.joinPathFragments(getTemporaryPath(), "sqoop-client");
+    File keytabDirFile = new File(keytabDir);
+    FileUtils.deleteDirectory(keytabDirFile);
+    FileUtils.forceMkdir(keytabDirFile);
+
+    String userName = "sqoopclient";
+    File userKeytabFile = new File(keytabDirFile, userName + ".keytab");
+    miniKdc.createPrincipal(userKeytabFile, userName);
+    sqoopClientPrincipal = userName + "@" + miniKdc.getRealm();
+    sqoopClientKeytabFile = userKeytabFile.getAbsolutePath();
+  }
+
+  private void createPrincipalsForSqoopServer() throws Exception {
+    String keytabDir = HdfsUtils.joinPathFragments(getTemporaryPath(), "sqoop-server");
+    File keytabDirFile = new File(keytabDir);
+    FileUtils.deleteDirectory(keytabDirFile);
+    FileUtils.forceMkdir(keytabDirFile);
+
+    String sqoopUserName = "sqoopserver";
+    File sqoopKeytabFile = new File(keytabDirFile, sqoopUserName + ".keytab");
+    String host = SqoopUtils.getLocalHostName();
+    miniKdc.createPrincipal(sqoopKeytabFile, "HTTP/" + host);
+    sqoopServerKeytabFile = sqoopKeytabFile.getAbsolutePath();
+    spnegoPrincipal = "HTTP/" + host + "@" + miniKdc.getRealm();
+  }
+
+  private <T> T doAsSqoopClient(Callable<T> callable) throws Exception {
+    return doAs(sqoopClientPrincipal, sqoopClientKeytabFile, callable);
+  }
+
+  private static <T> T doAs(String principal, String keytabFile, final Callable<T> callable) throws Exception {
+    LoginContext loginContext = null;
+    try {
+      Set<Principal> principals = new HashSet<Principal>();
+      principals.add(new KerberosPrincipal(principal));
+      Subject subject = new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());
+      loginContext = new LoginContext("", subject, null, new KerberosConfiguration(principal, keytabFile));
+      loginContext.login();
+      subject = loginContext.getSubject();
+      return Subject.doAs(subject, new PrivilegedExceptionAction<T>() {
+        @Override
+        public T run() throws Exception {
+          return callable.call();
+        }
+      });
+    } catch (PrivilegedActionException ex) {
+      throw ex.getException();
+    } finally {
+      if (loginContext != null) {
+        loginContext.logout();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/kdc/NoKerberosKdcRunner.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/kdc/NoKerberosKdcRunner.java b/test/src/main/java/org/apache/sqoop/test/kdc/NoKerberosKdcRunner.java
new file mode 100644
index 0000000..29a005e
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/kdc/NoKerberosKdcRunner.java
@@ -0,0 +1,66 @@
+/**
+ * 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.kdc;
+
+import java.net.URL;
+
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.Token;
+import org.apache.sqoop.client.SqoopClient;
+
+/**
+  * This class enables running tests without kerberos enabled.
+  */
+public class NoKerberosKdcRunner extends KdcRunner {
+
+  @Override
+  public void start() throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void stop() throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void authenticateWithSqoopServer(SqoopClient client) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void authenticateWithSqoopServer(URL url, Token authToken)
+      throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public boolean isKerberosEnabled() {
+    return false;
+  }
+
+  @Override
+  public String getSpnegoPrincipal() {
+    return null;
+  }
+
+  @Override
+  public String getSqoopServerKeytabFile() {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/minicluster/JettySqoopMiniCluster.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/minicluster/JettySqoopMiniCluster.java b/test/src/main/java/org/apache/sqoop/test/minicluster/JettySqoopMiniCluster.java
index 325a790..5bb0175 100644
--- a/test/src/main/java/org/apache/sqoop/test/minicluster/JettySqoopMiniCluster.java
+++ b/test/src/main/java/org/apache/sqoop/test/minicluster/JettySqoopMiniCluster.java
@@ -18,9 +18,12 @@
 
 package org.apache.sqoop.test.minicluster;
 
+import java.net.MalformedURLException;
+import java.net.URL;
+
 import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.Logger;
 import org.apache.sqoop.server.SqoopJettyServer;
+import org.apache.sqoop.test.utils.SqoopUtils;
 
 /**
 * Embedded jetty Sqoop server mini cluster.
@@ -57,7 +60,17 @@ public class JettySqoopMiniCluster extends SqoopMiniCluster {
   @Override
   public String getServerUrl() {
     if (sqoopJettyServer != null) {
-      return sqoopJettyServer.getServerUrl();
+      String serverUrl = sqoopJettyServer.getServerUrl();
+      // Replace the hostname of server url with FQDN
+      String host;
+      try {
+        host = new URL(serverUrl).getHost();
+      } catch (MalformedURLException e) {
+        throw new RuntimeException("Invalid sqoop server url: " + serverUrl);
+      }
+
+      String fqdn = SqoopUtils.getLocalHostName();
+      return serverUrl.replaceFirst(host, fqdn);
     }
     throw new RuntimeException("Jetty server wasn't started.");
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java b/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java
index 9ae941f..4ff97e7 100644
--- a/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java
+++ b/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java
@@ -22,6 +22,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sqoop.core.ConfigurationConstants;
 import org.apache.sqoop.common.test.repository.RepositoryProviderFactory;
+import org.apache.sqoop.test.kdc.KdcRunner;
 
 import java.io.File;
 import java.io.IOException;
@@ -49,6 +50,11 @@ public abstract class SqoopMiniCluster {
   private String temporaryPath;
 
   /**
+   * Kdc runner.
+   */
+  private KdcRunner kdc;
+
+  /**
    * Create Sqoop Mini cluster with default configuration
    *
    * @param temporaryPath Temporary path
@@ -87,6 +93,10 @@ public abstract class SqoopMiniCluster {
     return temporaryPath + "/log/";
   }
 
+  public void setKdc(KdcRunner kdc) {
+    this.kdc = kdc;
+  }
+
   /**
    * Start Sqoop Mini cluster
    *
@@ -198,8 +208,17 @@ public abstract class SqoopMiniCluster {
   protected Map<String, String> getSecurityConfiguration() {
     Map<String, String> properties = new HashMap<String, String>();
 
-    properties.put("org.apache.sqoop.authentication.type", "SIMPLE");
-    properties.put("org.apache.sqoop.authentication.handler", "org.apache.sqoop.security.SimpleAuthenticationHandler");
+    if (kdc != null && kdc.isKerberosEnabled()) {
+      // Sqoop Server is kerberos enabled
+      properties.put("org.apache.sqoop.security.authentication.type", "KERBEROS");
+      properties.put("org.apache.sqoop.security.authentication.kerberos.http.principal", kdc.getSpnegoPrincipal());
+      properties.put("org.apache.sqoop.security.authentication.kerberos.http.keytab", kdc.getSqoopServerKeytabFile());
+    } else {
+      properties.put("org.apache.sqoop.security.authentication.type", "SIMPLE");
+    }
+
+    // Sqoop Server do simple authentication with other services
+    properties.put("org.apache.sqoop.security.authentication.handler", "org.apache.sqoop.security.authentication.SimpleAuthenticationHandler");
 
     /**
      * Due to the fact that we share a JVM with hadoop during unit testing,

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniClusterFactory.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniClusterFactory.java b/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniClusterFactory.java
index 544d419..bdfc01a 100644
--- a/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniClusterFactory.java
+++ b/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniClusterFactory.java
@@ -18,6 +18,7 @@
 package org.apache.sqoop.test.minicluster;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.test.kdc.KdcRunner;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -29,10 +30,12 @@ public class SqoopMiniClusterFactory {
 
   public static final String MINICLUSTER_CLASS_PROPERTY = "sqoop.minicluster.class";
 
-  public static SqoopMiniCluster getSqoopMiniCluster(Properties properties, Class<? extends SqoopMiniCluster> defaultClusterClass, String temporaryPath, Configuration configuration) throws ClassNotFoundException, IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException {
+  public static SqoopMiniCluster getSqoopMiniCluster(Properties properties, Class<? extends SqoopMiniCluster> defaultClusterClass, String temporaryPath, Configuration configuration, KdcRunner kdc) throws ClassNotFoundException, IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException {
     String className = properties.getProperty(MINICLUSTER_CLASS_PROPERTY);
     Class<?> klass = className == null ? defaultClusterClass : Class.forName(className);
     Constructor konstructor = klass.getConstructor(String.class, Configuration.class);
-    return (SqoopMiniCluster)konstructor.newInstance(temporaryPath, configuration);
+    SqoopMiniCluster cluster = (SqoopMiniCluster)konstructor.newInstance(temporaryPath, configuration);
+    cluster.setKdc(kdc);
+    return cluster;
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/main/java/org/apache/sqoop/test/testcases/JettyTestCase.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/testcases/JettyTestCase.java b/test/src/main/java/org/apache/sqoop/test/testcases/JettyTestCase.java
index e3ee300..d4ca396 100644
--- a/test/src/main/java/org/apache/sqoop/test/testcases/JettyTestCase.java
+++ b/test/src/main/java/org/apache/sqoop/test/testcases/JettyTestCase.java
@@ -27,8 +27,10 @@ import org.apache.sqoop.test.asserts.HdfsAsserts;
 import org.apache.sqoop.test.hadoop.HadoopMiniClusterRunner;
 import org.apache.sqoop.test.hadoop.HadoopRunner;
 import org.apache.sqoop.test.hadoop.HadoopRunnerFactory;
+import org.apache.sqoop.test.kdc.KdcRunner;
+import org.apache.sqoop.test.kdc.KdcRunnerFactory;
+import org.apache.sqoop.test.kdc.MiniKdcRunner;
 import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
-import org.apache.sqoop.test.minicluster.RealSqoopCluster;
 import org.apache.sqoop.test.minicluster.SqoopMiniCluster;
 import org.apache.sqoop.test.minicluster.SqoopMiniClusterFactory;
 import org.apache.sqoop.test.utils.HdfsUtils;
@@ -93,6 +95,11 @@ abstract public class JettyTestCase implements ITest {
   private static SqoopClient client;
 
   /**
+   * Kdc
+   */
+  private static KdcRunner kdc;
+
+  /**
    * Use the method name as the test name
    */
   public String getTestName() {
@@ -112,6 +119,7 @@ abstract public class JettyTestCase implements ITest {
     LOG.debug("Temporary Directory: " + getTemporaryPath());
     FileUtils.deleteDirectory(new File(getTemporaryPath()));
 
+    startKdc();
     startHadoop();
     startSqoop();
   }
@@ -120,6 +128,17 @@ abstract public class JettyTestCase implements ITest {
   public void tearDownSuite() throws Exception {
     stopSqoop();
     stopHadoop();
+    stopKdc();
+  }
+
+  protected void startKdc() throws Exception {
+    kdc = KdcRunnerFactory.getKdc(System.getProperties(), MiniKdcRunner.class);
+    kdc.setTemporaryPath(getTemporaryPath());
+    kdc.start();
+  }
+
+  protected void stopKdc() throws Exception {
+    kdc.stop();
   }
 
   protected void startHadoop() throws Exception {
@@ -140,7 +159,7 @@ abstract public class JettyTestCase implements ITest {
     cluster.start();
 
     // Initialize Sqoop Client API
-    client = new SqoopClient(getServerUrl());
+    setClient(new SqoopClient(getServerUrl()));
   }
 
   protected void stopSqoop() throws Exception {
@@ -160,7 +179,7 @@ abstract public class JettyTestCase implements ITest {
    */
   public SqoopMiniCluster createSqoopMiniCluster() throws Exception {
     return SqoopMiniClusterFactory.getSqoopMiniCluster(System.getProperties(), JettySqoopMiniCluster.class,
-      getSqoopMiniClusterTemporaryPath(), hadoopCluster.getConfiguration());
+      getSqoopMiniClusterTemporaryPath(), hadoopCluster.getConfiguration(), kdc);
   }
 
   /**
@@ -172,8 +191,9 @@ abstract public class JettyTestCase implements ITest {
     return client;
   }
 
-  public static void setClient(SqoopClient sqoopClient) {
+  public static void setClient(SqoopClient sqoopClient) throws Exception {
     client = sqoopClient;
+    kdc.authenticateWithSqoopServer(sqoopClient);
   }
 
   public static SqoopMiniCluster getCluster() {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/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 5964bcd..6614b19 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
@@ -17,6 +17,9 @@
  */
 package org.apache.sqoop.test.utils;
 
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Locale;
 import java.util.Random;
 
 import org.apache.sqoop.model.MAccountableEntity;
@@ -40,4 +43,15 @@ public class SqoopUtils {
     }
     object.setName(prefix + rand.nextLong());
   }
+
+  //Retrieve the FQDN of the current host
+  public static String getLocalHostName() {
+    String fqdn;
+    try {
+      fqdn = InetAddress.getLocalHost().getCanonicalHostName().toLowerCase(Locale.getDefault());
+    } catch (UnknownHostException e1) {
+      fqdn = "localhost";
+    }
+    return fqdn;
+  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/test/java/org/apache/sqoop/integration/server/InformalObjectNameTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/InformalObjectNameTest.java b/test/src/test/java/org/apache/sqoop/integration/server/InformalObjectNameTest.java
index fe04df7..cb8c5de 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/InformalObjectNameTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/InformalObjectNameTest.java
@@ -28,6 +28,7 @@ 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.infrastructure.providers.SqoopInfrastructureProvider;
 import org.apache.sqoop.test.utils.ParametrizedUtils;
 import org.testng.annotations.AfterMethod;
@@ -35,7 +36,7 @@ import org.testng.annotations.DataProvider;
 import org.testng.annotations.Factory;
 import org.testng.annotations.Test;
 
-@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
 public class InformalObjectNameTest extends SqoopTestCase {
 
   private String target;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java b/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java
index 298ec09..49e26a2 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java
@@ -24,6 +24,7 @@ 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.infrastructure.providers.SqoopInfrastructureProvider;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.Test;
@@ -35,7 +36,7 @@ import static org.testng.Assert.assertEquals;
 /**
  * Ensure that jobs will be shown in order
  */
-@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
 public class ShowJobInOrderTest extends SqoopTestCase {
 
   public ShowJobInOrderTest() {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java b/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java
index b309c86..9adebea 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java
@@ -28,6 +28,7 @@ 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.infrastructure.providers.SqoopInfrastructureProvider;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
@@ -44,7 +45,7 @@ import static org.testng.Assert.fail;
  * Ensure that server will reject starting job when either job itself
  * or corresponding link is disabled.
  */
-@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
 public class SubmissionWithDisabledModelObjectsTest extends SqoopTestCase {
 
   private boolean enabledLink;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java b/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java
index e86a6cc..76002a5 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java
@@ -22,6 +22,7 @@ import org.apache.sqoop.common.VersionInfo;
 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.infrastructure.providers.SqoopInfrastructureProvider;
 import org.apache.sqoop.json.VersionBean;
 import org.testng.annotations.Test;
@@ -31,12 +32,12 @@ import static org.testng.Assert.assertEquals;
 /**
  * Basic test to check that server is working and returning correct version info.
  */
-@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class})
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class})
 public class VersionTest extends SqoopTestCase {
 
   @Test
   public void testVersion() {
-    VersionResourceRequest versionRequest = new VersionResourceRequest();
+    VersionResourceRequest versionRequest = new VersionResourceRequest(getAuthToken());
     VersionBean versionBean = versionRequest.read(getSqoopServerUrl());
 
     assertEquals(versionBean.getBuildVersion(), VersionInfo.getBuildVersion());

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5329c1b3/test/src/test/java/org/apache/sqoop/integration/server/rest/RestTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/rest/RestTest.java b/test/src/test/java/org/apache/sqoop/integration/server/rest/RestTest.java
index 20f09e8..ff6b30d 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/rest/RestTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/rest/RestTest.java
@@ -25,6 +25,7 @@ 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.infrastructure.providers.SqoopInfrastructureProvider;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.Test;
@@ -38,7 +39,7 @@ import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNotNull;
 import static org.testng.Assert.assertTrue;
 
-@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class})
+@Infrastructure(dependencies = {KdcInfrastructureProvider.class, HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class})
 public abstract class RestTest extends SqoopTestCase {
 
   private static final Logger LOG = Logger.getLogger(RestTest.class);
@@ -150,7 +151,7 @@ public abstract class RestTest extends SqoopTestCase {
     LOG.info("Start: " + getTestName());
 
     URL url = new URL(getSqoopServerUrl() +  desc.rest);
-    HttpURLConnection connection = new DelegationTokenAuthenticatedURL().openConnection(url, new DelegationTokenAuthenticatedURL.Token());
+    HttpURLConnection connection = new DelegationTokenAuthenticatedURL().openConnection(url, getAuthToken());
     connection.setRequestMethod(desc.method);
 
     if(desc.data != null) {