This is an automated email from the ASF dual-hosted git repository.
jshao pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/gravitino.git
The following commit(s) were added to refs/heads/main by this push:
new e0bce18f5 [#5596] Improvement(filesystem-hadoop3): Override the
`addDelegationTokens` method to fix the token expired issue (#5597)
e0bce18f5 is described below
commit e0bce18f5d3187f91c8979c592d36c58b5b8bb89
Author: xloya <[email protected]>
AuthorDate: Mon Nov 18 14:58:39 2024 +0800
[#5596] Improvement(filesystem-hadoop3): Override the `addDelegationTokens`
method to fix the token expired issue (#5597)
### What changes were proposed in this pull request?
Pass the delegation token to the FileSystem managed by GVFS.
### Why are the changes needed?
Fix: #5596
### How was this patch tested?
Tests it in the production environment.
---
.../filesystem/hadoop/GravitinoVirtualFileSystem.java | 17 +++++++++++++++++
1 file changed, 17 insertions(+)
diff --git
a/clients/filesystem-hadoop3/src/main/java/org/apache/gravitino/filesystem/hadoop/GravitinoVirtualFileSystem.java
b/clients/filesystem-hadoop3/src/main/java/org/apache/gravitino/filesystem/hadoop/GravitinoVirtualFileSystem.java
index aaa81ab55..a1e7bb5d5 100644
---
a/clients/filesystem-hadoop3/src/main/java/org/apache/gravitino/filesystem/hadoop/GravitinoVirtualFileSystem.java
+++
b/clients/filesystem-hadoop3/src/main/java/org/apache/gravitino/filesystem/hadoop/GravitinoVirtualFileSystem.java
@@ -25,12 +25,14 @@ import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.Scheduler;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.util.Arrays;
+import java.util.List;
import java.util.Map;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadFactory;
@@ -57,6 +59,8 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -559,6 +563,19 @@ public class GravitinoVirtualFileSystem extends FileSystem
{
return
context.getFileSystem().getDefaultBlockSize(context.getActualFileLocation());
}
+ @Override
+ public Token<?>[] addDelegationTokens(String renewer, Credentials
credentials) {
+ List<Token<?>> tokenList = Lists.newArrayList();
+ for (FileSystem fileSystem : internalFileSystemCache.asMap().values()) {
+ try {
+ tokenList.addAll(Arrays.asList(fileSystem.addDelegationTokens(renewer,
credentials)));
+ } catch (IOException e) {
+ Logger.warn("Failed to add delegation tokens for filesystem: {}",
fileSystem.getUri(), e);
+ }
+ }
+ return tokenList.stream().distinct().toArray(Token[]::new);
+ }
+
@Override
public synchronized void close() throws IOException {
// close all actual FileSystems