You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by su...@apache.org on 2020/11/20 19:20:07 UTC

[incubator-gobblin] branch master updated: [GOBBLIN-1314] Adds ldap group ownership service

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f81a2a2  [GOBBLIN-1314] Adds ldap group ownership service
f81a2a2 is described below

commit f81a2a2ac93091ac878985565bc010532418e2c2
Author: William Lo <wl...@linkedin.com>
AuthorDate: Fri Nov 20 11:19:55 2020 -0800

    [GOBBLIN-1314] Adds ldap group ownership service
    
    Closes #3152 from Will-Lo/add-ldap-group-ownership
---
 .../gobblin/service/GroupOwnershipService.java     |  11 --
 .../gobblin/service/LdapGroupOwnershipService.java |  64 +++++++++
 .../org/apache/gobblin/util/DummyTrustManager.java |  39 +++---
 .../java/org/apache/gobblin/util/LdapUtils.java    | 146 +++++++++++++++++++++
 .../gobblin/util/TrustManagerSocketFactory.java    |  88 +++++++++++++
 5 files changed, 318 insertions(+), 30 deletions(-)

diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/GroupOwnershipService.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/GroupOwnershipService.java
index a84290d..e448e4b 100644
--- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/GroupOwnershipService.java
+++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/GroupOwnershipService.java
@@ -29,15 +29,4 @@ public abstract class GroupOwnershipService {
     * @return true if any of the serviceRequesters belong in the group
     */
    public abstract boolean isMemberOfGroup(List<ServiceRequester> serviceRequesters, String group);
-
-   /**
-    * Extracts ServiceRequester names
-    * @param requesterList
-    * @return a list of service requester names
-    */
-   protected static List<String> extractRequesterNames(List<ServiceRequester> requesterList) {
-      return requesterList.stream()
-          .map(requester -> requester.getName())
-          .collect(Collectors.toList());
-   }
 }
diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LdapGroupOwnershipService.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LdapGroupOwnershipService.java
new file mode 100644
index 0000000..f97d12e
--- /dev/null
+++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LdapGroupOwnershipService.java
@@ -0,0 +1,64 @@
+/*
+ * 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.gobblin.service;
+
+
+import com.typesafe.config.Config;
+import java.util.List;
+import java.util.Set;
+import javax.naming.NamingException;
+import javax.naming.PartialResultException;
+import org.apache.gobblin.annotation.Alias;
+import org.apache.gobblin.util.LdapUtils;
+import org.apache.log4j.Logger;
+
+
+/**
+ * Queries external Active Directory service to check if the requester is part of the group
+ */
+@Alias("ldap")
+public class LdapGroupOwnershipService extends GroupOwnershipService {
+  LdapUtils ldapUtils;
+  private static final Logger logger = Logger.getLogger(LdapGroupOwnershipService.class);
+
+  public LdapGroupOwnershipService(Config config) {
+    this.ldapUtils = new LdapUtils(config);
+  }
+
+  @Override
+  public boolean isMemberOfGroup(List<ServiceRequester> serviceRequesters, String group) {
+    try {
+      Set<String> groupMemberships = ldapUtils.getGroupMembers(group);
+      if (!groupMemberships.isEmpty()) {
+        for (ServiceRequester requester: serviceRequesters) {
+          if (groupMemberships.contains(requester.getName())) {
+            return true;
+          }
+        }
+      }
+      return false;
+    } catch (NamingException e) {
+      logger.warn(String.format("Caught naming exception when parsing results from LDAP server. Message: %s",
+          e.getExplanation()));
+      if (e instanceof PartialResultException) {
+        logger.warn("Check that the Ldap group exists");
+        return false;
+      }
+      throw new RuntimeException(e);
+    }
+  }
+}
diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/GroupOwnershipService.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/DummyTrustManager.java
similarity index 52%
copy from gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/GroupOwnershipService.java
copy to gobblin-utility/src/main/java/org/apache/gobblin/util/DummyTrustManager.java
index a84290d..1386020 100644
--- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/GroupOwnershipService.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/DummyTrustManager.java
@@ -15,29 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.gobblin.service;
+package org.apache.gobblin.util;
+
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import javax.net.ssl.X509TrustManager;
 
-import java.util.List;
-import java.util.stream.Collectors;
 
 /**
- * Service for handling group ownership of flows
+ * A default trust manager used by {@link: TrustManagerSocketFactory}
  */
-public abstract class GroupOwnershipService {
+class DummyTrustManager implements X509TrustManager {
+
+  @Override
+  public void checkClientTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+    return;
+  }
 
-   /**
-    * @return true if any of the serviceRequesters belong in the group
-    */
-   public abstract boolean isMemberOfGroup(List<ServiceRequester> serviceRequesters, String group);
+  @Override
+  public void checkServerTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+    return;
+  }
 
-   /**
-    * Extracts ServiceRequester names
-    * @param requesterList
-    * @return a list of service requester names
-    */
-   protected static List<String> extractRequesterNames(List<ServiceRequester> requesterList) {
-      return requesterList.stream()
-          .map(requester -> requester.getName())
-          .collect(Collectors.toList());
-   }
+  @Override
+  public X509Certificate[] getAcceptedIssuers() {
+    return null;
+  }
 }
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/LdapUtils.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/LdapUtils.java
new file mode 100644
index 0000000..9d30d86
--- /dev/null
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/LdapUtils.java
@@ -0,0 +1,146 @@
+/*
+ * 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.gobblin.util;
+
+import com.typesafe.config.Config;
+import java.util.HashSet;
+import java.util.Hashtable;
+import java.util.Set;
+import javax.naming.Context;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.InitialDirContext;
+import javax.naming.directory.SearchControls;
+import javax.naming.directory.SearchResult;
+import org.apache.gobblin.password.PasswordManager;
+import org.apache.log4j.Logger;
+
+/**
+ * This is a utility class for accessing Active Directory.
+ * Utility factory which returns an instance of {@link LdapUtils}
+ */
+public class LdapUtils {
+  public static final String LDAP_PREFIX = "groupOwnershipService.ldap";
+  public static final String LDAP_BASE_DN_KEY = LDAP_PREFIX + ".baseDn";
+  public static final String LDAP_HOST_KEY = LDAP_PREFIX + ".host";
+  public static final String LDAP_PORT_KEY = LDAP_PREFIX + ".port";
+  public static final String LDAP_USER_KEY = LDAP_PREFIX + ".username";
+  public static final String LDAP_PASSWORD_KEY = LDAP_PREFIX + ".password";
+
+  private static final Logger logger = Logger.getLogger(LdapUtils.class);
+
+  private final String _ldapHost;
+  private final String _ldapPort;
+  private final String _ldapBaseDN;
+
+  // Creds of headless account for searching LDAP
+  private final String _ldapUser;
+  private final String _ldapPassword;
+
+  private final String _personSearchFilter = "(&(objectcategory=Person)(samaccountname=%s))";
+  private final String _groupSearchFilter = "(&(objectcategory=Group)(cn=%s))";
+  private final String _memberSearchFilter = "(&(objectcategory=Person)(memberof=%s))";
+
+  private final String _distinguishedName = "distinguishedName";
+  private final String _samAccount = "sAMAccountName";
+  private final String _memberOf = "memberof";
+
+  public LdapUtils(Config config) {
+    PasswordManager passwordManager = PasswordManager.getInstance(ConfigUtils.configToState(config));
+    String password = passwordManager.readPassword(config.getString(LDAP_PASSWORD_KEY));
+    _ldapHost = config.getString(LDAP_HOST_KEY);
+    _ldapPort = config.getString(LDAP_PORT_KEY);
+    _ldapUser = config.getString(LDAP_USER_KEY);
+    _ldapPassword = password;
+    _ldapBaseDN = config.getString(LDAP_BASE_DN_KEY);
+  }
+
+  /**
+   * Returns DirContext for making LDAP call
+   *
+   * @param username The LDAP sAMAccountName
+   * @param password The LDAP password
+   * @throws NamingException
+   */
+  private DirContext getDirContext(String username, String password) throws NamingException {
+    Hashtable<String, Object> env = new Hashtable<>();
+    env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory");
+    env.put(Context.PROVIDER_URL, String.format("ldaps://%s:%s", _ldapHost, _ldapPort));
+    env.put(Context.SECURITY_AUTHENTICATION, "simple");
+    env.put(Context.SECURITY_PROTOCOL, "ssl");
+    env.put(Context.SECURITY_PRINCIPAL, username);
+    env.put(Context.SECURITY_CREDENTIALS, password);
+
+    env.put("java.naming.ldap.factory.socket", TrustManagerSocketFactory.class.getCanonicalName());
+
+    return new InitialDirContext(env);
+  }
+
+  /**
+   * Returns LDAP SearchResult for given filter and ctx
+   *
+   * @param searchFilter The LDAP filter
+   * @param ctx The DirContext for LDAP
+   * @throws NamingException
+   */
+  private NamingEnumeration<SearchResult> searchLDAP(String searchFilter, DirContext ctx) throws NamingException {
+    String baseDN = _ldapBaseDN;
+    SearchControls controls = new SearchControls();
+    controls.setSearchScope(SearchControls.SUBTREE_SCOPE);
+    return ctx.search(baseDN, searchFilter, controls);
+  }
+
+  /**
+   * Returns String Attribute value
+   *
+   * @param result The LDAP SearchResult, could be either Person or Group
+   * @param attribute Attribute to find from SearchResult
+   * @throws NamingException
+   */
+  private String getAttribute(SearchResult result, String attribute) throws NamingException {
+    return result.getAttributes().get(attribute).get().toString();
+  }
+
+
+
+  public Set<String> getGroupMembers(String groupName) throws NamingException {
+    // Username and password for binding must exist
+    if (_ldapUser == null || _ldapPassword == null) {
+      throw new IllegalStateException("Username and password must be provided when initiating the class");
+    }
+
+    DirContext ctx;
+    Set<String> resultSet = new HashSet<>();
+    ctx = getDirContext(_ldapUser, _ldapPassword);
+    logger.info("Searching for groups");
+    String searchFilter = String.format(_groupSearchFilter, groupName);
+    NamingEnumeration<SearchResult> groupResults = searchLDAP(searchFilter, ctx);
+    SearchResult group = groupResults.next();
+    String distinguishedName = getAttribute(group, _distinguishedName);
+    String membersSearchFilter = String.format(_memberSearchFilter, distinguishedName);
+    logger.info("Searching for members");
+    NamingEnumeration<SearchResult> members = searchLDAP(membersSearchFilter, ctx);
+    while (members.hasMoreElements()) {
+      SearchResult member = members.next();
+      resultSet.add(getAttribute(member, _samAccount));
+    }
+    logger.info(String.format("Members part of group %s: %s", groupName, resultSet.toString()));
+    return resultSet;
+  }
+}
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/TrustManagerSocketFactory.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/TrustManagerSocketFactory.java
new file mode 100644
index 0000000..588565c
--- /dev/null
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/TrustManagerSocketFactory.java
@@ -0,0 +1,88 @@
+/*
+ * 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.gobblin.util;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import javax.net.SocketFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+
+
+/**
+ * A SSL socket wrapper used to create sockets with a {@link: javax.net.ssl.TrustManager}
+ */
+public class TrustManagerSocketFactory extends SSLSocketFactory {
+  private SSLSocketFactory _sslSocketFactory;
+
+  public TrustManagerSocketFactory() {
+    try {
+      SSLContext ctx = SSLContext.getInstance("TLS");
+      ctx.init(null, new TrustManager[]{new DummyTrustManager()}, new SecureRandom());
+      _sslSocketFactory = ctx.getSocketFactory();
+    } catch (KeyManagementException | NoSuchAlgorithmException e) {
+    }
+  }
+
+  public static SocketFactory getDefault() {
+    return new TrustManagerSocketFactory();
+  }
+
+  @Override
+  public Socket createSocket(Socket socket, String host, int port, boolean autoClose) throws IOException {
+    return _sslSocketFactory.createSocket(socket, host, port, autoClose);
+  }
+
+  @Override
+  public String[] getDefaultCipherSuites() {
+    return _sslSocketFactory.getDefaultCipherSuites();
+  }
+
+  @Override
+  public String[] getSupportedCipherSuites() {
+    return _sslSocketFactory.getSupportedCipherSuites();
+  }
+
+  @Override
+  public Socket createSocket(String host, int port) throws IOException, UnknownHostException {
+    return _sslSocketFactory.createSocket(host, port);
+  }
+
+  @Override
+  public Socket createSocket(InetAddress host, int port) throws IOException {
+    return _sslSocketFactory.createSocket(host, port);
+  }
+
+  @Override
+  public Socket createSocket(String host, int port, InetAddress localHost, int localPort)
+      throws IOException, UnknownHostException {
+    return _sslSocketFactory.createSocket(host, port, localHost, localPort);
+  }
+
+  @Override
+  public Socket createSocket(InetAddress address, int port, InetAddress localAddress, int localPort)
+      throws IOException {
+    return _sslSocketFactory.createSocket(address, port, localAddress, localPort);
+  }
+}