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 cm...@apache.org on 2013/10/29 01:49:22 UTC

svn commit: r1536572 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ src/main/docs/ src/main/java/ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/fs/permission/ src/main/java/org/apache/hadoop/io/ src/main/java/or...

Author: cmccabe
Date: Tue Oct 29 00:49:20 2013
New Revision: 1536572

URL: http://svn.apache.org/r1536572
Log:
Merge HDFS-4949 branch back into trunk

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchedRemoteIterator.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchedRemoteIterator.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HasEnhancedByteBufferAccess.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HasEnhancedByteBufferAccess.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ReadOption.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ReadOption.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ZeroCopyUnavailableException.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ZeroCopyUnavailableException.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdentityHashStore.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdentityHashStore.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestIdentityHashStore.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestIdentityHashStore.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java
      - copied unchanged from r1536569, hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt   (props changed)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/docs/   (props changed)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/   (props changed)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/core/   (props changed)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/CHANGES.txt:r1509426-1536569

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/docs/
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/docs:r1509426-1536569

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java:r1509426-1536569

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java Tue Oct 29 00:49:20 2013
@@ -304,6 +304,13 @@ public class FsPermission implements Wri
   }
 
   /**
+   * Get the default permission for cache pools.
+   */
+  public static FsPermission getCachePoolDefault() {
+    return new FsPermission((short)00755);
+  }
+
+  /**
    * Create a FsPermission from a Unix symbolic permission string
    * @param unixSymbolicPermission e.g. "-rw-rw-rw-"
    */

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java Tue Oct 29 00:49:20 2013
@@ -454,10 +454,7 @@ public class Text extends BinaryComparab
   /** Read a UTF8 encoded string from in
    */
   public static String readString(DataInput in) throws IOException {
-    int length = WritableUtils.readVInt(in);
-    byte [] bytes = new byte[length];
-    in.readFully(bytes, 0, length);
-    return decode(bytes);
+    return readString(in, Integer.MAX_VALUE);
   }
   
   /** Read a UTF8 encoded string with a maximum size

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java Tue Oct 29 00:49:20 2013
@@ -23,6 +23,7 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -145,6 +146,12 @@ public class NativeIO {
       return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded;
     }
 
+    private static void assertCodeLoaded() throws IOException {
+      if (!isAvailable()) {
+        throw new IOException("NativeIO was not loaded");
+      }
+    }
+
     /** Wrapper around open(2) */
     public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
     /** Wrapper around fstat(2) */
@@ -225,6 +232,46 @@ public class NativeIO {
       }
     }
 
+    static native void mlock_native(
+        ByteBuffer buffer, long len) throws NativeIOException;
+    static native void munlock_native(
+        ByteBuffer buffer, long len) throws NativeIOException;
+
+    /**
+     * Locks the provided direct ByteBuffer into memory, preventing it from
+     * swapping out. After a buffer is locked, future accesses will not incur
+     * a page fault.
+     * 
+     * See the mlock(2) man page for more information.
+     * 
+     * @throws NativeIOException
+     */
+    public static void mlock(ByteBuffer buffer, long len)
+        throws IOException {
+      assertCodeLoaded();
+      if (!buffer.isDirect()) {
+        throw new IOException("Cannot mlock a non-direct ByteBuffer");
+      }
+      mlock_native(buffer, len);
+    }
+
+    /**
+     * Unlocks a locked direct ByteBuffer, allowing it to swap out of memory.
+     * This is a no-op if the ByteBuffer was not previously locked.
+     * 
+     * See the munlock(2) man page for more information.
+     * 
+     * @throws NativeIOException
+     */
+    public static void munlock(ByteBuffer buffer, long len)
+        throws IOException {
+      assertCodeLoaded();
+      if (!buffer.isDirect()) {
+        throw new IOException("Cannot munlock a non-direct ByteBuffer");
+      }
+      munlock_native(buffer, len);
+    }
+
     /** Linux only methods used for getOwner() implementation */
     private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
     private static native String getUserName(long uid) throws IOException;
@@ -478,6 +525,20 @@ public class NativeIO {
   /** Initialize the JNI method ID and class ID cache */
   private static native void initNative();
 
+  /**
+   * Get the maximum number of bytes that can be locked into memory at any
+   * given point.
+   *
+   * @return 0 if no bytes can be locked into memory;
+   *         Long.MAX_VALUE if there is no limit;
+   *         The number of bytes that can be locked into memory otherwise.
+   */
+  public static long getMemlockLimit() {
+    return isAvailable() ? getMemlockLimit0() : 0;
+  }
+
+  private static native long getMemlockLimit0();
+  
   private static class CachedUid {
     final long timestamp;
     final String username;

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java Tue Oct 29 00:49:20 2013
@@ -1254,6 +1254,14 @@ public class UserGroupInformation {
     return null;
   }
 
+  public String getPrimaryGroupName() throws IOException {
+    String[] groups = getGroupNames();
+    if (groups.length == 0) {
+      throw new IOException("There is no primary group for UGI " + this);
+    }
+    return groups[0];
+  }
+
   /**
    * Get the user's full principal name.
    * @return the user's full principal name.

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java Tue Oct 29 00:49:20 2013
@@ -18,6 +18,7 @@
 package org.apache.hadoop.util;
 
 import java.util.Comparator;
+import java.util.Iterator;
 import java.util.PriorityQueue;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -235,4 +236,28 @@ public class LightWeightCache<K, E exten
     }
     return removed;
   }
+
+  @Override
+  public Iterator<E> iterator() {
+    final Iterator<E> iter = super.iterator();
+    return new Iterator<E>() {
+      @Override
+      public boolean hasNext() {
+        return iter.hasNext();
+      }
+
+      @Override
+      public E next() {
+        return iter.next();
+      }
+
+      @Override
+      public void remove() {
+        // It would be tricky to support this because LightWeightCache#remove
+        // may evict multiple elements via evictExpiredEntries.
+        throw new UnsupportedOperationException("Remove via iterator is " +
+            "not supported for LightWeightCache");
+      }
+    };
+  }
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java Tue Oct 29 00:49:20 2013
@@ -246,10 +246,10 @@ public class LightWeightGSet<K, E extend
 
   private class SetIterator implements Iterator<E> {
     /** The starting modification for fail-fast. */
-    private final int startModification = modification;
+    private int iterModification = modification;
     /** The current index of the entry array. */
     private int index = -1;
-    /** The next element to return. */
+    private LinkedElement cur = null;
     private LinkedElement next = nextNonemptyEntry();
 
     /** Find the next nonempty entry starting at (index + 1). */
@@ -258,30 +258,51 @@ public class LightWeightGSet<K, E extend
       return index < entries.length? entries[index]: null;
     }
 
+    private void ensureNext() {
+      if (modification != iterModification) {
+        throw new ConcurrentModificationException("modification=" + modification
+            + " != iterModification = " + iterModification);
+      }
+      if (next != null) {
+        return;
+      }
+      if (cur == null) {
+        return;
+      }
+      next = cur.getNext();
+      if (next == null) {
+        next = nextNonemptyEntry();
+      }
+    }
+
     @Override
     public boolean hasNext() {
+      ensureNext();
       return next != null;
     }
 
     @Override
     public E next() {
-      if (modification != startModification) {
-        throw new ConcurrentModificationException("modification=" + modification
-            + " != startModification = " + startModification);
+      ensureNext();
+      if (next == null) {
+        throw new IllegalStateException("There are no more elements");
       }
-
-      final E e = convert(next);
-
-      //find the next element
-      final LinkedElement n = next.getNext();
-      next = n != null? n: nextNonemptyEntry();
-
-      return e;
+      cur = next;
+      next = null;
+      return convert(cur);
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public void remove() {
-      throw new UnsupportedOperationException("Remove is not supported.");
+      ensureNext();
+      if (cur == null) {
+        throw new IllegalStateException("There is no current element " +
+            "to remove");
+      }
+      LightWeightGSet.this.remove((K)cur);
+      iterModification++;
+      cur = null;
     }
   }
   

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java Tue Oct 29 00:49:20 2013
@@ -905,4 +905,81 @@ public class StringUtils {
     }
     return str.toString();
   }
+
+  /**
+   * From a list of command-line arguments, remove both an option and the 
+   * next argument.
+   *
+   * @param name  Name of the option to remove.  Example: -foo.
+   * @param args  List of arguments.
+   * @return      null if the option was not found; the value of the 
+   *              option otherwise.
+   */
+  public static String popOptionWithArgument(String name, List<String> args) {
+    String val = null;
+    for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
+      String cur = iter.next();
+      if (cur.equals("--")) {
+        // stop parsing arguments when you see --
+        break;
+      } else if (cur.equals(name)) {
+        iter.remove();
+        if (!iter.hasNext()) {
+          throw new RuntimeException("option " + name + " requires 1 " +
+              "argument.");
+        }
+        val = iter.next();
+        iter.remove();
+        break;
+      }
+    }
+    return val;
+  }
+  
+  /**
+   * From a list of command-line arguments, remove an option.
+   *
+   * @param name  Name of the option to remove.  Example: -foo.
+   * @param args  List of arguments.
+   * @return      true if the option was found and removed; false otherwise.
+   */
+  public static boolean popOption(String name, List<String> args) {
+    for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
+      String cur = iter.next();
+      if (cur.equals("--")) {
+        // stop parsing arguments when you see --
+        break;
+      } else if (cur.equals(name)) {
+        iter.remove();
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  /**
+   * From a list of command-line arguments, return the first non-option
+   * argument.  Non-option arguments are those which either come after 
+   * a double dash (--) or do not start with a dash.
+   *
+   * @param args  List of arguments.
+   * @return      The first non-option argument, or null if there were none.
+   */
+  public static String popFirstNonOption(List<String> args) {
+    for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
+      String cur = iter.next();
+      if (cur.equals("--")) {
+        if (!iter.hasNext()) {
+          return null;
+        }
+        cur = iter.next();
+        iter.remove();
+        return cur;
+      } else if (!cur.startsWith("-")) {
+        iter.remove();
+        return cur;
+      }
+    }
+    return null;
+  }
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c Tue Oct 29 00:49:20 2013
@@ -16,8 +16,6 @@
  * limitations under the License.
  */
 
-#define _GNU_SOURCE
-
 #include "org_apache_hadoop.h"
 #include "org_apache_hadoop_io_nativeio_NativeIO.h"
 
@@ -28,11 +26,15 @@
 #include <grp.h>
 #include <jni.h>
 #include <pwd.h>
+#include <stdint.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
+#include <sys/mman.h>
+#include <sys/resource.h>
 #include <sys/stat.h>
 #include <sys/syscall.h>
+#include <sys/time.h>
 #include <sys/types.h>
 #include <unistd.h>
 #include "config.h"
@@ -360,6 +362,71 @@ Java_org_apache_hadoop_io_nativeio_Nativ
 #endif
 }
 
+#define CHECK_DIRECT_BUFFER_ADDRESS(buf) \
+  { \
+    if (!buf) { \
+      THROW(env, "java/lang/UnsupportedOperationException", \
+        "JNI access to direct buffers not available"); \
+      return; \
+    } \
+  }
+
+/**
+ * public static native void mlock_native(
+ *   ByteBuffer buffer, long offset);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_mlock_1native(
+  JNIEnv *env, jclass clazz,
+  jobject buffer, jlong len)
+{
+#ifdef UNIX
+  void* buf = (void*)(*env)->GetDirectBufferAddress(env, buffer);
+  PASS_EXCEPTIONS(env);
+
+  if (mlock(buf, len)) {
+    CHECK_DIRECT_BUFFER_ADDRESS(buf);
+    throw_ioe(env, errno);
+  }
+#endif
+
+#ifdef WINDOWS
+  THROW(env, "java/io/IOException",
+    "The function POSIX.mlock_native() is not supported on Windows");
+#endif
+}
+
+/**
+ * public static native void munlock_native(
+ *   ByteBuffer buffer, long offset);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_munlock_1native(
+  JNIEnv *env, jclass clazz,
+  jobject buffer, jlong len)
+{
+#ifdef UNIX
+  void* buf = (void*)(*env)->GetDirectBufferAddress(env, buffer);
+  PASS_EXCEPTIONS(env);
+
+  if (munlock(buf, len)) {
+    CHECK_DIRECT_BUFFER_ADDRESS(buf);
+    throw_ioe(env, errno);
+  }
+#endif
+
+#ifdef WINDOWS
+  THROW(env, "java/io/IOException",
+    "The function POSIX.munlock_native() is not supported on Windows");
+#endif
+}
+
 #ifdef __FreeBSD__
 static int toFreeBSDFlags(int flags)
 {
@@ -924,6 +991,24 @@ done:
 #endif
 }
 
+JNIEXPORT jlong JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_getMemlockLimit0(
+JNIEnv *env, jclass clazz)
+{
+#ifdef WINDOWS
+  return 0;
+#else
+  struct rlimit rlim;
+  int rc = getrlimit(RLIMIT_MEMLOCK, &rlim);
+  if (rc != 0) {
+    throw_ioe(env, errno);
+    return 0;
+  }
+  return (rlim.rlim_cur == RLIM_INFINITY) ?
+    INT64_MAX : rlim.rlim_cur;
+#endif
+}
+
 /**
  * vim: sw=2: ts=2: et:
  */

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/core/
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/core:r1509426-1536569

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java?rev=1536572&r1=1536571&r2=1536572&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java Tue Oct 29 00:49:20 2013
@@ -24,6 +24,9 @@ import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileChannel.MapMode;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -32,6 +35,7 @@ import java.util.List;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+
 import static org.junit.Assume.*;
 import static org.junit.Assert.*;
 
@@ -45,6 +49,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Time;
 
 public class TestNativeIO {
@@ -563,4 +568,60 @@ public class TestNativeIO {
 
     FileUtils.deleteQuietly(TEST_DIR);
   }
+
+  @Test(timeout=10000)
+  public void testMlock() throws Exception {
+    assumeTrue(NativeIO.isAvailable());
+    assumeTrue(Shell.LINUX);
+    final File TEST_FILE = new File(new File(
+        System.getProperty("test.build.data","build/test/data")),
+        "testMlockFile");
+    final int BUF_LEN = 12289;
+    byte buf[] = new byte[BUF_LEN];
+    int bufSum = 0;
+    for (int i = 0; i < buf.length; i++) {
+      buf[i] = (byte)(i % 60);
+      bufSum += buf[i];
+    }
+    FileOutputStream fos = new FileOutputStream(TEST_FILE);
+    try {
+      fos.write(buf);
+      fos.getChannel().force(true);
+    } finally {
+      fos.close();
+    }
+    
+    FileInputStream fis = null;
+    FileChannel channel = null;
+    try {
+      // Map file into memory
+      fis = new FileInputStream(TEST_FILE);
+      channel = fis.getChannel();
+      long fileSize = channel.size();
+      MappedByteBuffer mapbuf = channel.map(MapMode.READ_ONLY, 0, fileSize);
+      // mlock the buffer
+      NativeIO.POSIX.mlock(mapbuf, fileSize);
+      // Read the buffer
+      int sum = 0;
+      for (int i=0; i<fileSize; i++) {
+        sum += mapbuf.get(i);
+      }
+      assertEquals("Expected sums to be equal", bufSum, sum);
+      // munlock the buffer
+      NativeIO.POSIX.munlock(mapbuf, fileSize);
+    } finally {
+      if (channel != null) {
+        channel.close();
+      }
+      if (fis != null) {
+        fis.close();
+      }
+    }
+  }
+
+  @Test(timeout=10000)
+  public void testGetMemlockLimit() throws Exception {
+    assumeTrue(NativeIO.isAvailable());
+    NativeIO.getMemlockLimit();
+  }
 }