浏览代码

HADOOP-9004. Reverting the commit r1406202 to address patch issue

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1406379 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 年之前
父节点
当前提交
251230a126

+ 0 - 3
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -129,9 +129,6 @@ Trunk (Unreleased)
     HADOOP-8776. Provide an option in test-patch that can enable/disable
     compiling native code. (Chris Nauroth via suresh)
 
-    HADOOP-9004. Allow security unit tests to use external KDC. (Stephen Chu
-    via suresh)
-
   BUG FIXES
 
     HADOOP-8177. MBeans shouldn't try to register when it fails to create MBeanName.

+ 0 - 15
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java

@@ -27,19 +27,4 @@ public class SecurityUtilTestHelper {
   public static void setTokenServiceUseIp(boolean flag) {
     SecurityUtil.setTokenServiceUseIp(flag);
   }
-
-  /**
-   * Return true if externalKdc=true and the location of the krb5.conf
-   * file has been specified, and false otherwise.
-   */
-  public static boolean isExternalKdcRunning() {
-    String externalKdc = System.getProperty("externalKdc");
-    String krb5Conf = System.getProperty("java.security.krb5.conf");
-    if(externalKdc == null || !externalKdc.equals("true") ||
-       krb5Conf == null) {
-      return false;
-    }
-    return true;
-  }
-
 }

+ 0 - 74
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java

@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.security;
-
-import java.io.IOException;
-
-import junit.framework.Assert;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import static org.apache.hadoop.security.SecurityUtilTestHelper.isExternalKdcRunning;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Tests kerberos keytab login using a user-specified external KDC
- *
- * To run, users must specify the following system properties:
- *   externalKdc=true
- *   java.security.krb5.conf
- *   user.principal
- *   user.keytab
- */
-public class TestUGIWithExternalKdc {
-
-  @Before
-  public void testExternalKdcRunning() {
-    Assume.assumeTrue(isExternalKdcRunning());
-  }
-
-  @Test
-  public void testLogin() throws IOException {
-    String userPrincipal = System.getProperty("user.principal");
-    String userKeyTab = System.getProperty("user.keytab");
-    Assert.assertNotNull("User principal was not specified", userPrincipal);
-    Assert.assertNotNull("User keytab was not specified", userKeyTab);
-
-    Configuration conf = new Configuration();
-    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
-        "kerberos");
-    UserGroupInformation.setConfiguration(conf);
-
-    UserGroupInformation ugi = UserGroupInformation
-        .loginUserFromKeytabAndReturnUGI(userPrincipal, userKeyTab);
-
-    Assert.assertEquals(AuthenticationMethod.KERBEROS,
-        ugi.getAuthenticationMethod());
-    
-    try {
-      UserGroupInformation
-      .loginUserFromKeytabAndReturnUGI("bogus@EXAMPLE.COM", userKeyTab);
-      Assert.fail("Login should have failed");
-    } catch (Exception ex) {
-      ex.printStackTrace();
-    }
-  }
-
-}

+ 14 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -38,8 +38,6 @@ import org.mortbay.jetty.security.SslSocketConnector;
 
 import javax.net.ssl.SSLServerSocketFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * Utility class to start a datanode in a secure cluster, first obtaining 
  * privileged resources before main startup and handing them to the datanode.
@@ -75,25 +73,6 @@ public class SecureDataNodeStarter implements Daemon {
     // Stash command-line arguments for regular datanode
     args = context.getArguments();
     
-    sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
-    resources = getSecureResources(sslFactory, conf);
-  }
-
-  @Override
-  public void start() throws Exception {
-    System.err.println("Starting regular datanode initialization");
-    DataNode.secureMain(args, resources);
-  }
-  
-  @Override public void destroy() {
-    sslFactory.destroy();
-  }
-
-  @Override public void stop() throws Exception { /* Nothing to do */ }
-
-  @VisibleForTesting
-  public static SecureResources getSecureResources(final SSLFactory sslFactory,
-                                  Configuration conf) throws Exception {
     // Obtain secure port for data streaming to datanode
     InetSocketAddress streamingAddr  = DataNode.getStreamingAddr(conf);
     int socketWriteTimeout = conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
@@ -106,12 +85,13 @@ public class SecureDataNodeStarter implements Daemon {
     // Check that we got the port we need
     if (ss.getLocalPort() != streamingAddr.getPort()) {
       throw new RuntimeException("Unable to bind on specified streaming port in secure " +
-          "context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
+      		"context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
     }
 
     // Obtain secure listener for web server
     Connector listener;
     if (HttpConfig.isSecure()) {
+      sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
       try {
         sslFactory.init();
       } catch (GeneralSecurityException ex) {
@@ -146,7 +126,18 @@ public class SecureDataNodeStarter implements Daemon {
     }
     System.err.println("Opened streaming server at " + streamingAddr);
     System.err.println("Opened info server at " + infoSocAddr);
-    return new SecureResources(ss, listener);
+    resources = new SecureResources(ss, listener);
   }
 
+  @Override
+  public void start() throws Exception {
+    System.err.println("Starting regular datanode initialization");
+    DataNode.secureMain(args, resources);
+  }
+  
+  @Override public void destroy() {
+    sslFactory.destroy();
+  }
+
+  @Override public void stop() throws Exception { /* Nothing to do */ }
 }

+ 7 - 37
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -81,8 +81,6 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
-import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter;
-import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -97,7 +95,6 @@ import org.apache.hadoop.net.StaticMapping;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
@@ -148,7 +145,6 @@ public class MiniDFSCluster {
     private boolean setupHostsFile = false;
     private MiniDFSNNTopology nnTopology = null;
     private boolean checkExitOnShutdown = true;
-    private boolean checkDataNodeAddrConfig = false;
     private boolean checkDataNodeHostConfig = false;
     
     public Builder(Configuration conf) {
@@ -267,14 +263,6 @@ public class MiniDFSCluster {
       return this;
     }
 
-    /**
-     * Default: false
-     */
-    public Builder checkDataNodeAddrConfig(boolean val) {
-      this.checkDataNodeAddrConfig = val;
-      return this;
-    }
-
     /**
      * Default: false
      */
@@ -348,7 +336,6 @@ public class MiniDFSCluster {
                        builder.setupHostsFile,
                        builder.nnTopology,
                        builder.checkExitOnShutdown,
-                       builder.checkDataNodeAddrConfig,
                        builder.checkDataNodeHostConfig);
   }
   
@@ -356,14 +343,11 @@ public class MiniDFSCluster {
     DataNode datanode;
     Configuration conf;
     String[] dnArgs;
-    SecureResources secureResources;
 
-    DataNodeProperties(DataNode node, Configuration conf, String[] args,
-                       SecureResources secureResources) {
+    DataNodeProperties(DataNode node, Configuration conf, String[] args) {
       this.datanode = node;
       this.conf = conf;
       this.dnArgs = args;
-      this.secureResources = secureResources;
     }
   }
 
@@ -589,7 +573,7 @@ public class MiniDFSCluster {
         manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs,
         operation, racks, hosts,
         simulatedCapacities, null, true, false,
-        MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false, false);
+        MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false);
   }
 
   private void initMiniDFSCluster(
@@ -600,7 +584,6 @@ public class MiniDFSCluster {
       String[] hosts, long[] simulatedCapacities, String clusterId,
       boolean waitSafeMode, boolean setupHostsFile,
       MiniDFSNNTopology nnTopology, boolean checkExitOnShutdown,
-      boolean checkDataNodeAddrConfig,
       boolean checkDataNodeHostConfig)
   throws IOException {
     ExitUtil.disableSystemExit();
@@ -664,7 +647,7 @@ public class MiniDFSCluster {
 
     // Start the DataNodes
     startDataNodes(conf, numDataNodes, manageDataDfsDirs, operation, racks,
-        hosts, simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, checkDataNodeHostConfig);
+        hosts, simulatedCapacities, setupHostsFile, false, checkDataNodeHostConfig);
     waitClusterUp();
     //make sure ProxyUsers uses the latest conf
     ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
@@ -1178,18 +1161,7 @@ public class MiniDFSCluster {
       if (hosts != null) {
         NetUtils.addStaticResolution(hosts[i - curDatanodesNum], "localhost");
       }
-
-      SecureResources secureResources = null;
-      if (UserGroupInformation.isSecurityEnabled()) {
-        SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, dnConf);
-        try {
-          secureResources = SecureDataNodeStarter.getSecureResources(sslFactory, dnConf);
-        } catch (Exception ex) {
-          ex.printStackTrace();
-        }
-      }
-      DataNode dn = DataNode.instantiateDataNode(dnArgs, dnConf,
-                                                 secureResources);
+      DataNode dn = DataNode.instantiateDataNode(dnArgs, dnConf);
       if(dn == null)
         throw new IOException("Cannot start DataNode in "
             + dnConf.get(DFS_DATANODE_DATA_DIR_KEY));
@@ -1204,7 +1176,7 @@ public class MiniDFSCluster {
                                   racks[i-curDatanodesNum]);
       }
       dn.runDatanodeDaemon();
-      dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs, secureResources));
+      dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs));
     }
     curDatanodesNum += numDataNodes;
     this.numDataNodes += numDataNodes;
@@ -1635,16 +1607,14 @@ public class MiniDFSCluster {
       boolean keepPort) throws IOException {
     Configuration conf = dnprop.conf;
     String[] args = dnprop.dnArgs;
-    SecureResources secureResources = dnprop.secureResources;
     Configuration newconf = new HdfsConfiguration(conf); // save cloned config
     if (keepPort) {
       InetSocketAddress addr = dnprop.datanode.getXferAddress();
       conf.set(DFS_DATANODE_ADDRESS_KEY, 
           addr.getAddress().getHostAddress() + ":" + addr.getPort());
     }
-    dataNodes.add(new DataNodeProperties(
-        DataNode.createDataNode(args, conf, secureResources),
-        newconf, args, secureResources));
+    dataNodes.add(new DataNodeProperties(DataNode.createDataNode(args, conf),
+        newconf, args));
     numDataNodes++;
     return true;
   }

+ 0 - 117
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java

@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.hdfs.server.namenode;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-
-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.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import static org.apache.hadoop.security.SecurityUtilTestHelper.isExternalKdcRunning;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * This test starts a 1 NameNode 1 DataNode MiniDFSCluster with
- * kerberos authentication enabled using user-specified KDC,
- * principals, and keytabs.
- *
- * A secure DataNode has to be started by root, so this test needs to
- * be run by root.
- *
- * To run, users must specify the following system properties:
- *   externalKdc=true
- *   java.security.krb5.conf
- *   dfs.namenode.kerberos.principal
- *   dfs.namenode.kerberos.internal.spnego.principal
- *   dfs.namenode.keytab.file
- *   dfs.datanode.kerberos.principal
- *   dfs.datanode.keytab.file
- */
-public class TestStartSecureDataNode {
-  final static private int NUM_OF_DATANODES = 1;
-
-  @Before
-  public void testExternalKdcRunning() {
-    // Tests are skipped if external KDC is not running.
-    Assume.assumeTrue(isExternalKdcRunning());
-  }
-
-  @Test
-  public void testSecureNameNode() throws IOException, InterruptedException {
-    MiniDFSCluster cluster = null;
-    try {
-      String nnPrincipal =
-        System.getProperty("dfs.namenode.kerberos.principal");
-      String nnSpnegoPrincipal =
-        System.getProperty("dfs.namenode.kerberos.internal.spnego.principal");
-      String nnKeyTab = System.getProperty("dfs.namenode.keytab.file");
-      assertNotNull("NameNode principal was not specified", nnPrincipal);
-      assertNotNull("NameNode SPNEGO principal was not specified",
-                    nnSpnegoPrincipal);
-      assertNotNull("NameNode keytab was not specified", nnKeyTab);
-
-      String dnPrincipal = System.getProperty("dfs.datanode.kerberos.principal");
-      String dnKeyTab = System.getProperty("dfs.datanode.keytab.file");
-      assertNotNull("DataNode principal was not specified", dnPrincipal);
-      assertNotNull("DataNode keytab was not specified", dnKeyTab);
-
-      Configuration conf = new HdfsConfiguration();
-      conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
-        "kerberos");
-      conf.set(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, nnPrincipal);
-      conf.set(DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
-        nnSpnegoPrincipal);
-      conf.set(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY, nnKeyTab);
-      conf.set(DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY, dnPrincipal);
-      conf.set(DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY, dnKeyTab);
-      // Secure DataNode requires using ports lower than 1024.
-      conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:1004");
-      conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:1006");
-      conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, "700");
-
-      cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(NUM_OF_DATANODES)
-        .checkDataNodeAddrConfig(true)
-        .build();
-      cluster.waitActive();
-      assertTrue(cluster.isDataNodeUp());
-
-    } catch (Exception ex) {
-      ex.printStackTrace();
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-}

+ 0 - 129
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNodeWithExternalKdc.java

@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.hdfs.server.namenode;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-
-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.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import static org.apache.hadoop.security.SecurityUtilTestHelper.isExternalKdcRunning;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * This test brings up a MiniDFSCluster with 1 NameNode and 0
- * DataNodes with kerberos authentication enabled using user-specified
- * KDC, principals, and keytabs.
- *
- * To run, users must specify the following system properties:
- *   externalKdc=true
- *   java.security.krb5.conf
- *   dfs.namenode.kerberos.principal
- *   dfs.namenode.kerberos.internal.spnego.principal
- *   dfs.namenode.keytab.file
- *   user.principal (do not specify superuser!)
- *   user.keytab
- */
-public class TestSecureNameNodeWithExternalKdc {
-  final static private int NUM_OF_DATANODES = 0;
-
-  @Before
-  public void testExternalKdcRunning() {
-    // Tests are skipped if external KDC is not running.
-    Assume.assumeTrue(isExternalKdcRunning());
-  }
-
-  @Test
-  public void testSecureNameNode() throws IOException, InterruptedException {
-    MiniDFSCluster cluster = null;
-    try {
-      String nnPrincipal =
-        System.getProperty("dfs.namenode.kerberos.principal");
-      String nnSpnegoPrincipal =
-        System.getProperty("dfs.namenode.kerberos.internal.spnego.principal");
-      String nnKeyTab = System.getProperty("dfs.namenode.keytab.file");
-      assertNotNull("NameNode principal was not specified", nnPrincipal);
-      assertNotNull("NameNode SPNEGO principal was not specified",
-        nnSpnegoPrincipal);
-      assertNotNull("NameNode keytab was not specified", nnKeyTab);
-
-      Configuration conf = new HdfsConfiguration();
-      conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
-          "kerberos");
-      conf.set(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, nnPrincipal);
-      conf.set(DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
-          nnSpnegoPrincipal);
-      conf.set(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY, nnKeyTab);
-
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_OF_DATANODES)
-          .build();
-      final MiniDFSCluster clusterRef = cluster;
-      cluster.waitActive();
-      FileSystem fsForCurrentUser = cluster.getFileSystem();
-      fsForCurrentUser.mkdirs(new Path("/tmp"));
-      fsForCurrentUser.setPermission(new Path("/tmp"), new FsPermission(
-          (short) 511));
-
-      // The user specified should not be a superuser
-      String userPrincipal = System.getProperty("user.principal");
-      String userKeyTab = System.getProperty("user.keytab");
-      assertNotNull("User principal was not specified", userPrincipal);
-      assertNotNull("User keytab was not specified", userKeyTab);
-
-      UserGroupInformation ugi = UserGroupInformation
-          .loginUserFromKeytabAndReturnUGI(userPrincipal, userKeyTab);
-      FileSystem fs = ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
-        @Override
-        public FileSystem run() throws Exception {
-          return clusterRef.getFileSystem();
-        }
-      });
-      try {
-        Path p = new Path("/users");
-        fs.mkdirs(p);
-        fail("User must not be allowed to write in /");
-      } catch (IOException expected) {
-      }
-
-      Path p = new Path("/tmp/alpha");
-      fs.mkdirs(p);
-      assertNotNull(fs.listStatus(p));
-      assertEquals(AuthenticationMethod.KERBEROS,
-          ugi.getAuthenticationMethod());
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-}