You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@uniffle.apache.org by GitBox <gi...@apache.org> on 2022/08/02 12:01:09 UTC

[GitHub] [incubator-uniffle] jerqi commented on a diff in pull request #53: Support storing shuffle data to secured dfs cluster

jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r935489269


##########
client-mr/src/main/java/org/apache/hadoop/mapreduce/task/reduce/RssRemoteMergeManagerImpl.java:
##########
@@ -144,8 +144,9 @@ public RssRemoteMergeManagerImpl(String appId, TaskAttemptID reduceId, JobConf j
     try {
       remoteConf.setInt("dfs.replication", replication);
       remoteConf.setInt("dfs.client.block.write.retries", retries); // origin=3
-      this.remoteFS = ShuffleStorageUtils.getFileSystemForPath(new Path(basePath), remoteConf);
-    } catch (IOException e) {
+      HadoopAccessorProvider.init();
+      this.remoteFS = HadoopAccessorProvider.getFileSystem(new Path(basePath), remoteConf);

Review Comment:
   This part will write spill data to hdfs. 
   One question:
   If Hdfs use kerberos, it's necessary to use kerberos in Spark or MR client?



##########
common/src/main/java/org/apache/uniffle/common/config/RssBaseConf.java:
##########
@@ -157,6 +156,32 @@ public class RssBaseConf extends RssConf {
       .defaultValue(true)
       .withDescription("The switch for jvm metrics verbose");
 
+  public static final ConfigOption<Boolean> RSS_ACCESS_HADOOP_KERBEROS_ENABLE = ConfigOptions
+      .key("rss.access.hadoop.kerberos.enable")

Review Comment:
   rss.access.hadoop.kerberos.enable -> rss.security.hadoop.kerberos.enable



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.uniffle.common.provider;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The HadoopAccessorProvider will provide the only entrypoint to get the hadoop filesystem whether
+ * the hadoop cluster is kerberized or not.
+ * <p>
+ * It should be initialized when the shuffle server/coordinator starts. And in client, there is no need
+ * to login with keytab at startup, the authentication of client side should be managed by computing framework
+ * like Spark/MR.
+ */
+public class HadoopAccessorProvider implements Closeable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(HadoopAccessorProvider.class);
+
+  private static volatile HadoopAccessorProvider provider;
+
+  private final boolean kerberosEnabled;
+  // Only valid when in kerberized cluster.
+  private Map<String, UserGroupInformation> cache;
+  private ScheduledExecutorService scheduledExecutorService;
+
+  private HadoopAccessorProvider(SecurityInfo securityInfo) throws Exception {
+    if (securityInfo == null) {
+      this.kerberosEnabled = false;
+      return;
+    }
+
+    this.kerberosEnabled = true;
+
+    String keytabFile = securityInfo.getKeytabFilePath();
+    String principal = securityInfo.getPrincipal();
+    long reLoginIntervalSec = securityInfo.getReloginIntervalSec();
+
+    if (StringUtils.isEmpty(keytabFile)) {
+      throw new Exception("When hadoop kerberos is enabled, keytab must be set");
+    }
+
+    if (StringUtils.isEmpty(principal)) {
+      throw new Exception("When hadoop kerberos is enabled, principal must be set");
+    }
+
+    if (reLoginIntervalSec <= 0) {
+      throw new Exception("The relogin interval must be negative");
+    }
+
+    Configuration conf = new Configuration(false);
+    conf.set("hadoop.security.authentication", "kerberos");
+
+    UserGroupInformation.setConfiguration(conf);
+    UserGroupInformation.loginUserFromKeytab(principal, keytabFile);
+
+    cache = new ConcurrentHashMap<>();
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder()

Review Comment:
   Could we use ThreadUtils?



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.uniffle.common.provider;
+
+import static org.apache.uniffle.common.config.RssBaseConf.RSS_ACCESS_HADOOP_KERBEROS_ENABLE;
+import static org.apache.uniffle.common.config.RssBaseConf.RSS_ACCESS_HADOOP_KERBEROS_KEYTAB_FILE;
+import static org.apache.uniffle.common.config.RssBaseConf.RSS_ACCESS_HADOOP_KERBEROS_PRINCIPAL;
+import static org.apache.uniffle.common.config.RssBaseConf.RSS_ACCESS_HADOOP_KERBEROS_RELOGIN_INTERVAL;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.io.Closeable;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.uniffle.common.config.RssBaseConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The HadoopAccessorProvider will provide the only entrypoint to get the hadoop filesystem whether
+ * the hadoop cluster is kerberized or not.
+ * <p>
+ * It should be initialized when the shuffle server/coordinator starts. And in client, there is no need
+ * to login with keytab at startup, the authentication of client side should be managed by computing framework
+ * like Spark/MR.
+ */
+public class HadoopAccessorProvider implements Closeable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(HadoopAccessorProvider.class);
+
+  private static volatile HadoopAccessorProvider provider;
+
+  private Map<String, UserGroupInformation> cache = new ConcurrentHashMap<>();
+  private ScheduledExecutorService scheduledExecutorService;
+  private boolean kerberosEnabled = false;
+  private RssBaseConf rssBaseConf;
+  private long reLoginIntervalSec;
+
+  private HadoopAccessorProvider(RssBaseConf rssConf) throws Exception {
+    this.rssBaseConf = rssConf;
+    this.kerberosEnabled = rssConf.getBoolean(RSS_ACCESS_HADOOP_KERBEROS_ENABLE);
+
+    if (kerberosEnabled) {
+      String keytabFile = rssConf.getString(RSS_ACCESS_HADOOP_KERBEROS_KEYTAB_FILE);
+      if (StringUtils.isEmpty(keytabFile)) {
+        throw new Exception("When hadoop kerberos is enabled. the conf of "
+            + RSS_ACCESS_HADOOP_KERBEROS_KEYTAB_FILE.key() + " must be set");
+      }
+
+      String principal = rssConf.getString(RSS_ACCESS_HADOOP_KERBEROS_PRINCIPAL);
+      if (StringUtils.isEmpty(principal)) {
+        throw new Exception("When hadoop kerberos is enabled. the conf of "
+            + RSS_ACCESS_HADOOP_KERBEROS_PRINCIPAL.key() + " must be set");
+      }
+
+      this.reLoginIntervalSec = rssConf.getLong(RSS_ACCESS_HADOOP_KERBEROS_RELOGIN_INTERVAL);
+
+      Configuration conf = new Configuration(false);
+      conf.set("hadoop.security.authentication", "kerberos");
+
+      UserGroupInformation.setConfiguration(conf);
+      UserGroupInformation.loginUserFromKeytab(principal, keytabFile);
+
+      cache = new ConcurrentHashMap<>();
+
+      LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+          keytabFile, principal, UserGroupInformation.getLoginUser());
+
+      scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+          new ThreadFactoryBuilder()
+              .setDaemon(true).setNameFormat("Kerberos-Relogin-%d").build());
+      scheduledExecutorService.scheduleAtFixedRate(
+          HadoopAccessorProvider::kerberosRelogin,
+          reLoginIntervalSec,
+          reLoginIntervalSec,
+          TimeUnit.SECONDS);
+    }
+  }
+
+  @VisibleForTesting
+  static void kerberosRelogin() {
+    try {
+      LOGGER.info("Renewing kerberos token.");
+      UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
+    } catch (Throwable t) {
+      LOGGER.error("Error in token renewal task: ", t);
+    }
+  }
+
+  private static UserGroupInformation getProxyUser(String user) throws IOException {
+    provider.cache.putIfAbsent(

Review Comment:
   remind



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@uniffle.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@uniffle.apache.org
For additional commands, e-mail: issues-help@uniffle.apache.org