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:50:46 UTC

svn commit: r1077744 - in /hadoop/common/branches/branch-0.20-security-patches/src: core/ core/org/apache/hadoop/io/ core/org/apache/hadoop/io/nativeio/ mapred/org/apache/hadoop/mapred/ native/src/org/apache/hadoop/io/nativeio/ native/src/org/apache/ha...

Author: omalley
Date: Fri Mar  4 04:50:46 2011
New Revision: 1077744

URL: http://svn.apache.org/viewvc?rev=1077744&view=rev
Log:
commit 669c5b2151ecb1392df12bbda16b78699d86c33f
Author: Devaraj Das <dd...@yahoo-inc.com>
Date:   Thu Nov 18 13:53:22 2010 -0800

    . Reduces the number of calls to getpwuid_r and getpwgid_r, by implementing a cache in NativeIO.
    
    +++ b/YAHOO-CHANGES.txt
    +Release 0.20.201.7 - unreleased
    +
    +    . Reduces the number of calls to getpwuid_r and
    +    getpwgid_r, by implementing a cache in NativeIO. (ddas)
    +

Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/core/core-default.xml
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/SecureIOUtils.java
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/nativeio/NativeIO.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JvmManager.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapTask.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/SpillRecord.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLog.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLogsTruncater.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskTracker.java
    hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
    hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/security/getGroup.c
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/TestSecureIOUtils.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/core-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/core-default.xml?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/core-default.xml (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/core-default.xml Fri Mar  4 04:50:46 2011
@@ -459,6 +459,12 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.uid.cache.secs</name>
+  <value>14400</value>
+  <description> NativeIO maintains a cache from UID to UserName. This is
+  the timeout for an entry in that cache. </description>
+</property>
 
 
 </configuration>

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/SecureIOUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/SecureIOUtils.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/SecureIOUtils.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/SecureIOUtils.java Fri Mar  4 04:50:46 2011
@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.permission.F
 import org.apache.hadoop.io.nativeio.Errno;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIOException;
-import org.apache.hadoop.io.nativeio.NativeIO.Stat;
 import org.apache.hadoop.security.UserGroupInformation;
 
 /**
@@ -91,31 +90,31 @@ public class SecureIOUtils {
   private final static FileSystem rawFilesystem;
 
   /**
-   * Open the given File for read access, verifying the expected user/group
+   * Open the given File for read access, verifying the expected user
    * constraints.
    * @param f the file that we are trying to open
    * @param expectedOwner the expected user owner for the file
-   * @param expectedGroup the expected group owner for the file
-   * @throws IOException if an IO Error occurred, or the user/group does not 
+   * @throws IOException if an IO Error occurred, or the user does not 
    * match
    */
-  public static FileInputStream openForRead(File f, String expectedOwner, 
-      String expectedGroup) throws IOException {
+  public static FileInputStream openForRead(File f, String expectedOwner) 
+  throws IOException {
+    FileInputStream fis = new FileInputStream(f);
+    if (expectedOwner == null) { //no security checks
+      return fis;
+    }
     if (skipSecurity) {
       // Subject to race conditions but this is the best we can do
       FileStatus status =
         rawFilesystem.getFileStatus(new Path(f.getAbsolutePath()));
-      checkStat(f, status.getOwner(), status.getGroup(),
-          expectedOwner, expectedGroup);
-      return new FileInputStream(f);
+      checkStat(f, status.getOwner(), expectedOwner);
+      return fis;
     }
 
-    FileInputStream fis = new FileInputStream(f);
     boolean success = false;
     try {
-      Stat stat = NativeIO.fstat(fis.getFD());
-      checkStat(f, stat.getOwner(), stat.getGroup(), expectedOwner,
-          expectedGroup);
+      String owner = NativeIO.getOwner(fis.getFD());
+      checkStat(f, owner, expectedOwner);
       success = true;
       return fis;
     } finally {
@@ -172,21 +171,13 @@ public class SecureIOUtils {
     }
   }
 
-  private static void checkStat(File f, String owner, String group, 
-      String expectedOwner, 
-      String expectedGroup) throws IOException {
+  private static void checkStat(File f, String owner, String expectedOwner) throws IOException {
     if (expectedOwner != null &&
         !expectedOwner.equals(owner)) {
       throw new IOException(
         "Owner '" + owner + "' for path " + f + " did not match " +
         "expected owner '" + expectedOwner + "'");
     }
-    if (expectedGroup != null &&
-        !expectedGroup.equals(group)) {
-      throw new IOException(
-        "Group '" + group + "' for path " + f + " did not match " +
-        "expected group '" + expectedGroup + "'");
-    }
   }
 
   /**

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/nativeio/NativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/nativeio/NativeIO.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/nativeio/NativeIO.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/io/nativeio/NativeIO.java Fri Mar  4 04:50:46 2011
@@ -19,7 +19,10 @@ package org.apache.hadoop.io.nativeio;
 
 import java.io.FileDescriptor;
 import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.NativeCodeLoader;
 
 import org.apache.commons.logging.Log;
@@ -73,16 +76,60 @@ public class NativeIO {
   /** Wrapper around open(2) */
   public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
   /** Wrapper around fstat(2) */
+  //TODO: fstat is an old implementation. Doesn't use the cache. This should be 
+  //changed to use the cache.
   public static native Stat fstat(FileDescriptor fd) throws IOException;
+  private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
+  private static native String getUserName(long uid) throws IOException;
   /** Initialize the JNI method ID and class ID cache */
   private static native void initNative();
+  
+  private static class CachedUid {
+    final long timestamp;
+    final String username;
+    public CachedUid(String username, long timestamp) {
+      this.timestamp = timestamp;
+      this.username = username;
+    }
+  }
+  private static final Map<Long, CachedUid> uidCache = 
+    new ConcurrentHashMap<Long, CachedUid>();
+  private static long cacheTimeout;
+  private static boolean initialized = false;
+  
+  public static String getOwner(FileDescriptor fd) throws IOException {
+    ensureInitialized();
+    long uid = getUIDforFDOwnerforOwner(fd);
+    CachedUid cUid = uidCache.get(uid);
+    long now = System.currentTimeMillis();
+    if (cUid != null && (cUid.timestamp + cacheTimeout) > now) {
+      return cUid.username;
+    }
+    String user = getUserName(uid);
+    LOG.info("Got UserName " + user + " for UID " + uid + 
+        " from the native implementation");
+    cUid = new CachedUid(user, now);
+    uidCache.put(uid, cUid);
+    return user;
+  }
+    
+  private synchronized static void ensureInitialized() {
+    if (!initialized) {
+      cacheTimeout = 
+        new Configuration().getLong("hadoop.security.uid.cache.secs", 
+                                     4*60*60) * 1000;
+      LOG.info("Initialized cache for UID to User mapping with a cache" +
+      		" timeout of " + cacheTimeout/1000 + " seconds.");
+      initialized = true;
+    }
+  }
 
 
   /**
    * Result type of the fstat call
    */
   public static class Stat {
-    private String owner, group;
+    private String owner;
     private int mode;
 
     // Mode constants
@@ -102,23 +149,19 @@ public class NativeIO {
     public static final int S_IWUSR = 0000200;  /* write permission, owner */
     public static final int S_IXUSR = 0000100;  /* execute/search permission, owner */
 
-    Stat(String owner, String group, int mode) {
+    Stat(String owner, int mode) {
       this.owner = owner;
-      this.group = group;
       this.mode = mode;
     }
 
     public String toString() {
-      return "Stat(owner='" + owner + "', group='" + group + "'" +
+      return "Stat(owner='" + owner + "'" +
         ", mode=" + mode + ")";
     }
 
     public String getOwner() {
       return owner;
     }
-    public String getGroup() {
-      return group;
-    }
     public int getMode() {
       return mode;
     }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JvmManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JvmManager.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JvmManager.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JvmManager.java Fri Mar  4 04:50:46 2011
@@ -359,10 +359,11 @@ class JvmManager {
         return;
       }
       //*MUST* never reach this
-      throw new RuntimeException("Inconsistent state!!! " +
+      LOG.fatal("Inconsistent state!!! " +
       		"JVM Manager reached an unstable state " +
             "while reaping a JVM for task: " + t.getTask().getTaskID()+
-            " " + getDetails());
+            " " + getDetails() + ". Aborting. ");
+      System.exit(-1);
     }
     
     private String getDetails() {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapTask.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapTask.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapTask.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/MapTask.java Fri Mar  4 04:50:46 2011
@@ -1512,8 +1512,7 @@ class MapTask extends Task {
       // read in paged indices
       for (int i = indexCacheList.size(); i < numSpills; ++i) {
         Path indexFileName = mapOutputFile.getSpillIndexFile(i);
-        indexCacheList.add(new SpillRecord(indexFileName, job, 
-            UserGroupInformation.getCurrentUser().getShortUserName()));
+        indexCacheList.add(new SpillRecord(indexFileName, job, null));
       }
 
       //make correction in the length to include the sequence file header

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/SpillRecord.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/SpillRecord.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/SpillRecord.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/SpillRecord.java Fri Mar  4 04:50:46 2011
@@ -60,7 +60,7 @@ class SpillRecord {
     final FileSystem rfs = FileSystem.getLocal(job).getRaw();
     final DataInputStream in =
       new DataInputStream(SecureIOUtils.openForRead(
-         new File(indexFileName.toUri().getPath()), expectedIndexOwner, null));
+         new File(indexFileName.toUri().getPath()), expectedIndexOwner));
     try {
       final long length = rfs.getFileStatus(indexFileName).getLen();
       final int partitions = (int) length / MAP_OUTPUT_INDEX_RECORD_LENGTH;

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLog.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLog.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLog.java Fri Mar  4 04:50:46 2011
@@ -110,7 +110,7 @@ public class TaskLog {
 
     File indexFile = getIndexFile(taskid, isCleanup);
     BufferedReader fis = new BufferedReader(new InputStreamReader(
-      SecureIOUtils.openForRead(indexFile, obtainLogDirOwner(taskid), null)));
+      SecureIOUtils.openForRead(indexFile, obtainLogDirOwner(taskid))));
     //the format of the index file is
     //LOG_DIR: <the dir where the task logs are really stored>
     //stdout:<start-offset in the stdout file> <length>
@@ -351,7 +351,7 @@ public class TaskLog {
       bytesRemaining = end - start;
       String owner = obtainLogDirOwner(taskid);
       file = SecureIOUtils.openForRead(new File(fileDetail.location, kind.toString()), 
-          owner, null);
+          owner);
       // skip upto start
       long pos = 0;
       while (pos < start) {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLogsTruncater.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLogsTruncater.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLogsTruncater.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskLogsTruncater.java Fri Mar  4 04:50:46 2011
@@ -185,7 +185,7 @@ public class TaskLogsTruncater {
 
       // ////// Open logFile for reading //////
       try {
-        logFileInputStream = SecureIOUtils.openForRead(logFile, owner, null);
+        logFileInputStream = SecureIOUtils.openForRead(logFile, owner);
       } catch (IOException ioe) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Cannot open " + logFile.getAbsolutePath()

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskTracker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskTracker.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskTracker.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/TaskTracker.java Fri Mar  4 04:50:46 2011
@@ -3468,7 +3468,7 @@ public class TaskTracker implements MRCo
          */
         //open the map-output file
         mapOutputIn = SecureIOUtils.openForRead(
-            new File(mapOutputFileName.toUri().getPath()), runAsUserName, null);
+            new File(mapOutputFileName.toUri().getPath()), runAsUserName);
 
         //seek to the correct offset for the reduce
         mapOutputIn.skip(info.startOffset);

Modified: hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c Fri Mar  4 04:50:46 2011
@@ -55,7 +55,7 @@ static void stat_init(JNIEnv *env) {
   PASS_EXCEPTIONS(env);
   stat_clazz = (*env)->NewGlobalRef(env, clazz);
   stat_ctor = (*env)->GetMethodID(env, stat_clazz, "<init>",
-    "(Ljava/lang/String;Ljava/lang/String;I)V");
+    "(Ljava/lang/String;I)V");
 }
 
 static void stat_deinit(JNIEnv *env) {
@@ -153,33 +153,17 @@ Java_org_apache_hadoop_io_nativeio_Nativ
       goto cleanup;
     }
   }
-  assert(pwdp == &pwd);
+  if (rc == 0 && pwdp == NULL) {
+    throw_ioe(env, ENOENT);
+    goto cleanup;
+  }
 
   jstring jstr_username = (*env)->NewStringUTF(env, pwd.pw_name);
   if (jstr_username == NULL) goto cleanup;
 
-  // Grab group
-  struct group grp, *grpp;
-  while ((rc = getgrgid_r(s.st_gid, &grp, pw_buf, pw_buflen, &grpp)) != 0) {
-    if (rc != ERANGE) {
-      throw_ioe(env, rc);
-      goto cleanup;
-    }
-    free(pw_buf);
-    pw_buflen *= 2;
-    if ((pw_buf = malloc(pw_buflen)) == NULL) {
-      THROW(env, "java/lang/OutOfMemoryError", "Couldn't allocate memory for pw buffer");
-      goto cleanup;
-    }
-  }
-  assert(grpp == &grp);
-
-  jstring jstr_groupname = (*env)->NewStringUTF(env, grp.gr_name);
-  PASS_EXCEPTIONS_GOTO(env, cleanup);
-
   // Construct result
   ret = (*env)->NewObject(env, stat_clazz, stat_ctor,
-    jstr_username, jstr_groupname, s.st_mode);
+    jstr_username, s.st_mode);
 
 cleanup:
   if (pw_buf != NULL) free(pw_buf);
@@ -222,6 +206,67 @@ cleanup:
 }
 
 /*
+ * private static native long getUIDforFDOwnerforOwner(FileDescriptor fd);
+ */
+JNIEXPORT jlong JNICALL 
+Java_org_apache_hadoop_io_nativeio_NativeIO_getUIDforFDOwnerforOwner(JNIEnv *env, jclass clazz,
+ jobject fd_object) {
+  int fd = fd_get(env, fd_object);
+  PASS_EXCEPTIONS_GOTO(env, cleanup);
+
+  struct stat s;
+  int rc = fstat(fd, &s);
+  if (rc != 0) {
+    throw_ioe(env, errno);
+    goto cleanup;
+  }
+  return (jlong)(s.st_uid);
+cleanup:
+  return (jlong)(-1);
+}
+
+/*
+ * private static native String getUserName(long uid);
+ */
+JNIEXPORT jstring JNICALL 
+Java_org_apache_hadoop_io_nativeio_NativeIO_getUserName(JNIEnv *env, 
+jclass clazz, jlong uid) {
+   
+  char *pw_buf = NULL;
+  int rc;
+  size_t pw_buflen = get_pw_buflen();
+  if ((pw_buf = malloc(pw_buflen)) == NULL) {
+    THROW(env, "java/lang/OutOfMemoryError", "Couldn't allocate memory for pw buffer");
+    goto cleanup;
+  }
+
+  // Grab username
+  struct passwd pwd, *pwdp;
+  while ((rc = getpwuid_r((uid_t)uid, &pwd, pw_buf, pw_buflen, &pwdp)) != 0) {
+    if (rc != ERANGE) {
+      throw_ioe(env, rc);
+      goto cleanup;
+    }
+    free(pw_buf);
+    pw_buflen *= 2;
+    if ((pw_buf = malloc(pw_buflen)) == NULL) {
+      THROW(env, "java/lang/OutOfMemoryError", "Couldn't allocate memory for pw buffer");
+      goto cleanup;
+    }
+  }
+  if (rc == 0 && pwdp == NULL) {
+    throw_ioe(env, ENOENT);
+    goto cleanup;
+  }
+
+  jstring jstr_username = (*env)->NewStringUTF(env, pwd.pw_name);
+
+cleanup:
+  if (pw_buf != NULL) free(pw_buf);
+  return jstr_username;
+}
+
+/*
  * Throw a java.IO.IOException, generating the message from errno.
  */
 static void throw_ioe(JNIEnv* env, int errnum)

Modified: hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/security/getGroup.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/security/getGroup.c?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/security/getGroup.c (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/native/src/org/apache/hadoop/security/getGroup.c Fri Mar  4 04:50:46 2011
@@ -23,6 +23,7 @@
 #include <string.h>
 #include <stdlib.h>
 
+#define MAX(a, b) (a > b ? a : b)
 /*Helper functions for the JNI implementation of unix group mapping service*/
 
 
@@ -75,7 +76,7 @@ int getGroupIDList(const char *user, int
  */
 int getGroupDetails(gid_t group, char **grpBuf) {
   struct group * grp = NULL;
-  size_t currBufferSize = sysconf(_SC_GETGR_R_SIZE_MAX);
+  size_t currBufferSize = MAX(sysconf(_SC_GETGR_R_SIZE_MAX), 2048);
   *grpBuf = NULL; 
   char *buf = (char*)malloc(sizeof(char) * currBufferSize);
 
@@ -117,7 +118,7 @@ int getGroupDetails(gid_t group, char **
  */
 int getPW(const char *user, char **pwbuf) {
   struct passwd *pwbufp = NULL;
-  size_t currBufferSize = sysconf(_SC_GETPW_R_SIZE_MAX);
+  size_t currBufferSize = MAX(sysconf(_SC_GETPW_R_SIZE_MAX), 2048);
   *pwbuf = NULL;
   char *buf = (char*)malloc(sizeof(char) * currBufferSize);
   

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/TestSecureIOUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/TestSecureIOUtils.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/TestSecureIOUtils.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/TestSecureIOUtils.java Fri Mar  4 04:50:46 2011
@@ -35,7 +35,7 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 
 public class TestSecureIOUtils {
-  private static String realOwner, realGroup; 
+  private static String realOwner; 
   private static final File testFilePath =
       new File(System.getProperty("test.build.data"), "TestSecureIOContext");
 
@@ -50,25 +50,24 @@ public class TestSecureIOUtils {
     FileStatus stat = rawFS.getFileStatus(
       new Path(testFilePath.toString()));
     realOwner = stat.getOwner();
-    realGroup = stat.getGroup();
   }
 
   @Test
   public void testReadUnrestricted() throws IOException {
-    SecureIOUtils.openForRead(testFilePath, null, null).close();
+    SecureIOUtils.openForRead(testFilePath, null).close();
   }
 
   @Test
   public void testReadCorrectlyRestrictedWithSecurity() throws IOException {
     SecureIOUtils
-      .openForRead(testFilePath, realOwner, realGroup).close();
+      .openForRead(testFilePath, realOwner).close();
   }
 
   @Test
   public void testReadIncorrectlyRestrictedWithSecurity() throws IOException {
     try {
       SecureIOUtils
-        .openForRead(testFilePath, "invalidUser", null).close();
+        .openForRead(testFilePath, "invalidUser").close();
       fail("Didn't throw expection for wrong ownership!");
     } catch (IOException ioe) {
       // expected

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java?rev=1077744&r1=1077743&r2=1077744&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java Fri Mar  4 04:50:46 2011
@@ -57,10 +57,19 @@ public class TestNativeIO {
     LOG.info("Stat: " + String.valueOf(stat));
 
     assertEquals(System.getProperty("user.name"), stat.getOwner());
-    assertNotNull(stat.getGroup());
-    assertTrue(!"".equals(stat.getGroup()));
     assertEquals(NativeIO.Stat.S_IFREG, stat.getMode() & NativeIO.Stat.S_IFMT);
   }
+  
+  @Test
+  public void testGetOwner() throws Exception {
+    FileOutputStream fos = new FileOutputStream(
+      new File(TEST_DIR, "testfstat"));
+    String owner = NativeIO.getOwner(fos.getFD());
+    fos.close();
+    LOG.info("Owner: " + owner);
+
+    assertEquals(System.getProperty("user.name"), owner);
+  }
 
   @Test
   public void testFstatClosedFd() throws Exception {