You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@livy.apache.org by js...@apache.org on 2019/10/31 02:35:10 UTC

[incubator-livy] branch master updated: [LIVY-678] Thrift ldap authentication, based on ldapurl, basedn, domain

This is an automated email from the ASF dual-hosted git repository.

jshao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-livy.git


The following commit(s) were added to refs/heads/master by this push:
     new 9b776a5  [LIVY-678] Thrift ldap authentication, based on ldapurl, basedn, domain
9b776a5 is described below

commit 9b776a58fdeac63261f988dde70093bdf0191721
Author: captainzmc <mi...@tencent.com>
AuthorDate: Thu Oct 31 10:34:53 2019 +0800

    [LIVY-678] Thrift ldap authentication, based on ldapurl, basedn, domain
    
    ## What changes were proposed in this pull request?
    Currently, livy thrift-server doesn't support LDAP Authentication. We need to add LDAP authentication as that's preferable method due to security reasons.
    The authentication was based on ldapUrl, basedn, domain.
    User-based filters are also supported in this PR.
    
    ## How was this patch tested?
    
    UTs tests for this part have been added. We can test in UTs
    
    Author: captainzmc <mi...@tencent.com>
    
    Closes #236 from captainzmc/thrift-ldap-impl.
---
 .../src/main/scala/org/apache/livy/LivyConf.scala  |   3 +-
 .../auth/LdapAuthenticationHandlerImpl.scala       |  20 +--
 .../org/apache/livy/server/auth/LdapUtils.scala    |  87 ++++++++++++
 thriftserver/server/pom.xml                        |  44 ++++++
 .../livy/thriftserver/auth/AuthFactory.scala       |   5 +-
 .../thriftserver/auth/AuthenticationProvider.scala |   5 +-
 .../auth/LdapAuthenticationProviderImpl.scala      |  65 +++++++++
 .../livy/thriftserver/auth/ldap/ChainFilter.scala  |  29 ++++
 .../thriftserver/auth/ldap/DirSearchFactory.scala  |  37 +++++
 .../livy/thriftserver/auth/ldap/Filter.scala       |  34 +++++
 .../thriftserver/auth/ldap/LdapSearchFactory.scala |  59 ++++++++
 .../livy/thriftserver/auth/ldap/UserFilter.scala   |  44 ++++++
 .../livy/thriftserver/cli/ThriftHttpServlet.scala  |   2 +-
 .../auth/TestLdapAuthenticationProviderImpl.scala  | 155 +++++++++++++++++++++
 14 files changed, 564 insertions(+), 25 deletions(-)

diff --git a/server/src/main/scala/org/apache/livy/LivyConf.scala b/server/src/main/scala/org/apache/livy/LivyConf.scala
index 0a52d51..683bccc 100644
--- a/server/src/main/scala/org/apache/livy/LivyConf.scala
+++ b/server/src/main/scala/org/apache/livy/LivyConf.scala
@@ -174,7 +174,8 @@ object LivyConf {
     Entry("livy.server.thrift.delegation.token.max-lifetime", "7d")
   val THRIFT_DELEGATION_TOKEN_RENEW_INTERVAL =
     Entry("livy.server.thrift.delegation.token.renew-interval", "1d")
-
+  val THRIFT_LDAP_AUTHENTICATION_USERFILTER =
+    Entry("livy.server.thrift.ldap.authentication.userfilter", null)
   /**
    * Recovery mode of Livy. Possible values:
    * off: Default. Turn off recovery. Every time Livy shuts down, it stops and forgets all sessions.
diff --git a/server/src/main/scala/org/apache/livy/server/auth/LdapAuthenticationHandlerImpl.scala b/server/src/main/scala/org/apache/livy/server/auth/LdapAuthenticationHandlerImpl.scala
index 4fe8c01..53b4df8 100644
--- a/server/src/main/scala/org/apache/livy/server/auth/LdapAuthenticationHandlerImpl.scala
+++ b/server/src/main/scala/org/apache/livy/server/auth/LdapAuthenticationHandlerImpl.scala
@@ -43,24 +43,6 @@ object LdapAuthenticationHandlerImpl {
   val BASE_DN = "ldap.basedn"
   val LDAP_BIND_DOMAIN = "ldap.binddomain"
   val ENABLE_START_TLS = "ldap.enablestarttls"
-
-  private def hasDomain(userName: String): Boolean = {
-    indexOfDomainMatch(userName) > 0
-  }
-
-  /**
-    * Get the index separating the user name from domain name (the user's name up
-    * to the first '/' or '@').
-    */
-  private def indexOfDomainMatch(userName: String): Int = {
-    val idx = userName.indexOf('/')
-    val idx2 = userName.indexOf('@')
-    // Use the earlier match.
-    val endIdx = Math.min(idx, idx2)
-
-    // If neither '/' nor '@' was found, using the latter
-    if (endIdx == -1) Math.max(idx, idx2) else endIdx
-  }
 }
 
 class LdapAuthenticationHandlerImpl extends AuthenticationHandler with Logging {
@@ -142,7 +124,7 @@ class LdapAuthenticationHandlerImpl extends AuthenticationHandler with Logging {
     }
 
     var principle = userName
-    if (!LdapAuthenticationHandlerImpl.hasDomain(userName) && ldapDomain != null) {
+    if (!LdapUtils.hasDomain(userName) && ldapDomain != null) {
       principle = userName + "@" + ldapDomain
     }
     val bindDN = if (baseDN != null) {
diff --git a/server/src/main/scala/org/apache/livy/server/auth/LdapUtils.scala b/server/src/main/scala/org/apache/livy/server/auth/LdapUtils.scala
new file mode 100644
index 0000000..ef770bf
--- /dev/null
+++ b/server/src/main/scala/org/apache/livy/server/auth/LdapUtils.scala
@@ -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.livy.server.auth
+
+import org.apache.livy.LivyConf
+
+/**
+ * Static utility methods related to LDAP authentication module.
+ */
+object LdapUtils {
+
+  def hasDomain(userName: String): Boolean = {
+    indexOfDomainMatch(userName) > 0
+  }
+
+  /**
+   * Get the index separating the user name from domain name (the user's name up
+   * to the first '/' or '@').
+   */
+  def indexOfDomainMatch(userName: String): Int = {
+    val idx = userName.indexOf('/')
+    val idx2 = userName.indexOf('@')
+    // Use the earlier match.
+    val endIdx = Math.min(idx, idx2)
+    // If neither '/' nor '@' was found, using the latter
+    if (endIdx == -1) Math.max(idx, idx2) else endIdx
+  }
+
+  /**
+   * Extracts username from user DN.
+   */
+  def extractUserName(userDn: String): String = {
+    if (!isDn(userDn) && !hasDomain(userDn)) {
+      userDn
+    } else {
+      val domainIdx = indexOfDomainMatch(userDn)
+      if (domainIdx > 0) {
+        userDn.substring(0, domainIdx)
+      } else if (userDn.contains("=")) {
+        userDn.substring(userDn.indexOf("=") + 1, userDn.indexOf(","))
+      } else {
+        userDn
+      }
+    }
+  }
+
+  /**
+   * Detects DN names.
+   */
+  def isDn(name: String): Boolean = {
+    name.contains("=")
+  }
+
+  /**
+   * Creates a principal to be used for user authentication.
+   */
+  def createCandidatePrincipal(conf: LivyConf, user: String): String = {
+    val ldapDomain = conf.get(LivyConf.AUTH_LDAP_USERNAME_DOMAIN)
+    val ldapBaseDN = conf.get(LivyConf.AUTH_LDAP_BASE_DN )
+    val principle = if (!hasDomain(user) && ldapDomain != null) {
+      user + "@" + ldapDomain
+    } else {
+      user
+    }
+
+    if (ldapBaseDN != null) {
+      "uid=" + principle + "," + ldapBaseDN
+    } else {
+      principle
+    }
+  }
+}
+
diff --git a/thriftserver/server/pom.xml b/thriftserver/server/pom.xml
index 86f0b86..f533fc0 100644
--- a/thriftserver/server/pom.xml
+++ b/thriftserver/server/pom.xml
@@ -151,6 +151,50 @@
         </exclusion>
       </exclusions>
     </dependency>
+
+    <!-- needed for LDAP testing -->
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-core</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-protocol-ldap</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-ldif-partition</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.api</groupId>
+      <artifactId>api-ldap-codec-core</artifactId>
+      <version>${ldap-api.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.api</groupId>
+      <artifactId>api-ldap-model</artifactId>
+      <version>${ldap-api.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-server-integ</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-core-integ</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <build>
     <plugins>
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/AuthFactory.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/AuthFactory.scala
index 6ac61d2..0c7ee83 100644
--- a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/AuthFactory.scala
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/AuthFactory.scala
@@ -77,8 +77,9 @@ class AuthFactory(val conf: LivyConf) extends Logging {
   def getAuthTransFactory: TTransportFactory = {
     val isAuthKerberos = authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName)
     val isAuthNoSASL = authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName)
-    // TODO: add LDAP and PAM when supported
-    val isAuthOther = authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName) ||
+    // TODO: add PAM when supported
+    val isAuthOther = authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName) ||
+      authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName) ||
       authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName)
 
     saslServer.map { server =>
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/AuthenticationProvider.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/AuthenticationProvider.scala
index 9464af5..9e6f140 100644
--- a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/AuthenticationProvider.scala
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/AuthenticationProvider.scala
@@ -25,14 +25,15 @@ import org.apache.hive.service.auth.PasswdAuthenticationProvider
 import org.apache.livy.LivyConf
 
 object AuthenticationProvider {
-  // TODO: support LDAP and PAM
-  val AUTH_METHODS = Seq("NONE", "CUSTOM")
+  // TODO: support PAM
+  val AUTH_METHODS = Seq("NONE", "CUSTOM", "LDAP")
 
   @throws[AuthenticationException]
   def getAuthenticationProvider(method: String, conf: LivyConf): PasswdAuthenticationProvider = {
     method match {
       case "NONE" => new NoneAuthenticationProvider
       case "CUSTOM" => new CustomAuthenticationProvider(conf)
+      case "LDAP" => new LdapAuthenticationProviderImpl(conf)
       case _ => throw new AuthenticationException("Unsupported authentication method")
     }
   }
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/LdapAuthenticationProviderImpl.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/LdapAuthenticationProviderImpl.scala
new file mode 100644
index 0000000..966eb29
--- /dev/null
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/LdapAuthenticationProviderImpl.scala
@@ -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.livy.thriftserver.auth
+
+import javax.security.sasl.AuthenticationException
+
+import org.apache.commons.lang.StringUtils
+import org.apache.hive.service.auth.PasswdAuthenticationProvider
+
+import org.apache.livy.thriftserver.auth.ldap._
+import org.apache.livy.LivyConf
+import org.apache.livy.server.auth.LdapUtils
+
+class LdapAuthenticationProviderImpl(val conf: LivyConf) extends PasswdAuthenticationProvider {
+  private val filter: Filter = new ChainFilter(List(new UserFilter(conf)))
+  private val searchFactory: DirSearchFactory = new LdapSearchFactory()
+
+  @throws[AuthenticationException]
+  def Authenticate(user: String, password: String): Unit = {
+    createDirSearch(user, password)
+    applyFilter(user)
+  }
+
+  @throws[AuthenticationException]
+  private def createDirSearch(user: String, password: String): Unit = {
+    if (StringUtils.isBlank(user) || StringUtils.isEmpty(user)) {
+      throw new AuthenticationException(
+        "Error validating LDAP: a null or blank user name has been provided")
+    }
+    if (StringUtils.isBlank(password) || StringUtils.isEmpty(password)) {
+      throw new AuthenticationException(
+        "Error validating LDAP: a null or blank password has been provided")
+    }
+    val principal = LdapUtils.createCandidatePrincipal(conf, user)
+    try {
+      searchFactory.getInstance(conf, principal, password)
+    } catch {
+      case e: AuthenticationException =>
+        throw new AuthenticationException(s"Error validating LDAP user: $user", e)
+    }
+  }
+
+  @throws[AuthenticationException]
+  private def applyFilter(user: String): Unit = {
+    if (LdapUtils.hasDomain(user)) {
+      filter(LdapUtils.extractUserName(user))
+    } else {
+      filter(user)
+    }
+  }
+}
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/ChainFilter.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/ChainFilter.scala
new file mode 100644
index 0000000..41318d3
--- /dev/null
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/ChainFilter.scala
@@ -0,0 +1,29 @@
+/*
+ * 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.livy.thriftserver.auth.ldap
+
+import javax.security.sasl.AuthenticationException
+
+/**
+ * Applies all the filters passed as param to the `ChainFilter`
+ */
+class ChainFilter(val chainedFilters: Seq[Filter]) extends Filter {
+  @throws[AuthenticationException]
+  def apply(user: String): Unit = {
+    chainedFilters.foreach { f => f(user) }
+  }
+}
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/DirSearchFactory.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/DirSearchFactory.scala
new file mode 100644
index 0000000..7d4eed0
--- /dev/null
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/DirSearchFactory.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.livy.thriftserver.auth.ldap
+
+import javax.naming.directory.InitialDirContext
+import javax.security.sasl.AuthenticationException
+
+import org.apache.livy.LivyConf
+
+/**
+ * A factory for DirSearch.
+ */
+trait DirSearchFactory {
+  /**
+   * Returns an instance of DirSearch.
+   *
+   * @param conf     Livy configuration
+   * @param user     username
+   * @param password user password
+   */
+  @throws[AuthenticationException]
+  def getInstance(conf: LivyConf, user: String, password: String): InitialDirContext
+}
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/Filter.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/Filter.scala
new file mode 100644
index 0000000..8e13310
--- /dev/null
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/Filter.scala
@@ -0,0 +1,34 @@
+/*
+ * 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.livy.thriftserver.auth.ldap
+
+import javax.security.sasl.AuthenticationException
+
+/**
+ * The object that filters LDAP users.
+ * The assumption is that this user was already authenticated by a previous bind operation.
+ */
+trait Filter {
+  /**
+   * Applies this filter to the authenticated user.
+   *
+   * @param user   username
+   * @throws AuthenticationException
+   */
+  @throws[AuthenticationException]
+  def apply(user: String): Unit
+}
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/LdapSearchFactory.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/LdapSearchFactory.scala
new file mode 100644
index 0000000..cba5aae
--- /dev/null
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/LdapSearchFactory.scala
@@ -0,0 +1,59 @@
+/*
+ * 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.livy.thriftserver.auth.ldap
+
+import java.util.Hashtable
+import javax.naming.{Context, NamingException}
+import javax.naming.directory.InitialDirContext
+import javax.security.sasl.AuthenticationException
+
+import org.apache.livy.{LivyConf, Logging}
+
+/**
+ * A factory for LDAP search objects.
+ */
+object LdapSearchFactory extends Logging {
+
+  @throws[NamingException]
+  private def createDirContext(
+      conf: LivyConf,
+      principal: String,
+      password: String): InitialDirContext = {
+    val env = new Hashtable[String, String]
+    val ldapUrl = conf.get(LivyConf.AUTH_LDAP_URL)
+    env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory")
+    env.put(Context.PROVIDER_URL, ldapUrl)
+    env.put(Context.SECURITY_AUTHENTICATION, "simple")
+    env.put(Context.SECURITY_CREDENTIALS, password)
+    env.put(Context.SECURITY_PRINCIPAL, principal)
+
+    new InitialDirContext(env)
+  }
+}
+
+class LdapSearchFactory extends DirSearchFactory with Logging {
+
+  @throws(classOf[AuthenticationException])
+  def getInstance(conf: LivyConf, principal: String, password: String): InitialDirContext = {
+    try {
+      LdapSearchFactory.createDirContext(conf, principal, password)
+    } catch {
+      case e: NamingException =>
+        throw new AuthenticationException("Error validating LDAP user", e)
+    }
+  }
+}
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/UserFilter.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/UserFilter.scala
new file mode 100644
index 0000000..a968a48
--- /dev/null
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/auth/ldap/UserFilter.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.livy.thriftserver.auth.ldap
+
+import javax.security.sasl.AuthenticationException
+
+import org.apache.livy.{LivyConf, Logging}
+import org.apache.livy.server.auth.LdapUtils
+
+/**
+ * Filter out all users that are not in the provided in Livy configuration list.
+ */
+class UserFilter(conf: LivyConf) extends Filter with Logging {
+  private val userFilterStr = conf.get(LivyConf.THRIFT_LDAP_AUTHENTICATION_USERFILTER)
+  private val userFilter: Set[String] =
+      if (userFilterStr != null) userFilterStr.split(",").toSet else Set()
+
+  @throws[AuthenticationException]
+  def apply(user: String): Unit = {
+    if (!userFilter.isEmpty) {
+      info("Authenticating user '{}' using user filter", user)
+      val userName = LdapUtils.extractUserName(user).toLowerCase
+      if (!userFilter.contains(userName)) {
+        info("Authentication failed based on user membership")
+        throw new AuthenticationException(
+          "Authentication failed: User not a member of specified list")
+      }
+    }
+  }
+}
diff --git a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/cli/ThriftHttpServlet.scala b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/cli/ThriftHttpServlet.scala
index ae83917..06303e5 100644
--- a/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/cli/ThriftHttpServlet.scala
+++ b/thriftserver/server/src/main/scala/org/apache/livy/thriftserver/cli/ThriftHttpServlet.scala
@@ -264,7 +264,7 @@ class ThriftHttpServlet(
 
 
   /**
-   * Do the authentication (LDAP/PAM not yet supported)
+   * Do the authentication (PAM not yet supported)
    */
   private def doPasswdAuth(request: HttpServletRequest, authType: String): String = {
     val userName = getUsername(request, authType)
diff --git a/thriftserver/server/src/test/scala/org/apache/livy/thriftserver/auth/TestLdapAuthenticationProviderImpl.scala b/thriftserver/server/src/test/scala/org/apache/livy/thriftserver/auth/TestLdapAuthenticationProviderImpl.scala
new file mode 100644
index 0000000..a7b27a0
--- /dev/null
+++ b/thriftserver/server/src/test/scala/org/apache/livy/thriftserver/auth/TestLdapAuthenticationProviderImpl.scala
@@ -0,0 +1,155 @@
+/*
+ * 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.livy.thriftserver.auth
+
+import javax.security.sasl.AuthenticationException
+
+import org.apache.directory.server.annotations.CreateLdapServer
+import org.apache.directory.server.annotations.CreateTransport
+import org.apache.directory.server.core.annotations.ApplyLdifs
+import org.apache.directory.server.core.annotations.ContextEntry
+import org.apache.directory.server.core.annotations.CreateDS
+import org.apache.directory.server.core.annotations.CreatePartition
+import org.apache.directory.server.core.integ.AbstractLdapTestUnit
+import org.apache.directory.server.core.integ.FrameworkRunner
+import org.junit.Assert
+import org.junit.Before
+import org.junit.Test
+import org.junit.runner.RunWith
+
+import org.apache.livy.LivyConf
+
+/**
+ * This unit test verifies the functionality of LdapAuthenticationProviderImpl.
+ */
+@RunWith(classOf[FrameworkRunner])
+@CreateLdapServer(transports = Array(
+  new CreateTransport(
+    protocol = "LDAP",
+    address = "localhost"
+  )))
+@CreateDS(
+  allowAnonAccess = true,
+  partitions = Array(
+    new CreatePartition(
+      name = "Test_Partition",
+      suffix = "dc=example,dc=com",
+      contextEntry = new ContextEntry(entryLdif = "dn: dc=example," +
+        "dc=com \ndc: example\nobjectClass: top\nobjectClass: domain\n\n")
+    )))
+@ApplyLdifs(Array(
+  "dn: uid=bjones,dc=example,dc=com",
+  "cn: Bob Jones",
+  "sn: Jones",
+  "objectClass: inetOrgPerson",
+  "uid: bjones",
+  "userPassword: p@ssw0rd"
+))
+class TestLdapAuthenticationProviderImpl extends AbstractLdapTestUnit {
+  private var handler: LdapAuthenticationProviderImpl = null
+  val livyConf = new LivyConf()
+
+  @Before
+  def setup(): Unit = {
+    livyConf.set(LivyConf.AUTH_LDAP_BASE_DN, "dc=example,dc=com")
+    livyConf.set(LivyConf.AUTH_LDAP_URL, String.format("ldap://%s:%s", "localhost",
+      AbstractLdapTestUnit.getLdapServer.getPort.toString))
+  }
+
+  @Test
+  def testAuthenticatePasses(): Unit = {
+    val user = "bjones"
+    val pwd = "p@ssw0rd"
+
+    try {
+      handler = new LdapAuthenticationProviderImpl(livyConf)
+      handler.Authenticate(user, pwd)
+    } catch {
+      case e: AuthenticationException =>
+        val message = String.format("Authentication failed for user '%s'",
+          user)
+        throw new AssertionError(message, e)
+    }
+  }
+
+  @Test
+  def testAuthenticateWithGroupPasses(): Unit = {
+    val user = "bjones"
+    val pwd = "p@ssw0rd"
+
+    livyConf.set(LivyConf.THRIFT_LDAP_AUTHENTICATION_USERFILTER, "bjones,jake")
+    try {
+      handler = new LdapAuthenticationProviderImpl(livyConf)
+      handler.Authenticate(user, pwd)
+    } catch {
+      case e: AuthenticationException =>
+        val message = String.format("Authentication failed for user '%s'",
+          user)
+        throw new AssertionError(message, e)
+    }
+  }
+
+  @Test
+  def testAuthenticateWithWrongUser(): Unit = {
+    val wrongUser = "jake"
+    val pwd = "p@ssw0rd"
+
+    try {
+      handler = new LdapAuthenticationProviderImpl(livyConf)
+      handler.Authenticate(wrongUser, pwd)
+    } catch {
+      case ex: AuthenticationException =>
+      // Expected
+      case ex: Exception =>
+        Assert.fail
+    }
+  }
+
+  @Test
+  def testAuthenticateWithWrongPassword(): Unit = {
+    val user = "bjones"
+    val wrongPwd = "wrongPwd"
+
+    try {
+      handler = new LdapAuthenticationProviderImpl(livyConf)
+      handler.Authenticate(user, wrongPwd)
+    } catch {
+      case ex: AuthenticationException =>
+      // Expected
+      case ex: Exception =>
+        Assert.fail
+    }
+  }
+
+  @Test
+  def testAuthenticateWithWrongGroup(): Unit = {
+    val user = "bjones"
+    val pwd = "p@ssw0rd"
+    livyConf.set(LivyConf.THRIFT_LDAP_AUTHENTICATION_USERFILTER, "user1,user2")
+
+    try {
+      handler = new LdapAuthenticationProviderImpl(livyConf)
+      handler.Authenticate(user, pwd)
+    } catch {
+      case ex: AuthenticationException =>
+      // Expected
+      case ex: Exception =>
+        Assert.fail
+    }
+  }
+}