You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ab...@apache.org on 2014/10/31 04:58:01 UTC

git commit: SQOOP-1526: Sqoop2: Kerberos support when starting service

Repository: sqoop
Updated Branches:
  refs/heads/sqoop2 8a7d950ce -> 21d4a9518


SQOOP-1526: Sqoop2: Kerberos support when starting service

(Richard Zhou via Abraham Elmahrek)


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

Branch: refs/heads/sqoop2
Commit: 21d4a9518ba8724126c8417400439d4d6f6c0371
Parents: 8a7d950
Author: Abraham Elmahrek <ab...@elmahrek.com>
Authored: Thu Oct 30 20:57:03 2014 -0700
Committer: Abraham Elmahrek <ab...@elmahrek.com>
Committed: Thu Oct 30 20:57:03 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/sqoop/core/SqoopServer.java |   3 +
 .../sqoop/security/AuthenticationConstants.java |  76 +++++++++++++
 .../sqoop/security/AuthenticationError.java     |  52 +++++++++
 .../sqoop/security/AuthenticationHandler.java   |  47 ++++++++
 .../security/AuthenticationHandlerFactory.java  |  46 ++++++++
 .../sqoop/security/AuthenticationManager.java   | 111 +++++++++++++++++++
 dist/src/main/server/conf/sqoop.properties      |  10 ++
 pom.xml                                         |   7 ++
 security/pom.xml                                |  86 ++++++++++++++
 .../security/KerberosAuthenticationHandler.java |  87 +++++++++++++++
 .../security/SimpleAuthenticationHandler.java   |  41 +++++++
 server/pom.xml                                  |   5 +
 .../test/minicluster/SqoopMiniCluster.java      |  10 ++
 13 files changed, 581 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/core/src/main/java/org/apache/sqoop/core/SqoopServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/core/SqoopServer.java b/core/src/main/java/org/apache/sqoop/core/SqoopServer.java
index ac836c7..fe467e3 100644
--- a/core/src/main/java/org/apache/sqoop/core/SqoopServer.java
+++ b/core/src/main/java/org/apache/sqoop/core/SqoopServer.java
@@ -23,6 +23,7 @@ import org.apache.sqoop.connector.ConnectorManager;
 import org.apache.sqoop.driver.Driver;
 import org.apache.sqoop.driver.JobManager;
 import org.apache.sqoop.repository.RepositoryManager;
+import org.apache.sqoop.security.AuthenticationManager;
 
 /**
  * Entry point for initializing and destroying Sqoop server
@@ -38,6 +39,7 @@ public class SqoopServer {
     ConnectorManager.getInstance().destroy();
     RepositoryManager.getInstance().destroy();
     AuditLoggerManager.getInstance().destroy();
+    AuthenticationManager.getInstance().destroy();
     SqoopConfiguration.getInstance().destroy();
     LOG.info("Sqoop server has been correctly terminated");
   }
@@ -46,6 +48,7 @@ public class SqoopServer {
     try {
       LOG.info("Booting up Sqoop server");
       SqoopConfiguration.getInstance().initialize();
+      AuthenticationManager.getInstance().initialize();
       AuditLoggerManager.getInstance().initialize();
       RepositoryManager.getInstance().initialize();
       ConnectorManager.getInstance().initialize();

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/core/src/main/java/org/apache/sqoop/security/AuthenticationConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/security/AuthenticationConstants.java b/core/src/main/java/org/apache/sqoop/security/AuthenticationConstants.java
new file mode 100644
index 0000000..645555f
--- /dev/null
+++ b/core/src/main/java/org/apache/sqoop/security/AuthenticationConstants.java
@@ -0,0 +1,76 @@
+/**
+ * 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.security;
+
+import org.apache.sqoop.core.ConfigurationConstants;
+
+/**
+ * Constants that are used in authentication module.
+ */
+public final class AuthenticationConstants {
+
+  /**
+   * All authentication related configuration is prefixed with this:
+   * <tt>org.apache.sqoop.authentication.</tt>
+   */
+  public static final String PREFIX_AUTHENTICATION_CONFIG =
+    ConfigurationConstants.PREFIX_GLOBAL_CONFIG + "authentication.";
+
+  /**
+   * The config specifies the sqoop authentication type (SIMPLE, KERBEROS).
+   * The default type is SIMPLE
+   * <tt>org.apache.sqoop.authentication.type</tt>.
+   */
+  public static final String AUTHENTICATION_TYPE =
+    PREFIX_AUTHENTICATION_CONFIG + "type";
+
+  /**
+   * The config specifies the sqoop authentication handler class.
+   * The default type is org.apache.sqoop.security.SimpleAuthenticationHandler
+   * <tt>org.apache.sqoop.authentication.handler</tt>.
+   */
+  public static final String AUTHENTICATION_HANDLER =
+          PREFIX_AUTHENTICATION_CONFIG + "handler";
+
+  /**
+   * All kerberos authentication related configuration is prefixed with this:
+   * <tt>org.apache.sqoop.authentication.kerberos.</tt>
+   */
+  public static final String PREFIX_AUTHENTICATION_KERBEROS_CONFIG =
+          PREFIX_AUTHENTICATION_CONFIG + "kerberos.";
+
+  /**
+   * The config specifies the kerberos principal.
+   * <tt>org.apache.sqoop.authentication.kerberos.principal</tt>.
+   */
+  public static final String AUTHENTICATION_KERBEROS_PRINCIPAL =
+    PREFIX_AUTHENTICATION_KERBEROS_CONFIG + "principal";
+
+  /**
+   * The config specifies the kerberos keytab.
+   * <tt>org.apache.sqoop.authentication.kerberos.principal</tt>.
+   */
+  public static final String AUTHENTICATION_KERBEROS_KEYTAB =
+    PREFIX_AUTHENTICATION_KERBEROS_CONFIG + "keytab";
+
+  public static enum TYPE {SIMPLE, KERBEROS}
+
+  private AuthenticationConstants() {
+    // Instantiation of this class is prohibited
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/core/src/main/java/org/apache/sqoop/security/AuthenticationError.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/security/AuthenticationError.java b/core/src/main/java/org/apache/sqoop/security/AuthenticationError.java
new file mode 100644
index 0000000..338db06
--- /dev/null
+++ b/core/src/main/java/org/apache/sqoop/security/AuthenticationError.java
@@ -0,0 +1,52 @@
+/**
+ * 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.security;
+
+import org.apache.sqoop.common.ErrorCode;
+
+public enum AuthenticationError implements ErrorCode {
+
+  /** An unknown error has occurred. */
+  AUTH_0000("An unknown error has occurred"),
+
+  /** The system was not able to find Kerberos keytab in sqoop configuration. */
+  AUTH_0001("Unable to find Kerberos keytab"),
+
+  /** The system was not able to find Kerberos principal in sqoop configuration. */
+  AUTH_0002("Unable to find Kerberos principal"),
+
+  /** The system was not able to login using Kerberos keytab and principal in sqoop configuration. */
+  AUTH_0003("Unable to login using Kerberos keytab and principal"),
+
+  /** Invalid FileSystemAccess security mode {simple, Kerberos}. */
+  AUTH_0004("Invalid FileSystemAccess security mode");
+
+  private final String message;
+
+  private AuthenticationError(String message) {
+    this.message = message;
+  }
+
+  public String getCode() {
+    return name();
+  }
+
+  public String getMessage() {
+    return message;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/core/src/main/java/org/apache/sqoop/security/AuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/security/AuthenticationHandler.java b/core/src/main/java/org/apache/sqoop/security/AuthenticationHandler.java
new file mode 100644
index 0000000..6fb6d1d
--- /dev/null
+++ b/core/src/main/java/org/apache/sqoop/security/AuthenticationHandler.java
@@ -0,0 +1,47 @@
+/**
+ * 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.security;
+
+import org.apache.log4j.Logger;
+
+/***
+ * AuthenticationHandler is responsible for secure checking.
+ * KerberosAuthenticationHandler and SimpleAuthenticationHandler have
+ * implemented this abstract class.
+ */
+public abstract class AuthenticationHandler {
+
+  private static final Logger LOG = Logger.getLogger(AuthenticationHandler.class);
+
+  /**
+   * Security enabled option value
+   */
+  protected boolean securityEnabled = false;
+
+  public boolean isSecurityEnabled() {
+    return securityEnabled;
+  }
+
+  public abstract void doInitialize();
+
+  public abstract void secureLogin();
+
+  public String get_hadoop_security_authentication() {
+    return "hadoop.security.authentication";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/core/src/main/java/org/apache/sqoop/security/AuthenticationHandlerFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/security/AuthenticationHandlerFactory.java b/core/src/main/java/org/apache/sqoop/security/AuthenticationHandlerFactory.java
new file mode 100644
index 0000000..b62fe18
--- /dev/null
+++ b/core/src/main/java/org/apache/sqoop/security/AuthenticationHandlerFactory.java
@@ -0,0 +1,46 @@
+/**
+ * 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.security;
+
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.utils.ClassUtils;
+
+/**
+ * Create authentication manager.
+ */
+public class AuthenticationHandlerFactory {
+
+  public static AuthenticationHandler getAuthenticationHandler(String handler) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
+
+    Class<?> handlerClass = ClassUtils.loadClass(handler);
+
+    if (handlerClass == null) {
+      throw new SqoopException(AuthenticationError.AUTH_0004,
+              "Authentication Handler Class: " + handler);
+    }
+
+    AuthenticationHandler newHandler;
+    try {
+      newHandler = (AuthenticationHandler) handlerClass.newInstance();
+    } catch (Exception ex) {
+      throw new SqoopException(AuthenticationError.AUTH_0004,
+              "Authentication Handler Class: " + handler, ex);
+    }
+    return newHandler;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/core/src/main/java/org/apache/sqoop/security/AuthenticationManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/security/AuthenticationManager.java b/core/src/main/java/org/apache/sqoop/security/AuthenticationManager.java
new file mode 100644
index 0000000..1f7248b
--- /dev/null
+++ b/core/src/main/java/org/apache/sqoop/security/AuthenticationManager.java
@@ -0,0 +1,111 @@
+/**
+ * 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.security;
+
+import org.apache.log4j.Logger;
+import org.apache.sqoop.core.Reconfigurable;
+import org.apache.sqoop.core.SqoopConfiguration;
+
+/***
+ * AuthenticationManager is responsible for manager AuthenticationHandler.
+ */
+public class AuthenticationManager implements Reconfigurable {
+
+  private static final Logger LOG = Logger.getLogger(AuthenticationManager.class);
+
+
+  /**
+   * Default authentication auto upgrade option value
+   */
+  protected static boolean DEFAULT_AUTO_UPGRADE = false;
+
+  /**
+   * Private instance to singleton of this class.
+   */
+  private static AuthenticationManager instance;
+
+  /**
+   * Create default object
+   */
+  static {
+    instance = new AuthenticationManager();
+  }
+
+  /**
+   * Return current instance.
+   *
+   * @return Current instance
+   */
+  public static AuthenticationManager getInstance() {
+    return instance;
+  }
+
+  /**
+   * Allows to set instance in case that it's need.
+   * <p/>
+   * This method should not be normally used as the default instance should be sufficient. One target
+   * user use case for this method are unit tests.
+   *
+   * @param newInstance New instance
+   */
+  public static void setInstance(AuthenticationManager newInstance) {
+    instance = newInstance;
+  }
+
+  /**
+   * Private AuthenticiationHandler to singleton of this class.
+   */
+  private static AuthenticationHandler authenticationHandler;
+
+  /**
+   * Return current authentication handler.
+   *
+   * @return Current authentication handler
+   */
+  public static AuthenticationHandler getAuthenticationHandler() {
+    return authenticationHandler;
+  }
+
+  public synchronized void initialize() throws ClassNotFoundException, IllegalAccessException, InstantiationException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Begin authentication manager initialization");
+    }
+
+    String handler = SqoopConfiguration.getInstance().getContext().getString(AuthenticationConstants.AUTHENTICATION_HANDLER.trim());
+    authenticationHandler = AuthenticationHandlerFactory.getAuthenticationHandler(handler);
+    authenticationHandler.doInitialize();
+    authenticationHandler.secureLogin();
+
+    if (LOG.isInfoEnabled()) {
+      LOG.info("Authentication loaded.");
+    }
+  }
+
+  public synchronized void destroy() {
+    LOG.trace("Begin authentication manager destroy");
+  }
+
+  @Override
+  public synchronized void configurationChanged() {
+    LOG.info("Begin authentication manager reconfiguring");
+    // If there are configuration options for AuthenticationManager,
+    // implement the reconfiguration procedure right here.
+    LOG.info("Authentication manager reconfigured");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/dist/src/main/server/conf/sqoop.properties
----------------------------------------------------------------------
diff --git a/dist/src/main/server/conf/sqoop.properties b/dist/src/main/server/conf/sqoop.properties
index bb01016..3ba50ef 100755
--- a/dist/src/main/server/conf/sqoop.properties
+++ b/dist/src/main/server/conf/sqoop.properties
@@ -135,3 +135,13 @@ org.apache.sqoop.submission.engine.mapreduce.configuration.directory=/etc/hadoop
 # Execution engine configuration
 #
 org.apache.sqoop.execution.engine=org.apache.sqoop.execution.mapreduce.MapreduceExecutionEngine
+
+#
+# Authentication configuration
+#
+org.apache.sqoop.authentication.type=SIMPLE
+org.apache.sqoop.authentication.handler=org.apache.sqoop.security.SimpleAuthenticationHandler
+#org.apache.sqoop.authentication.type=KERBEROS
+#org.apache.sqoop.authentication.handler=org.apache.sqoop.security.KerberosAuthenticationHandler
+#org.apache.sqoop.authentication.kerberos.principal=sqoop/_HOST@NOVALOCAL
+#org.apache.sqoop.authentication.kerberos.keytab=/home/kerberos/sqoop.keytab
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f25a29f..b2259cf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -163,6 +163,7 @@ limitations under the License.
             <artifactId>hadoop-minicluster</artifactId>
             <version>${hadoop.1.version}</version>
           </dependency>
+
         </dependencies>
       </dependencyManagement>
     </profile>
@@ -322,6 +323,11 @@ limitations under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.sqoop</groupId>
+        <artifactId>sqoop-security</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.sqoop</groupId>
         <artifactId>sqoop-tomcat</artifactId>
         <version>${project.version}</version>
       </dependency>
@@ -474,6 +480,7 @@ limitations under the License.
     <module>connector</module>
     <module>execution</module>
     <module>submission</module>
+    <module>security</module>
     <module>dist</module>
     <module>test</module>
     <module>tools</module>

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/security/pom.xml
----------------------------------------------------------------------
diff --git a/security/pom.xml b/security/pom.xml
new file mode 100644
index 0000000..6ec4994
--- /dev/null
+++ b/security/pom.xml
@@ -0,0 +1,86 @@
+<?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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache</groupId>
+    <artifactId>sqoop</artifactId>
+    <version>2.0.0-SNAPSHOT</version>
+  </parent>
+
+  <groupId>org.apache.sqoop</groupId>
+  <artifactId>sqoop-security</artifactId>
+  <name>Sqoop Security</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.sqoop</groupId>
+      <artifactId>sqoop-core</artifactId>
+    </dependency>
+  </dependencies>
+
+  <!-- Profiles for various supported Hadoop distributions -->
+  <profiles>
+
+    <!-- Hadoop 1.x -->
+    <profile>
+      <id>hadoop100</id>
+
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>100</value>
+        </property>
+      </activation>
+
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Hadoop 2.x (active by default) -->
+    <profile>
+      <id>hadoop200</id>
+
+      <activation>
+        <activeByDefault>true</activeByDefault>
+        <property>
+          <name>hadoop.profile</name>
+          <value>200</value>
+        </property>
+      </activation>
+
+      <properties>
+        <hadoop.profile>200</hadoop.profile>
+      </properties>
+
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/security/src/main/java/org/apache/sqoop/security/KerberosAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/security/src/main/java/org/apache/sqoop/security/KerberosAuthenticationHandler.java b/security/src/main/java/org/apache/sqoop/security/KerberosAuthenticationHandler.java
new file mode 100644
index 0000000..4b920f0
--- /dev/null
+++ b/security/src/main/java/org/apache/sqoop/security/KerberosAuthenticationHandler.java
@@ -0,0 +1,87 @@
+/**
+ * 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.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.log4j.Logger;
+import org.apache.sqoop.common.MapContext;
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.core.SqoopConfiguration;
+
+import java.io.IOException;
+
+public class KerberosAuthenticationHandler extends AuthenticationHandler {
+
+  private static final Logger LOG = Logger.getLogger(KerberosAuthenticationHandler.class);
+
+  /**
+   * Principal for Kerberos option value
+   */
+  private String keytabPrincipal;
+
+  public String getKeytabPrincipal() {
+    return keytabPrincipal;
+  }
+
+  /**
+   * Keytab for Kerberos option value
+   */
+  private String keytabFile;
+
+  public String getKeytabFile() {
+    return keytabFile;
+  }
+
+  public void doInitialize() {
+    securityEnabled = true;
+  }
+
+  public void secureLogin() {
+    MapContext mapContext = SqoopConfiguration.getInstance().getContext();
+    String keytab = mapContext.getString(
+            AuthenticationConstants.AUTHENTICATION_KERBEROS_KEYTAB).trim();
+    if (keytab.length() == 0) {
+      throw new SqoopException(AuthenticationError.AUTH_0001,
+              AuthenticationConstants.AUTHENTICATION_KERBEROS_KEYTAB);
+    }
+    keytabFile = keytab;
+
+    String principal = mapContext.getString(
+            AuthenticationConstants.AUTHENTICATION_KERBEROS_PRINCIPAL).trim();
+    if (principal.length() == 0) {
+      throw new SqoopException(AuthenticationError.AUTH_0002,
+              AuthenticationConstants.AUTHENTICATION_KERBEROS_PRINCIPAL);
+    }
+    keytabPrincipal = principal;
+
+    Configuration conf = new Configuration();
+    conf.set(get_hadoop_security_authentication(),
+            AuthenticationConstants.TYPE.KERBEROS.name());
+    UserGroupInformation.setConfiguration(conf);
+    try {
+      String hostPrincipal = SecurityUtil.getServerPrincipal(principal, "0.0.0.0");
+      UserGroupInformation.loginUserFromKeytab(hostPrincipal, keytab);
+    } catch (IOException ex) {
+      throw new SqoopException(AuthenticationError.AUTH_0003);
+    }
+    LOG.info("Using FileSystemAccess Kerberos authentication, principal ["
+            + principal + "] keytab [" + keytab + "]");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/security/src/main/java/org/apache/sqoop/security/SimpleAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/security/src/main/java/org/apache/sqoop/security/SimpleAuthenticationHandler.java b/security/src/main/java/org/apache/sqoop/security/SimpleAuthenticationHandler.java
new file mode 100644
index 0000000..44209bf
--- /dev/null
+++ b/security/src/main/java/org/apache/sqoop/security/SimpleAuthenticationHandler.java
@@ -0,0 +1,41 @@
+/**
+ * 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.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.log4j.Logger;
+
+public class SimpleAuthenticationHandler extends AuthenticationHandler {
+
+  private static final Logger LOG = Logger.getLogger(SimpleAuthenticationHandler.class);
+
+  public void doInitialize() {
+    securityEnabled = false;
+  }
+
+  public void secureLogin() {
+    //no secureLogin, just set configurations
+    Configuration conf = new Configuration();
+    conf.set(get_hadoop_security_authentication(),
+            AuthenticationConstants.TYPE.SIMPLE.name());
+    UserGroupInformation.setConfiguration(conf);
+    LOG.info("Using FileSystemAccess simple/pseudo authentication, principal ["
+            + System.getProperty("user.name") + "]");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index 67baaa5..be2ee9b 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -49,6 +49,11 @@ limitations under the License.
     </dependency>
 
     <dependency>
+      <groupId>org.apache.sqoop</groupId>
+      <artifactId>sqoop-security</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>org.apache.sqoop.submission</groupId>
       <artifactId>sqoop-submission-mapreduce</artifactId>
       <classifier>hadoop${hadoop.profile}</classifier>

http://git-wip-us.apache.org/repos/asf/sqoop/blob/21d4a951/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 9c71688..4322b1c 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
@@ -129,6 +129,7 @@ public abstract class SqoopMiniCluster {
     mapToProperties(sqoopProperties, getRepositoryConfiguration());
     mapToProperties(sqoopProperties, getSubmissionEngineConfiguration());
     mapToProperties(sqoopProperties, getExecutionEngineConfiguration());
+    mapToProperties(sqoopProperties, getSecurityConfiguration());
 
     FileUtils.writeLines(f, sqoopProperties);
 
@@ -198,4 +199,13 @@ public abstract class SqoopMiniCluster {
     return properties;
   }
 
+  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");
+
+    return properties;
+  }
+
 }