|
@@ -26,6 +26,7 @@ import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.EnumSet;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.IdentityHashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
@@ -51,6 +52,7 @@ import org.apache.hadoop.ipc.UnexpectedServerException;
|
|
|
import org.apache.hadoop.fs.InvalidPathException;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.security.token.Token;
|
|
|
|
|
|
/**
|
|
|
* The FileContext class provides an interface to the application writer for
|
|
@@ -2196,6 +2198,30 @@ public final class FileContext {
|
|
|
}
|
|
|
}.resolve(this, f).getPath();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns the list of AbstractFileSystems accessed in the path. The list may
|
|
|
+ * contain more than one AbstractFileSystems objects in case of symlinks.
|
|
|
+ *
|
|
|
+ * @param f
|
|
|
+ * Path which needs to be resolved
|
|
|
+ * @return List of AbstractFileSystems accessed in the path
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ Set<AbstractFileSystem> resolveAbstractFileSystems(final Path f)
|
|
|
+ throws IOException {
|
|
|
+ final Path absF = fixRelativePart(f);
|
|
|
+ final HashSet<AbstractFileSystem> result = new HashSet<AbstractFileSystem>();
|
|
|
+ new FSLinkResolver<Void>() {
|
|
|
+ public Void next(final AbstractFileSystem fs, final Path p)
|
|
|
+ throws IOException, UnresolvedLinkException {
|
|
|
+ result.add(fs);
|
|
|
+ fs.getFileStatus(p);
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }.resolve(this, absF);
|
|
|
+ return result;
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Class used to perform an operation on and resolve symlinks in a
|
|
@@ -2289,4 +2315,25 @@ public final class FileContext {
|
|
|
public static Map<URI, Statistics> getAllStatistics() {
|
|
|
return AbstractFileSystem.getAllStatistics();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get delegation tokens for the file systems accessed for a given
|
|
|
+ * path.
|
|
|
+ * @param p Path for which delegations tokens are requested.
|
|
|
+ * @param renewer the account name that is allowed to renew the token.
|
|
|
+ * @return List of delegation tokens.
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ @InterfaceAudience.LimitedPrivate( { "HDFS", "MapReduce" })
|
|
|
+ public List<Token<?>> getDelegationTokens(
|
|
|
+ Path p, String renewer) throws IOException {
|
|
|
+ Set<AbstractFileSystem> afsSet = resolveAbstractFileSystems(p);
|
|
|
+ List<Token<?>> tokenList =
|
|
|
+ new ArrayList<Token<?>>();
|
|
|
+ for (AbstractFileSystem afs : afsSet) {
|
|
|
+ List<Token<?>> afsTokens = afs.getDelegationTokens(renewer);
|
|
|
+ tokenList.addAll(afsTokens);
|
|
|
+ }
|
|
|
+ return tokenList;
|
|
|
+ }
|
|
|
}
|