HDFS-15585: ViewDFS#getDelegationToken should not throw UnsupportedOperationException. (#2312). Contributed by Uma Maheswara Rao G.

This commit is contained in:
Uma Maheswara Rao G 2020-09-18 02:48:10 -07:00 committed by GitHub
parent eacbe07b56
commit 7bba4c609c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 34 additions and 3 deletions

View File

@ -1032,15 +1032,24 @@ protected int getDefaultPort() {
return super.getDefaultPort();
}
/**
* If no mount points configured, it works same as
* {@link DistributedFileSystem#getDelegationToken(String)}. If
* there are mount points configured and if default fs(linkFallback)
* configured, then it will return default fs delegation token. Otherwise
* it will return null.
*/
@Override
public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
throws IOException {
if (this.vfs == null) {
return super.getDelegationToken(renewer);
}
//Let applications call getDelegationTokenIssuers and get respective
// delegation tokens from child fs.
throw new UnsupportedOperationException();
if (defaultDFS != null) {
return defaultDFS.getDelegationToken(renewer);
}
return null;
}
@Override

View File

@ -18,14 +18,17 @@
package org.apache.hadoop.hdfs;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathHandle;
import org.apache.hadoop.fs.viewfs.ConfigUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.test.Whitebox;
import org.junit.Test;
import java.io.IOException;
import java.net.URI;
public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
@Override
@ -67,4 +70,23 @@ public void testOpenWithPathHandle() throws Exception {
}
}
}
@Override
public void testEmptyDelegationToken() throws IOException {
Configuration conf = getTestConfiguration();
MiniDFSCluster cluster = null;
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
URI defaultUri =
URI.create(conf.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY));
ConfigUtil.addLinkFallback(conf, defaultUri.getHost(), defaultUri);
try (FileSystem fileSys = FileSystem.get(conf)) {
fileSys.getDelegationToken("");
}
} finally {
if (cluster != null) {
cluster.shutdown();
}
}
}
}