HADOOP-18169. getDelegationTokens in ViewFs should also fetch the token from fallback FS (#4094)
Signed-off-by: Owen O'Malley <oomalley@linkedin.com>
This commit is contained in:
parent
ecafd38c09
commit
20483f6dc7
|
@ -746,6 +746,17 @@ public class ViewFs extends AbstractFileSystem {
|
|||
result.addAll(tokens);
|
||||
}
|
||||
}
|
||||
|
||||
// Add tokens from fallback FS
|
||||
if (this.fsState.getRootFallbackLink() != null) {
|
||||
AbstractFileSystem rootFallbackFs =
|
||||
this.fsState.getRootFallbackLink().getTargetFileSystem();
|
||||
List<Token<?>> tokens = rootFallbackFs.getDelegationTokens(renewer);
|
||||
if (tokens != null) {
|
||||
result.addAll(tokens);
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.net.URI;
|
|||
import java.net.URISyntaxException;
|
||||
import java.util.EnumSet;
|
||||
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.AbstractFileSystem;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
|
@ -45,6 +46,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
|
@ -182,6 +184,26 @@ public class TestViewFsLinkFallback {
|
|||
assertTrue(fsTarget.exists(test));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test getDelegationToken when fallback is configured.
|
||||
*/
|
||||
@Test
|
||||
public void testGetDelegationToken() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
conf.setBoolean(Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS, false);
|
||||
ConfigUtil.addLink(conf, "/user",
|
||||
new Path(targetTestRoot.toString(), "user").toUri());
|
||||
ConfigUtil.addLink(conf, "/data",
|
||||
new Path(targetTestRoot.toString(), "data").toUri());
|
||||
ConfigUtil.addLinkFallback(conf, targetTestRoot.toUri());
|
||||
|
||||
FileContext fcView =
|
||||
FileContext.getFileContext(FsConstants.VIEWFS_URI, conf);
|
||||
List<Token<?>> tokens = fcView.getDelegationTokens(new Path("/"), "tester");
|
||||
// Two tokens from the two mount points and one token from fallback
|
||||
assertEquals(3, tokens.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that when the parent dirs does not exist in fallback but the parent
|
||||
* dir is same as mount internal directory, then we create parent structure
|
||||
|
|
Loading…
Reference in New Issue