|
@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
@@ -45,6 +46,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
import org.apache.hadoop.ipc.RetriableException;
|
|
|
+import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.log4j.Level;
|
|
@@ -482,4 +484,43 @@ public class TestWebHDFS {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDTInInsecureClusterWithFallback()
|
|
|
+ throws IOException, URISyntaxException {
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ final Configuration conf = WebHdfsTestUtil.createConf();
|
|
|
+ conf.setBoolean(CommonConfigurationKeys
|
|
|
+ .IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY, true);
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
|
|
|
+ final FileSystem webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
|
|
|
+ WebHdfsFileSystem.SCHEME);
|
|
|
+ Assert.assertNull(webHdfs.getDelegationToken(null));
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDTInInsecureCluster() throws Exception {
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ final Configuration conf = WebHdfsTestUtil.createConf();
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
|
|
|
+ final FileSystem webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
|
|
|
+ WebHdfsFileSystem.SCHEME);
|
|
|
+ webHdfs.getDelegationToken(null);
|
|
|
+ fail("No exception is thrown.");
|
|
|
+ } catch (AccessControlException ace) {
|
|
|
+ Assert.assertTrue(ace.getMessage().startsWith(
|
|
|
+ WebHdfsFileSystem.CANT_FALLBACK_TO_INSECURE_MSG));
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|