You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 05:12:17 UTC

svn commit: r1077412 - in /hadoop/common/branches/branch-0.20-security-patches/src: core/org/apache/hadoop/conf/ core/org/apache/hadoop/security/ core/org/apache/hadoop/security/authorize/ hdfs/org/apache/hadoop/hdfs/ hdfs/org/apache/hadoop/hdfs/server...

Author: omalley
Date: Fri Mar  4 04:12:16 2011
New Revision: 1077412

URL: http://svn.apache.org/viewvc?rev=1077412&view=rev
Log:
commit f2acac9b846cacd4e9bb36a39e6b67706157c5cb
Author: Boris Shkolnik <bo...@yahoo-inc.com>
Date:   Mon Apr 19 16:38:48 2010 -0700

    HDFS:1096 https://issues.apache.org/jira/secure/attachment/12442244/HDFS-1096-BP20-7.patch
    
    +++ b/YAHOO-CHANGES.txt
    +    HDFS-1096. allow dfsadmin/mradmin refresh of superuser proxy group
    +    mappings(boryas).
    +

Added:
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/RefreshUserMappingsProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java
Removed:
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/RefreshUserToGroupMappingsProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestGroupMappingServiceRefresh.java
Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/conf/Configuration.java
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/ProxyUsers.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapReducePolicyProvider.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/conf/TestConfiguration.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestDoAsEffectiveUser.java

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/conf/Configuration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/conf/Configuration.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/conf/Configuration.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/conf/Configuration.java Fri Mar  4 04:12:16 2011
@@ -1402,5 +1402,27 @@ public class Configuration implements It
       org.apache.hadoop.io.Text.writeString(out, (String) item.getValue());
     }
   }
-
+  
+  /**
+   * get keys matching the the regex 
+   * @param regex
+   * @return Map<String,String> with matching keys
+   */
+  public Map<String,String> getValByRegex(String regex) {
+    Pattern p = Pattern.compile(regex);
+    
+    Map<String,String> result = new HashMap<String,String>();
+    Matcher m;
+    
+    for(Map.Entry<Object,Object> item: getProps().entrySet()) {
+      if (item.getKey() instanceof String && 
+          item.getValue() instanceof String) {
+        m = p.matcher((String)item.getKey());
+        if(m.find()) { // match
+          result.put((String) item.getKey(), (String) item.getValue());
+        }
+      }
+    }
+    return result;
+  }
 }

Added: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/RefreshUserMappingsProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/RefreshUserMappingsProtocol.java?rev=1077412&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/RefreshUserMappingsProtocol.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/RefreshUserMappingsProtocol.java Fri Mar  4 04:12:16 2011
@@ -0,0 +1,53 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ipc.VersionedProtocol;
+
+/**
+ * Protocol use 
+ *
+ */
+@KerberosInfo(
+    serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+public interface RefreshUserMappingsProtocol extends VersionedProtocol {
+  
+  /**
+   * Version 1: Initial version.
+   */
+  public static final long versionID = 1L;
+
+  /**
+   * Refresh user to group mappings.
+   * @param conf
+   * @throws IOException
+   */
+  public void refreshUserToGroupsMappings(Configuration conf) throws IOException;
+  
+  /**
+   * Refresh superuser proxy group list
+   * @param conf
+   * @throws IOException
+   */
+  public void refreshSuperUserGroupsConfiguration(Configuration conf) 
+  throws IOException;
+}

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/ProxyUsers.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/ProxyUsers.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/ProxyUsers.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/ProxyUsers.java Fri Mar  4 04:12:16 2011
@@ -21,52 +21,98 @@ package org.apache.hadoop.security.autho
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
 
 public class ProxyUsers {
+  private static final String CONF_HOSTS = ".hosts";
+  public static final String CONF_GROUPS = ".groups";
+  public static final String CONF_HADOOP_PROXYUSER = "hadoop.proxyuser.";
+  public static final String CONF_HADOOP_PROXYUSER_RE = "hadoop\\.proxyuser\\.";
+  private static Configuration conf=null;
+  // list of groups and hosts per proxyuser
+  private static Map<String, Collection<String>> proxyGroups = 
+    new HashMap<String, Collection<String>>();
+  private static Map<String, Collection<String>> proxyHosts = 
+    new HashMap<String, Collection<String>>();
+  
+  /**
+   * reread the conf and get new values for "hadoop.proxyuser.*.groups/hosts"
+   */
+  public static synchronized void refreshSuperUserGroupsConfiguration(Configuration cn) {
+    conf = cn;
+    
+    // remove alle existing stuff
+    proxyGroups.clear();
+    proxyHosts.clear();
+    
+    // get all the new keys for groups
+    String regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_GROUPS;
+    Map<String,String> allMatchKeys = conf.getValByRegex(regex);
+    for(Entry<String, String> entry : allMatchKeys.entrySet()) {
+      proxyGroups.put(entry.getKey(), 
+          StringUtils.getStringCollection(entry.getValue()));
+    }
+    
+    // now hosts
+    regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_HOSTS;
+    allMatchKeys = conf.getValByRegex(regex);
+    for(Entry<String, String> entry : allMatchKeys.entrySet()) {
+      proxyHosts.put(entry.getKey(),
+          StringUtils.getStringCollection(entry.getValue()));
+    }
+  }
 
-  /*
+  /**
    * Returns configuration key for effective user groups allowed for a superuser
    * 
    * @param userName name of the superuser
    * @return configuration key for superuser groups
    */
   public static String getProxySuperuserGroupConfKey(String userName) {
-    return "hadoop.proxyuser."+userName+".groups";
+    return ProxyUsers.CONF_HADOOP_PROXYUSER+userName+ProxyUsers.CONF_GROUPS;
   }
   
-  /*
+  /**
    * Return configuration key for superuser ip addresses
    * 
    * @param userName name of the superuser
    * @return configuration key for superuser ip-addresses
    */
   public static String getProxySuperuserIpConfKey(String userName) {
-    return "hadoop.proxyuser."+userName+".hosts";
+    return ProxyUsers.CONF_HADOOP_PROXYUSER+userName+ProxyUsers.CONF_HOSTS;
   }
   
-  /*
+  /**
    * Authorize the superuser which is doing doAs
    * 
    * @param user ugi of the effective or proxy user which contains a real user
    * @param remoteAddress the ip address of client
-   * @param conf configuration
+   * @param newConf configuration
    * @throws AuthorizationException
    */
-  public static void authorize(UserGroupInformation user, String remoteAddress,
-      Configuration conf) throws AuthorizationException {
+  public static synchronized void authorize(UserGroupInformation user, String remoteAddress,
+      Configuration newConf) throws AuthorizationException {
+    
+    if(conf == null) {
+      refreshSuperUserGroupsConfiguration(newConf); 
+    }
 
     if (user.getRealUser() == null) {
       return;
     }
     boolean groupAuthorized = false;
+    boolean ipAuthorized = false;
     UserGroupInformation superUser = user.getRealUser();
 
-    Collection<String> allowedUserGroups = conf
-        .getStringCollection(getProxySuperuserGroupConfKey(superUser
-            .getShortUserName()));
+    Collection<String> allowedUserGroups = proxyGroups.get(
+        getProxySuperuserGroupConfKey(superUser.getShortUserName()));
+    
     if (!allowedUserGroups.isEmpty()) {
       for (String group : user.getGroupNames()) {
         if (allowedUserGroups.contains(group)) {
@@ -75,15 +121,15 @@ public class ProxyUsers {
         }
       }
     }
-
+    
     if (!groupAuthorized) {
       throw new AuthorizationException("User: " + superUser.getUserName()
           + " is not allowed to impersonate " + user.getUserName());
     }
     
-    Collection<String> ipList = conf
-        .getStringCollection(getProxySuperuserIpConfKey(superUser
-            .getShortUserName()));
+    Collection<String> ipList = proxyHosts.get(
+        getProxySuperuserIpConfKey(superUser.getShortUserName()));
+    
     if (!ipList.isEmpty()) {
       for (String allowedHost : ipList) {
         InetAddress hostAddr;
@@ -94,11 +140,13 @@ public class ProxyUsers {
         }
         if (hostAddr.getHostAddress().equals(remoteAddress)) {
           // Authorization is successful
-          return;
+          ipAuthorized = true;
         }
       }
     }
-    throw new AuthorizationException("Unauthorized connection for super-user: "
-        + superUser.getUserName() + " from IP " + remoteAddress);
+    if(!ipAuthorized) {
+      throw new AuthorizationException("Unauthorized connection for super-user: "
+          + superUser.getUserName() + " from IP " + remoteAddress);
+    }
   }
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HDFSPolicyProvider.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HDFSPolicyProvider.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HDFSPolicyProvider.java Fri Mar  4 04:12:16 2011
@@ -22,7 +22,7 @@ import org.apache.hadoop.hdfs.protocol.C
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
-import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.Service;
@@ -43,7 +43,7 @@ public class HDFSPolicyProvider extends 
     new Service("security.refresh.policy.protocol.acl", 
                 RefreshAuthorizationPolicyProtocol.class),
     new Service("security.refresh.usertogroups.mappings.protocol.acl", 
-                RefreshUserToGroupMappingsProtocol.class),
+                RefreshUserMappingsProtocol.class),
   };
   
   @Override

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java Fri Mar  4 04:12:16 2011
@@ -52,13 +52,14 @@ import org.apache.hadoop.net.NetworkTopo
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.security.Groups;
-import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.SecurityUtil;
 
 import java.io.*;
@@ -104,7 +105,7 @@ import java.util.Iterator;
 public class NameNode implements ClientProtocol, DatanodeProtocol,
                                  NamenodeProtocol, FSConstants,
                                  RefreshAuthorizationPolicyProtocol,
-                                 RefreshUserToGroupMappingsProtocol {
+                                 RefreshUserMappingsProtocol {
   static{
     Configuration.addDefaultResource("hdfs-default.xml");
     Configuration.addDefaultResource("hdfs-site.xml");
@@ -120,8 +121,8 @@ public class NameNode implements ClientP
       return NamenodeProtocol.versionID;
     } else if (protocol.equals(RefreshAuthorizationPolicyProtocol.class.getName())){
       return RefreshAuthorizationPolicyProtocol.versionID;
-    } else if (protocol.equals(RefreshUserToGroupMappingsProtocol.class.getName())){
-      return RefreshUserToGroupMappingsProtocol.versionID;
+    } else if (protocol.equals(RefreshUserMappingsProtocol.class.getName())){
+      return RefreshUserMappingsProtocol.versionID;
     } else {
       throw new IOException("Unknown protocol to name node: " + protocol);
     }
@@ -1000,6 +1001,13 @@ public class NameNode implements ClientP
              UserGroupInformation.getCurrentUser().getShortUserName());
     Groups.getUserToGroupsMappingService(conf).refresh();
   }
+  
+  @Override
+  public void refreshSuperUserGroupsConfiguration(Configuration conf) {
+    LOG.info("Refreshing SuperUser proxy group mapping list ");
+    
+    ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
+  }
 
   private static void printUsage() {
     System.err.println(

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java Fri Mar  4 04:12:16 2011
@@ -21,25 +21,26 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.Command;
+import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem.DiskStatus;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.shell.Command;
-import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.util.StringUtils;
@@ -430,6 +431,7 @@ public class DFSAdmin extends FsShell {
       "\t[" + ClearSpaceQuotaCommand.USAGE +"]\n" +
       "\t[-refreshServiceAcl]\n" +
       "\t[-refreshUserToGroupsMappings]\n" +
+      "\t[refreshSuperUserGroupsConfiguration]\n" +
       "\t[-help [cmd]]\n";
 
     String report ="-report: \tReports basic filesystem information and statistics.\n";
@@ -482,6 +484,9 @@ public class DFSAdmin extends FsShell {
     String refreshUserToGroupsMappings =
       "-refreshUserToGroupsMappings: Refresh user-to-groups mappings\n";
     
+    String refreshSuperUserGroupsConfiguration = 
+      "-refreshSuperUserGroupsConfiguration: Refresh superuser proxy groups mappings\n";
+    
     String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
       "\t\tis specified.\n";
 
@@ -511,6 +516,8 @@ public class DFSAdmin extends FsShell {
       System.out.println(refreshServiceAcl);
     } else if ("refreshUserToGroupsMappings".equals(cmd)) {
       System.out.println(refreshUserToGroupsMappings);
+    } else if ("refreshSuperUserGroupsConfiguration".equals(cmd)) {
+      System.out.println(refreshSuperUserGroupsConfiguration);
     } else if ("help".equals(cmd)) {
       System.out.println(help);
     } else {
@@ -528,6 +535,7 @@ public class DFSAdmin extends FsShell {
       System.out.println(ClearSpaceQuotaCommand.DESCRIPTION);
       System.out.println(refreshServiceAcl);
       System.out.println(refreshUserToGroupsMappings);
+      System.out.println(refreshSuperUserGroupsConfiguration);
       System.out.println(help);
       System.out.println();
       ToolRunner.printGenericCommandUsage(System.out);
@@ -642,6 +650,7 @@ public class DFSAdmin extends FsShell {
                                              RefreshAuthorizationPolicyProtocol.class));
     
     // Refresh the authorization policy in-effect
+    
     refreshProtocol.refreshServiceAcl();
     
     return 0;
@@ -663,20 +672,51 @@ public class DFSAdmin extends FsShell {
         conf.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, ""));
     
     // Create the client
-    RefreshUserToGroupMappingsProtocol refreshProtocol = 
-      (RefreshUserToGroupMappingsProtocol) 
-      RPC.getProxy(RefreshUserToGroupMappingsProtocol.class, 
-                   RefreshUserToGroupMappingsProtocol.versionID, 
+    RefreshUserMappingsProtocol refreshProtocol = 
+      (RefreshUserMappingsProtocol) 
+      RPC.getProxy(RefreshUserMappingsProtocol.class, 
+                   RefreshUserMappingsProtocol.versionID, 
                    NameNode.getAddress(conf), getUGI(), conf,
                    NetUtils.getSocketFactory(conf, 
-                                             RefreshUserToGroupMappingsProtocol.class));
+                                             RefreshUserMappingsProtocol.class));
     
     // Refresh the user-to-groups mappings
     refreshProtocol.refreshUserToGroupsMappings(conf);
     
     return 0;
   }
+
+  /**
+   * refreshSuperUserGroupsConfiguration {@link NameNode}.
+   * @return exitcode 0 on success, non-zero on failure
+   * @throws IOException
+   */
+  public int refreshSuperUserGroupsConfiguration() throws IOException {
+    // Get the current configuration
+    Configuration conf = getConf();
+    
+    // for security authorization
+    // server principal for this call 
+    // should be NAMENODE's one.
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY, 
+        conf.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, ""));
+    
+    // Create the client
+    RefreshUserMappingsProtocol refreshProtocol = 
+      (RefreshUserMappingsProtocol) 
+      RPC.getProxy(RefreshUserMappingsProtocol.class, 
+                   RefreshUserMappingsProtocol.versionID, 
+                   NameNode.getAddress(conf), getUGI(), conf,
+                   NetUtils.getSocketFactory(conf, 
+                       RefreshUserMappingsProtocol.class));
+    
+    // Refresh the user-to-groups mappings
+    refreshProtocol.refreshSuperUserGroupsConfiguration(conf);
+    
+    return 0;
+  }
   
+
   /**
    * Displays format of commands.
    * @param cmd The command that is being executed.
@@ -721,6 +761,9 @@ public class DFSAdmin extends FsShell {
     } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
       System.err.println("Usage: java DFSAdmin"
                          + " [-refreshUserToGroupsMappings]");
+    } else if ("-refreshSuperUserGroupsConfiguration".equals(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+                         + " [-refreshSuperUserGroupsConfiguration]");
     } else {
       System.err.println("Usage: java DFSAdmin");
       System.err.println("           [-report]");
@@ -732,6 +775,7 @@ public class DFSAdmin extends FsShell {
       System.err.println("           [-metasave filename]");
       System.err.println("           [-refreshServiceAcl]");
       System.err.println("           [-refreshUserToGroupsMappings]");
+      System.err.println("           [-refreshSuperUserGroupsConfiguration]");
       System.err.println("           ["+SetQuotaCommand.USAGE+"]");
       System.err.println("           ["+ClearQuotaCommand.USAGE+"]");
       System.err.println("           ["+SetSpaceQuotaCommand.USAGE+"]");
@@ -849,6 +893,8 @@ public class DFSAdmin extends FsShell {
         exitCode = refreshServiceAcl();
       } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
         exitCode = refreshUserToGroupsMappings();
+      } else if ("-refreshSuperUserGroupsConfiguration".equals(cmd)) {
+        exitCode = refreshSuperUserGroupsConfiguration();
       } else if ("-help".equals(cmd)) {
         if (i < argv.length) {
           printHelp(argv[i]);

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java Fri Mar  4 04:12:16 2011
@@ -92,11 +92,12 @@ import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Groups;
-import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.security.token.Token;
@@ -119,7 +120,7 @@ import org.apache.hadoop.security.Creden
  *
  *******************************************************/
 public class JobTracker implements MRConstants, InterTrackerProtocol,
-    JobSubmissionProtocol, TaskTrackerManager, RefreshUserToGroupMappingsProtocol,
+    JobSubmissionProtocol, TaskTrackerManager, RefreshUserMappingsProtocol,
     RefreshAuthorizationPolicyProtocol, AdminOperationsProtocol {
 
   static{
@@ -310,8 +311,8 @@ public class JobTracker implements MRCon
       return RefreshAuthorizationPolicyProtocol.versionID;
     } else if (protocol.equals(AdminOperationsProtocol.class.getName())){
       return AdminOperationsProtocol.versionID;
-    } else if (protocol.equals(RefreshUserToGroupMappingsProtocol.class.getName())){
-      return RefreshUserToGroupMappingsProtocol.versionID;
+    } else if (protocol.equals(RefreshUserMappingsProtocol.class.getName())){
+      return RefreshUserMappingsProtocol.versionID;
     } else {
       throw new IOException("Unknown protocol to job tracker: " + protocol);
     }
@@ -4846,6 +4847,12 @@ public class JobTracker implements MRCon
             limitMaxMemForReduceTasks).append(")"));
   }
 
+  @Override
+  public void refreshSuperUserGroupsConfiguration(Configuration conf) {
+    LOG.info("Refreshing superuser proxy groups mapping ");
+    
+    ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
+  }
     
   @Override
   public void refreshUserToGroupsMappings(Configuration conf) throws IOException {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapReducePolicyProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapReducePolicyProvider.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapReducePolicyProvider.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapReducePolicyProvider.java Fri Mar  4 04:12:16 2011
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.mapred;
 
-import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.Service;
@@ -37,7 +37,7 @@ public class MapReducePolicyProvider ext
       new Service("security.refresh.policy.protocol.acl", 
                   RefreshAuthorizationPolicyProtocol.class),
       new Service("security.refresh.usertogroups.mappings.protocol.acl", 
-                  RefreshUserToGroupMappingsProtocol.class),
+                  RefreshUserMappingsProtocol.class),
       new Service("security.admin.operations.protocol.acl", 
                   AdminOperationsProtocol.class),
   };

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java Fri Mar  4 04:12:16 2011
@@ -22,13 +22,14 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.mapred.AdminOperationsProtocol;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobTracker;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.util.Tool;
@@ -55,7 +56,8 @@ public class MRAdmin extends Configured 
     String summary = "hadoop mradmin is the command to execute Map-Reduce administrative commands.\n" +
     "The full syntax is: \n\n" +
     "hadoop mradmin [-refreshServiceAcl] [-refreshQueueAcls] " +
-    "[-refreshNodes] [-refreshUserToGroupsMappings] [-help [cmd]]\n";
+    "[-refreshNodes] [-refreshUserToGroupsMappings] " +
+    "[-refreshSuperUserGroupsConfiguration] [-help [cmd]]\n";
 
   String refreshServiceAcl = "-refreshServiceAcl: Reload the service-level authorization policy file\n" +
     "\t\tJobtracker will reload the authorization policy file.\n";
@@ -67,6 +69,9 @@ public class MRAdmin extends Configured 
   String refreshUserToGroupsMappings = 
     "-refreshUserToGroupsMappings: Refresh user-to-groups mappings\n";
   
+  String refreshSuperUserGroupsConfiguration = 
+    "-refreshSuperUserGroupsConfiguration: Refresh superuser proxy groups mappings\n";
+  
   String refreshNodes =
     "-refreshNodes: Refresh the hosts information at the jobtracker.\n";
   
@@ -79,6 +84,8 @@ public class MRAdmin extends Configured 
     System.out.println(refreshQueueAcls);
   } else if ("refreshUserToGroupsMappings".equals(cmd)) {
     System.out.println(refreshUserToGroupsMappings);
+  } else if ("refreshSuperUserGroupsConfiguration".equals(cmd)) {
+    System.out.println(refreshSuperUserGroupsConfiguration);
   }  else if ("refreshNodes".equals(cmd)) {
     System.out.println(refreshNodes);
   } else if ("help".equals(cmd)) {
@@ -88,6 +95,7 @@ public class MRAdmin extends Configured 
     System.out.println(refreshServiceAcl);
     System.out.println(refreshQueueAcls);
     System.out.println(refreshUserToGroupsMappings);
+    System.out.println(refreshSuperUserGroupsConfiguration);
     System.out.println(refreshNodes);
     System.out.println(help);
     System.out.println();
@@ -107,6 +115,9 @@ public class MRAdmin extends Configured 
       System.err.println("Usage: java MRAdmin" + " [-refreshQueueAcls]");
     } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
       System.err.println("Usage: java MRAdmin" + " [-refreshUserToGroupsMappings]");
+    } else if ("-refreshSuperUserGroupsConfiguration".equals(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+                         + " [-refreshSuperUserGroupsConfiguration]");
     } else if ("-refreshNodes".equals(cmd)) {
       System.err.println("Usage: java MRAdmin" + " [-refreshNodes]");
     } else {
@@ -114,6 +125,7 @@ public class MRAdmin extends Configured 
       System.err.println("           [-refreshServiceAcl]");
       System.err.println("           [-refreshQueueAcls]");
       System.err.println("           [-refreshUserToGroupsMappings]");
+      System.err.println("           [-refreshSuperUserGroupsConfiguration]");
       System.err.println("           [-refreshNodes]");
       System.err.println("           [-help [cmd]]");
       System.err.println();
@@ -197,6 +209,38 @@ public class MRAdmin extends Configured 
     return 0;
   }
 
+  
+  /**
+   * refreshSuperUserGroupsConfiguration {@link JobTracker}.
+   * @return exitcode 0 on success, non-zero on failure
+   * @throws IOException
+   */
+  public int refreshSuperUserGroupsConfiguration() throws IOException {
+    // Get the current configuration
+    Configuration conf = getConf();
+    
+    // for security authorization
+    // server principal for this call   
+    // should be JT's one.
+    JobConf jConf = new JobConf(conf);
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY, 
+        jConf.get(JobTracker.JT_USER_NAME, ""));
+    
+    // Create the client
+    RefreshUserMappingsProtocol refreshProtocol = 
+      (RefreshUserMappingsProtocol) 
+      RPC.getProxy(RefreshUserMappingsProtocol.class, 
+                   RefreshUserMappingsProtocol.versionID, 
+                   JobTracker.getAddress(conf), getUGI(conf), conf,
+                   NetUtils.getSocketFactory(conf, 
+                       RefreshUserMappingsProtocol.class));
+    
+    // Refresh the user-to-groups mappings
+    refreshProtocol.refreshSuperUserGroupsConfiguration(conf);
+    
+    return 0;
+  }
+  
   /**
    * Refresh the user-to-groups mappings on the {@link JobTracker}.
    * @return exitcode 0 on success, non-zero on failure
@@ -216,13 +260,13 @@ public class MRAdmin extends Configured 
     
     
     // Create the client
-    RefreshUserToGroupMappingsProtocol refreshProtocol =
-      (RefreshUserToGroupMappingsProtocol)
-      RPC.getProxy(RefreshUserToGroupMappingsProtocol.class,
-                   RefreshUserToGroupMappingsProtocol.versionID,
+    RefreshUserMappingsProtocol refreshProtocol =
+      (RefreshUserMappingsProtocol)
+      RPC.getProxy(RefreshUserMappingsProtocol.class,
+                   RefreshUserMappingsProtocol.versionID,
                    JobTracker.getAddress(conf), getUGI(conf), conf,
                    NetUtils.getSocketFactory(conf,
-                                             RefreshUserToGroupMappingsProtocol.class));
+                                             RefreshUserMappingsProtocol.class));
 
     // Refresh the user-to-groups mappings
     refreshProtocol.refreshUserToGroupsMappings(conf);
@@ -245,7 +289,10 @@ public class MRAdmin extends Configured 
     // verify that we have enough command line parameters
     //
     if ("-refreshServiceAcl".equals(cmd) || "-refreshQueueAcls".equals(cmd)
-        || "-refreshNodes".equals(cmd) || "-refreshUserToGroupsMappings".equals(cmd)) {
+        || "-refreshNodes".equals(cmd) ||
+        "-refreshUserToGroupsMappings".equals(cmd) ||
+        "-refreshSuperUserGroupsConfiguration".equals(cmd)
+        ) {
       if (args.length != 1) {
         printUsage(cmd);
         return exitCode;
@@ -260,6 +307,8 @@ public class MRAdmin extends Configured 
         exitCode = refreshQueueAcls();
       } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
         exitCode = refreshUserToGroupsMappings();
+      } else if ("-refreshSuperUserGroupsConfiguration".equals(cmd)) {
+        exitCode = refreshSuperUserGroupsConfiguration();
       } else if ("-refreshNodes".equals(cmd)) {
         exitCode = refreshNodes();
       } else if ("-help".equals(cmd)) {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/conf/TestConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/conf/TestConfiguration.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/conf/TestConfiguration.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/conf/TestConfiguration.java Fri Mar  4 04:12:16 2011
@@ -28,6 +28,7 @@ import java.io.DataOutputStream;
 import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Map;
 import java.util.Random;
 
 import junit.framework.TestCase;
@@ -606,5 +607,23 @@ public class TestConfiguration extends T
       assertEquals(fileResource.toString(),prop.getResource());
     }
   }
+  
+  public void testGetValByRegex() {
+    Configuration conf = new Configuration();
+    String key1 = "t.abc.key1";
+    String key2 = "t.abc.key2";
+    String key3 = "tt.abc.key3";
+    String key4 = "t.abc.ey3";
+    conf.set(key1, "value1");
+    conf.set(key2, "value2");
+    conf.set(key3, "value3");
+    conf.set(key4, "value3");
+    
+    Map<String,String> res = conf.getValByRegex("^t\\..*\\.key\\d");
+    assertTrue("Conf didn't get key " + key1, res.containsKey(key1));
+    assertTrue("Conf didn't get key " + key2, res.containsKey(key2));
+    assertTrue("Picked out wrong key " + key3, !res.containsKey(key3));
+    assertTrue("Picked out wrong key " + key4, !res.containsKey(key4));
+  }
 }
 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestDoAsEffectiveUser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestDoAsEffectiveUser.java?rev=1077412&r1=1077411&r2=1077412&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestDoAsEffectiveUser.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestDoAsEffectiveUser.java Fri Mar  4 04:12:16 2011
@@ -29,6 +29,7 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
@@ -224,14 +225,16 @@ public class TestDoAsEffectiveUser {
    */
   @Test
   public void testRealUserIPAuthorizationFailure() throws IOException {
-    final Configuration conf = new Configuration();
+    final Configuration conf = new Configuration(masterConf);
     conf.setStrings(ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_SHORT_NAME),
         "20.20.20.20"); //Authorized IP address
     conf.setStrings(ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
         "group1");
     Server server = RPC.getServer(new TestImpl(), ADDRESS,
         0, 2, false, conf, null);
-
+    
+    refreshConf(conf);
+    
     try {
       server.start();
 
@@ -410,6 +413,9 @@ public class TestDoAsEffectiveUser {
     UserGroupInformation proxyUserUgi = UserGroupInformation
         .createProxyUserForTesting(PROXY_USER_NAME, current, GROUP_NAMES);
     proxyUserUgi.addToken(token);
+    
+    refreshConf(conf);
+    
     String retVal = proxyUserUgi.doAs(new PrivilegedExceptionAction<String>() {
       @Override
       public String run() throws Exception {
@@ -451,6 +457,8 @@ public class TestDoAsEffectiveUser {
 
     final UserGroupInformation current = UserGroupInformation
         .createUserForTesting(REAL_USER_NAME, GROUP_NAMES);
+    refreshConf(newConf);
+    
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
     TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
         .getUserName()), new Text("SomeSuperUser"));
@@ -479,6 +487,12 @@ public class TestDoAsEffectiveUser {
         }
       }
     });
-    Assert.assertEquals(REAL_USER_NAME + " via SomeSuperUser", retVal);
+    String expected = REAL_USER_NAME + " via SomeSuperUser";
+    Assert.assertEquals(retVal + "!=" + expected, expected, retVal);
+  }
+  
+  //
+  private void refreshConf(Configuration conf) throws IOException {
+    ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
   }
 }

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java?rev=1077412&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java Fri Mar  4 04:12:16 2011
@@ -0,0 +1,201 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestRefreshUserMappings {
+  private MiniDFSCluster cluster;
+  Configuration config;
+  private static long groupRefreshTimeoutSec = 1;
+  
+  public static class MockUnixGroupsMapping implements GroupMappingServiceProvider {
+    private int i=0;
+    
+    @Override
+    public List<String> getGroups(String user) throws IOException {
+      System.out.println("Getting groups in MockUnixGroupsMapping");
+      String g1 = user + (10 * i + 1);
+      String g2 = user + (10 * i + 2);
+      List<String> l = new ArrayList<String>(2);
+      l.add(g1);
+      l.add(g2);
+      i++;
+      return l;
+    }
+  }
+  
+  @Before
+  public void setUp() throws Exception {
+    config = new Configuration();
+    config.setClass("hadoop.security.group.mapping",
+        TestRefreshUserMappings.MockUnixGroupsMapping.class,
+        GroupMappingServiceProvider.class);
+    config.setLong("hadoop.security.groups.cache.secs", groupRefreshTimeoutSec);
+    Groups.getUserToGroupsMappingService(config);
+    
+    FileSystem.setDefaultUri(config, "hdfs://localhost:" + "0");
+    cluster = new MiniDFSCluster(0, config, 1, true, true, true,  null, null, null, null);
+    cluster.waitActive();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if(cluster!=null) {
+      cluster.shutdown();
+    }
+  }
+    
+  @Test
+  public void testGroupMappingRefresh() throws Exception {
+    DFSAdmin admin = new DFSAdmin(config);
+    String [] args =  new String[]{"-refreshUserToGroupsMappings"};
+    Groups groups = Groups.getUserToGroupsMappingService(config);
+    String user = UserGroupInformation.getCurrentUser().getUserName();
+    System.out.println("first attempt:");
+    List<String> g1 = groups.getGroups(user);
+    String [] str_groups = new String [g1.size()];
+    g1.toArray(str_groups);
+    System.out.println(Arrays.toString(str_groups));
+    
+    System.out.println("second attempt, should be same:");
+    List<String> g2 = groups.getGroups(user);
+    g2.toArray(str_groups);
+    System.out.println(Arrays.toString(str_groups));
+    for(int i=0; i<g2.size(); i++) {
+      assertEquals("Should be same group ", g1.get(i), g2.get(i));
+    }
+    admin.run(args);
+    System.out.println("third attempt(after refresh command), should be different:");
+    List<String> g3 = groups.getGroups(user);
+    g3.toArray(str_groups);
+    System.out.println(Arrays.toString(str_groups));
+    for(int i=0; i<g3.size(); i++) {
+      assertFalse("Should be different group: " + g1.get(i) + " and " + g3.get(i), 
+          g1.get(i).equals(g3.get(i)));
+    }
+    
+    // test time out
+    Thread.sleep(groupRefreshTimeoutSec*1100);
+    System.out.println("fourth attempt(after timeout), should be different:");
+    List<String> g4 = groups.getGroups(user);
+    g4.toArray(str_groups);
+    System.out.println(Arrays.toString(str_groups));
+    for(int i=0; i<g4.size(); i++) {
+      assertFalse("Should be different group ", g3.get(i).equals(g4.get(i)));
+    }
+  }
+  
+  @Test
+  public void testRefreshSuperUserGroupsConfiguration() throws Exception {
+    final String SUPER_USER = "super_user";
+    final String [] GROUP_NAMES1 = new String [] {"gr1" , "gr2"};
+    final String [] GROUP_NAMES2 = new String [] {"gr3" , "gr4"};
+    
+    //keys in conf
+    String userKeyGroups = ProxyUsers.getProxySuperuserGroupConfKey(SUPER_USER);
+    String userKeyHosts = ProxyUsers.getProxySuperuserIpConfKey (SUPER_USER);
+    
+    config.set(userKeyGroups, "gr3,gr4,gr5"); // superuser can proxy for this group
+    config.set(userKeyHosts,"127.0.0.1");
+    
+    UserGroupInformation ugi1 = mock(UserGroupInformation.class);
+    UserGroupInformation ugi2 = mock(UserGroupInformation.class);
+    UserGroupInformation suUgi = mock(UserGroupInformation.class);
+    when(ugi1.getRealUser()).thenReturn(suUgi);
+    when(ugi2.getRealUser()).thenReturn(suUgi);
+
+    when(suUgi.getShortUserName()).thenReturn(SUPER_USER); // super user
+    when(suUgi.getUserName()).thenReturn(SUPER_USER+"L"); // super user
+     
+    when(ugi1.getShortUserName()).thenReturn("user1");
+    when(ugi2.getShortUserName()).thenReturn("user2");
+    
+    when(ugi1.getUserName()).thenReturn("userL1");
+    when(ugi2.getUserName()).thenReturn("userL2");
+   
+    // set groups for users
+    when(ugi1.getGroupNames()).thenReturn(GROUP_NAMES1);
+    when(ugi2.getGroupNames()).thenReturn(GROUP_NAMES2);
+   
+    
+    // check before
+    try {
+      ProxyUsers.authorize(ugi1, "127.0.0.1", config);
+      fail("first auth for " + ugi1.getShortUserName() + " should've failed ");
+    } catch (AuthorizationException e) {
+      // expected
+      System.err.println("auth for " + ugi1.getUserName() + " failed");
+    }
+    try {
+      ProxyUsers.authorize(ugi2, "127.0.0.1", config);
+      System.err.println("auth for " + ugi2.getUserName() + " succeeded");
+      // expected
+    } catch (AuthorizationException e) {
+      fail("first auth for " + ugi2.getShortUserName() + " should've succeeded: " + e.getLocalizedMessage());
+    }
+    
+    DFSAdmin admin = new DFSAdmin(config);
+    String [] args = new String[]{"-refreshSuperUserGroupsConfiguration"};
+    NameNode nn = cluster.getNameNode();
+    Configuration conf = new Configuration(config);
+    conf.set(userKeyGroups, "gr2"); // superuser can proxy for this group
+    admin.setConf(conf);
+    admin.run(args);
+    
+    //check after...
+    
+    try {
+      ProxyUsers.authorize(ugi2, "127.0.0.1", config);
+      fail("second auth for " + ugi2.getShortUserName() + " should've failed ");
+    } catch (AuthorizationException e) {
+      // expected
+      System.err.println("auth for " + ugi2.getUserName() + " failed");
+    }
+    try {
+      ProxyUsers.authorize(ugi1, "127.0.0.1", config);
+      System.err.println("auth for " + ugi1.getUserName() + " succeeded");
+      // expected
+    } catch (AuthorizationException e) {
+      fail("second auth for " + ugi1.getShortUserName() + " should've succeeded: " + e.getLocalizedMessage());
+    }    
+  }
+
+}