瀏覽代碼

HDFS-3733. Audit logs should include WebHDFS access. Contributed by Andy Isaacson

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1379278 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 12 年之前
父節點
當前提交
7d1c8d92f9

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -698,6 +698,8 @@ Branch-2 ( Unreleased changes )
 
     HDFS-3837. Fix DataNode.recoverBlock findbugs warning. (eli)
 
+    HDFS-3733. Audit logs should include WebHDFS access. (Andy Isaacson via eli)
+
   BREAKDOWN OF HDFS-3042 SUBTASKS
 
     HDFS-2185. HDFS portion of ZK-based FailoverController (todd)

+ 55 - 30
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -169,6 +169,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
@@ -1071,7 +1072,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "setPermission", src, null, null);
       }
       throw e;
@@ -1100,7 +1101,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "setPermission", src, null, resultingStat);
     }
   }
@@ -1117,7 +1118,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "setOwner", src, null, null);
       }
       throw e;
@@ -1155,7 +1156,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "setOwner", src, null, resultingStat);
     }
   }
@@ -1190,7 +1191,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "open", src, null, null);
       }
       throw e;
@@ -1216,7 +1217,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         offset, length, doAccessTime, needBlockToken);  
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "open", src, null, null);
     }
     if (checkSafeMode && isInSafeMode()) {
@@ -1301,7 +1302,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getLoginUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "concat", Arrays.toString(srcs), target, null);
       }
       throw e;
@@ -1351,7 +1352,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getLoginUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "concat", Arrays.toString(srcs), target, resultingStat);
     }
   }
@@ -1468,7 +1469,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "setTimes", src, null, null);
       }
       throw e;
@@ -1495,7 +1496,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         if (auditLog.isInfoEnabled() && isExternalInvocation()) {
           final HdfsFileStatus stat = dir.getFileInfo(src, false);
           logAuditEvent(UserGroupInformation.getCurrentUser(),
-                        Server.getRemoteIp(),
+                        getRemoteIp(),
                         "setTimes", src, null, stat);
         }
       } else {
@@ -1517,7 +1518,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "createSymlink", link, target, null);
       }
       throw e;
@@ -1545,7 +1546,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "createSymlink", link, target, resultingStat);
     }
   }
@@ -1601,7 +1602,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "setReplication", src, null, null);
       }
       throw e;
@@ -1637,7 +1638,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     if (isFile && auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "setReplication", src, null, null);
     }
     return isFile;
@@ -1694,7 +1695,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "create", src, null, null);
       }
       throw e;
@@ -1719,7 +1720,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       final HdfsFileStatus stat = dir.getFileInfo(src, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "create", src, null, stat);
     }
   }
@@ -2017,7 +2018,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "append", src, null, null);
       }
       throw e;
@@ -2055,7 +2056,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "append", src, null, null);
     }
     return lb;
@@ -2521,7 +2522,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "rename", src, dst, null);
       }
       throw e;
@@ -2550,7 +2551,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "rename", src, dst, resultingStat);
     }
     return status;
@@ -2610,7 +2611,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       for (Rename option : options) {
         cmd.append(option.value()).append(" ");
       }
-      logAuditEvent(UserGroupInformation.getCurrentUser(), Server.getRemoteIp(),
+      logAuditEvent(UserGroupInformation.getCurrentUser(), getRemoteIp(),
                     cmd.toString(), src, dst, resultingStat);
     }
   }
@@ -2648,7 +2649,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "delete", src, null, null);
       }
       throw e;
@@ -2664,7 +2665,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     boolean status = deleteInternal(src, recursive, true);
     if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "delete", src, null, null);
     }
     return status;
@@ -2802,8 +2803,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   HdfsFileStatus getFileInfo(String src, boolean resolveLink) 
     throws AccessControlException, UnresolvedLinkException,
-           StandbyException {
+           StandbyException, IOException {
+    HdfsFileStatus stat = null;
+
     readLock();
+
     try {
       checkOperation(OperationCategory.READ);
 
@@ -2813,10 +2817,23 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (isPermissionEnabled) {
         checkTraverse(src);
       }
-      return dir.getFileInfo(src, resolveLink);
+      stat = dir.getFileInfo(src, resolveLink);
+    } catch (AccessControlException e) {
+      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+        logAuditEvent(false, UserGroupInformation.getCurrentUser(),
+                      getRemoteIp(),
+                      "getfileinfo", src, null, null);
+      }
+      throw e;
     } finally {
       readUnlock();
     }
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      logAuditEvent(UserGroupInformation.getCurrentUser(),
+                    getRemoteIp(),
+                    "getfileinfo", src, null, null);
+    }
+    return stat;
   }
 
   /**
@@ -2829,7 +2846,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "mkdirs", src, null, null);
       }
       throw e;
@@ -2854,7 +2871,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
       final HdfsFileStatus stat = dir.getFileInfo(src, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
-                    Server.getRemoteIp(),
+                    getRemoteIp(),
                     "mkdirs", src, null, stat);
     }
     return status;
@@ -3295,7 +3312,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } catch (AccessControlException e) {
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "listStatus", src, null, null);
       }
       throw e;
@@ -3319,7 +3336,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         logAuditEvent(UserGroupInformation.getCurrentUser(),
-                      Server.getRemoteIp(),
+                      getRemoteIp(),
                       "listStatus", src, null, null);
       }
       dl = dir.getListing(src, startAfter, needLocation);
@@ -5250,7 +5267,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * RPC call context even if the client exits.
    */
   private boolean isExternalInvocation() {
-    return Server.isRpcInvocation();
+    return Server.isRpcInvocation() || NamenodeWebHdfsMethods.isWebHdfsInvocation();
+  }
+
+  private static InetAddress getRemoteIp() {
+    InetAddress ip = Server.getRemoteIp();
+    if (ip != null) {
+      return ip;
+    }
+    return NamenodeWebHdfsMethods.getRemoteIp();
   }
   
   /**

+ 16 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintStream;
+import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
@@ -92,6 +93,7 @@ import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
 import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
 import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
+import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
@@ -116,9 +118,20 @@ public class NamenodeWebHdfsMethods {
     return REMOTE_ADDRESS.get();
   }
 
-  /** Set the remote client address. */
-  static void setRemoteAddress(String remoteAddress) {
-    REMOTE_ADDRESS.set(remoteAddress);
+  public static InetAddress getRemoteIp() {
+    try {
+      return InetAddress.getByName(getRemoteAddress());
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  /**
+   * Returns true if a WebHdfs request is in progress.  Akin to
+   * {@link Server#isRpcInvocation()}.
+   */
+  public static boolean isWebHdfsInvocation() {
+    return getRemoteAddress() != null;
   }
 
   private @Context ServletContext context;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java

@@ -61,7 +61,7 @@ public class NameNodeAdapter {
   
   public static HdfsFileStatus getFileInfo(NameNode namenode, String src,
       boolean resolveLink) throws AccessControlException, UnresolvedLinkException,
-        StandbyException {
+        StandbyException, IOException {
     return namenode.getNamesystem().getFileInfo(src, resolveLink);
   }
   

+ 119 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java

@@ -32,13 +32,17 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 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.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
@@ -83,6 +87,7 @@ public class TestAuditLogs {
     final long precision = 1L;
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
+    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     util = new DFSTestUtil.Builder().setName("TestAuditAllowed").
         setNumFiles(20).build();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
@@ -115,6 +120,18 @@ public class TestAuditLogs {
     assertTrue("failed to read from file", val > 0);
   }
 
+  /** test that allowed stat puts proper entry in audit log */
+  @Test
+  public void testAuditAllowedStat() throws Exception {
+    final Path file = new Path(fnames[0]);
+    FileSystem userfs = DFSTestUtil.getFileSystemAs(userGroupInfo, conf);
+
+    setupAuditLogs();
+    FileStatus st = userfs.getFileStatus(file);
+    verifyAuditLogs(true);
+    assertTrue("failed to stat file", st != null && st.isFile());
+  }
+
   /** test that denied operation puts proper entry in audit log */
   @Test
   public void testAuditDenied() throws Exception {
@@ -135,6 +152,85 @@ public class TestAuditLogs {
     verifyAuditLogs(false);
   }
 
+  /** test that access via webhdfs puts proper entry in audit log */
+  @Test
+  public void testAuditWebHdfs() throws Exception {
+    final Path file = new Path(fnames[0]);
+
+    fs.setPermission(file, new FsPermission((short)0644));
+    fs.setOwner(file, "root", null);
+
+    setupAuditLogs();
+
+    WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(userGroupInfo, conf);
+    InputStream istream = webfs.open(file);
+    int val = istream.read();
+    istream.close();
+
+    verifyAuditLogsRepeat(true, 3);
+    assertTrue("failed to read from file", val > 0);
+  }
+
+  /** test that stat via webhdfs puts proper entry in audit log */
+  @Test
+  public void testAuditWebHdfsStat() throws Exception {
+    final Path file = new Path(fnames[0]);
+
+    fs.setPermission(file, new FsPermission((short)0644));
+    fs.setOwner(file, "root", null);
+
+    setupAuditLogs();
+
+    WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(userGroupInfo, conf);
+    FileStatus st = webfs.getFileStatus(file);
+
+    verifyAuditLogs(true);
+    assertTrue("failed to stat file", st != null && st.isFile());
+  }
+
+  /** test that access via Hftp puts proper entry in audit log */
+  @Test
+  public void testAuditHftp() throws Exception {
+    final Path file = new Path(fnames[0]);
+
+    final String hftpUri =
+      "hftp://" + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+
+    HftpFileSystem hftpFs = null;
+
+    setupAuditLogs();
+    try {
+      hftpFs = (HftpFileSystem) new Path(hftpUri).getFileSystem(conf);
+      InputStream istream = hftpFs.open(file);
+      int val = istream.read();
+      istream.close();
+
+      verifyAuditLogs(true);
+    } finally {
+      if (hftpFs != null) hftpFs.close();
+    }
+  }
+
+  /** test that denied access via webhdfs puts proper entry in audit log */
+  @Test
+  public void testAuditWebHdfsDenied() throws Exception {
+    final Path file = new Path(fnames[0]);
+
+    fs.setPermission(file, new FsPermission((short)0600));
+    fs.setOwner(file, "root", null);
+
+    setupAuditLogs();
+    try {
+      WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(userGroupInfo, conf);
+      InputStream istream = webfs.open(file);
+      int val = istream.read();
+      fail("open+read must not succeed, got " + val);
+    } catch(AccessControlException E) {
+      System.out.println("got access denied, as expected.");
+    }
+    verifyAuditLogsRepeat(false, 2);
+  }
+
   /** Sets up log4j logger for auditlogs */
   private void setupAuditLogs() throws IOException {
     File file = new File(auditLogFile);
@@ -148,19 +244,34 @@ public class TestAuditLogs {
     logger.addAppender(appender);
   }
 
+  // Ensure audit log has only one entry
   private void verifyAuditLogs(boolean expectSuccess) throws IOException {
+    verifyAuditLogsRepeat(expectSuccess, 1);
+  }
+
+  // Ensure audit log has exactly N entries
+  private void verifyAuditLogsRepeat(boolean expectSuccess, int ndupe)
+      throws IOException {
     // Turn off the logs
     Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
     logger.setLevel(Level.OFF);
     
-    // Ensure audit log has only one entry
     BufferedReader reader = new BufferedReader(new FileReader(auditLogFile));
-    String line = reader.readLine();
-    assertNotNull(line);
-    assertTrue("Expected audit event not found in audit log",
-        auditPattern.matcher(line).matches());
-    assertTrue("Expected success=" + expectSuccess,
-               successPattern.matcher(line).matches() == expectSuccess);
-    assertNull("Unexpected event in audit log", reader.readLine());
+    String line = null;
+    boolean ret = true;
+   
+    try {
+      for (int i = 0; i < ndupe; i++) {
+        line = reader.readLine();
+        assertNotNull(line);
+        assertTrue("Expected audit event not found in audit log",
+            auditPattern.matcher(line).matches());
+        ret &= successPattern.matcher(line).matches();
+      }
+      assertNull("Unexpected event in audit log", reader.readLine());
+      assertTrue("Expected success=" + expectSuccess, ret == expectSuccess);
+    } finally {
+      reader.close();
+    }
   }
 }

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -95,6 +95,12 @@ public class TestFsck {
       "ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\s" + 
       "cmd=fsck\\ssrc=\\/\\sdst=null\\s" + 
       "perm=null");
+  static final Pattern getfileinfoPattern = Pattern.compile(
+      "allowed=.*?\\s" +
+      "ugi=.*?\\s" + 
+      "ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\s" + 
+      "cmd=getfileinfo\\ssrc=\\/\\sdst=null\\s" + 
+      "perm=null");
   
   static final Pattern numCorruptBlocksPattern = Pattern.compile(
       ".*Corrupt blocks:\t\t([0123456789]*).*");
@@ -180,10 +186,14 @@ public class TestFsck {
     Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
     logger.setLevel(Level.OFF);
     
-    // Ensure audit log has only one for FSCK
+    // Audit log should contain one getfileinfo and one fsck
     BufferedReader reader = new BufferedReader(new FileReader(auditLogFile));
     String line = reader.readLine();
     assertNotNull(line);
+    assertTrue("Expected getfileinfo event not found in audit log",
+        getfileinfoPattern.matcher(line).matches());
+    line = reader.readLine();
+    assertNotNull(line);
     assertTrue("Expected fsck event not found in audit log",
         fsckPattern.matcher(line).matches());
     assertNull("Unexpected event in audit log", reader.readLine());

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java

@@ -89,7 +89,6 @@ public class TestWebHdfsDataLocality {
           //set client address to a particular datanode
           final DataNode dn = cluster.getDataNodes().get(i);
           final String ipAddr = dm.getDatanode(dn.getDatanodeId()).getIpAddr();
-          NamenodeWebHdfsMethods.setRemoteAddress(ipAddr);
 
           //The chosen datanode must be the same as the client address
           final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(