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/07/12 10:07:59 UTC

[GitHub] [incubator-uniffle] zuston opened a new pull request, #53: Support storing shuffle data to secured dfs cluster

zuston opened a new pull request, #53:
URL: https://github.com/apache/incubator-uniffle/pull/53

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://github.com/Tencent/Firestorm/blob/master/CONTRIBUTING.md
     2. Ensure you have added or run the appropriate tests for your PR
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]XXXX Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
   -->
   
   ### What changes were proposed in this pull request?
   Support storing shuffle data to secured dfs cluster
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   Now uniffle dont support visiting shuffle data to secured dfs cluster.
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   Manual tests.
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919806792


##########
storage/src/main/java/org/apache/uniffle/storage/common/HdfsStorage.java:
##########
@@ -37,10 +37,14 @@ public class HdfsStorage extends AbstractStorage {
   private final String storagePath;
   private final Configuration conf;
   private String storageHost;
+  private final String user;
+  private final boolean securityEnable;
 
-  public HdfsStorage(String path, Configuration conf) {
+  public HdfsStorage(String path, Configuration conf, String user, boolean securityEnable) {
     this.storagePath = path;
     this.conf = conf;
+    this.user = user;
+    this.securityEnable = securityEnable;

Review Comment:
   securityEnable can be got from conf
   user shouldn't be a member of HdfsStorage, and I prefer HdfsShuffleWriteHandler to hold the user info



##########
proto/src/main/proto/Rss.proto:
##########
@@ -151,6 +151,8 @@ message ShuffleRegisterRequest {
   int32 shuffleId = 2;
   repeated ShufflePartitionRange partitionRanges = 3;
   RemoteStorage remoteStorage = 4;
+  string user = 5;
+  google.protobuf.BoolValue securityEnable = 6;

Review Comment:
   securityEnable can be got from configuration?



##########
storage/src/main/java/org/apache/uniffle/storage/handler/impl/UploadedStorageHdfsShuffleReadHandler.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.storage.handler.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.roaringbitmap.longlong.Roaring64NavigableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.ShuffleIndexResult;
+
+public class UploadedStorageHdfsShuffleReadHandler extends HdfsShuffleReadHandler {

Review Comment:
   Upload feature is removed and we needn't to support it any more.



##########
server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java:
##########
@@ -126,18 +128,28 @@ public ShuffleTaskManager(
     thread.start();
   }
 
+  public StatusCode registerShuffle(
+          String appId,
+          int shuffleId,
+          List<PartitionRange> partitionRanges,
+          RemoteStorageInfo remoteStorageInfo) {
+    return registerShuffle(appId, shuffleId, partitionRanges, remoteStorageInfo, null, false);
+  }
+
   public StatusCode registerShuffle(
       String appId,
       int shuffleId,
       List<PartitionRange> partitionRanges,
-      RemoteStorageInfo remoteStorageInfo) {
+      RemoteStorageInfo remoteStorageInfo,
+      String user, boolean securityEnable) {
     refreshAppId(appId);
+    appUserMap.put(appId, user);

Review Comment:
   put -> putIfAbsent



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * 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 final long RELOGIN_CHECK_INTERVAL_SEC = 60L;

Review Comment:
   RELOGIN_CHECK_INTERVAL_SEC should be configurable



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * 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 final long RELOGIN_CHECK_INTERVAL_SEC = 60L;
+
+    private static volatile HadoopAccessorProvider provider;
+
+    private Map<String, UserGroupInformation> cache = new ConcurrentHashMap<>();
+    private ScheduledExecutorService scheduledExecutorService;
+    private boolean kerberosEnabled = false;
+    private RssBaseConf rssBaseConf;
+
+    private HadoopAccessorProvider(RssBaseConf rssConf) throws Exception {

Review Comment:
   HadoopAccessorProvider is used for HDFS access, how about replace RssBaseConf with Configuration?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919955938


##########
proto/src/main/proto/Rss.proto:
##########
@@ -151,6 +151,8 @@ message ShuffleRegisterRequest {
   int32 shuffleId = 2;
   repeated ShufflePartitionRange partitionRanges = 3;
   RemoteStorage remoteStorage = 4;
+  string user = 5;

Review Comment:
   Could we put user and securityEnable into the field `remoteStorage`?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r938562722


##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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 = Maps.newConcurrentMap();
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {
+      synchronized (HadoopAccessorProvider.class) {
+        if (provider == null) {
+          final HadoopAccessorProvider hadoopAccessorProvider = new HadoopAccessorProvider(securityInfo);
+          provider = hadoopAccessorProvider;
+        }
+      }
+    }
+    LOGGER.info("The {} has been initialized, kerberos enable: {}",
+        HadoopAccessorProvider.class.getSimpleName(),
+        provider.kerberosEnabled);
+  }
+
+  /**
+   * No need to
+   * For non-kerberized cluster access, like client side reading shuffle data.
+   */
+  public static void init() throws Exception {
+    init(null);
+  }
+
+  @VisibleForTesting
+  public static void cleanup() throws Exception {
+    if (provider != null) {
+      provider.close();
+      provider = null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (cache != null) {
+      for (UserGroupInformation ugi : cache.values()) {
+        try {
+          FileSystem.closeAllForUGI(ugi);
+        } catch (IOException ioException) {
+          LOGGER.error("Exception occurred while closing filesystems for {}", ugi.getUserName(), ioException);
+        }
+      }
+      cache.clear();

Review Comment:
   Got it. I think no cache is also OK. I will remove it.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1187417504

   > Access to secured clusters and isolation of user shuffle data are orthogonal, you can still do the isolation in a non-securere cluster (using the proxy user way in your pr).
   
   It could use the proxy user to make the data written to unsecured dfs cluster, to make data isolation? Please let me know whether i am right.
   
   > Could you please help to write the pr's description in more detail, and write a design doc to describe your motivation, design, workflow, compatibility, and test case in different scenarios since it's a huge and critical change.
   
   Do i need to introduce a new page to submit proposal like RFC? Or just write it in description? Or open a new issue to track and discuss.
   
   @duanmeng 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1195360622

   If you have time, could u help review this proposal? @jerqi


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922839114


##########
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 {

Review Comment:
   I read some Hadoop code about Kerberos, Maybe the constructor method should be
   ```
   new HadoopAccessorProvider(String principalName, String keytabName, Configuration conf);
   ```



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922629943


##########
proto/src/main/proto/Rss.proto:
##########
@@ -151,6 +151,8 @@ message ShuffleRegisterRequest {
   int32 shuffleId = 2;
   repeated ShufflePartitionRange partitionRanges = 3;
   RemoteStorage remoteStorage = 4;
+  string user = 5;

Review Comment:
   Sounds great! I will do it.
   
   Besides it looks we dont need to populate securityEnabled, it can be inferred by remote conf.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r923349401


##########
common/src/main/java/org/apache/uniffle/common/RemoteStorageInfo.java:
##########
@@ -34,8 +34,10 @@ public class RemoteStorageInfo implements Serializable {
   private final String path;
   private final Map<String, String> confItems;
 
+  private String user;

Review Comment:
   You are right. It looks the RemoteStorageInfo should only store storage metadata instead of spark job info like user.
   
   Maybe we should still add the param of `user` in the method of `registerShuffle`. Do u think so?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r923354116


##########
common/src/main/java/org/apache/uniffle/common/RemoteStorageInfo.java:
##########
@@ -34,8 +34,10 @@ public class RemoteStorageInfo implements Serializable {
   private final String path;
   private final Map<String, String> confItems;
 
+  private String user;

Review Comment:
   > user shouldn't be member of RemoteStorageInfo which is used for describe storage only.
   
   User seems only to be used for storage, what's other usage for shuffle server?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919933812


##########
storage/src/main/java/org/apache/uniffle/storage/handler/impl/UploadedStorageHdfsShuffleReadHandler.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.storage.handler.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.roaringbitmap.longlong.Roaring64NavigableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.ShuffleIndexResult;
+
+public class UploadedStorageHdfsShuffleReadHandler extends HdfsShuffleReadHandler {

Review Comment:
   Sorry for that. I rebased the latest master but forget to delete it.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937294464


##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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 = Maps.newConcurrentMap();
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {
+      synchronized (HadoopAccessorProvider.class) {
+        if (provider == null) {
+          final HadoopAccessorProvider hadoopAccessorProvider = new HadoopAccessorProvider(securityInfo);
+          provider = hadoopAccessorProvider;
+        }
+      }
+    }
+    LOGGER.info("The {} has been initialized, kerberos enable: {}",
+        HadoopAccessorProvider.class.getSimpleName(),
+        provider.kerberosEnabled);
+  }
+
+  /**
+   * No need to
+   * For non-kerberized cluster access, like client side reading shuffle data.
+   */
+  public static void init() throws Exception {
+    init(null);
+  }
+
+  @VisibleForTesting
+  public static void cleanup() throws Exception {
+    if (provider != null) {
+      provider.close();
+      provider = null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (cache != null) {
+      for (UserGroupInformation ugi : cache.values()) {
+        try {
+          FileSystem.closeAllForUGI(ugi);
+        } catch (IOException ioException) {
+          LOGGER.error("Exception occurred while closing filesystems for {}", ugi.getUserName(), ioException);
+        }
+      }
+      cache.clear();

Review Comment:
   No cache is also OK. But I think the number of users may be not too much and no need to consider this.



-- 
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


[GitHub] [incubator-uniffle] jerqi merged pull request #53: Support storing shuffle data to secured dfs cluster

Posted by GitBox <gi...@apache.org>.
jerqi merged PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r946288876


##########
server/src/main/java/org/apache/uniffle/server/storage/StorageManager.java:
##########
@@ -45,9 +45,11 @@ public interface StorageManager {
 
   void stop();
 
-  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo);
+  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo, String user);

Review Comment:
   `WriteHandler` is created with `CreateShuffleWriteHandlerRequest` in `ShuffleFlushManager`, and `shuffleTaskManager` can be accessed in `ShuffleFlushManager` with `ShuffleServer.getTaskManager()`



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r936199819


##########
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:
   Only will be cleaned when stopping server/coordinator.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1220167783

   PTAL @colinmjj @jerqi . If having any problems, I think I will fix in this weekend.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922839114


##########
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 {

Review Comment:
   I read some Hadoop code about Kerberos, Maybe the constructor method should be
   ```
   new HadoopAccessorProvider(String principal, String keyTab, Configuration conf);
   ```



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919934574


##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * 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 final long RELOGIN_CHECK_INTERVAL_SEC = 60L;
+
+    private static volatile HadoopAccessorProvider provider;
+
+    private Map<String, UserGroupInformation> cache = new ConcurrentHashMap<>();
+    private ScheduledExecutorService scheduledExecutorService;
+    private boolean kerberosEnabled = false;
+    private RssBaseConf rssBaseConf;
+
+    private HadoopAccessorProvider(RssBaseConf rssConf) throws Exception {

Review Comment:
   It's OK. But we still should set these config items into RssBaseConf.



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * 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 final long RELOGIN_CHECK_INTERVAL_SEC = 60L;

Review Comment:
   Got it.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919933113


##########
storage/src/main/java/org/apache/uniffle/storage/common/HdfsStorage.java:
##########
@@ -37,10 +37,14 @@ public class HdfsStorage extends AbstractStorage {
   private final String storagePath;
   private final Configuration conf;
   private String storageHost;
+  private final String user;
+  private final boolean securityEnable;
 
-  public HdfsStorage(String path, Configuration conf) {
+  public HdfsStorage(String path, Configuration conf, String user, boolean securityEnable) {
     this.storagePath = path;
     this.conf = conf;
+    this.user = user;
+    this.securityEnable = securityEnable;

Review Comment:
   Got it.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1220656683

   Thanks for your patience and kind @jerqi @colinmjj . 😄


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1206079392

   @jerqi . Updated, and all CI test passed.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r936210982


##########
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:
   Got it.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1201095903

   > @zuston Do you commit the latest source? Some previous comments seems not be fixed, eg, user shouldn't be a member in RemoteStorage
   > 
   > ```
   > message RemoteStorage {
   >   string path = 1;
   >   repeated RemoteStorageConfItem remoteStorageConf = 2;
   >   string user = 3;
   > }
   > ```
   
   Sorry, i forget to submit, but i think i can submit new commit tomorrow. 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1214695294

   > > 2. Add an api in Storage to check if it is secured, eg, isSecuredStorage()
   > > 3. Create new writeHandler/readHandler to deal with secured write/read, eg, SecuredHdfsShuffleWriteHandler extends HdfsShuffleWriteHandler
   > 
   > I think there is no need to introduce extra secured method and writer handler for security.
   > 
   > All the secured operation can be handled by filesystem. And the difference of secured and no-secured fs is the way of initialization which will be handled by HadoopFilesystemProvider [52c53d5](https://github.com/apache/incubator-uniffle/commit/52c53d513c494990ad2d9e2ad3b56584e9473ea8)
   > 
   > Please let me what you think @jerqi @colinmjj . Thanks
   
   @zuston How secured operation is handled by HadoopFilesystemProvider? When do the such operation, the common way is like:
   ```
   ugi.doAs() {
       // Real action
   }
   ```
   You mean the real action will be passed to HadoopFilesystemProvider as a parameter?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerryshao commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1182869639

   I think what @zuston mentioned here is the case, for example:
   
   * user A launched a Spark application, and send shuffle data to shuffle server.
   * Shuffle server should write shuffle data with "user A" into HDFS.
   * Reduce tasks (launched by user A) could read shuffle data in HDFS with user A.
   
   Otherwise, user A may not have the permission to ready shuffle data written by shuffle server (another user) if it it is not owned by A.
   
   I'm not sure if I understand correctly or not? @zuston 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937295648


##########
coordinator/src/test/java/org/apache/uniffle/coordinator/AccessCandidatesCheckerTest.java:
##########
@@ -38,6 +41,11 @@
 
 public class AccessCandidatesCheckerTest {
 
+  @BeforeAll

Review Comment:
   Yes. I add some tests in previous commits. But I found some conflicts and CI will fail when we use the non-kerberized HDFS and kerberilized HDFS together. Maybe I should dig the cause.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r938473804


##########
server/src/main/java/org/apache/uniffle/server/storage/LocalStorageManager.java:
##########
@@ -170,6 +170,11 @@ public Checker getStorageChecker() {
     return checker;
   }
 
+  @Override
+  public String getStorageUser(String appId) {
+    return null;

Review Comment:
   Empty string may be better.



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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 = Maps.newConcurrentMap();
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {
+      synchronized (HadoopAccessorProvider.class) {
+        if (provider == null) {
+          final HadoopAccessorProvider hadoopAccessorProvider = new HadoopAccessorProvider(securityInfo);
+          provider = hadoopAccessorProvider;
+        }
+      }
+    }
+    LOGGER.info("The {} has been initialized, kerberos enable: {}",
+        HadoopAccessorProvider.class.getSimpleName(),
+        provider.kerberosEnabled);
+  }
+
+  /**
+   * No need to
+   * For non-kerberized cluster access, like client side reading shuffle data.
+   */
+  public static void init() throws Exception {
+    init(null);
+  }
+
+  @VisibleForTesting
+  public static void cleanup() throws Exception {
+    if (provider != null) {
+      provider.close();
+      provider = null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (cache != null) {
+      for (UserGroupInformation ugi : cache.values()) {
+        try {
+          FileSystem.closeAllForUGI(ugi);
+        } catch (IOException ioException) {
+          LOGGER.error("Exception occurred while closing filesystems for {}", ugi.getUserName(), ioException);
+        }
+      }
+      cache.clear();

Review Comment:
   A shuffle server is a long time service, I worried about the problems, more and more people use the service, user occupy too much memory.



##########
server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java:
##########
@@ -158,6 +158,7 @@ private void flushToFile(ShuffleDataFlushEvent event) {
           writeSuccess = true;
           LOG.warn("AppId {} was removed already, event {} should be dropped", event.getAppId(), event);
         } else {
+          String user = storageManager.getStorageUser(event.getAppId());

Review Comment:
   Remind.



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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 = Maps.newConcurrentMap();
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {
+      synchronized (HadoopAccessorProvider.class) {
+        if (provider == null) {
+          final HadoopAccessorProvider hadoopAccessorProvider = new HadoopAccessorProvider(securityInfo);
+          provider = hadoopAccessorProvider;
+        }
+      }
+    }
+    LOGGER.info("The {} has been initialized, kerberos enable: {}",
+        HadoopAccessorProvider.class.getSimpleName(),
+        provider.kerberosEnabled);
+  }
+
+  /**
+   * No need to
+   * For non-kerberized cluster access, like client side reading shuffle data.
+   */
+  public static void init() throws Exception {
+    init(null);
+  }
+
+  @VisibleForTesting
+  public static void cleanup() throws Exception {
+    if (provider != null) {
+      provider.close();
+      provider = null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (cache != null) {
+      for (UserGroupInformation ugi : cache.values()) {
+        try {
+          FileSystem.closeAllForUGI(ugi);
+        } catch (IOException ioException) {
+          LOGGER.error("Exception occurred while closing filesystems for {}", ugi.getUserName(), ioException);
+        }
+      }
+      cache.clear();

Review Comment:
   If we want to use a cache, I think we need a LRU cache.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1211510881

   @zuston Is it possible to split the PR according to following function:
   1. register remote storage to shuffle server with user info and stores the mapping about `appId -> user`
   2. Add an api in Storage to check if it is secured, eg, `isSecuredStorage()`
   3. Create new writeHandler/readHandler to deal with secured write/read, eg, `SecuredHdfsShuffleWriteHandler extends HdfsShuffleWriteHandler` 
   4. For Shuffle Server, there has an executor service to deal with kerberos login if secured is enabled.
   Just for your reference to split the PR and will be easy to review.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r948588367


##########
.gitignore:
##########
@@ -23,3 +23,6 @@ dependency-reduced-pom.xml
 rss-*.tgz
 hadoop-*.tar.gz
 deploy/kubernetes/docker/hadoopconfig/*
+common/build/

Review Comment:
   These folders are the dfs data dirs



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1220269468

   > It's ok for me. Could you resolve conflicts first?
   
   Done


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1214540938

   > > 2. Add an api in Storage to check if it is secured, eg, isSecuredStorage()
   > > 3. Create new writeHandler/readHandler to deal with secured write/read, eg, SecuredHdfsShuffleWriteHandler extends HdfsShuffleWriteHandler
   > 
   > I think there is no need to introduce extra secured method and writer handler for security.
   > 
   > All the secured operation can be handled by filesystem. And the difference of secured and no-secured fs is the way of initialization which will be handled by HadoopFilesystemProvider [52c53d5](https://github.com/apache/incubator-uniffle/commit/52c53d513c494990ad2d9e2ad3b56584e9473ea8)
   > 
   > Please let me what you think @jerqi @colinmjj . Thanks
   
   LGTM, cc @colinmjj 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1212791890

   > The changed first commit: [52c53d5](https://github.com/apache/incubator-uniffle/commit/52c53d513c494990ad2d9e2ad3b56584e9473ea8)
   > 
   > Changelog:
   > 
   > 1. Introduce the `SecurityContextFactory` to manage security.
   > 2. In `HadoopSecurityContext`, it will do kerberos login and refresh auth. The `hadoopSecurityContext` will only be initialized when kerberos security is enabled. Default securityContext is `NoOpSecurityContext`.
   > 3. `HadoopFileSystemProvider` will be a static class for invoking.
   > 
   > This commit make the security and filesystem retrieval separate.
   > 
   > Could u help review it? @jerqi @colinmjj And I will submit commits one by one in next days.
   
   I have left some comments. General code is ok.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937298545


##########
integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerGrpcTest.java:
##########
@@ -626,4 +627,21 @@ private void addExpectedBlockIds(Roaring64NavigableMap bitmap, List<Long> blockI
       bitmap.addLong(blockIds.get(i));
     }
   }
+
+  @Test
+  public void testGettingUserWhenUsingHdfsStorage() throws Exception {

Review Comment:
   Yes. After I solve the above problems, i will attach more test cases.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r938600894


##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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 = Maps.newConcurrentMap();
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {
+      synchronized (HadoopAccessorProvider.class) {
+        if (provider == null) {
+          final HadoopAccessorProvider hadoopAccessorProvider = new HadoopAccessorProvider(securityInfo);
+          provider = hadoopAccessorProvider;
+        }
+      }
+    }
+    LOGGER.info("The {} has been initialized, kerberos enable: {}",
+        HadoopAccessorProvider.class.getSimpleName(),
+        provider.kerberosEnabled);
+  }
+
+  /**
+   * No need to
+   * For non-kerberized cluster access, like client side reading shuffle data.
+   */
+  public static void init() throws Exception {
+    init(null);
+  }
+
+  @VisibleForTesting
+  public static void cleanup() throws Exception {
+    if (provider != null) {
+      provider.close();
+      provider = null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (cache != null) {
+      for (UserGroupInformation ugi : cache.values()) {
+        try {
+          FileSystem.closeAllForUGI(ugi);
+        } catch (IOException ioException) {
+          LOGGER.error("Exception occurred while closing filesystems for {}", ugi.getUserName(), ioException);
+        }
+      }
+      cache.clear();

Review Comment:
   Removed.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919932906


##########
proto/src/main/proto/Rss.proto:
##########
@@ -151,6 +151,8 @@ message ShuffleRegisterRequest {
   int32 shuffleId = 2;
   repeated ShufflePartitionRange partitionRanges = 3;
   RemoteStorage remoteStorage = 4;
+  string user = 5;
+  google.protobuf.BoolValue securityEnable = 6;

Review Comment:
   Got it. This is can be gotten by remote storage conf.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1182745110

   > For spark client, it can depend on spark's implementation, and read data according to delegation token.
   
   yes. There is no need to retrieve fs by ugi proxy user. The credentials have been fetched in spark driver side when starting.
   
   > For Shuffle server, Hadoop conf can be updated with security enable when write data to HDFS
   What's the advantage to add HadoopAccessorProvider?
   
   Now shuffle server can’t access secured cluster due to lacking keytab. 
   
   To solve this, i introduced some config and then shuffle server can login and delegate user to write hdfs files by using the proxy user.
   
   Current uniffle don’t support writing shuffle data to secured dfs cluster.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922805915


##########
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 {

Review Comment:
   You can look at the method implement. you can use as below.
   ```
   new HadoopAccessorProvider(rssConf.getHadoopConf());
   ```



-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1182810234

   
   
   
   > @colinmjj @jerqi
   > 
   > > For spark client, it can depend on spark's implementation, and read data according to delegation token.
   > 
   > yes. There is no need to retrieve fs by ugi proxy user. The credentials have been fetched in spark driver side when starting.
   > 
   > > For Shuffle server, Hadoop conf can be updated with security enable when write data to HDFS
   > > What's the advantage to add HadoopAccessorProvider?
   > 
   > Now shuffle server can’t access secured cluster due to lacking keytab.
   > 
   > To solve this, i introduced some config and then shuffle server can login and delegate user to write hdfs files by using the proxy user.
   
   Have you try the following way to update Hadoop configuration:
   1. Option 1, in Shuffle server, conf prefix with `rss.server.hadoop` will be added to Hadoop configuration for data writing.
   2. Option 2, Coordinator can manage the information for different Hdfs, you can check the related API `rpc fetchRemoteStorage(FetchRemoteStorageRequest)` for more detail.
   
   For the keytab problem, do you mean it can't be existed in Shuffle Server?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
duanmeng commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1182850800

   We may not need to support secured hdfs in Uniffle, the DFSClient would create ugi and spawn a renew thread automatically when the shuffler server gets a FileSystem. Normally you could use the hdfs cli to read/write files in secured hdfs if the kerberos-related config is right in you $HADOOP_HOME. Could you check the code in `DFSClient` and `UserGroupInformation` and test without any code change using a secured hdfs. @zuston 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1203573421

   I test this PR in our internal env.
   
   - [x] coordinator read the exclude-node-file from kerberlized HDFS
   - [x] shuffle server write shuffle data to kerbeilized HDFS and client read it directly.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922941849


##########
common/src/main/java/org/apache/uniffle/common/RemoteStorageInfo.java:
##########
@@ -34,8 +34,10 @@ public class RemoteStorageInfo implements Serializable {
   private final String path;
   private final Map<String, String> confItems;
 
+  private String user;

Review Comment:
   user shouldn't be member of RemoteStorageInfo which is used for describe storage only.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r943052816


##########
server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java:
##########
@@ -433,6 +437,10 @@ public int getRequireBufferSize(long requireId) {
     return pabi.getRequireSize();
   }
 
+  public String getUserByAppID(String appid) {

Review Comment:
   getUserByAppID(String appid) -> getUserByAppI**d**(String app**I**d)



##########
server/src/main/java/org/apache/uniffle/server/storage/StorageManager.java:
##########
@@ -45,9 +45,11 @@ public interface StorageManager {
 
   void stop();
 
-  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo);
+  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo, String user);

Review Comment:
   StorageManager only care about info about storage, and `appId -> user` can be stored in ShuffleTaskManager



##########
.gitignore:
##########
@@ -23,3 +23,6 @@ dependency-reduced-pom.xml
 rss-*.tgz
 hadoop-*.tar.gz
 deploy/kubernetes/docker/hadoopconfig/*
+common/build/

Review Comment:
   what's the usage of these folders?



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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;

Review Comment:
   It's strange to have HadoopAccessorProvider as a member here.



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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 {

Review Comment:
   As a singleton class, there shouldn't have lots of static method, except `getInstance()`
   As a static class, there shouldn't have method like `init()`
   It's kind of strange of the implementation of this class.



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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;
+  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);
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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 {
+    return UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser());
+  }
+
+  private static FileSystem getInternalFileSystem(

Review Comment:
   It's better to split implementation of getProxyUser and getFileSystem



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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;
+  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);
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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 {
+    return UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser());
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {
+      synchronized (HadoopAccessorProvider.class) {
+        if (provider == null) {
+          final HadoopAccessorProvider hadoopAccessorProvider = new HadoopAccessorProvider(securityInfo);
+          provider = hadoopAccessorProvider;
+        }
+      }
+    }
+    LOGGER.info("The {} has been initialized, kerberos enable: {}",
+        HadoopAccessorProvider.class.getSimpleName(),
+        provider.kerberosEnabled);
+  }
+
+  /**
+   * No need to
+   * For non-kerberized cluster access, like client side reading shuffle data.
+   */
+  public static void init() throws Exception {

Review Comment:
   It's kind of tricky for the static class which should be invoked init() method before using it 



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1182908394

   Now our Uniffle already support multiple HDFS. Is it possible to use the secured hdfs cluster and unsecured hdfds cluster?
   And some configuration may be added to RemoteStorageConfItem.https://github.com/apache/incubator-uniffle/blob/0f6a896efbd3a435de5e7a5d28843ecd05c38bde/proto/src/main/proto/Rss.proto#L360


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919953816


##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * 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 final long RELOGIN_CHECK_INTERVAL_SEC = 60L;
+
+    private static volatile HadoopAccessorProvider provider;
+
+    private Map<String, UserGroupInformation> cache = new ConcurrentHashMap<>();
+    private ScheduledExecutorService scheduledExecutorService;
+    private boolean kerberosEnabled = false;
+    private RssBaseConf rssBaseConf;
+
+    private HadoopAccessorProvider(RssBaseConf rssConf) throws Exception {

Review Comment:
   Yes. I think this conf should be belonged to shuffle server or coordinator. If configured in hadoop conf, where to get the conf when starting the coordinator/shuffle server?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r936193761


##########
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")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Whether enable visiting secured hadoop cluster.");
+
+  public static final ConfigOption<String> RSS_ACCESS_HADOOP_KERBEROS_KEYTAB_FILE = ConfigOptions
+      .key("rss.access.hadoop.kerberos.keytab.file")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("The kerberos keytab file path. And only when "
+          + RSS_ACCESS_HADOOP_KERBEROS_ENABLE.key() + " enabled, the option will be valid.");
+
+  public static final ConfigOption<String> RSS_ACCESS_HADOOP_KERBEROS_PRINCIPAL = ConfigOptions
+      .key("rss.access.hadoop.kerberos.principal")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("The kerberos keytab principal. And only when "
+          + RSS_ACCESS_HADOOP_KERBEROS_ENABLE.key() + " enabled, the option will be valid.");
+
+  public static final ConfigOption<Long> RSS_ACCESS_HADOOP_KERBEROS_RELOGIN_INTERVAL = ConfigOptions

Review Comment:
   Got it.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1206215699

   LGTM, I have no other suggestion. cc @colinmjj @duanmeng 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922840055


##########
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:
   When do we clean cache?



##########
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 {

Review Comment:
   I feel like this class is a delegation for Hadoop Filesystem.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922839394


##########
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 {

Review Comment:
   Should we have the class HadoopAccessorProvider?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
duanmeng commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1186731252

   Access to secured clusters and isolation of user shuffle data are orthogonal, you can still do the isolation innon-securered cluster (using the proxy user way in your pr).
   Could you please help to write the pr's description in more detail, and write a design doc to describe your motivation, design, workflow, compatibility, and test case in different scenarios since it's a huge and critical change.
   @zuston 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1193318100

   Update the description and submit a single proposal (https://docs.google.com/document/d/1pIDwCwv8iwnXmFQeTZKA5tK55SRc_f0SPKYFONA5K70). Feel free to discuss or comment more.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r949995218


##########
storage/src/main/java/org/apache/uniffle/storage/handler/impl/HdfsFileWriter.java:
##########
@@ -39,35 +39,37 @@ public class HdfsFileWriter implements Closeable {
 
   private static final Logger LOG = LoggerFactory.getLogger(HdfsFileWriter.class);
 
+  private final FileSystem fileSystem;
+
   private Path path;
   private Configuration hadoopConf;
   private FSDataOutputStream fsDataOutputStream;
   private long nextOffset;
 
-  public HdfsFileWriter(Path path, Configuration hadoopConf) throws IOException, IllegalStateException {
-    // init fsDataOutputStream
+  public HdfsFileWriter(FileSystem fileSystem, Path path, Configuration hadoopConf) throws IOException {
     this.path = path;
     this.hadoopConf = hadoopConf;
+    this.fileSystem = fileSystem;
     initStream();
   }
 
   private void initStream() throws IOException, IllegalStateException {
-    FileSystem fileSystem = ShuffleStorageUtils.getFileSystemForPath(path, hadoopConf);
-    if (fileSystem.isFile(path)) {
+    final FileSystem writerFs = fileSystem;
+    if (writerFs.isFile(path)) {
       if (hadoopConf.getBoolean("dfs.support.append", true)) {
-        fsDataOutputStream = fileSystem.append(path);
+        fsDataOutputStream = writerFs.append(path);

Review Comment:
   ok



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1206014798

   Changelog
   1. Add the check of rss.security.hadoop.kerberos.relogin.interval.sec
   2. Add more test cases about kerberos HDFS cluster
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1207958642

   Gentle ping @jerqi 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1214760527

   @colinmjj Thanks for your comment. 
   
   The secured write/read only depends on the secured filesystem. And the secured filesystem can be initialized in HadoopFilesystemProvider. As I know, there is no need to wrap all the real action into the `doAs` method. Same operations from Oozie: https://github.com/apache/oozie/blob/f1e01a9e155692aa5632f4573ab1b3ebeab7ef45/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java#L638
   
   Filesystem will be created like this:
   ```java
   public class HadoopFilesystemProvider {
     private static final Logger LOGGER = LoggerFactory.getLogger([HadoopFilesystemProvider.class](https://github.com/apache/incubator-uniffle/pull/HadoopFilesystemProvider.class));
   
     public static FileSystem getFilesystem(String user, Path path, Configuration configuration) throws Exception {
       UserGroupInformation.AuthenticationMethod authenticationMethod =
           SecurityUtil.getAuthenticationMethod(configuration);
       boolean needSecurity = authenticationMethod != [UserGroupInformation.AuthenticationMethod.SIMPLE](https://github.com/apache/incubator-uniffle/pull/UserGroupInformation.AuthenticationMethod.SIMPLE);
   
       Callable<FileSystem> callable = () -> [FileSystem.get](https://github.com/apache/incubator-uniffle/pull/FileSystem.get)([path.toUri](https://github.com/apache/incubator-uniffle/pull/path.toUri)(), configuration);
   
       FileSystem fileSystem = null;
       if (needSecurity) {
        // this will return a secured filesystem, right?
         fileSystem = SecurityContextFactory
             .get()
             .getSecurityContext()
             .runSecured(user, callable);
       } else {
         fileSystem = [callable.call](https://github.com/apache/incubator-uniffle/pull/callable.call)();
       }
   
       if (fileSystem instanceof LocalFileSystem) {
         [LOGGER.debug](https://github.com/apache/incubator-uniffle/pull/LOGGER.debug)("{} is local file system", path);
         return ((LocalFileSystem) fileSystem).getRawFileSystem();
       }
   
       return fileSystem;
     }
   }
   ```


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1211840410

   Thanks for your review @colinmjj .
   I will update the patch as soon as possible with your comments and maybe split to several commits


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1213176883

   > 2. Add an api in Storage to check if it is secured, eg, isSecuredStorage()
   > 3. Create new writeHandler/readHandler to deal with secured write/read, eg, SecuredHdfsShuffleWriteHandler extends HdfsShuffleWriteHandler
   
   I think there is no need to introduce extra secured method and writer handler for security. 
   
   All the secured operation can be handled by filesystem. And the difference of secured and no-secured fs is the way of initialization which will be handled by HadoopFilesystemProvider https://github.com/apache/incubator-uniffle/commit/52c53d513c494990ad2d9e2ad3b56584e9473ea8 
   
   Please let me what you think @jerqi @colinmjj . Thanks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937289449


##########
integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerGrpcTest.java:
##########
@@ -626,4 +627,21 @@ private void addExpectedBlockIds(Roaring64NavigableMap bitmap, List<Long> blockI
       bitmap.addLong(blockIds.get(i));
     }
   }
+
+  @Test
+  public void testGettingUserWhenUsingHdfsStorage() throws Exception {

Review Comment:
   Could we add some Spark integration tests?



##########
server/src/test/java/org/apache/uniffle/server/storage/HdfsStorageManagerTest.java:
##########
@@ -61,13 +62,19 @@ public void testRegisterRemoteStorage() {
     final String remoteStoragePath3 = "hdfs://path3";
     hdfsStorageManager.registerRemoteStorage(
         "app1",
-        new RemoteStorageInfo(remoteStoragePath1, ImmutableMap.of("k1", "v1", "k2", "v2")));
+        new RemoteStorageInfo(remoteStoragePath1, ImmutableMap.of("k1", "v1", "k2", "v2")),

Review Comment:
   Could we add more test cases?



##########
server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java:
##########
@@ -158,6 +158,7 @@ private void flushToFile(ShuffleDataFlushEvent event) {
           writeSuccess = true;
           LOG.warn("AppId {} was removed already, event {} should be dropped", event.getAppId(), event);
         } else {
+          String user = storageManager.getStorageUser(event.getAppId());

Review Comment:
   Could the user be `null`?



##########
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:
   Could the client use this `fs` to write data to secured HDFS?



##########
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_SECURITY_HADOOP_KERBEROS_ENABLE = ConfigOptions
+      .key("rss.security.hadoop.kerberos.enable")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Whether enable visiting secured hadoop cluster.");
+
+  public static final ConfigOption<String> RSS_SECURITY_HADOOP_KERBEROS_KEYTAB_FILE = ConfigOptions
+      .key("rss.security.hadoop.kerberos.keytab.file")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("The kerberos keytab file path. And only when "
+          + RSS_SECURITY_HADOOP_KERBEROS_ENABLE.key() + " enabled, the option will be valid.");
+
+  public static final ConfigOption<String> RSS_SECURITY_HADOOP_KERBEROS_PRINCIPAL = ConfigOptions
+      .key("rss.security.hadoop.kerberos.principal")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("The kerberos keytab principal. And only when "
+          + RSS_SECURITY_HADOOP_KERBEROS_ENABLE.key() + " enabled, the option will be valid.");
+
+  public static final ConfigOption<Long> RSS_SECURITY_HADOOP_KERBEROS_RELOGIN_INTERVAL_SEC = ConfigOptions
+      .key("rss.security.hadoop.kerberos.relogin.interval.sec")
+      .longType()
+      .defaultValue(60L)

Review Comment:
   Do we need checkValue for this ConfigOption?



##########
server/src/main/java/org/apache/uniffle/server/storage/LocalStorageManager.java:
##########
@@ -170,6 +170,11 @@ public Checker getStorageChecker() {
     return checker;
   }
 
+  @Override
+  public String getStorageUser(String appId) {
+    return null;

Review Comment:
   Should we use the user who start the server?



##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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 = Maps.newConcurrentMap();
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {
+      synchronized (HadoopAccessorProvider.class) {
+        if (provider == null) {
+          final HadoopAccessorProvider hadoopAccessorProvider = new HadoopAccessorProvider(securityInfo);
+          provider = hadoopAccessorProvider;
+        }
+      }
+    }
+    LOGGER.info("The {} has been initialized, kerberos enable: {}",
+        HadoopAccessorProvider.class.getSimpleName(),
+        provider.kerberosEnabled);
+  }
+
+  /**
+   * No need to
+   * For non-kerberized cluster access, like client side reading shuffle data.
+   */
+  public static void init() throws Exception {
+    init(null);
+  }
+
+  @VisibleForTesting
+  public static void cleanup() throws Exception {
+    if (provider != null) {
+      provider.close();
+      provider = null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (cache != null) {
+      for (UserGroupInformation ugi : cache.values()) {
+        try {
+          FileSystem.closeAllForUGI(ugi);
+        } catch (IOException ioException) {
+          LOGGER.error("Exception occurred while closing filesystems for {}", ugi.getUserName(), ioException);
+        }
+      }
+      cache.clear();

Review Comment:
   If there are many users, cache may occupy too much memory.



##########
storage/src/main/java/org/apache/uniffle/storage/handler/impl/HdfsFileWriter.java:
##########
@@ -39,35 +39,37 @@ public class HdfsFileWriter implements Closeable {
 
   private static final Logger LOG = LoggerFactory.getLogger(HdfsFileWriter.class);
 
+  private final FileSystem fileSystem;
+
   private Path path;
   private Configuration hadoopConf;
   private FSDataOutputStream fsDataOutputStream;
   private long nextOffset;
 
-  public HdfsFileWriter(Path path, Configuration hadoopConf) throws IOException, IllegalStateException {
-    // init fsDataOutputStream
+  public HdfsFileWriter(FileSystem fileSystem, Path path, Configuration hadoopConf) throws IOException {

Review Comment:
   Could we add more tests for secured HDFS?



##########
coordinator/src/test/java/org/apache/uniffle/coordinator/AccessCandidatesCheckerTest.java:
##########
@@ -38,6 +41,11 @@
 
 public class AccessCandidatesCheckerTest {
 
+  @BeforeAll

Review Comment:
   Could we add some tests for Coordinator when it use Kerboros?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937298280


##########
server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java:
##########
@@ -158,6 +158,7 @@ private void flushToFile(ShuffleDataFlushEvent event) {
           writeSuccess = true;
           LOG.warn("AppId {} was removed already, event {} should be dropped", event.getAppId(), event);
         } else {
+          String user = storageManager.getStorageUser(event.getAppId());

Review Comment:
   User can be null. But if we apply this PR into production, user will not be null.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1196372423

   > If you have time, could u help review this proposal? @jerqi
   
   I have left some comments.  


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922787031


##########
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 {

Review Comment:
   I think we should still keep the conf in rssconf. The rss keytab maybe different from the node hadoop conf.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1186531339

   > ### What changes were proposed in this pull request?
   > Support storing shuffle data to secured dfs cluster
   > 
   > ### Why are the changes needed?
   > Now uniffle dont support visiting shuffle data to secured dfs cluster.
   > 
   > ### Does this PR introduce _any_ user-facing change?
   > Yes
   > 
   > ### How was this patch tested?
   > Manual tests.
   
   Should we provide more detailed description?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r936173653


##########
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")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Whether enable visiting secured hadoop cluster.");
+
+  public static final ConfigOption<String> RSS_ACCESS_HADOOP_KERBEROS_KEYTAB_FILE = ConfigOptions
+      .key("rss.access.hadoop.kerberos.keytab.file")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("The kerberos keytab file path. And only when "
+          + RSS_ACCESS_HADOOP_KERBEROS_ENABLE.key() + " enabled, the option will be valid.");
+
+  public static final ConfigOption<String> RSS_ACCESS_HADOOP_KERBEROS_PRINCIPAL = ConfigOptions
+      .key("rss.access.hadoop.kerberos.principal")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("The kerberos keytab principal. And only when "
+          + RSS_ACCESS_HADOOP_KERBEROS_ENABLE.key() + " enabled, the option will be valid.");
+
+  public static final ConfigOption<Long> RSS_ACCESS_HADOOP_KERBEROS_RELOGIN_INTERVAL = ConfigOptions

Review Comment:
   RSS_ACCESS_HADOOP_KERBEROS_RELOGIN_INTERVAL -> RSS_ACCESS_HADOOP_KERBEROS_RELOGIN_INTERVAL_SEC
   
   Could we add unit to its name to help people understand the code easily?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1207988812

   > Gentle ping @jerqi
   
   Should you ping @colinmjj @duanmeng ?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919947120


##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * 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 final long RELOGIN_CHECK_INTERVAL_SEC = 60L;
+
+    private static volatile HadoopAccessorProvider provider;
+
+    private Map<String, UserGroupInformation> cache = new ConcurrentHashMap<>();
+    private ScheduledExecutorService scheduledExecutorService;
+    private boolean kerberosEnabled = false;
+    private RssBaseConf rssBaseConf;
+
+    private HadoopAccessorProvider(RssBaseConf rssConf) throws Exception {

Review Comment:
   why Hadoop secured related conf should be set in RssBaseConf?
   for Coordinator to access HDFS?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919936074


##########
common/src/main/java/org/apache/uniffle/common/config/RssBaseConf.java:
##########
@@ -20,6 +20,8 @@
 import java.util.List;
 import java.util.Map;
 
+import com.sun.org.apache.xpath.internal.operations.Bool;

Review Comment:
   Forgot to delete it. All unused import will be deleted in next commit.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922615117


##########
.gitignore:
##########
@@ -21,4 +21,5 @@ metastore_db/
 derby.log
 dependency-reduced-pom.xml
 rss-*.tgz
+common/build/

Review Comment:
   Generated by test case of minikdc and dfs cluster



-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r924093812


##########
common/src/main/java/org/apache/uniffle/common/RemoteStorageInfo.java:
##########
@@ -34,8 +34,10 @@ public class RemoteStorageInfo implements Serializable {
   private final String path;
   private final Map<String, String> confItems;
 
+  private String user;

Review Comment:
   @zuston I'm ok with add the param of `user` in the method of `registerShuffle`



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1186224573

   Updated. @jerqi. 
   
   Besides, i think i will introduce some UTs later.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922819728


##########
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 {

Review Comment:
   But the coordinator also use this HadoopAccessorProvider. So maybe we should define the general hadoop prefix for coordinator and shuffle server. Right?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1220454337

   LGTM, please resolve the conflict


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r949975346


##########
storage/src/main/java/org/apache/uniffle/storage/handler/impl/HdfsFileWriter.java:
##########
@@ -39,35 +39,37 @@ public class HdfsFileWriter implements Closeable {
 
   private static final Logger LOG = LoggerFactory.getLogger(HdfsFileWriter.class);
 
+  private final FileSystem fileSystem;
+
   private Path path;
   private Configuration hadoopConf;
   private FSDataOutputStream fsDataOutputStream;
   private long nextOffset;
 
-  public HdfsFileWriter(Path path, Configuration hadoopConf) throws IOException, IllegalStateException {
-    // init fsDataOutputStream
+  public HdfsFileWriter(FileSystem fileSystem, Path path, Configuration hadoopConf) throws IOException {
     this.path = path;
     this.hadoopConf = hadoopConf;
+    this.fileSystem = fileSystem;
     initStream();
   }
 
   private void initStream() throws IOException, IllegalStateException {
-    FileSystem fileSystem = ShuffleStorageUtils.getFileSystemForPath(path, hadoopConf);
-    if (fileSystem.isFile(path)) {
+    final FileSystem writerFs = fileSystem;
+    if (writerFs.isFile(path)) {
       if (hadoopConf.getBoolean("dfs.support.append", true)) {
-        fsDataOutputStream = fileSystem.append(path);
+        fsDataOutputStream = writerFs.append(path);

Review Comment:
   With delegation mode(`proxy user` delegate `real user`), the security FileSystem will write data with real user?
   Do we need create a new UserGroupInformation, and wapper the action in `ugi.doAs()`?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1219042879

   Updated the latest commits. (These changes are hard to split meaningful parts, so I make the change shown into the commit log message) @colinmjj @jerqi 
   
   Changelog compared with last commit:
   1. Store the app->user mapping into the `ShuffleTaskManager`, invoked by `ShuffleFlushManager` by `shuffleServer.getShuffleTaskManager.getUserByAppId()`
   2. Add the new test cases for abstract `SecurityContext`
   3. Make all invoking areas use the `HadoopFilesystemProvider` to get filesystem


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r936194774


##########
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()
+            .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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {

Review Comment:
   LazyHolder only works with non-args constructors.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r936169893


##########
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:
   No need. The auth of HDFS should be managed by computing framework.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r938420061


##########
server/src/main/java/org/apache/uniffle/server/storage/LocalStorageManager.java:
##########
@@ -170,6 +170,11 @@ public Checker getStorageChecker() {
     return checker;
   }
 
+  @Override
+  public String getStorageUser(String appId) {
+    return null;

Review Comment:
   The user only will be used in kerberos HDFS. And when in kerberos HDFS and user is null or empty, it will directly check and throw exception while flushing shuffle data.
   
   So i think we need not to consider the NPE. Or return the empty of string? 



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919935452


##########
common/pom.xml:
##########
@@ -76,5 +76,35 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+    </dependency>
   </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.felix</groupId>

Review Comment:
   Due to the minikdc dependency. If having no it, it will compile failed



-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1182685318

   @zuston To access the secured DFS cluster:
   1. For spark client, it can depend on spark's implementation, and read data according to delegation token.
   2. For Shuffle server, Hadoop conf can be updated with security enable when write data to HDFS
   What's the advantage to add HadoopAccessorProvider?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937293659


##########
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:
   Yes.The spark/mr will manage the credentials of HDFS delegation token for all executors, so there is no need to relogin in client side.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937297636


##########
server/src/main/java/org/apache/uniffle/server/storage/LocalStorageManager.java:
##########
@@ -170,6 +170,11 @@ public Checker getStorageChecker() {
     return checker;
   }
 
+  @Override
+  public String getStorageUser(String appId) {
+    return null;

Review Comment:
   No need. If the user is null, fs will use the process startup user by default



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1183063524

   > Now our Uniffle already support multiple HDFS. Is it possible to use the secured hdfs cluster and unsecured hdfds cluster? And some configuration may be added to RemoteStorageConfItem.
   > 
   > https://github.com/apache/incubator-uniffle/blob/0f6a896efbd3a435de5e7a5d28843ecd05c38bde/proto/src/main/proto/Rss.proto#L360
   
   I think the logged in ugi can get the unsecured hdfs filesystem. But this case is not involved in current test case, maybe added it later.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r919852670


##########
.gitignore:
##########
@@ -21,4 +21,5 @@ metastore_db/
 derby.log
 dependency-reduced-pom.xml
 rss-*.tgz
+common/build/

Review Comment:
   why do we need this?



##########
common/src/main/java/org/apache/uniffle/common/config/RssBaseConf.java:
##########
@@ -20,6 +20,8 @@
 import java.util.List;
 import java.util.Map;
 
+import com.sun.org.apache.xpath.internal.operations.Bool;

Review Comment:
   Why do we need this?



##########
common/pom.xml:
##########
@@ -76,5 +76,35 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+    </dependency>
   </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.felix</groupId>

Review Comment:
   Why do we need this?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1200438077

   > > If you have time, could u help review this proposal? @jerqi
   > 
   > I have left some comments.
   
   updated the comments.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1182900553

   > @jerryshao You have caught my thoughts and thanks very much for explaining it in detail. Sorry I didn't explain clearly in advance.
   > 
   > So I think using the super user to delegate proxy user will be better in access control.
   
   Got you points, I think the feature is necessary to support proxy user when using secured HDFS


-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937310605


##########
server/src/main/java/org/apache/uniffle/server/storage/LocalStorageManager.java:
##########
@@ -170,6 +170,11 @@ public Checker getStorageChecker() {
     return checker;
   }
 
+  @Override
+  public String getStorageUser(String appId) {
+    return null;

Review Comment:
   I just want to reduce `return null`. Because if we use `return null`, we should check every place to avoid NullException.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937293659


##########
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:
   Yes.The spark/mr will manage the credentials of HDFS delegation token for all executors, so there is no need to relogin in shuffle manager client of uniffle.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r938561732


##########
server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java:
##########
@@ -158,6 +158,7 @@ private void flushToFile(ShuffleDataFlushEvent event) {
           writeSuccess = true;
           LOG.warn("AppId {} was removed already, event {} should be dropped", event.getAppId(), event);
         } else {
+          String user = storageManager.getStorageUser(event.getAppId());

Review Comment:
   OK. I will make it empty string when user is null. like
   ```
   String user = StringUtils.defaultString(storageManager.getStorageUser(event.getAppId()), StringUtils.EMPTY);
   ```



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r946296838


##########
server/src/main/java/org/apache/uniffle/server/storage/StorageManager.java:
##########
@@ -45,9 +45,11 @@ public interface StorageManager {
 
   void stop();
 
-  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo);
+  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo, String user);

Review Comment:
   Got it. Thanks for your advices. I missed this way.



##########
server/src/main/java/org/apache/uniffle/server/storage/StorageManager.java:
##########
@@ -45,9 +45,11 @@ public interface StorageManager {
 
   void stop();
 
-  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo);
+  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo, String user);

Review Comment:
   Got it. Thanks for your advices.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1186710502

   @zuston To access secured DFS, I think the process should be:
   1. get remote storage from coordinator
   2. register remote storage to shuffle server with **user info**
   3. shuffle server stores the mapping about  `appId -> user`
   4. when shuffle server write DFS with write handler, get user info and write data with proxy mode by `doAs()`
   what do you think?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r945566596


##########
server/src/main/java/org/apache/uniffle/server/storage/StorageManager.java:
##########
@@ -45,9 +45,11 @@ public interface StorageManager {
 
   void stop();
 
-  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo);
+  void registerRemoteStorage(String appId, RemoteStorageInfo remoteStorageInfo, String user);

Review Comment:
   If storing in shuffleTaskManager, there is no chance to get it when creating writer handler in ShuffleFlushManager.
   The writer handler is trigger by the shuffleDataFlushEvent created by ShuffleBufferManager. There are 2 ways to solve
   
   1. Store the appId -> user into ShuffleBufferManager
   2. Make the ShuffleTaskManager as a member of ShuffleBufferManager, and then the invoking like this `shuffleBufferManager.getShuffleTaskManager.getUserByAppId()` 
   
   What do u think @colinmjj 



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1212780688

   The changed first commit: https://github.com/apache/incubator-uniffle/pull/53/commits/52c53d513c494990ad2d9e2ad3b56584e9473ea8
   
   Changelog:
   1. Introduce the `SecurityContextFactory` to manage security.
   2. In `HadoopSecurityContext`, it will do kerberos login and refresh auth. The `hadoopSecurityContext` will only be initialized when kerberos security is enabled. Default securityContext is `NoOpSecurityContext`.
   3. `HadoopFileSystemProvider` will be a static class for invoking.
   
   This commit make the security and filesystem retrieval separate.
   
   Could u help review it? @jerqi @colinmjj  And I will submit commit one by one in next days.
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1212924821

   Thanks for your review @jerqi. Do u have any suggestions? @colinmjj . If OK, i think i will continue.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r949989631


##########
storage/src/main/java/org/apache/uniffle/storage/handler/impl/HdfsFileWriter.java:
##########
@@ -39,35 +39,37 @@ public class HdfsFileWriter implements Closeable {
 
   private static final Logger LOG = LoggerFactory.getLogger(HdfsFileWriter.class);
 
+  private final FileSystem fileSystem;
+
   private Path path;
   private Configuration hadoopConf;
   private FSDataOutputStream fsDataOutputStream;
   private long nextOffset;
 
-  public HdfsFileWriter(Path path, Configuration hadoopConf) throws IOException, IllegalStateException {
-    // init fsDataOutputStream
+  public HdfsFileWriter(FileSystem fileSystem, Path path, Configuration hadoopConf) throws IOException {
     this.path = path;
     this.hadoopConf = hadoopConf;
+    this.fileSystem = fileSystem;
     initStream();
   }
 
   private void initStream() throws IOException, IllegalStateException {
-    FileSystem fileSystem = ShuffleStorageUtils.getFileSystemForPath(path, hadoopConf);
-    if (fileSystem.isFile(path)) {
+    final FileSystem writerFs = fileSystem;
+    if (writerFs.isFile(path)) {
       if (hadoopConf.getBoolean("dfs.support.append", true)) {
-        fsDataOutputStream = fileSystem.append(path);
+        fsDataOutputStream = writerFs.append(path);

Review Comment:
   > With delegation mode(proxy user delegate real user), the security FileSystem will write data with real user?
   
   Yes. The data file's owner is real user.
   
   > Do we need create a new UserGroupInformation, and wapper the action in ugi.doAs()?
   
   No need to wrap operation in `ugi.doAs().` Once secured filesystem is created, all secured data operation are all handled by it. I have tested it.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922695368


##########
client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java:
##########
@@ -136,6 +139,12 @@ private synchronized void putBlockId(
   };
 
   public RssShuffleManager(SparkConf conf, boolean isDriver) {
+    try {
+      HadoopAccessorProvider.init(new RssBaseConf());

Review Comment:
   RssBaseConf is used for Shuffle Server and Coordinator. We shouldn't use it  in the client.



##########
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(

Review Comment:
   We can reuse the method `getThreadFactory`
   https://github.com/apache/incubator-uniffle/blob/e48f74ee657eca1c6dc05d558d88525c01733c3a/common/src/main/java/org/apache/uniffle/common/util/ThreadUtils.java#L29



##########
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 {

Review Comment:
   ShuffleServerConf have a method `getHadoopConf()`, maybe we can use the class `Configuration`.



-- 
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


[GitHub] [incubator-uniffle] codecov-commenter commented on pull request #53: Support storing shuffle data to secured dfs cluster

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1186103868

   # [Codecov](https://codecov.io/gh/apache/incubator-uniffle/pull/53?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#53](https://codecov.io/gh/apache/incubator-uniffle/pull/53?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5f85ff9) into [master](https://codecov.io/gh/apache/incubator-uniffle/commit/f4ce2edd2d4e3355a6f1e12f0e7f712964fe83b4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (f4ce2ed) will **increase** coverage by `0.76%`.
   > The diff coverage is `71.34%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master      #53      +/-   ##
   ============================================
   + Coverage     55.21%   55.97%   +0.76%     
   + Complexity     1111     1013      -98     
   ============================================
     Files           148      136      -12     
     Lines          7953     6869    -1084     
     Branches        760      657     -103     
   ============================================
   - Hits           4391     3845     -546     
   + Misses         3321     2810     -511     
   + Partials        241      214      -27     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-uniffle/pull/53?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...he/uniffle/client/impl/ShuffleWriteClientImpl.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Y2xpZW50L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS91bmlmZmxlL2NsaWVudC9pbXBsL1NodWZmbGVXcml0ZUNsaWVudEltcGwuamF2YQ==) | `25.38% <0.00%> (-0.60%)` | :arrow_down: |
   | [...che/uniffle/coordinator/ClusterManagerFactory.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Y29vcmRpbmF0b3Ivc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3VuaWZmbGUvY29vcmRpbmF0b3IvQ2x1c3Rlck1hbmFnZXJGYWN0b3J5LmphdmE=) | `100.00% <ø> (ø)` | |
   | [...g/apache/uniffle/coordinator/CoordinatorUtils.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Y29vcmRpbmF0b3Ivc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3VuaWZmbGUvY29vcmRpbmF0b3IvQ29vcmRpbmF0b3JVdGlscy5qYXZh) | `65.21% <ø> (+4.10%)` | :arrow_up: |
   | [...pache/uniffle/server/ShuffleServerGrpcService.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2VydmVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS91bmlmZmxlL3NlcnZlci9TaHVmZmxlU2VydmVyR3JwY1NlcnZpY2UuamF2YQ==) | `1.00% <0.00%> (-0.01%)` | :arrow_down: |
   | [...he/uniffle/server/buffer/ShuffleBufferManager.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2VydmVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS91bmlmZmxlL3NlcnZlci9idWZmZXIvU2h1ZmZsZUJ1ZmZlck1hbmFnZXIuamF2YQ==) | `81.81% <ø> (ø)` | |
   | [...he/uniffle/server/storage/MultiStorageManager.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2VydmVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS91bmlmZmxlL3NlcnZlci9zdG9yYWdlL011bHRpU3RvcmFnZU1hbmFnZXIuamF2YQ==) | `36.58% <0.00%> (-0.92%)` | :arrow_down: |
   | [...org/apache/uniffle/storage/common/HdfsStorage.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3RvcmFnZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvdW5pZmZsZS9zdG9yYWdlL2NvbW1vbi9IZGZzU3RvcmFnZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...storage/handler/impl/HdfsShuffleDeleteHandler.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3RvcmFnZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvdW5pZmZsZS9zdG9yYWdlL2hhbmRsZXIvaW1wbC9IZGZzU2h1ZmZsZURlbGV0ZUhhbmRsZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...e/storage/handler/impl/HdfsShuffleReadHandler.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3RvcmFnZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvdW5pZmZsZS9zdG9yYWdlL2hhbmRsZXIvaW1wbC9IZGZzU2h1ZmZsZVJlYWRIYW5kbGVyLmphdmE=) | `51.02% <ø> (ø)` | |
   | [...rage/request/CreateShuffleWriteHandlerRequest.java](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3RvcmFnZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvdW5pZmZsZS9zdG9yYWdlL3JlcXVlc3QvQ3JlYXRlU2h1ZmZsZVdyaXRlSGFuZGxlclJlcXVlc3QuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | ... and [35 more](https://codecov.io/gh/apache/incubator-uniffle/pull/53/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-uniffle/pull/53?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-uniffle/pull/53?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [f4ce2ed...5f85ff9](https://codecov.io/gh/apache/incubator-uniffle/pull/53?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922762723


##########
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 {

Review Comment:
   OK. I will follow it.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922819728


##########
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 {

Review Comment:
   But the coordinator also use this HadoopAccessorProvider. So maybe we should define the general hadoop prefix for coordinator and shuffle server. Right?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r922818978


##########
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 {

Review Comment:
   Sounds great!



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1182878228

   @jerryshao 
   You have caught my thoughts and thanks very much for explaining it in detail. 
   Sorry I didn't explain clearly in advance.
   
   So I think using the super user to delegate proxy user will be better in access control.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1187377347

   > @zuston To access secured DFS, I think the process should be:
   > 
   > 1. get remote storage from coordinator
   > 2. register remote storage to shuffle server with **user info**
   > 3. shuffle server stores the mapping about  `appId -> user`
   > 4. when shuffle server write DFS with write handler, get user info and write data with proxy mode by `doAs()`
   >    what do you think?
   
   Got your points.
   
   As I know the HdfsStorage is the shareable in the entire JVM. So now the user only will be added in `CreateShuffleWriteHandlerRequest`, which is bound to the app's w/r operations.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
duanmeng commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1188528009

   >It could use the proxy user to make the data written to unsecured dfs cluster, to make data isolation? Please let me know whether i am right.
   
   Yes, you're right, say user B can not access the path created by user A, they are both proxy user of the user running the uniflle cluster. 
   
   >Do i need to introduce a new page to submit proposal like RFC? Or just write it in description? Or open a new issue to track and discuss.
   
   I suggest you enrich your description and add your design doc's (google doc or tencent doc) link to it.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r938600429


##########
server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java:
##########
@@ -158,6 +158,7 @@ private void flushToFile(ShuffleDataFlushEvent event) {
           writeSuccess = true;
           LOG.warn("AppId {} was removed already, event {} should be dropped", event.getAppId(), event);
         } else {
+          String user = storageManager.getStorageUser(event.getAppId());

Review Comment:
   Updated.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r938600690


##########
server/src/main/java/org/apache/uniffle/server/storage/LocalStorageManager.java:
##########
@@ -170,6 +170,11 @@ public Checker getStorageChecker() {
     return checker;
   }
 
+  @Override
+  public String getStorageUser(String appId) {
+    return null;

Review Comment:
   Done.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r936172663


##########
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()
+            .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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {

Review Comment:
   If this is singleton, could we use lazy holder to keep consistent code style with other code?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937294464


##########
common/src/main/java/org/apache/uniffle/common/provider/HadoopAccessorProvider.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+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;
+
+import org.apache.uniffle.common.util.ThreadUtils;
+
+/**
+ * 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 = Maps.newConcurrentMap();
+
+    LOGGER.info("Got Kerberos ticket, keytab [{}], principal [{}], user [{}]",
+        keytabFile, principal, UserGroupInformation.getLoginUser());
+
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+        ThreadUtils.getThreadFactory("Kerberos-relogin-%d")
+    );
+    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(
+        user,
+        UserGroupInformation.createProxyUser(
+            user, UserGroupInformation.getLoginUser()
+        )
+    );
+    return provider.cache.get(user);
+  }
+
+  private static FileSystem getInternalFileSystem(
+      final String user,
+      final boolean retrievedByProxyUser,
+      final Path path,
+      final Configuration configuration) throws Exception {
+    if (provider == null) {
+      throw new Exception("HadoopAccessorProvider should be initialized.");
+    }
+
+    if (retrievedByProxyUser && StringUtils.isEmpty(user)) {
+      throw new Exception("User must be set when security is enabled");
+    }
+    if (retrievedByProxyUser && !provider.kerberosEnabled) {
+      String msg = String.format("There is need to be interactive with secured DFS by user: %s, path: %s "
+          + "but the HadoopAccessProvider's kerberos config is disabled and can't retrieve the "
+          + "secured filesystem", user, path);
+      throw new Exception(msg);
+    }
+
+    // For local file systems, return the raw local file system, such calls to flush()
+    // actually flushes the stream.
+    try {
+      FileSystem fs;
+      if (retrievedByProxyUser) {
+        LOGGER.info("Fetching the proxy user ugi of {} when getting filesystem of [{}]", user, path);
+        UserGroupInformation proxyUserUGI = provider.getProxyUser(user);
+        fs = proxyUserUGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          @Override
+          public FileSystem run() throws Exception {
+            return path.getFileSystem(configuration);
+          }
+        });
+      } else {
+        fs = path.getFileSystem(configuration);
+      }
+      if (fs instanceof LocalFileSystem) {
+        LOGGER.debug("{} is local file system", path);
+        return ((LocalFileSystem) fs).getRawFileSystem();
+      }
+      return fs;
+    } catch (IOException e) {
+      LOGGER.error("Fail to get filesystem of {}", path);
+      throw e;
+    }
+  }
+
+  /**
+   * The only entrypoint is to get the hadoop filesystem instance and is compatible with
+   * the kerberos security.
+   *
+   * When to invoke this method?
+   * 1. For shuffle server side, it needs to get filesystem before writing the shuffle data to secured HDFS
+   * with the spark job's user auth.
+   *
+   * @param user
+   * @param path
+   * @param conf
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFilesystem(
+      final String user,
+      final Path path,
+      final Configuration conf) throws Exception {
+    UserGroupInformation.AuthenticationMethod authenticationMethod =
+        SecurityUtil.getAuthenticationMethod(conf);
+    boolean securityEnable = authenticationMethod != UserGroupInformation.AuthenticationMethod.SIMPLE;
+    return getInternalFileSystem(user, securityEnable, path, conf);
+  }
+
+  /**
+   * The method is to return the Hadoop Filesystem directly which is not retrieved by
+   * ugi proxy user.
+   *
+   * When to invoke this method?
+   * 1. In client side, spark shuffle-reader getting filesystem before reading shuffle data stored in HDFS.
+   * 2. In shuffle-server/coordinator side, it reads the config file stored in HDFS.
+   *
+   * @param path
+   * @param configuration
+   * @return
+   * @throws Exception
+   */
+  public static FileSystem getFileSystem(
+      final Path path,
+      final Configuration configuration) throws Exception {
+    return getInternalFileSystem(null, false, path, configuration);
+  }
+
+  /**
+   * For kerberized cluster access
+   * @param securityInfo
+   * @throws Exception
+   */
+  public static void init(SecurityInfo securityInfo) throws Exception {
+    if (provider == null) {
+      synchronized (HadoopAccessorProvider.class) {
+        if (provider == null) {
+          final HadoopAccessorProvider hadoopAccessorProvider = new HadoopAccessorProvider(securityInfo);
+          provider = hadoopAccessorProvider;
+        }
+      }
+    }
+    LOGGER.info("The {} has been initialized, kerberos enable: {}",
+        HadoopAccessorProvider.class.getSimpleName(),
+        provider.kerberosEnabled);
+  }
+
+  /**
+   * No need to
+   * For non-kerberized cluster access, like client side reading shuffle data.
+   */
+  public static void init() throws Exception {
+    init(null);
+  }
+
+  @VisibleForTesting
+  public static void cleanup() throws Exception {
+    if (provider != null) {
+      provider.close();
+      provider = null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (cache != null) {
+      for (UserGroupInformation ugi : cache.values()) {
+        try {
+          FileSystem.closeAllForUGI(ugi);
+        } catch (IOException ioException) {
+          LOGGER.error("Exception occurred while closing filesystems for {}", ugi.getUserName(), ioException);
+        }
+      }
+      cache.clear();

Review Comment:
   No cache is also OK. But i think the number of users may be not too much.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937294010


##########
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_SECURITY_HADOOP_KERBEROS_ENABLE = ConfigOptions
+      .key("rss.security.hadoop.kerberos.enable")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Whether enable visiting secured hadoop cluster.");
+
+  public static final ConfigOption<String> RSS_SECURITY_HADOOP_KERBEROS_KEYTAB_FILE = ConfigOptions
+      .key("rss.security.hadoop.kerberos.keytab.file")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("The kerberos keytab file path. And only when "
+          + RSS_SECURITY_HADOOP_KERBEROS_ENABLE.key() + " enabled, the option will be valid.");
+
+  public static final ConfigOption<String> RSS_SECURITY_HADOOP_KERBEROS_PRINCIPAL = ConfigOptions
+      .key("rss.security.hadoop.kerberos.principal")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("The kerberos keytab principal. And only when "
+          + RSS_SECURITY_HADOOP_KERBEROS_ENABLE.key() + " enabled, the option will be valid.");
+
+  public static final ConfigOption<Long> RSS_SECURITY_HADOOP_KERBEROS_RELOGIN_INTERVAL_SEC = ConfigOptions
+      .key("rss.security.hadoop.kerberos.relogin.interval.sec")
+      .longType()
+      .defaultValue(60L)

Review Comment:
   Yes. it should > 0



-- 
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


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

Posted by GitBox <gi...@apache.org>.
jerqi commented on code in PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#discussion_r937309489


##########
server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java:
##########
@@ -158,6 +158,7 @@ private void flushToFile(ShuffleDataFlushEvent event) {
           writeSuccess = true;
           LOG.warn("AppId {} was removed already, event {} should be dropped", event.getAppId(), event);
         } else {
+          String user = storageManager.getStorageUser(event.getAppId());

Review Comment:
   It will be null when shuffle was deleted.  We ever fix similar issue. You can see https://github.com/apache/incubator-uniffle/commit/55ec437aed9a465f0ad014534e17b82f501b0464 for more details.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zuston commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1208958038

   @colinmjj  @duanmeng  If u have time, could you help review this?
   
    I have spent some time on this current implementation and looking forward to your reply. Thanks.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
colinmjj commented on PR #53:
URL: https://github.com/apache/incubator-uniffle/pull/53#issuecomment-1201071612

   @zuston Do you commit the latest source? Some previous comments seems not be fixed, eg, user shouldn't be a member in RemoteStorage
   ```
   message RemoteStorage {
     string path = 1;
     repeated RemoteStorageConfItem remoteStorageConf = 2;
     string user = 3;
   }
   ```


-- 
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