ソースを参照

Merge r1569890 through r1573813 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1573821 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 年 前
コミット
9ad8dd1e8a
69 ファイル変更3635 行追加813 行削除
  1. 6 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
  3. 12 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
  4. 7 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/SharedFileDescriptorFactory.java
  5. 14 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocketWatcher.java
  6. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestSharedFileDescriptorFactory.java
  7. 5 13
      hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
  8. 6 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  9. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
  10. 42 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
  11. 121 100
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  12. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
  13. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  14. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  15. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  16. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java
  17. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
  18. 630 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ShortCircuitShm.java
  19. 20 26
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ClientMmap.java
  20. 115 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShm.java
  21. 474 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShmManager.java
  22. 178 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitCache.java
  23. 81 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitReplica.java
  24. 0 302
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitSharedMemorySegment.java
  25. 19 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
  26. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java
  27. 27 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
  28. 28 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  29. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  30. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  31. 143 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  32. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
  33. 332 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
  34. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
  35. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
  36. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
  37. 12 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
  38. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
  39. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  40. 151 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
  41. 27 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
  42. 47 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  43. 92 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
  44. 25 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
  45. 177 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitCache.java
  46. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
  47. 0 104
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitSharedMemorySegment.java
  48. 0 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitShm.java
  49. 16 43
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
  50. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  51. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
  52. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  53. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  54. 8 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  55. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  56. 21 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
  57. 13 0
      hadoop-yarn-project/CHANGES.txt
  58. 97 0
      hadoop-yarn-project/LICENSE.txt
  59. 15 0
      hadoop-yarn-project/NOTICE.txt
  60. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/FileSystemBasedConfigurationProvider.java
  61. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
  62. 8 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  63. 8 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  64. 14 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  65. 8 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  66. 10 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  67. 416 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java
  68. 28 56
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
  69. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml

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

@@ -315,6 +315,9 @@ Release 2.5.0 - UNRELEASED
 
   BUG FIXES 
 
+    HADOOP-10378. Typo in help printed by hdfs dfs -help.
+    (Mit Desai via suresh)
+
 Release 2.4.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -352,6 +355,9 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10285. Admin interface to swap callqueue at runtime. (Chris Li via
     Arpit Agarwal)
 
+    HADOOP-10314. The ls command help still shows outdated 0.16 format.
+    (Rushabh S Shah via kihwal)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java

@@ -48,7 +48,7 @@ public class Count extends FsCommand {
       "Count the number of directories, files and bytes under the paths\n" +
       "that match the specified file pattern.  The output columns are:\n" +
       "DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME or\n" +
-      "QUOTA REMAINING_QUATA SPACE_QUOTA REMAINING_SPACE_QUOTA \n" +
+      "QUOTA REMAINING_QUOTA SPACE_QUOTA REMAINING_SPACE_QUOTA \n" +
       "      DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME";
   
   private boolean showQuotas;

+ 12 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java

@@ -51,18 +51,18 @@ class Ls extends FsCommand {
   public static final String NAME = "ls";
   public static final String USAGE = "[-d] [-h] [-R] [<path> ...]";
   public static final String DESCRIPTION =
-    "List the contents that match the specified file pattern. If\n" + 
-    "path is not specified, the contents of /user/<currentUser>\n" +
-    "will be listed. Directory entries are of the form \n" +
-    "\tdirName (full path) <dir> \n" +
-    "and file entries are of the form \n" + 
-    "\tfileName(full path) <r n> size \n" +
-    "where n is the number of replicas specified for the file \n" + 
-    "and size is the size of the file, in bytes.\n" +
-    "  -d  Directories are listed as plain files.\n" +
-    "  -h  Formats the sizes of files in a human-readable fashion\n" +
-    "      rather than a number of bytes.\n" +
-    "  -R  Recursively list the contents of directories.";
+		    "List the contents that match the specified file pattern. If\n" + 
+		    "path is not specified, the contents of /user/<currentUser>\n" +
+		    "will be listed. Directory entries are of the form \n" +
+		    "\tpermissions - userid groupid size_of_directory(in bytes) modification_date(yyyy-MM-dd HH:mm) directoryName \n" +
+		    "and file entries are of the form \n" + 
+		    "\tpermissions number_of_replicas userid groupid size_of_file(in bytes) modification_date(yyyy-MM-dd HH:mm) fileName \n" +
+		    "  -d  Directories are listed as plain files.\n" +
+		    "  -h  Formats the sizes of files in a human-readable fashion\n" +
+		    "      rather than a number of bytes.\n" +
+		    "  -R  Recursively list the contents of directories.";
+		  
+  
 
   protected static final SimpleDateFormat dateFormat = 
     new SimpleDateFormat("yyyy-MM-dd HH:mm");

+ 7 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/SharedFileDescriptorFactory.java

@@ -66,14 +66,18 @@ public class SharedFileDescriptorFactory {
   /**
    * Create a shared file descriptor which will be both readable and writable.
    *
+   * @param info           Information to include in the path of the 
+   *                         generated descriptor.
    * @param length         The starting file length.
    *
    * @return               The file descriptor, wrapped in a FileInputStream.
    * @throws IOException   If there was an I/O or configuration error creating
-   *                       the descriptor.
+   *                         the descriptor.
    */
-  public FileInputStream createDescriptor(int length) throws IOException {
-    return new FileInputStream(createDescriptor0(prefix, path, length));
+  public FileInputStream createDescriptor(String info, int length)
+      throws IOException {
+    return new FileInputStream(
+        createDescriptor0(prefix + info, path, length));
   }
 
   /**

+ 14 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocketWatcher.java

@@ -81,7 +81,7 @@ public final class DomainSocketWatcher implements Closeable {
    */
   private static native void anchorNative();
 
-  interface Handler {
+  public interface Handler {
     /**
      * Handles an event on a socket.  An event may be the socket becoming
      * readable, or the remote end being closed.
@@ -228,9 +228,9 @@ public final class DomainSocketWatcher implements Closeable {
     if (loadingFailureReason != null) {
       throw new UnsupportedOperationException(loadingFailureReason);
     }
-    notificationSockets = DomainSocket.socketpair();
-    this.interruptCheckPeriodMs = interruptCheckPeriodMs;
     Preconditions.checkArgument(interruptCheckPeriodMs > 0);
+    this.interruptCheckPeriodMs = interruptCheckPeriodMs;
+    notificationSockets = DomainSocket.socketpair();
     watcherThread.start();
   }
 
@@ -241,8 +241,8 @@ public final class DomainSocketWatcher implements Closeable {
    */
   @Override
   public void close() throws IOException {
+    lock.lock();
     try {
-      lock.lock();
       if (closed) return;
       LOG.info(this + ": closing");
       closed = true;
@@ -266,15 +266,17 @@ public final class DomainSocketWatcher implements Closeable {
    *                   called any time after this function is called.
    */
   public void add(DomainSocket sock, Handler handler) {
+    lock.lock();
     try {
-      lock.lock();
       checkNotClosed();
       Entry entry = new Entry(sock, handler);
       try {
         sock.refCount.reference();
-      } catch (ClosedChannelException e) {
-        Preconditions.checkArgument(false,
-            "tried to add a closed DomainSocket to " + this);
+      } catch (ClosedChannelException e1) {
+        // If the socket is already closed before we add it, invoke the
+        // handler immediately.  Then we're done.
+        handler.handle(sock);
+        return;
       }
       toAdd.add(entry);
       kick();
@@ -300,8 +302,8 @@ public final class DomainSocketWatcher implements Closeable {
    * @param sock     The socket to remove.
    */
   public void remove(DomainSocket sock) {
+    lock.lock();
     try {
-      lock.lock();
       checkNotClosed();
       toRemove.put(sock.fd, sock);
       kick();
@@ -328,7 +330,9 @@ public final class DomainSocketWatcher implements Closeable {
     try {
       notificationSockets[0].getOutputStream().write(0);
     } catch (IOException e) {
-      LOG.error(this + ": error writing to notificationSockets[0]", e);
+      if (!closed) {
+        LOG.error(this + ": error writing to notificationSockets[0]", e);
+      }
     }
   }
 

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestSharedFileDescriptorFactory.java

@@ -44,7 +44,8 @@ public class TestSharedFileDescriptorFactory {
     path.mkdirs();
     SharedFileDescriptorFactory factory =
         new SharedFileDescriptorFactory("woot_", path.getAbsolutePath());
-    FileInputStream inStream = factory.createDescriptor(4096);
+    FileInputStream inStream =
+        factory.createDescriptor("testReadAndWrite", 4096);
     FileOutputStream outStream = new FileOutputStream(inStream.getFD());
     outStream.write(101);
     inStream.getChannel().position(0);

+ 5 - 13
hadoop-common-project/hadoop-common/src/test/resources/testConf.xml

@@ -70,23 +70,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*dirName \(full path\) &lt;dir&gt;( )*</expected-output>
+          <expected-output>^( |\t)*permissions - userid groupid size_of_directory\(in bytes\) modification_date\(yyyy-MM-dd HH:mm\) directoryName( )*</expected-output>
         </comparator>
-        <comparator>
+         <comparator>
           <type>RegexpComparator</type>
           <expected-output>^( |\t)*and file entries are of the form( )*</expected-output>
         </comparator>
-        <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^( |\t)*fileName\(full path\) &lt;r n&gt; size( )*</expected-output>
-        </comparator>
-        <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^( |\t)*where n is the number of replicas specified for the file( )*</expected-output>
-        </comparator>
-        <comparator>
+          <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*and size is the size of the file, in bytes.( )*</expected-output>
+          <expected-output>^( |\t)*permissions number_of_replicas userid groupid size_of_file\(in bytes\) modification_date\(yyyy-MM-dd HH:mm\) fileName( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -242,7 +234,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*QUOTA REMAINING_QUATA SPACE_QUOTA REMAINING_SPACE_QUOTA( )*</expected-output>
+          <expected-output>^( |\t)*QUOTA REMAINING_QUOTA SPACE_QUOTA REMAINING_SPACE_QUOTA( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>

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

@@ -370,6 +370,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-4200. Reduce the size of synchronized sections in PacketResponder.
     (suresh)
 
+    HDFS-5950. The DFSClient and DataNode should use shared memory segments to
+    communicate short-circuit information. (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
@@ -514,6 +517,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5956. A file size is multiplied by the replication factor in 'hdfs oiv
     -p FileDistribution' option. (Akira Ajisaka via wheat9)
 
+    HDFS-5866. '-maxSize' and '-step' option fail in OfflineImageViewer.
+    (Akira Ajisaka via wheat9)
+
   BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS
 
     HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9)

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java

@@ -23,7 +23,6 @@ import java.util.EnumSet;
 import org.apache.hadoop.fs.ByteBufferReadable;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.client.ClientMmap;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 
 /**
  * A BlockReader is responsible for reading a single block

+ 42 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -24,6 +24,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
+import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -32,6 +33,8 @@ import org.apache.hadoop.hdfs.client.ShortCircuitCache;
 import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
+import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -410,7 +413,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         setBlock(block).
         setStartOffset(startOffset).
         setShortCircuitReplica(info.getReplica()).
-        setDatanodeID(datanode).
         setVerifyChecksum(verifyChecksum).
         setCachingStrategy(cachingStrategy).
         build();
@@ -438,12 +440,31 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     while (true) {
       curPeer = nextDomainPeer();
       if (curPeer == null) break;
+      if (curPeer.fromCache) remainingCacheTries--;
       DomainPeer peer = (DomainPeer)curPeer.peer;
+      Slot slot = null;
+      ShortCircuitCache cache = clientContext.getShortCircuitCache();
       try {
-        ShortCircuitReplicaInfo info = requestFileDescriptors(peer);
+        MutableBoolean usedPeer = new MutableBoolean(false);
+        slot = cache.allocShmSlot(datanode, peer, usedPeer,
+            new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()),
+            clientName);
+        if (usedPeer.booleanValue()) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": allocShmSlot used up our previous socket " +
+              peer.getDomainSocket() + ".  Allocating a new one...");
+          }
+          curPeer = nextDomainPeer();
+          if (curPeer == null) break;
+          peer = (DomainPeer)curPeer.peer;
+        }
+        ShortCircuitReplicaInfo info = requestFileDescriptors(peer, slot);
         clientContext.getPeerCache().put(datanode, peer);
         return info;
       } catch (IOException e) {
+        if (slot != null) {
+          cache.freeSlot(slot);
+        }
         if (curPeer.fromCache) {
           // Handle an I/O error we got when using a cached socket.
           // These are considered less serious, because the socket may be stale.
@@ -470,16 +491,22 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   /**
    * Request file descriptors from a DomainPeer.
    *
+   * @param peer   The peer to use for communication.
+   * @param slot   If non-null, the shared memory slot to associate with the 
+   *               new ShortCircuitReplica.
+   * 
    * @return  A ShortCircuitReplica object if we could communicate with the
    *          datanode; null, otherwise. 
    * @throws  IOException If we encountered an I/O exception while communicating
    *          with the datanode.
    */
-  private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer)
-        throws IOException {
+  private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
+          Slot slot) throws IOException {
+    ShortCircuitCache cache = clientContext.getShortCircuitCache();
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
-    new Sender(out).requestShortCircuitFds(block, token, 1);
+    SlotId slotId = slot == null ? null : slot.getSlotId();
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1);
     DataInputStream in = new DataInputStream(peer.getInputStream());
     BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
@@ -491,9 +518,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       sock.recvFileInputStreams(fis, buf, 0, buf.length);
       ShortCircuitReplica replica = null;
       try {
-        ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
-        replica = new ShortCircuitReplica(key, fis[0], fis[1],
-            clientContext.getShortCircuitCache(), Time.monotonicNow());
+        ExtendedBlockId key =
+            new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+        replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
+            Time.monotonicNow(), slot);
       } catch (IOException e) {
         // This indicates an error reading from disk, or a format error.  Since
         // it's not a socket communication problem, we return null rather than
@@ -527,8 +555,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       }
       return new ShortCircuitReplicaInfo(new InvalidToken(msg));
     default:
-      LOG.warn(this + "unknown response code " + resp.getStatus() + " while " +
-          "attempting to set up short-circuit access. " + resp.getMessage());
+      LOG.warn(this + ": unknown response code " + resp.getStatus() +
+          " while attempting to set up short-circuit access. " +
+          resp.getMessage());
       clientContext.getDomainSocketFactory()
           .disableShortCircuitForPath(pathInfo.getPath());
       return null;
@@ -565,6 +594,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     while (true) {
       BlockReaderPeer curPeer = nextDomainPeer();
       if (curPeer == null) break;
+      if (curPeer.fromCache) remainingCacheTries--;
       DomainPeer peer = (DomainPeer)curPeer.peer;
       BlockReader blockReader = null;
       try {
@@ -630,6 +660,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       try {
         curPeer = nextTcpPeer();
         if (curPeer == null) break;
+        if (curPeer.fromCache) remainingCacheTries--;
         peer = curPeer.peer;
         blockReader = getRemoteBlockReader(peer);
         return blockReader;
@@ -662,7 +693,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     return null;
   }
 
-  private static class BlockReaderPeer {
+  public static class BlockReaderPeer {
     final Peer peer;
     final boolean fromCache;
     
@@ -681,7 +712,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, true);
       if (peer != null) {
-        remainingCacheTries--;
         if (LOG.isTraceEnabled()) {
           LOG.trace("nextDomainPeer: reusing existing peer " + peer);
         }
@@ -706,7 +736,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, false);
       if (peer != null) {
-        remainingCacheTries--;
         if (LOG.isTraceEnabled()) {
           LOG.trace("nextTcpPeer: reusing existing peer " + peer);
         }

+ 121 - 100
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -17,26 +17,21 @@
  */
 package org.apache.hadoop.hdfs;
 
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.EnumSet;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.client.ClientMmap;
-import org.apache.hadoop.hdfs.client.ShortCircuitCache;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
 import org.apache.hadoop.hdfs.DFSClient.Conf;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.DirectBufferPool;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -70,8 +65,6 @@ class BlockReaderLocal implements BlockReader {
     private String filename;
     private ShortCircuitReplica replica;
     private long dataPos;
-    private DatanodeID datanodeID;
-    private boolean mlocked;
     private ExtendedBlock block;
 
     public Builder(Conf conf) {
@@ -108,16 +101,6 @@ class BlockReaderLocal implements BlockReader {
       return this;
     }
 
-    public Builder setDatanodeID(DatanodeID datanodeID) {
-      this.datanodeID = datanodeID;
-      return this;
-    }
-
-    public Builder setMlocked(boolean mlocked) {
-      this.mlocked = mlocked;
-      return this;
-    }
-
     public Builder setBlock(ExtendedBlock block) {
       this.block = block;
       return this;
@@ -164,20 +147,10 @@ class BlockReaderLocal implements BlockReader {
    */
   private final boolean verifyChecksum;
 
-  /**
-   * If true, this block is mlocked on the DataNode.
-   */
-  private final AtomicBoolean mlocked;
-
   /**
    * Name of the block, for logging purposes.
    */
   private final String filename;
-
-  /**
-   * DataNode which contained this block.
-   */
-  private final DatanodeID datanodeID;
   
   /**
    * Block ID and Block Pool ID.
@@ -220,8 +193,6 @@ class BlockReaderLocal implements BlockReader {
    */
   private int maxReadaheadLength;
 
-  private ClientMmap clientMmap;
-
   /**
    * Buffers data starting at the current dataPos and extending on
    * for dataBuf.limit().
@@ -247,9 +218,7 @@ class BlockReaderLocal implements BlockReader {
     this.checksum = header.getChecksum();
     this.verifyChecksum = builder.verifyChecksum &&
         (this.checksum.getChecksumType().id != DataChecksum.CHECKSUM_NULL);
-    this.mlocked = new AtomicBoolean(builder.mlocked);
     this.filename = builder.filename;
-    this.datanodeID = builder.datanodeID;
     this.block = builder.block;
     this.bytesPerChecksum = checksum.getBytesPerChecksum();
     this.checksumSize = checksum.getChecksumSize();
@@ -380,42 +349,55 @@ class BlockReaderLocal implements BlockReader {
     return total;
   }
 
-  private boolean getCanSkipChecksum() {
-    return (!verifyChecksum) || mlocked.get();
+  private boolean createNoChecksumContext() {
+    if (verifyChecksum) {
+      return replica.addNoChecksumAnchor();
+    } else {
+      return true;
+    }
   }
-  
+
+  private void releaseNoChecksumContext() {
+    if (verifyChecksum) {
+      replica.removeNoChecksumAnchor();
+    }
+  }
+
   @Override
   public synchronized int read(ByteBuffer buf) throws IOException {
-    boolean canSkipChecksum = getCanSkipChecksum();
-    
-    String traceString = null;
-    if (LOG.isTraceEnabled()) {
-      traceString = new StringBuilder().
-          append("read(").
-          append("buf.remaining=").append(buf.remaining()).
-          append(", block=").append(block).
-          append(", filename=").append(filename).
-          append(", canSkipChecksum=").append(canSkipChecksum).
-          append(")").toString();
-      LOG.info(traceString + ": starting");
-    }
-    int nRead;
+    boolean canSkipChecksum = createNoChecksumContext();
     try {
-      if (canSkipChecksum && zeroReadaheadRequested) {
-        nRead = readWithoutBounceBuffer(buf);
-      } else {
-        nRead = readWithBounceBuffer(buf, canSkipChecksum);
+      String traceString = null;
+      if (LOG.isTraceEnabled()) {
+        traceString = new StringBuilder().
+            append("read(").
+            append("buf.remaining=").append(buf.remaining()).
+            append(", block=").append(block).
+            append(", filename=").append(filename).
+            append(", canSkipChecksum=").append(canSkipChecksum).
+            append(")").toString();
+        LOG.info(traceString + ": starting");
+      }
+      int nRead;
+      try {
+        if (canSkipChecksum && zeroReadaheadRequested) {
+          nRead = readWithoutBounceBuffer(buf);
+        } else {
+          nRead = readWithBounceBuffer(buf, canSkipChecksum);
+        }
+      } catch (IOException e) {
+        if (LOG.isTraceEnabled()) {
+          LOG.info(traceString + ": I/O error", e);
+        }
+        throw e;
       }
-    } catch (IOException e) {
       if (LOG.isTraceEnabled()) {
-        LOG.info(traceString + ": I/O error", e);
+        LOG.info(traceString + ": returning " + nRead);
       }
-      throw e;
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.info(traceString + ": returning " + nRead);
+      return nRead;
+    } finally {
+      if (canSkipChecksum) releaseNoChecksumContext();
     }
-    return nRead;
   }
 
   private synchronized int readWithoutBounceBuffer(ByteBuffer buf)
@@ -531,34 +513,38 @@ class BlockReaderLocal implements BlockReader {
   @Override
   public synchronized int read(byte[] arr, int off, int len)
         throws IOException {
-    boolean canSkipChecksum = getCanSkipChecksum();
-    String traceString = null;
-    if (LOG.isTraceEnabled()) {
-      traceString = new StringBuilder().
-          append("read(arr.length=").append(arr.length).
-          append(", off=").append(off).
-          append(", len=").append(len).
-          append(", filename=").append(filename).
-          append(", block=").append(block).
-          append(", canSkipChecksum=").append(canSkipChecksum).
-          append(")").toString();
-      LOG.trace(traceString + ": starting");
-    }
+    boolean canSkipChecksum = createNoChecksumContext();
     int nRead;
     try {
-      if (canSkipChecksum && zeroReadaheadRequested) {
-        nRead = readWithoutBounceBuffer(arr, off, len);
-      } else {
-        nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
+      String traceString = null;
+      if (LOG.isTraceEnabled()) {
+        traceString = new StringBuilder().
+            append("read(arr.length=").append(arr.length).
+            append(", off=").append(off).
+            append(", len=").append(len).
+            append(", filename=").append(filename).
+            append(", block=").append(block).
+            append(", canSkipChecksum=").append(canSkipChecksum).
+            append(")").toString();
+        LOG.trace(traceString + ": starting");
+      }
+      try {
+        if (canSkipChecksum && zeroReadaheadRequested) {
+          nRead = readWithoutBounceBuffer(arr, off, len);
+        } else {
+          nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
+        }
+      } catch (IOException e) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(traceString + ": I/O error", e);
+        }
+        throw e;
       }
-    } catch (IOException e) {
       if (LOG.isTraceEnabled()) {
-        LOG.trace(traceString + ": I/O error", e);
+        LOG.trace(traceString + ": returning " + nRead);
       }
-      throw e;
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(traceString + ": returning " + nRead);
+    } finally {
+      if (canSkipChecksum) releaseNoChecksumContext();
     }
     return nRead;
   }
@@ -648,28 +634,45 @@ class BlockReaderLocal implements BlockReader {
     return true;
   }
 
+  /**
+   * Get or create a memory map for this replica.
+   * 
+   * There are two kinds of ClientMmap objects we could fetch here: one that 
+   * will always read pre-checksummed data, and one that may read data that
+   * hasn't been checksummed.
+   *
+   * If we fetch the former, "safe" kind of ClientMmap, we have to increment
+   * the anchor count on the shared memory slot.  This will tell the DataNode
+   * not to munlock the block until this ClientMmap is closed.
+   * If we fetch the latter, we don't bother with anchoring.
+   *
+   * @param opts     The options to use, such as SKIP_CHECKSUMS.
+   * 
+   * @return         null on failure; the ClientMmap otherwise.
+   */
   @Override
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
-    if ((!opts.contains(ReadOption.SKIP_CHECKSUMS)) &&
-          verifyChecksum && (!mlocked.get())) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("can't get an mmap for " + block + " of " + filename + 
-            " since SKIP_CHECKSUMS was not given, " +
-            "we aren't skipping checksums, and the block is not mlocked.");
+    boolean anchor = verifyChecksum &&
+        (opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
+    if (anchor) {
+      if (!createNoChecksumContext()) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("can't get an mmap for " + block + " of " + filename + 
+              " since SKIP_CHECKSUMS was not given, " +
+              "we aren't skipping checksums, and the block is not mlocked.");
+        }
+        return null;
       }
-      return null;
     }
-    return replica.getOrCreateClientMmap();
-  }
-
-  /**
-   * Set the mlocked state of the BlockReader.
-   * This method does NOT need to be synchronized because mlocked is atomic.
-   *
-   * @param mlocked  the new mlocked state of the BlockReader.
-   */
-  public void setMlocked(boolean mlocked) {
-    this.mlocked.set(mlocked);
+    ClientMmap clientMmap = null;
+    try {
+      clientMmap = replica.getOrCreateClientMmap(anchor);
+    } finally {
+      if ((clientMmap == null) && anchor) {
+        releaseNoChecksumContext();
+      }
+    }
+    return clientMmap;
   }
   
   @VisibleForTesting
@@ -681,4 +684,22 @@ class BlockReaderLocal implements BlockReader {
   int getMaxReadaheadLength() {
     return this.maxReadaheadLength;
   }
+  
+  /**
+   * Make the replica anchorable.  Normally this can only be done by the
+   * DataNode.  This method is only for testing.
+   */
+  @VisibleForTesting
+  void forceAnchorable() {
+    replica.getSlot().makeAnchorable();
+  }
+
+  /**
+   * Make the replica unanchorable.  Normally this can only be done by the
+   * DataNode.  This method is only for testing.
+   */
+  @VisibleForTesting
+  void forceUnanchorable() {
+    replica.getSlot().makeUnanchorable();
+  }
 }

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java

@@ -99,7 +99,8 @@ public class ClientContext {
         conf.shortCircuitMmapCacheSize,
         conf.shortCircuitMmapCacheExpiryMs,
         conf.shortCircuitMmapCacheRetryTimeout,
-        conf.shortCircuitCacheStaleThresholdMs);
+        conf.shortCircuitCacheStaleThresholdMs,
+        conf.shortCircuitSharedMemoryWatcherInterruptCheckMs);
     this.peerCache =
           new PeerCache(conf.socketCacheCapacity, conf.socketCacheExpiry);
     this.useLegacyBlockReaderLocal = conf.useLegacyBlockReaderLocal;
@@ -129,7 +130,9 @@ public class ClientContext {
       append(", useLegacyBlockReaderLocal = ").
       append(conf.useLegacyBlockReaderLocal).
       append(", domainSocketDataTraffic = ").
-      append(conf.domainSocketDataTraffic);
+      append(conf.domainSocketDataTraffic).
+      append(", shortCircuitSharedMemoryWatcherInterruptCheckMs = ").
+      append(conf.shortCircuitSharedMemoryWatcherInterruptCheckMs);
 
     return builder.toString();
   }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -282,6 +282,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     final boolean domainSocketDataTraffic;
     final int shortCircuitStreamsCacheSize;
     final long shortCircuitStreamsCacheExpiryMs; 
+    final int shortCircuitSharedMemoryWatcherInterruptCheckMs;
     
     final int shortCircuitMmapCacheSize;
     final long shortCircuitMmapCacheExpiryMs;
@@ -414,6 +415,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
       shortCircuitCacheStaleThresholdMs = conf.getLong(
           DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
           DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT);
+      shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
+          DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
+          DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
 
       datanodeRestartTimeout = conf.getLong(
           DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -477,6 +477,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_STARTUP_KEY = "dfs.namenode.startup";
   public static final String  DFS_DATANODE_KEYTAB_FILE_KEY = "dfs.datanode.keytab.file";
   public static final String  DFS_DATANODE_USER_NAME_KEY = "dfs.datanode.kerberos.principal";
+  public static final String  DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH = "dfs.datanode.shared.file.descriptor.path";
+  public static final String  DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH_DEFAULT = "/dev/shm";
+  public static final String  DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS = "dfs.short.circuit.shared.memory.watcher.interrupt.check.ms";
+  public static final int     DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT = 60000;
   public static final String  DFS_NAMENODE_KEYTAB_FILE_KEY = "dfs.namenode.keytab.file";
   public static final String  DFS_NAMENODE_USER_NAME_KEY = "dfs.namenode.kerberos.principal";
   public static final String  DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY = "dfs.namenode.kerberos.internal.spnego.principal";

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -39,6 +39,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ByteBufferReadable;
 import org.apache.hadoop.fs.ByteBufferUtil;
@@ -1630,7 +1631,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       success = true;
     } finally {
       if (!success) {
-        clientMmap.unref();
+        IOUtils.closeQuietly(clientMmap);
       }
     }
     return buffer;
@@ -1644,7 +1645,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           "that was not created by this stream, " + buffer);
     }
     if (val instanceof ClientMmap) {
-      ((ClientMmap)val).unref();
+      IOUtils.closeQuietly((ClientMmap)val);
     } else if (val instanceof ByteBufferPool) {
       ((ByteBufferPool)val).putBuffer(buffer);
     }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 
 /**
  * An immutable key which identifies a block.
@@ -34,6 +35,10 @@ final public class ExtendedBlockId {
    */
   private final String bpId;
 
+  public static ExtendedBlockId fromExtendedBlock(ExtendedBlock block) {
+    return new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+  }
+  
   public ExtendedBlockId(long blockId, String bpId) {
     this.blockId = blockId;
     this.bpId = bpId;

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java

@@ -20,9 +20,7 @@ package org.apache.hadoop.hdfs;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.security.UserGroupInformation;
 
 public interface RemotePeerFactory {
   /**

+ 630 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ShortCircuitShm.java

@@ -0,0 +1,630 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Random;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIO.POSIX;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.primitives.Ints;
+
+import sun.misc.Unsafe;
+
+/**
+ * A shared memory segment used to implement short-circuit reads.
+ */
+public class ShortCircuitShm {
+  private static final Log LOG = LogFactory.getLog(ShortCircuitShm.class);
+
+  protected static final int BYTES_PER_SLOT = 64;
+
+  private static final Unsafe unsafe = safetyDance();
+
+  private static Unsafe safetyDance() {
+    try {
+      Field f = Unsafe.class.getDeclaredField("theUnsafe");
+      f.setAccessible(true);
+      return (Unsafe)f.get(null);
+    } catch (Throwable e) {
+      LOG.error("failed to load misc.Unsafe", e);
+    }
+    return null;
+  }
+
+  /**
+   * Calculate the usable size of a shared memory segment.
+   * We round down to a multiple of the slot size and do some validation.
+   *
+   * @param stream The stream we're using.
+   * @return       The usable size of the shared memory segment.
+   */
+  private static int getUsableLength(FileInputStream stream)
+      throws IOException {
+    int intSize = Ints.checkedCast(stream.getChannel().size());
+    int slots = intSize / BYTES_PER_SLOT;
+    if (slots == 0) {
+      throw new IOException("size of shared memory segment was " +
+          intSize + ", but that is not enough to hold even one slot.");
+    }
+    return slots * BYTES_PER_SLOT;
+  }
+
+  /**
+   * Identifies a DfsClientShm.
+   */
+  public static class ShmId implements Comparable<ShmId> {
+    private static final Random random = new Random();
+    private final long hi;
+    private final long lo;
+
+    /**
+     * Generate a random ShmId.
+     * 
+     * We generate ShmIds randomly to prevent a malicious client from
+     * successfully guessing one and using that to interfere with another
+     * client.
+     */
+    public static ShmId createRandom() {
+      return new ShmId(random.nextLong(), random.nextLong());
+    }
+
+    public ShmId(long hi, long lo) {
+      this.hi = hi;
+      this.lo = lo;
+    }
+    
+    public long getHi() {
+      return hi;
+    }
+    
+    public long getLo() {
+      return lo;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if ((o == null) || (o.getClass() != this.getClass())) {
+        return false;
+      }
+      ShmId other = (ShmId)o;
+      return new EqualsBuilder().
+          append(hi, other.hi).
+          append(lo, other.lo).
+          isEquals();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().
+          append(this.hi).
+          append(this.lo).
+          toHashCode();
+    }
+
+    @Override
+    public String toString() {
+      return String.format("%016x%016x", hi, lo);
+    }
+
+    @Override
+    public int compareTo(ShmId other) {
+      return ComparisonChain.start().
+          compare(hi, other.hi).
+          compare(lo, other.lo).
+          result();
+    }
+  };
+
+  /**
+   * Uniquely identifies a slot.
+   */
+  public static class SlotId {
+    private final ShmId shmId;
+    private final int slotIdx;
+    
+    public SlotId(ShmId shmId, int slotIdx) {
+      this.shmId = shmId;
+      this.slotIdx = slotIdx;
+    }
+
+    public ShmId getShmId() {
+      return shmId;
+    }
+
+    public int getSlotIdx() {
+      return slotIdx;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if ((o == null) || (o.getClass() != this.getClass())) {
+        return false;
+      }
+      SlotId other = (SlotId)o;
+      return new EqualsBuilder().
+          append(shmId, other.shmId).
+          append(slotIdx, other.slotIdx).
+          isEquals();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().
+          append(this.shmId).
+          append(this.slotIdx).
+          toHashCode();
+    }
+
+    @Override
+    public String toString() {
+      return String.format("SlotId(%s:%d)", shmId.toString(), slotIdx);
+    }
+  }
+
+  public class SlotIterator implements Iterator<Slot> {
+    int slotIdx = -1;
+
+    @Override
+    public boolean hasNext() {
+      synchronized (ShortCircuitShm.this) {
+        return allocatedSlots.nextSetBit(slotIdx + 1) != -1;
+      }
+    }
+
+    @Override
+    public Slot next() {
+      synchronized (ShortCircuitShm.this) {
+        int nextSlotIdx = allocatedSlots.nextSetBit(slotIdx + 1);
+        if (nextSlotIdx == -1) {
+          throw new NoSuchElementException();
+        }
+        slotIdx = nextSlotIdx;
+        return slots[nextSlotIdx];
+      }
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException("SlotIterator " +
+          "doesn't support removal");
+    }
+  }
+  
+  /**
+   * A slot containing information about a replica.
+   *
+   * The format is:
+   * word 0
+   *   bit 0:32   Slot flags (see below).
+   *   bit 33:63  Anchor count.
+   * word 1:7
+   *   Reserved for future use, such as statistics.
+   *   Padding is also useful for avoiding false sharing.
+   *
+   * Little-endian versus big-endian is not relevant here since both the client
+   * and the server reside on the same computer and use the same orientation.
+   */
+  public class Slot {
+    /**
+     * Flag indicating that the slot is valid.  
+     * 
+     * The DFSClient sets this flag when it allocates a new slot within one of
+     * its shared memory regions.
+     * 
+     * The DataNode clears this flag when the replica associated with this slot
+     * is no longer valid.  The client itself also clears this flag when it
+     * believes that the DataNode is no longer using this slot to communicate.
+     */
+    private static final long VALID_FLAG =          1L<<63;
+
+    /**
+     * Flag indicating that the slot can be anchored.
+     */
+    private static final long ANCHORABLE_FLAG =     1L<<62;
+
+    /**
+     * The slot address in memory.
+     */
+    private final long slotAddress;
+
+    /**
+     * BlockId of the block this slot is used for.
+     */
+    private final ExtendedBlockId blockId;
+
+    Slot(long slotAddress, ExtendedBlockId blockId) {
+      this.slotAddress = slotAddress;
+      this.blockId = blockId;
+    }
+
+    /**
+     * Get the short-circuit memory segment associated with this Slot.
+     *
+     * @return      The enclosing short-circuit memory segment.
+     */
+    public ShortCircuitShm getShm() {
+      return ShortCircuitShm.this;
+    }
+
+    /**
+     * Get the ExtendedBlockId associated with this slot.
+     *
+     * @return      The ExtendedBlockId of this slot.
+     */
+    public ExtendedBlockId getBlockId() {
+      return blockId;
+    }
+
+    /**
+     * Get the SlotId of this slot, containing both shmId and slotIdx.
+     *
+     * @return      The SlotId of this slot.
+     */
+    public SlotId getSlotId() {
+      return new SlotId(getShmId(), getSlotIdx());
+    }
+
+    /**
+     * Get the Slot index.
+     *
+     * @return      The index of this slot.
+     */
+    public int getSlotIdx() {
+      return Ints.checkedCast(
+          (slotAddress - baseAddress) / BYTES_PER_SLOT);
+    }
+
+    private boolean isSet(long flag) {
+      long prev = unsafe.getLongVolatile(null, this.slotAddress);
+      return (prev & flag) != 0;
+    }
+
+    private void setFlag(long flag) {
+      long prev;
+      do {
+        prev = unsafe.getLongVolatile(null, this.slotAddress);
+        if ((prev & flag) != 0) {
+          return;
+        }
+      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
+                  prev, prev | flag));
+    }
+
+    private void clearFlag(long flag) {
+      long prev;
+      do {
+        prev = unsafe.getLongVolatile(null, this.slotAddress);
+        if ((prev & flag) == 0) {
+          return;
+        }
+      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
+                  prev, prev & (~flag)));
+    }
+    
+    public boolean isValid() {
+      return isSet(VALID_FLAG);
+    }
+
+    public void makeValid() {
+      setFlag(VALID_FLAG);
+    }
+
+    public void makeInvalid() {
+      clearFlag(VALID_FLAG);
+    }
+
+    public boolean isAnchorable() {
+      return isSet(ANCHORABLE_FLAG);
+    }
+
+    public void makeAnchorable() {
+      setFlag(ANCHORABLE_FLAG);
+    }
+
+    public void makeUnanchorable() {
+      clearFlag(ANCHORABLE_FLAG);
+    }
+
+    public boolean isAnchored() {
+      long prev = unsafe.getLongVolatile(null, this.slotAddress);
+      if ((prev & VALID_FLAG) == 0) {
+        // Slot is no longer valid.
+        return false;
+      }
+      return ((prev & 0x7fffffff) != 0);
+    }
+
+    /**
+     * Try to add an anchor for a given slot.
+     *
+     * When a slot is anchored, we know that the block it refers to is resident
+     * in memory.
+     *
+     * @return          True if the slot is anchored.
+     */
+    public boolean addAnchor() {
+      long prev;
+      do {
+        prev = unsafe.getLongVolatile(null, this.slotAddress);
+        if ((prev & VALID_FLAG) == 0) {
+          // Slot is no longer valid.
+          return false;
+        }
+        if ((prev & ANCHORABLE_FLAG) == 0) {
+          // Slot can't be anchored right now.
+          return false;
+        }
+        if ((prev & 0x7fffffff) == 0x7fffffff) {
+          // Too many other threads have anchored the slot (2 billion?)
+          return false;
+        }
+      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
+                  prev, prev + 1));
+      return true;
+    }
+
+    /**
+     * Remove an anchor for a given slot.
+     */
+    public void removeAnchor() {
+      long prev;
+      do {
+        prev = unsafe.getLongVolatile(null, this.slotAddress);
+        Preconditions.checkState((prev & 0x7fffffff) != 0,
+            "Tried to remove anchor for slot " + slotAddress +", which was " +
+            "not anchored.");
+      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
+                  prev, prev - 1));
+    }
+
+    @Override
+    public String toString() {
+      return "Slot(slotIdx=" + getSlotIdx() + ", shm=" + getShm() + ")";
+    }
+  }
+
+  /**
+   * ID for this SharedMemorySegment.
+   */
+  private final ShmId shmId;
+
+  /**
+   * The base address of the memory-mapped file.
+   */
+  private final long baseAddress;
+
+  /**
+   * The mmapped length of the shared memory segment
+   */
+  private final int mmappedLength;
+
+  /**
+   * The slots associated with this shared memory segment.
+   * slot[i] contains the slot at offset i * BYTES_PER_SLOT,
+   * or null if that slot is not allocated.
+   */
+  private final Slot slots[];
+
+  /**
+   * A bitset where each bit represents a slot which is in use.
+   */
+  private final BitSet allocatedSlots;
+
+  /**
+   * Create the ShortCircuitShm.
+   * 
+   * @param shmId       The ID to use.
+   * @param stream      The stream that we're going to use to create this 
+   *                    shared memory segment.
+   *                    
+   *                    Although this is a FileInputStream, we are going to
+   *                    assume that the underlying file descriptor is writable
+   *                    as well as readable. It would be more appropriate to use
+   *                    a RandomAccessFile here, but that class does not have
+   *                    any public accessor which returns a FileDescriptor,
+   *                    unlike FileInputStream.
+   */
+  public ShortCircuitShm(ShmId shmId, FileInputStream stream)
+        throws IOException {
+    if (!NativeIO.isAvailable()) {
+      throw new UnsupportedOperationException("NativeIO is not available.");
+    }
+    if (Shell.WINDOWS) {
+      throw new UnsupportedOperationException(
+          "DfsClientShm is not yet implemented for Windows.");
+    }
+    if (unsafe == null) {
+      throw new UnsupportedOperationException(
+          "can't use DfsClientShm because we failed to " +
+          "load misc.Unsafe.");
+    }
+    this.shmId = shmId;
+    this.mmappedLength = getUsableLength(stream);
+    this.baseAddress = POSIX.mmap(stream.getFD(), 
+        POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, mmappedLength);
+    this.slots = new Slot[mmappedLength / BYTES_PER_SLOT];
+    this.allocatedSlots = new BitSet(slots.length);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("creating " + this.getClass().getSimpleName() +
+          "(shmId=" + shmId +
+          ", mmappedLength=" + mmappedLength +
+          ", baseAddress=" + String.format("%x", baseAddress) +
+          ", slots.length=" + slots.length + ")");
+    }
+  }
+
+  public final ShmId getShmId() {
+    return shmId;
+  }
+  
+  /**
+   * Determine if this shared memory object is empty.
+   *
+   * @return    True if the shared memory object is empty.
+   */
+  synchronized final public boolean isEmpty() {
+    return allocatedSlots.nextSetBit(0) == -1;
+  }
+
+  /**
+   * Determine if this shared memory object is full.
+   *
+   * @return    True if the shared memory object is full.
+   */
+  synchronized final public boolean isFull() {
+    return allocatedSlots.nextClearBit(0) >= slots.length;
+  }
+
+  /**
+   * Calculate the base address of a slot.
+   *
+   * @param slotIdx   Index of the slot.
+   * @return          The base address of the slot.
+   */
+  private final long calculateSlotAddress(int slotIdx) {
+    return this.baseAddress + (slotIdx * BYTES_PER_SLOT);
+  }
+
+  /**
+   * Allocate a new slot and register it.
+   *
+   * This function chooses an empty slot, initializes it, and then returns
+   * the relevant Slot object.
+   *
+   * @return    The new slot.
+   */
+  synchronized public final Slot allocAndRegisterSlot(
+      ExtendedBlockId blockId) {
+    int idx = allocatedSlots.nextClearBit(0);
+    if (idx >= slots.length) {
+      throw new RuntimeException(this + ": no more slots are available.");
+    }
+    allocatedSlots.set(idx, true);
+    Slot slot = new Slot(calculateSlotAddress(idx), blockId);
+    slot.makeValid();
+    slots[idx] = slot;
+    if (LOG.isTraceEnabled()) {
+      //LOG.trace(this + ": allocAndRegisterSlot " + idx);
+      LOG.trace(this + ": allocAndRegisterSlot " + idx + ": allocatedSlots=" + allocatedSlots +
+                  StringUtils.getStackTrace(Thread.currentThread()));
+    }
+    return slot;
+  }
+
+  synchronized public final Slot getSlot(int slotIdx)
+      throws InvalidRequestException {
+    if (!allocatedSlots.get(slotIdx)) {
+      throw new InvalidRequestException(this + ": slot " + slotIdx +
+          " does not exist.");
+    }
+    return slots[slotIdx];
+  }
+
+  /**
+   * Register a slot.
+   *
+   * This function looks at a slot which has already been initialized (by
+   * another process), and registers it with us.  Then, it returns the 
+   * relevant Slot object.
+   *
+   * @return    The slot.
+   *
+   * @throws InvalidRequestException
+   *            If the slot index we're trying to allocate has not been
+   *            initialized, or is already in use.
+   */
+  synchronized public final Slot registerSlot(int slotIdx,
+      ExtendedBlockId blockId) throws InvalidRequestException {
+    if (allocatedSlots.get(slotIdx)) {
+      throw new InvalidRequestException(this + ": slot " + slotIdx +
+          " is already in use.");
+    }
+    Slot slot = new Slot(calculateSlotAddress(slotIdx), blockId);
+    if (!slot.isValid()) {
+      throw new InvalidRequestException(this + ": slot " + slotIdx +
+          " has not been allocated.");
+    }
+    slots[slotIdx] = slot;
+    allocatedSlots.set(slotIdx, true);
+    if (LOG.isTraceEnabled()) {
+      //LOG.trace(this + ": registerSlot " + slotIdx);
+      LOG.trace(this + ": registerSlot " + slotIdx + ": allocatedSlots=" + allocatedSlots +
+                  StringUtils.getStackTrace(Thread.currentThread()));
+    }
+    return slot;
+  }
+
+  /**
+   * Unregisters a slot.
+   * 
+   * This doesn't alter the contents of the slot.  It just means
+   *
+   * @param slotIdx  Index of the slot to unregister.
+   */
+  synchronized public final void unregisterSlot(int slotIdx) {
+    Preconditions.checkState(allocatedSlots.get(slotIdx),
+        "tried to unregister slot " + slotIdx + ", which was not registered.");
+    allocatedSlots.set(slotIdx, false);
+    slots[slotIdx] = null;
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": unregisterSlot " + slotIdx);
+    }
+  }
+  
+  /**
+   * Iterate over all allocated slots.
+   * 
+   * Note that this method isn't safe if 
+   *
+   * @return        The slot iterator.
+   */
+  public SlotIterator slotIterator() {
+    return new SlotIterator();
+  }
+
+  public void free() {
+    try {
+      POSIX.munmap(baseAddress, mmappedLength);
+    } catch (IOException e) {
+      LOG.warn(this + ": failed to munmap", e);
+    }
+    LOG.trace(this + ": freed");
+  }
+  
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "(" + shmId + ")";
+  }
+}

+ 20 - 26
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ClientMmap.java

@@ -19,26 +19,23 @@ package org.apache.hadoop.hdfs.client;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 
+import java.io.Closeable;
 import java.nio.MappedByteBuffer;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 /**
- * A memory-mapped region used by an HDFS client.
- * 
- * This class includes a reference count and some other information used by
- * ClientMmapManager to track and cache mmaps.
+ * A reference to a memory-mapped region used by an HDFS client.
  */
 @InterfaceAudience.Private
-public class ClientMmap {
+public class ClientMmap implements Closeable {
   static final Log LOG = LogFactory.getLog(ClientMmap.class);
   
   /**
    * A reference to the block replica which this mmap relates to.
    */
-  private final ShortCircuitReplica replica;
+  private ShortCircuitReplica replica;
   
   /**
    * The java ByteBuffer object.
@@ -46,33 +43,30 @@ public class ClientMmap {
   private final MappedByteBuffer map;
 
   /**
-   * Reference count of this ClientMmap object.
+   * Whether or not this ClientMmap anchors the replica into memory while
+   * it exists.  Closing an anchored ClientMmap unanchors the replica.
    */
-  private final AtomicInteger refCount = new AtomicInteger(1);
+  private final boolean anchored;
 
-  ClientMmap(ShortCircuitReplica replica, MappedByteBuffer map) {
+  ClientMmap(ShortCircuitReplica replica, MappedByteBuffer map,
+      boolean anchored) {
     this.replica = replica;
     this.map = map;
+    this.anchored = anchored;
   }
 
   /**
-   * Increment the reference count.
-   *
-   * @return   The new reference count.
+   * Close the ClientMmap object.
    */
-  void ref() {
-    refCount.addAndGet(1);
-  }
-
-  /**
-   * Decrement the reference count.
-   *
-   * The parent replica gets unreferenced each time the reference count 
-   * of this object goes to 0.
-   */
-  public void unref() {
-    refCount.addAndGet(-1);
-    replica.unref();
+  @Override
+  public void close() {
+    if (replica != null) {
+      if (anchored) {
+        replica.removeNoChecksumAnchor();
+      }
+      replica.unref();
+    }
+    replica = null;
   }
 
   public MappedByteBuffer getMappedByteBuffer() {

+ 115 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShm.java

@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.hdfs.ShortCircuitShm;
+import org.apache.hadoop.hdfs.client.DfsClientShmManager.EndpointShmManager;
+import org.apache.hadoop.hdfs.net.DomainPeer;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.DomainSocketWatcher;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * DfsClientShm is a subclass of ShortCircuitShm which is used by the
+ * DfsClient.
+ * When the UNIX domain socket associated with this shared memory segment
+ * closes unexpectedly, we mark the slots inside this segment as stale.
+ * ShortCircuitReplica objects that contain stale slots are themselves stale,
+ * and will not be used to service new reads or mmap operations.
+ * However, in-progress read or mmap operations will continue to proceed.
+ * Once the last slot is deallocated, the segment can be safely munmapped.
+ */
+public class DfsClientShm extends ShortCircuitShm
+    implements DomainSocketWatcher.Handler {
+  /**
+   * The EndpointShmManager associated with this shared memory segment.
+   */
+  private final EndpointShmManager manager;
+
+  /**
+   * The UNIX domain socket associated with this DfsClientShm.
+   * We rely on the DomainSocketWatcher to close the socket associated with
+   * this DomainPeer when necessary.
+   */
+  private final DomainPeer peer;
+
+  /**
+   * True if this shared memory segment has lost its connection to the
+   * DataNode.
+   *
+   * {@link DfsClientShm#handle} sets this to true.
+   */
+  private boolean stale = false;
+
+  DfsClientShm(ShmId shmId, FileInputStream stream, EndpointShmManager manager,
+      DomainPeer peer) throws IOException {
+    super(shmId, stream);
+    this.manager = manager;
+    this.peer = peer;
+  }
+
+  public EndpointShmManager getEndpointShmManager() {
+    return manager;
+  }
+
+  public DomainPeer getPeer() {
+    return peer;
+  }
+
+  /**
+   * Determine if the shared memory segment is stale.
+   *
+   * This must be called with the DfsClientShmManager lock held.
+   *
+   * @return   True if the shared memory segment is stale.
+   */
+  public synchronized boolean isStale() {
+    return stale;
+  }
+
+  /**
+   * Handle the closure of the UNIX domain socket associated with this shared
+   * memory segment by marking this segment as stale.
+   *
+   * If there are no slots associated with this shared memory segment, it will
+   * be freed immediately in this function.
+   */
+  @Override
+  public boolean handle(DomainSocket sock) {
+    manager.unregisterShm(getShmId());
+    synchronized (this) {
+      Preconditions.checkState(!stale);
+      stale = true;
+      boolean hadSlots = false;
+      for (Iterator<Slot> iter = slotIterator(); iter.hasNext(); ) {
+        Slot slot = iter.next();
+        slot.makeInvalid();
+        hadSlots = true;
+      }
+      if (!hadSlots) {
+        free();
+      }
+    }
+    return true;
+  }
+}

+ 474 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShmManager.java

@@ -0,0 +1,474 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.TreeMap;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.lang.mutable.MutableBoolean;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
+import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.net.DomainPeer;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.DomainSocketWatcher;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Manages short-circuit memory segments for an HDFS client.
+ * 
+ * Clients are responsible for requesting and releasing shared memory segments used
+ * for communicating with the DataNode. The client will try to allocate new slots
+ * in the set of existing segments, falling back to getting a new segment from the
+ * DataNode via {@link DataTransferProtocol#requestShortCircuitFds}.
+ * 
+ * The counterpart to this class on the DataNode is {@link ShortCircuitRegistry}.
+ * See {@link ShortCircuitRegistry} for more information on the communication protocol.
+ */
+@InterfaceAudience.Private
+public class DfsClientShmManager {
+  private static final Log LOG = LogFactory.getLog(DfsClientShmManager.class);
+
+  /**
+   * Manages short-circuit memory segments that pertain to a given DataNode.
+   */
+  class EndpointShmManager {
+    /**
+     * The datanode we're managing.
+     */
+    private final DatanodeInfo datanode;
+
+    /**
+     * Shared memory segments which have no empty slots.
+     *
+     * Protected by the manager lock.
+     */
+    private final TreeMap<ShmId, DfsClientShm> full =
+        new TreeMap<ShmId, DfsClientShm>();
+
+    /**
+     * Shared memory segments which have at least one empty slot.
+     *
+     * Protected by the manager lock.
+     */
+    private final TreeMap<ShmId, DfsClientShm> notFull =
+        new TreeMap<ShmId, DfsClientShm>();
+
+    /**
+     * True if this datanode doesn't support short-circuit shared memory
+     * segments.
+     *
+     * Protected by the manager lock.
+     */
+    private boolean disabled = false;
+
+    /**
+     * True if we're in the process of loading a shared memory segment from
+     * this DataNode.
+     *
+     * Protected by the manager lock.
+     */
+    private boolean loading = false;
+
+    EndpointShmManager (DatanodeInfo datanode) {
+      this.datanode = datanode;
+    }
+
+    /**
+     * Pull a slot out of a preexisting shared memory segment.
+     *
+     * Must be called with the manager lock held.
+     *
+     * @param blockId     The blockId to put inside the Slot object.
+     *
+     * @return            null if none of our shared memory segments contain a
+     *                      free slot; the slot object otherwise.
+     */
+    private Slot allocSlotFromExistingShm(ExtendedBlockId blockId) {
+      if (notFull.isEmpty()) {
+        return null;
+      }
+      Entry<ShmId, DfsClientShm> entry = notFull.firstEntry();
+      DfsClientShm shm = entry.getValue();
+      ShmId shmId = shm.getShmId();
+      Slot slot = shm.allocAndRegisterSlot(blockId);
+      if (shm.isFull()) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + ": pulled the last slot " + slot.getSlotIdx() +
+              " out of " + shm);
+        }
+        DfsClientShm removedShm = notFull.remove(shmId);
+        Preconditions.checkState(removedShm == shm);
+        full.put(shmId, shm);
+      } else {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + ": pulled slot " + slot.getSlotIdx() +
+              " out of " + shm);
+        }
+      }
+      return slot;
+    }
+
+    /**
+     * Ask the DataNode for a new shared memory segment.  This function must be
+     * called with the manager lock held.  We will release the lock while
+     * communicating with the DataNode.
+     *
+     * @param clientName    The current client name.
+     * @param peer          The peer to use to talk to the DataNode.
+     *
+     * @return              Null if the DataNode does not support shared memory
+     *                        segments, or experienced an error creating the
+     *                        shm.  The shared memory segment itself on success.
+     * @throws IOException  If there was an error communicating over the socket.
+     *                        We will not throw an IOException unless the socket
+     *                        itself (or the network) is the problem.
+     */
+    private DfsClientShm requestNewShm(String clientName, DomainPeer peer)
+        throws IOException {
+      final DataOutputStream out = 
+          new DataOutputStream(
+              new BufferedOutputStream(peer.getOutputStream()));
+      new Sender(out).requestShortCircuitShm(clientName);
+      ShortCircuitShmResponseProto resp = 
+          ShortCircuitShmResponseProto.parseFrom(
+              PBHelper.vintPrefixed(peer.getInputStream()));
+      String error = resp.hasError() ? resp.getError() : "(unknown)";
+      switch (resp.getStatus()) {
+      case SUCCESS:
+        DomainSocket sock = peer.getDomainSocket();
+        byte buf[] = new byte[1];
+        FileInputStream fis[] = new FileInputStream[1];
+        if (sock.recvFileInputStreams(fis, buf, 0, buf.length) < 0) {
+          throw new EOFException("got EOF while trying to transfer the " +
+              "file descriptor for the shared memory segment.");
+        }
+        if (fis[0] == null) {
+          throw new IOException("the datanode " + datanode + " failed to " +
+              "pass a file descriptor for the shared memory segment.");
+        }
+        try {
+          DfsClientShm shm = 
+              new DfsClientShm(PBHelper.convert(resp.getId()),
+                  fis[0], this, peer);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": createNewShm: created " + shm);
+          }
+          return shm;
+        } finally {
+          IOUtils.cleanup(LOG,  fis[0]);
+        }
+      case ERROR_UNSUPPORTED:
+        // The DataNode just does not support short-circuit shared memory
+        // access, and we should stop asking.
+        LOG.info(this + ": datanode does not support short-circuit " +
+            "shared memory access: " + error);
+        disabled = true;
+        return null;
+      default:
+        // The datanode experienced some kind of unexpected error when trying to
+        // create the short-circuit shared memory segment.
+        LOG.warn(this + ": error requesting short-circuit shared memory " +
+            "access: " + error);
+        return null;
+      }
+    }
+
+    /**
+     * Allocate a new shared memory slot connected to this datanode.
+     *
+     * Must be called with the EndpointShmManager lock held.
+     *
+     * @param peer          The peer to use to talk to the DataNode.
+     * @param clientName    The client name.
+     * @param usedPeer      (out param) Will be set to true if we used the peer.
+     *                        When a peer is used
+     *
+     * @return              null if the DataNode does not support shared memory
+     *                        segments, or experienced an error creating the
+     *                        shm.  The shared memory segment itself on success.
+     * @throws IOException  If there was an error communicating over the socket.
+     */
+    Slot allocSlot(DomainPeer peer, MutableBoolean usedPeer,
+        String clientName, ExtendedBlockId blockId) throws IOException {
+      while (true) {
+        if (disabled) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": shared memory segment access is disabled.");
+          }
+          return null;
+        }
+        // Try to use an existing slot.
+        Slot slot = allocSlotFromExistingShm(blockId);
+        if (slot != null) {
+          return slot;
+        }
+        // There are no free slots.  If someone is loading more slots, wait
+        // for that to finish.
+        if (loading) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": waiting for loading to finish...");
+          }
+          finishedLoading.awaitUninterruptibly();
+        } else {
+          // Otherwise, load the slot ourselves.
+          loading = true;
+          lock.unlock();
+          DfsClientShm shm;
+          try {
+            shm = requestNewShm(clientName, peer);
+            if (shm == null) continue;
+            // See #{DfsClientShmManager#domainSocketWatcher} for details
+            // about why we do this before retaking the manager lock.
+            domainSocketWatcher.add(peer.getDomainSocket(), shm);
+            // The DomainPeer is now our responsibility, and should not be
+            // closed by the caller.
+            usedPeer.setValue(true);
+          } finally {
+            lock.lock();
+            loading = false;
+            finishedLoading.signalAll();
+          }
+          if (shm.isStale()) {
+            // If the peer closed immediately after the shared memory segment
+            // was created, the DomainSocketWatcher callback might already have
+            // fired and marked the shm as stale.  In this case, we obviously
+            // don't want to add the SharedMemorySegment to our list of valid
+            // not-full segments.
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(this + ": the UNIX domain socket associated with " +
+                  "this short-circuit memory closed before we could make " +
+                  "use of the shm.");
+            }
+          } else {
+            notFull.put(shm.getShmId(), shm);
+          }
+        }
+      }
+    }
+    
+    /**
+     * Stop tracking a slot.
+     *
+     * Must be called with the EndpointShmManager lock held.
+     *
+     * @param slot          The slot to release.
+     */
+    void freeSlot(Slot slot) {
+      DfsClientShm shm = (DfsClientShm)slot.getShm();
+      shm.unregisterSlot(slot.getSlotIdx());
+      if (shm.isStale()) {
+        // Stale shared memory segments should not be tracked here.
+        Preconditions.checkState(!full.containsKey(shm.getShmId()));
+        Preconditions.checkState(!notFull.containsKey(shm.getShmId()));
+        if (shm.isEmpty()) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": freeing empty stale " + shm);
+          }
+          shm.free();
+        }
+      } else {
+        ShmId shmId = shm.getShmId();
+        full.remove(shmId); // The shm can't be full if we just freed a slot.
+        if (shm.isEmpty()) {
+          notFull.remove(shmId);
+  
+          // If the shared memory segment is now empty, we call shutdown(2) on
+          // the UNIX domain socket associated with it.  The DomainSocketWatcher,
+          // which is watching this socket, will call DfsClientShm#handle,
+          // cleaning up this shared memory segment.
+          //
+          // See #{DfsClientShmManager#domainSocketWatcher} for details about why
+          // we don't want to call DomainSocketWatcher#remove directly here.
+          //
+          // Note that we could experience 'fragmentation' here, where the
+          // DFSClient allocates a bunch of slots in different shared memory
+          // segments, and then frees most of them, but never fully empties out
+          // any segment.  We make some attempt to avoid this fragmentation by
+          // always allocating new slots out of the shared memory segment with the
+          // lowest ID, but it could still occur.  In most workloads,
+          // fragmentation should not be a major concern, since it doesn't impact
+          // peak file descriptor usage or the speed of allocation.
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": shutting down UNIX domain socket for " +
+                "empty " + shm);
+          }
+          shutdown(shm);
+        } else {
+          notFull.put(shmId, shm);
+        }
+      }
+    }
+    
+    /**
+     * Unregister a shared memory segment.
+     *
+     * Once a segment is unregistered, we will not allocate any more slots
+     * inside that segment.
+     *
+     * The DomainSocketWatcher calls this while holding the DomainSocketWatcher
+     * lock.
+     *
+     * @param shmId         The ID of the shared memory segment to unregister.
+     */
+    void unregisterShm(ShmId shmId) {
+      lock.lock();
+      try {
+        full.remove(shmId);
+        notFull.remove(shmId);
+      } finally {
+        lock.unlock();
+      }
+    }
+
+    @Override
+    public String toString() {
+      return String.format("EndpointShmManager(%s, parent=%s)",
+          datanode, DfsClientShmManager.this);
+    }
+
+    PerDatanodeVisitorInfo getVisitorInfo() {
+      return new PerDatanodeVisitorInfo(full, notFull, disabled);
+    }
+
+    final void shutdown(DfsClientShm shm) {
+      try {
+        shm.getPeer().getDomainSocket().shutdown();
+      } catch (IOException e) {
+        LOG.warn(this + ": error shutting down shm: got IOException calling " +
+            "shutdown(SHUT_RDWR)", e);
+      }
+    }
+  }
+
+  private final ReentrantLock lock = new ReentrantLock();
+
+  /**
+   * A condition variable which is signalled when we finish loading a segment
+   * from the Datanode.
+   */
+  private final Condition finishedLoading = lock.newCondition();
+
+  /**
+   * Information about each Datanode.
+   */
+  private final HashMap<DatanodeInfo, EndpointShmManager> datanodes =
+      new HashMap<DatanodeInfo, EndpointShmManager>(1);
+  
+  /**
+   * The DomainSocketWatcher which keeps track of the UNIX domain socket
+   * associated with each shared memory segment.
+   *
+   * Note: because the DomainSocketWatcher makes callbacks into this
+   * DfsClientShmManager object, you must MUST NOT attempt to take the
+   * DomainSocketWatcher lock while holding the DfsClientShmManager lock,
+   * or else deadlock might result.   This means that most DomainSocketWatcher
+   * methods are off-limits unless you release the manager lock first.
+   */
+  private final DomainSocketWatcher domainSocketWatcher;
+  
+  DfsClientShmManager(int interruptCheckPeriodMs) throws IOException {
+    this.domainSocketWatcher = new DomainSocketWatcher(interruptCheckPeriodMs);
+  }
+  
+  public Slot allocSlot(DatanodeInfo datanode, DomainPeer peer,
+      MutableBoolean usedPeer, ExtendedBlockId blockId,
+      String clientName) throws IOException {
+    lock.lock();
+    try {
+      EndpointShmManager shmManager = datanodes.get(datanode);
+      if (shmManager == null) {
+        shmManager = new EndpointShmManager(datanode);
+        datanodes.put(datanode, shmManager);
+      }
+      return shmManager.allocSlot(peer, usedPeer, clientName, blockId);
+    } finally {
+      lock.unlock();
+    }
+  }
+  
+  public void freeSlot(Slot slot) {
+    lock.lock();
+    try {
+      DfsClientShm shm = (DfsClientShm)slot.getShm();
+      shm.getEndpointShmManager().freeSlot(slot);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @VisibleForTesting
+  public static class PerDatanodeVisitorInfo {
+    public final TreeMap<ShmId, DfsClientShm> full;
+    public final TreeMap<ShmId, DfsClientShm> notFull;
+    public final boolean disabled;
+
+    PerDatanodeVisitorInfo(TreeMap<ShmId, DfsClientShm> full,
+        TreeMap<ShmId, DfsClientShm> notFull, boolean disabled) {
+      this.full = full;
+      this.notFull = notFull;
+      this.disabled = disabled;
+    }
+  }
+
+  @VisibleForTesting
+  public interface Visitor {
+    void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
+        throws IOException;
+  }
+
+  @VisibleForTesting
+  public void visit(Visitor visitor) throws IOException {
+    lock.lock();
+    try {
+      HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info = 
+          new HashMap<DatanodeInfo, PerDatanodeVisitorInfo>();
+      for (Entry<DatanodeInfo, EndpointShmManager> entry :
+            datanodes.entrySet()) {
+        info.put(entry.getKey(), entry.getValue().getVisitorInfo());
+      }
+      visitor.visit(info);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return String.format("ShortCircuitShmManager(%08x)",
+        System.identityHashCode(this));
+  }
+}

+ 178 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitCache.java

@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hdfs.client;
 
+import java.io.BufferedOutputStream;
 import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 
@@ -33,14 +36,23 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
+import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.net.DomainPeer;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RetriableException;
+import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -154,6 +166,69 @@ public class ShortCircuitCache implements Closeable {
     }
   }
 
+  /**
+   * A task which asks the DataNode to release a short-circuit shared memory
+   * slot.  If successful, this will tell the DataNode to stop monitoring
+   * changes to the mlock status of the replica associated with the slot.
+   * It will also allow us (the client) to re-use this slot for another
+   * replica.  If we can't communicate with the DataNode for some reason,
+   * we tear down the shared memory segment to avoid being in an inconsistent
+   * state.
+   */
+  private class SlotReleaser implements Runnable {
+    /**
+     * The slot that we need to release.
+     */
+    private final Slot slot;
+
+    SlotReleaser(Slot slot) {
+      this.slot = slot;
+    }
+
+    @Override
+    public void run() {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(ShortCircuitCache.this + ": about to release " + slot);
+      }
+      final DfsClientShm shm = (DfsClientShm)slot.getShm();
+      final DomainSocket shmSock = shm.getPeer().getDomainSocket();
+      DomainSocket sock = null;
+      DataOutputStream out = null;
+      final String path = shmSock.getPath();
+      boolean success = false;
+      try {
+        sock = DomainSocket.connect(path);
+        out = new DataOutputStream(
+            new BufferedOutputStream(sock.getOutputStream()));
+        new Sender(out).releaseShortCircuitFds(slot.getSlotId());
+        DataInputStream in = new DataInputStream(sock.getInputStream());
+        ReleaseShortCircuitAccessResponseProto resp =
+            ReleaseShortCircuitAccessResponseProto.parseFrom(
+                PBHelper.vintPrefixed(in));
+        if (resp.getStatus() != Status.SUCCESS) {
+          String error = resp.hasError() ? resp.getError() : "(unknown)";
+          throw new IOException(resp.getStatus().toString() + ": " + error);
+        }
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(ShortCircuitCache.this + ": released " + slot);
+        }
+        success = true;
+      } catch (IOException e) {
+        LOG.error(ShortCircuitCache.this + ": failed to release " +
+            "short-circuit shared memory slot " + slot + " by sending " +
+            "ReleaseShortCircuitAccessRequestProto to " + path +
+            ".  Closing shared memory segment.", e);
+      } finally {
+        if (success) {
+          shmManager.freeSlot(slot);
+        } else {
+          shm.getEndpointShmManager().shutdown(shm);
+        }
+        IOUtils.cleanup(LOG, sock, out);
+      }
+    }
+  }
+
   public interface ShortCircuitReplicaCreator {
     /**
      * Attempt to create a ShortCircuitReplica object.
@@ -173,9 +248,17 @@ public class ShortCircuitCache implements Closeable {
   /**
    * The executor service that runs the cacheCleaner.
    */
-  private final ScheduledThreadPoolExecutor executor
+  private final ScheduledThreadPoolExecutor cleanerExecutor
+  = new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
+          setDaemon(true).setNameFormat("ShortCircuitCache_Cleaner").
+          build());
+
+  /**
+   * The executor service that runs the cacheCleaner.
+   */
+  private final ScheduledThreadPoolExecutor releaserExecutor
       = new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
-          setDaemon(true).setNameFormat("ShortCircuitCache Cleaner").
+          setDaemon(true).setNameFormat("ShortCircuitCache_SlotReleaser").
           build());
 
   /**
@@ -252,6 +335,11 @@ public class ShortCircuitCache implements Closeable {
    */
   private int outstandingMmapCount = 0;
 
+  /**
+   * Manages short-circuit shared memory segments for the client.
+   */
+  private final DfsClientShmManager shmManager;
+
   /**
    * Create a {@link ShortCircuitCache} object from a {@link Configuration}
    */
@@ -268,12 +356,14 @@ public class ShortCircuitCache implements Closeable {
         conf.getLong(DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
             DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT),
         conf.getLong(DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
-            DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT));
+            DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT),
+        conf.getInt(DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
+            DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT));
   }
 
   public ShortCircuitCache(int maxTotalSize, long maxNonMmappedEvictableLifespanMs,
       int maxEvictableMmapedSize, long maxEvictableMmapedLifespanMs,
-      long mmapRetryTimeoutMs, long staleThresholdMs) {
+      long mmapRetryTimeoutMs, long staleThresholdMs, int shmInterruptCheckMs) {
     Preconditions.checkArgument(maxTotalSize >= 0);
     this.maxTotalSize = maxTotalSize;
     Preconditions.checkArgument(maxNonMmappedEvictableLifespanMs >= 0);
@@ -284,6 +374,15 @@ public class ShortCircuitCache implements Closeable {
     this.maxEvictableMmapedLifespanMs = maxEvictableMmapedLifespanMs;
     this.mmapRetryTimeoutMs = mmapRetryTimeoutMs;
     this.staleThresholdMs = staleThresholdMs;
+    DfsClientShmManager shmManager = null;
+    if (shmInterruptCheckMs > 0) {
+      try {
+        shmManager = new DfsClientShmManager(shmInterruptCheckMs);
+      } catch (IOException e) {
+        LOG.error("failed to create ShortCircuitShmManager", e);
+      }
+    }
+    this.shmManager = shmManager;
   }
 
   public long getMmapRetryTimeoutMs() {
@@ -339,7 +438,14 @@ public class ShortCircuitCache implements Closeable {
   void unref(ShortCircuitReplica replica) {
     lock.lock();
     try {
+      // If the replica is stale, but we haven't purged it yet, let's do that.
+      // It would be a shame to evict a non-stale replica so that we could put
+      // a stale one into the cache.
+      if ((!replica.purged) && replica.isStale()) {
+        purge(replica);
+      }
       String addedString = "";
+      boolean shouldTrimEvictionMaps = false;
       int newRefCount = --replica.refCount;
       if (newRefCount == 0) {
         // Close replica, since there are no remaining references to it.
@@ -362,7 +468,7 @@ public class ShortCircuitCache implements Closeable {
             insertEvictable(System.nanoTime(), replica, evictable);
             addedString = "added to evictable, ";
           }
-          trimEvictionMaps();
+          shouldTrimEvictionMaps = true;
         }
       } else {
         Preconditions.checkArgument(replica.refCount >= 0,
@@ -375,6 +481,9 @@ public class ShortCircuitCache implements Closeable {
             (newRefCount + 1) + " -> " + newRefCount +
             StringUtils.getStackTrace(Thread.currentThread()));
       }
+      if (shouldTrimEvictionMaps) {
+        trimEvictionMaps();
+      }
     } finally {
       lock.unlock();
     }
@@ -442,7 +551,7 @@ public class ShortCircuitCache implements Closeable {
        replica = evictable.firstEntry().getValue();
       }
       if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": trimEvictionMaps is purging " +
+        LOG.trace(this + ": trimEvictionMaps is purging " + replica +
           StringUtils.getStackTrace(Thread.currentThread()));
       }
       purge(replica);
@@ -542,7 +651,7 @@ public class ShortCircuitCache implements Closeable {
     }
     if (LOG.isTraceEnabled()) {
       StringBuilder builder = new StringBuilder();
-      builder.append(this).append(": ").append(": removed ").
+      builder.append(this).append(": ").append(": purged ").
           append(replica).append(" from the cache.");
       if (removedFromInfoMap) {
         builder.append("  Removed from the replicaInfoMap.");
@@ -706,7 +815,7 @@ public class ShortCircuitCache implements Closeable {
       cacheCleaner = new CacheCleaner();
       long rateMs = cacheCleaner.getRateInMs();
       ScheduledFuture<?> future =
-          executor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
+          cleanerExecutor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
               TimeUnit.MILLISECONDS);
       cacheCleaner.setFuture(future);
       if (LOG.isDebugEnabled()) {
@@ -716,16 +825,16 @@ public class ShortCircuitCache implements Closeable {
     }
   }
 
-  ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica) {
+  ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica,
+      boolean anchored) {
     Condition newCond;
     lock.lock();
     try {
       while (replica.mmapData != null) {
-        if (replica.mmapData instanceof ClientMmap) {
+        if (replica.mmapData instanceof MappedByteBuffer) {
           ref(replica);
-          ClientMmap clientMmap = (ClientMmap)replica.mmapData;
-          clientMmap.ref();
-          return clientMmap;
+          MappedByteBuffer mmap = (MappedByteBuffer)replica.mmapData;
+          return new ClientMmap(replica, mmap, anchored);
         } else if (replica.mmapData instanceof Long) {
           long lastAttemptTimeMs = (Long)replica.mmapData;
           long delta = Time.monotonicNow() - lastAttemptTimeMs;
@@ -762,12 +871,11 @@ public class ShortCircuitCache implements Closeable {
         newCond.signalAll();
         return null;
       } else {
-        ClientMmap clientMmap = new ClientMmap(replica, map);
         outstandingMmapCount++;
-        replica.mmapData = clientMmap;
+        replica.mmapData = map;
         ref(replica);
         newCond.signalAll();
-        return clientMmap;
+        return new ClientMmap(replica, map, anchored);
       }
     } finally {
       lock.unlock();
@@ -878,4 +986,58 @@ public class ShortCircuitCache implements Closeable {
     return "ShortCircuitCache(0x" +
         Integer.toHexString(System.identityHashCode(this)) + ")";
   }
+
+  /**
+   * Allocate a new shared memory slot.
+   *
+   * @param datanode       The datanode to allocate a shm slot with.
+   * @param peer           A peer connected to the datanode.
+   * @param usedPeer       Will be set to true if we use up the provided peer.
+   * @param blockId        The block id and block pool id of the block we're 
+   *                         allocating this slot for.
+   * @param clientName     The name of the DFSClient allocating the shared
+   *                         memory.
+   * @return               Null if short-circuit shared memory is disabled;
+   *                         a short-circuit memory slot otherwise.
+   * @throws IOException   An exception if there was an error talking to 
+   *                         the datanode.
+   */
+  public Slot allocShmSlot(DatanodeInfo datanode,
+        DomainPeer peer, MutableBoolean usedPeer,
+        ExtendedBlockId blockId, String clientName) throws IOException {
+    if (shmManager != null) {
+      return shmManager.allocSlot(datanode, peer, usedPeer,
+          blockId, clientName);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Free a slot immediately.
+   *
+   * ONLY use this if the DataNode is not yet aware of the slot.
+   * 
+   * @param slot           The slot to free.
+   */
+  public void freeSlot(Slot slot) {
+    Preconditions.checkState(shmManager != null);
+    slot.makeInvalid();
+    shmManager.freeSlot(slot);
+  }
+  
+  /**
+   * Schedule a shared memory slot to be released.
+   *
+   * @param slot           The slot to release.
+   */
+  public void scheduleSlotReleaser(Slot slot) {
+    Preconditions.checkState(shmManager != null);
+    releaserExecutor.execute(new SlotReleaser(slot));
+  }
+
+  @VisibleForTesting
+  public DfsClientShmManager getDfsClientShmManager() {
+    return shmManager;
+  }
 }

+ 81 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitReplica.java

@@ -28,6 +28,7 @@ import java.nio.channels.FileChannel.MapMode;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -80,6 +81,11 @@ public class ShortCircuitReplica {
    */
   private final long creationTimeMs;
 
+  /**
+   * If non-null, the shared memory slot associated with this replica.
+   */
+  private final Slot slot;
+  
   /**
    * Current mmap state.
    *
@@ -114,7 +120,7 @@ public class ShortCircuitReplica {
 
   public ShortCircuitReplica(ExtendedBlockId key,
       FileInputStream dataStream, FileInputStream metaStream,
-      ShortCircuitCache cache, long creationTimeMs) throws IOException {
+      ShortCircuitCache cache, long creationTimeMs, Slot slot) throws IOException {
     this.key = key;
     this.dataStream = dataStream;
     this.metaStream = metaStream;
@@ -126,6 +132,7 @@ public class ShortCircuitReplica {
     }
     this.cache = cache;
     this.creationTimeMs = creationTimeMs;
+    this.slot = slot;
   }
 
   /**
@@ -141,21 +148,61 @@ public class ShortCircuitReplica {
    * Must be called with the cache lock held.
    */
   boolean isStale() {
-    long deltaMs = Time.monotonicNow() - creationTimeMs;
-    long staleThresholdMs = cache.getStaleThresholdMs();
-    if (deltaMs > staleThresholdMs) {
+    if (slot != null) {
+      // Check staleness by looking at the shared memory area we use to
+      // communicate with the DataNode.
+      boolean stale = !slot.isValid();
       if (LOG.isTraceEnabled()) {
-        LOG.trace(this + " is stale because it's " + deltaMs +
-            " ms old, and staleThresholdMs = " + staleThresholdMs);
+        LOG.trace(this + ": checked shared memory segment.  isStale=" + stale);
       }
-      return true;
+      return stale;
     } else {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + " is not stale because it's only " + deltaMs +
-            " ms old, and staleThresholdMs = " + staleThresholdMs);
+      // Fall back to old, time-based staleness method.
+      long deltaMs = Time.monotonicNow() - creationTimeMs;
+      long staleThresholdMs = cache.getStaleThresholdMs();
+      if (deltaMs > staleThresholdMs) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + " is stale because it's " + deltaMs +
+              " ms old, and staleThresholdMs = " + staleThresholdMs);
+        }
+        return true;
+      } else {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + " is not stale because it's only " + deltaMs +
+              " ms old, and staleThresholdMs = " + staleThresholdMs);
+        }
+        return false;
       }
+    }
+  }
+  
+  /**
+   * Try to add a no-checksum anchor to our shared memory slot.
+   *
+   * It is only possible to add this anchor when the block is mlocked on the Datanode.
+   * The DataNode will not munlock the block until the number of no-checksum anchors
+   * for the block reaches zero.
+   * 
+   * This method does not require any synchronization.
+   *
+   * @return     True if we successfully added a no-checksum anchor.
+   */
+  public boolean addNoChecksumAnchor() {
+    if (slot == null) {
       return false;
     }
+    return slot.addAnchor();
+  }
+
+  /**
+   * Remove a no-checksum anchor for our shared memory slot.
+   *
+   * This method does not require any synchronization.
+   */
+  public void removeNoChecksumAnchor() {
+    if (slot != null) {
+      slot.removeAnchor();
+    }
   }
 
   /**
@@ -165,7 +212,7 @@ public class ShortCircuitReplica {
    */
   @VisibleForTesting
   public boolean hasMmap() {
-    return ((mmapData != null) && (mmapData instanceof ClientMmap));
+    return ((mmapData != null) && (mmapData instanceof MappedByteBuffer));
   }
 
   /**
@@ -174,8 +221,8 @@ public class ShortCircuitReplica {
    * Must be called with the cache lock held.
    */
   void munmap() {
-    ClientMmap clientMmap = (ClientMmap)mmapData;
-    NativeIO.POSIX.munmap(clientMmap.getMappedByteBuffer());
+    MappedByteBuffer mmap = (MappedByteBuffer)mmapData;
+    NativeIO.POSIX.munmap(mmap);
     mmapData = null;
   }
 
@@ -186,12 +233,25 @@ public class ShortCircuitReplica {
    * cache or elsewhere.
    */
   void close() {
+    String suffix = "";
+    
     Preconditions.checkState(refCount == 0,
         "tried to close replica with refCount " + refCount + ": " + this);
+    refCount = -1;
     Preconditions.checkState(purged,
         "tried to close unpurged replica " + this);
-    if (hasMmap()) munmap();
+    if (hasMmap()) {
+      munmap();
+      suffix += "  munmapped.";
+    }
     IOUtils.cleanup(LOG, dataStream, metaStream);
+    if (slot != null) {
+      cache.scheduleSlotReleaser(slot);
+      suffix += "  scheduling " + slot + " for later release.";
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("closed " + this + suffix);
+    }
   }
 
   public FileInputStream getDataStream() {
@@ -210,8 +270,8 @@ public class ShortCircuitReplica {
     return key;
   }
 
-  public ClientMmap getOrCreateClientMmap() {
-    return cache.getOrCreateClientMmap(this);
+  public ClientMmap getOrCreateClientMmap(boolean anchor) {
+    return cache.getOrCreateClientMmap(this, anchor);
   }
 
   MappedByteBuffer loadMmapInternal() {
@@ -250,6 +310,11 @@ public class ShortCircuitReplica {
     this.evictableTimeNs = evictableTimeNs;
   }
 
+  @VisibleForTesting
+  public Slot getSlot() {
+    return slot;
+  }
+
   /**
    * Convert the replica to a string for debugging purposes.
    * Note that we can't take the lock here.

+ 0 - 302
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitSharedMemorySegment.java

@@ -1,302 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.client;
-
-import java.io.Closeable;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.lang.reflect.Field;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
-import org.apache.hadoop.io.nativeio.NativeIO.POSIX;
-import org.apache.hadoop.util.CloseableReferenceCount;
-import org.apache.hadoop.util.Shell;
-
-import com.google.common.base.Preconditions;
-import com.google.common.primitives.Ints;
-
-import sun.misc.Unsafe;
-
-public class ShortCircuitSharedMemorySegment implements Closeable {
-  private static final Log LOG =
-    LogFactory.getLog(ShortCircuitSharedMemorySegment.class);
-
-  private static final int BYTES_PER_SLOT = 64;
-
-  private static final Unsafe unsafe;
-
-  static {
-    Unsafe theUnsafe = null;
-    try {
-      Field f = Unsafe.class.getDeclaredField("theUnsafe");
-      f.setAccessible(true);
-      theUnsafe = (Unsafe)f.get(null);
-    } catch (Throwable e) {
-      LOG.error("failed to load misc.Unsafe", e);
-    }
-    unsafe = theUnsafe;
-  }
-
-  /**
-   * A slot containing information about a replica.
-   *
-   * The format is:
-   * word 0
-   *   bit 0:32   Slot flags (see below).
-   *   bit 33:63  Anchor count.
-   * word 1:7
-   *   Reserved for future use, such as statistics.
-   *   Padding is also useful for avoiding false sharing.
-   *
-   * Little-endian versus big-endian is not relevant here since both the client
-   * and the server reside on the same computer and use the same orientation.
-   */
-  public class Slot implements Closeable {
-    /**
-     * Flag indicating that the slot is in use.
-     */
-    private static final long SLOT_IN_USE_FLAG =    1L<<63;
-
-    /**
-     * Flag indicating that the slot can be anchored.
-     */
-    private static final long ANCHORABLE_FLAG =     1L<<62;
-
-    private long slotAddress;
-
-    Slot(long slotAddress) {
-      this.slotAddress = slotAddress;
-    }
-
-    /**
-     * Make a given slot anchorable.
-     */
-    public void makeAnchorable() {
-      Preconditions.checkState(slotAddress != 0,
-          "Called makeAnchorable on a slot that was closed.");
-      long prev;
-      do {
-        prev = unsafe.getLongVolatile(null, this.slotAddress);
-        if ((prev & ANCHORABLE_FLAG) != 0) {
-          return;
-        }
-      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
-                  prev, prev | ANCHORABLE_FLAG));
-    }
-
-    /**
-     * Make a given slot unanchorable.
-     */
-    public void makeUnanchorable() {
-      Preconditions.checkState(slotAddress != 0,
-          "Called makeUnanchorable on a slot that was closed.");
-      long prev;
-      do {
-        prev = unsafe.getLongVolatile(null, this.slotAddress);
-        if ((prev & ANCHORABLE_FLAG) == 0) {
-          return;
-        }
-      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
-                  prev, prev & (~ANCHORABLE_FLAG)));
-    }
-
-    /**
-     * Try to add an anchor for a given slot.
-     *
-     * When a slot is anchored, we know that the block it refers to is resident
-     * in memory.
-     *
-     * @return          True if the slot is anchored.
-     */
-    public boolean addAnchor() {
-      long prev;
-      do {
-        prev = unsafe.getLongVolatile(null, this.slotAddress);
-        if ((prev & 0x7fffffff) == 0x7fffffff) {
-          // Too many other threads have anchored the slot (2 billion?)
-          return false;
-        }
-        if ((prev & ANCHORABLE_FLAG) == 0) {
-          // Slot can't be anchored right now.
-          return false;
-        }
-      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
-                  prev, prev + 1));
-      return true;
-    }
-
-    /**
-     * Remove an anchor for a given slot.
-     */
-    public void removeAnchor() {
-      long prev;
-      do {
-        prev = unsafe.getLongVolatile(null, this.slotAddress);
-        Preconditions.checkState((prev & 0x7fffffff) != 0,
-            "Tried to remove anchor for slot " + slotAddress +", which was " +
-            "not anchored.");
-      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
-                  prev, prev - 1));
-    }
-
-    /**
-     * @return      The index of this slot.
-     */
-    public int getIndex() {
-      Preconditions.checkState(slotAddress != 0);
-      return Ints.checkedCast(
-          (slotAddress - baseAddress) / BYTES_PER_SLOT);
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (slotAddress == 0) return;
-      long prev;
-      do {
-        prev = unsafe.getLongVolatile(null, this.slotAddress);
-        Preconditions.checkState((prev & SLOT_IN_USE_FLAG) != 0,
-            "tried to close slot that wasn't open");
-      } while (!unsafe.compareAndSwapLong(null, this.slotAddress,
-                  prev, 0));
-      slotAddress = 0;
-      if (ShortCircuitSharedMemorySegment.this.refCount.unreference()) {
-        ShortCircuitSharedMemorySegment.this.free();
-      }
-    }
-  }
-
-  /**
-   * The stream that we're going to use to create this shared memory segment.
-   *
-   * Although this is a FileInputStream, we are going to assume that the
-   * underlying file descriptor is writable as well as readable.
-   * It would be more appropriate to use a RandomAccessFile here, but that class
-   * does not have any public accessor which returns a FileDescriptor, unlike
-   * FileInputStream.
-   */
-  private final FileInputStream stream;
-
-  /**
-   * Length of the shared memory segment.
-   */
-  private final int length;
-
-  /**
-   * The base address of the memory-mapped file.
-   */
-  private final long baseAddress;
-
-  /**
-   * Reference count and 'closed' status.
-   */
-  private final CloseableReferenceCount refCount = new CloseableReferenceCount();
-
-  public ShortCircuitSharedMemorySegment(FileInputStream stream)
-        throws IOException {
-    if (!NativeIO.isAvailable()) {
-      throw new UnsupportedOperationException("NativeIO is not available.");
-    }
-    if (Shell.WINDOWS) {
-      throw new UnsupportedOperationException(
-          "ShortCircuitSharedMemorySegment is not yet implemented " +
-          "for Windows.");
-    }
-    if (unsafe == null) {
-      throw new UnsupportedOperationException(
-          "can't use ShortCircuitSharedMemorySegment because we failed to " +
-          "load misc.Unsafe.");
-    }
-    this.refCount.reference();
-    this.stream = stream;
-    this.length = getEffectiveLength(stream);
-    this.baseAddress = POSIX.mmap(this.stream.getFD(), 
-      POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, this.length);
-  }
-
-  /**
-   * Calculate the effective usable size of the shared memory segment.
-   * We round down to a multiple of the slot size and do some validation.
-   *
-   * @param stream The stream we're using.
-   * @return       The effective usable size of the shared memory segment.
-   */
-  private static int getEffectiveLength(FileInputStream stream)
-      throws IOException {
-    int intSize = Ints.checkedCast(stream.getChannel().size());
-    int slots = intSize / BYTES_PER_SLOT;
-    Preconditions.checkState(slots > 0, "size of shared memory segment was " +
-        intSize + ", but that is not enough to hold even one slot.");
-    return slots * BYTES_PER_SLOT;
-  }
-
-  private boolean allocateSlot(long address) {
-    long prev;
-    do {
-      prev = unsafe.getLongVolatile(null, address);
-      if ((prev & Slot.SLOT_IN_USE_FLAG) != 0) {
-        return false;
-      }
-    } while (!unsafe.compareAndSwapLong(null, address,
-                prev, prev | Slot.SLOT_IN_USE_FLAG));
-    return true;
-  }
-
-  /**
-   * Allocate a new Slot in this shared memory segment.
-   *
-   * @return        A newly allocated Slot, or null if there were no available
-   *                slots.
-   */
-  public Slot allocateNextSlot() throws IOException {
-    ShortCircuitSharedMemorySegment.this.refCount.reference();
-    Slot slot = null;
-    try {
-      final int numSlots = length / BYTES_PER_SLOT;
-      for (int i = 0; i < numSlots; i++) {
-        long address = this.baseAddress + (i * BYTES_PER_SLOT);
-        if (allocateSlot(address)) {
-          slot = new Slot(address);
-          break;
-        }
-      }
-    } finally {
-      if (slot == null) {
-        if (refCount.unreference()) {
-          free();
-        }
-      }
-    }
-    return slot;
-  }
-
-  @Override
-  public void close() throws IOException {
-    refCount.setClosed();
-    if (refCount.unreference()) {
-      free();
-    }
-  }
-
-  void free() throws IOException {
-    IOUtils.cleanup(LOG, stream);
-    POSIX.munmap(baseAddress, length);
-  }
-}

+ 19 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java

@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -116,13 +117,29 @@ public interface DataTransferProtocol {
    *
    * @param blk             The block to get file descriptors for.
    * @param blockToken      Security token for accessing the block.
+   * @param slotId          The shared memory slot id to use, or null 
+   *                          to use no slot id.
    * @param maxVersion      Maximum version of the block data the client 
-   *                        can understand.
+   *                          can understand.
    */
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      int maxVersion) throws IOException;
+      SlotId slotId, int maxVersion) throws IOException;
 
+  /**
+   * Release a pair of short-circuit FDs requested earlier.
+   *
+   * @param slotId          SlotID used by the earlier file descriptors.
+   */
+  public void releaseShortCircuitFds(final SlotId slotId) throws IOException;
+
+  /**
+   * Request a short circuit shared memory area from a DataNode.
+   * 
+   * @pram clientName       The name of the client.
+   */
+  public void requestShortCircuitShm(String clientName) throws IOException;
+  
   /**
    * Receive a block from a source datanode
    * and then notifies the namenode

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java

@@ -35,7 +35,9 @@ public enum Op {
   COPY_BLOCK((byte)84),
   BLOCK_CHECKSUM((byte)85),
   TRANSFER_BLOCK((byte)86),
-  REQUEST_SHORT_CIRCUIT_FDS((byte)87);
+  REQUEST_SHORT_CIRCUIT_FDS((byte)87),
+  RELEASE_SHORT_CIRCUIT_FDS((byte)88),
+  REQUEST_SHORT_CIRCUIT_SHM((byte)89);
 
   /** The code for this operation. */
   public final byte code;

+ 27 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java

@@ -25,6 +25,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
@@ -33,6 +34,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmRequestProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 
@@ -82,6 +85,12 @@ public abstract class Receiver implements DataTransferProtocol {
     case REQUEST_SHORT_CIRCUIT_FDS:
       opRequestShortCircuitFds(in);
       break;
+    case RELEASE_SHORT_CIRCUIT_FDS:
+      opReleaseShortCircuitFds(in);
+      break;
+    case REQUEST_SHORT_CIRCUIT_SHM:
+      opRequestShortCircuitShm(in);
+      break;
     default:
       throw new IOException("Unknown op " + op + " in data stream");
     }
@@ -141,9 +150,26 @@ public abstract class Receiver implements DataTransferProtocol {
   private void opRequestShortCircuitFds(DataInputStream in) throws IOException {
     final OpRequestShortCircuitAccessProto proto =
       OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in));
+    SlotId slotId = (proto.hasSlotId()) ? 
+        PBHelper.convert(proto.getSlotId()) : null;
     requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
         PBHelper.convert(proto.getHeader().getToken()),
-        proto.getMaxVersion());
+        slotId, proto.getMaxVersion());
+  }
+
+  /** Receive {@link Op#RELEASE_SHORT_CIRCUIT_FDS} */
+  private void opReleaseShortCircuitFds(DataInputStream in)
+      throws IOException {
+    final ReleaseShortCircuitAccessRequestProto proto =
+      ReleaseShortCircuitAccessRequestProto.parseFrom(vintPrefixed(in));
+    releaseShortCircuitFds(PBHelper.convert(proto.getSlotId()));
+  }
+
+  /** Receive {@link Op#REQUEST_SHORT_CIRCUIT_SHM} */
+  private void opRequestShortCircuitShm(DataInputStream in) throws IOException {
+    final ShortCircuitShmRequestProto proto =
+        ShortCircuitShmRequestProto.parseFrom(vintPrefixed(in));
+    requestShortCircuitShm(proto.getClientName());
   }
 
   /** Receive OP_REPLACE_BLOCK */

+ 28 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java

@@ -25,6 +25,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
@@ -37,6 +38,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmRequestProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
@@ -161,14 +164,36 @@ public class Sender implements DataTransferProtocol {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      int maxVersion) throws IOException {
-    OpRequestShortCircuitAccessProto proto =
+      SlotId slotId, int maxVersion) throws IOException {
+    OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
           .setHeader(DataTransferProtoUtil.buildBaseHeader(
-            blk, blockToken)).setMaxVersion(maxVersion).build();
+            blk, blockToken)).setMaxVersion(maxVersion);
+    if (slotId != null) {
+      builder.setSlotId(PBHelper.convert(slotId));
+    }
+    OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }
   
+  @Override
+  public void releaseShortCircuitFds(SlotId slotId) throws IOException {
+    ReleaseShortCircuitAccessRequestProto proto = 
+        ReleaseShortCircuitAccessRequestProto.newBuilder().
+        setSlotId(PBHelper.convert(slotId)).
+        build();
+    send(out, Op.RELEASE_SHORT_CIRCUIT_FDS, proto);
+  }
+
+  @Override
+  public void requestShortCircuitShm(String clientName) throws IOException {
+    ShortCircuitShmRequestProto proto =
+        ShortCircuitShmRequestProto.newBuilder().
+        setClientName(clientName).
+        build();
+    send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);
+  }
+  
   @Override
   public void replaceBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -41,6 +41,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
+import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -91,6 +93,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsS
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@@ -2055,5 +2059,29 @@ public class PBHelper {
         .addAllEntries(convertAclEntryProto(e.getEntries())).build();
     return GetAclStatusResponseProto.newBuilder().setResult(r).build();
   }
+
+  public static ShortCircuitShmSlotProto convert(SlotId slotId) {
+    return ShortCircuitShmSlotProto.newBuilder().
+        setShmId(convert(slotId.getShmId())).
+        setSlotIdx(slotId.getSlotIdx()).
+        build();
+  }
+
+  public static ShortCircuitShmIdProto convert(ShmId shmId) {
+    return ShortCircuitShmIdProto.newBuilder().
+        setHi(shmId.getHi()).
+        setLo(shmId.getLo()).
+        build();
+
+  }
+
+  public static SlotId convert(ShortCircuitShmSlotProto slotId) {
+    return new SlotId(PBHelper.convert(slotId.getShmId()),
+        slotId.getSlotIdx());
+  }
+
+  public static ShmId convert(ShortCircuitShmIdProto shmId) {
+    return new ShmId(shmId.getHi(), shmId.getLo());
+  }
 }
 

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -185,6 +185,7 @@ public class DataNode extends Configured
   AtomicInteger xmitsInProgress = new AtomicInteger();
   Daemon dataXceiverServer = null;
   Daemon localDataXceiverServer = null;
+  ShortCircuitRegistry shortCircuitRegistry = null;
   ThreadGroup threadGroup = null;
   private DNConf dnConf;
   private volatile boolean heartbeatsDisabledForTests = false;
@@ -540,6 +541,7 @@ public class DataNode extends Configured
             domainPeerServer.getBindPath());
       }
     }
+    this.shortCircuitRegistry = new ShortCircuitRegistry(conf);
   }
 
   static DomainPeerServer getDomainPeerServer(Configuration conf,
@@ -1304,6 +1306,7 @@ public class DataNode extends Configured
       MBeans.unregister(dataNodeInfoBeanName);
       dataNodeInfoBeanName = null;
     }
+    if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown();
     LOG.info("Shutdown complete.");
     synchronized(this) {
       // it is already false, but setting it again to avoid a findbug warning.
@@ -1957,7 +1960,8 @@ public class DataNode extends Configured
    * 
    * @return the fsdataset that stores the blocks
    */
-  FsDatasetSpi<?> getFSDataset() {
+  @VisibleForTesting
+  public FsDatasetSpi<?> getFSDataset() {
     return data;
   }
 
@@ -2568,4 +2572,8 @@ public class DataNode extends Configured
   DataStorage getStorage() {
     return storage;
   }
+
+  public ShortCircuitRegistry getShortCircuitRegistry() {
+    return shortCircuitRegistry;
+  }
 }

+ 143 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_ACCESS_TOKEN;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
@@ -42,6 +43,9 @@ import java.nio.channels.ClosedChannelException;
 import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -58,6 +62,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -65,11 +71,13 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException;
 import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.NewShmInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -84,7 +92,7 @@ class DataXceiver extends Receiver implements Runnable {
   public static final Log LOG = DataNode.LOG;
   static final Log ClientTraceLog = DataNode.ClientTraceLog;
   
-  private final Peer peer;
+  private Peer peer;
   private final String remoteAddress; // address of remote side
   private final String localAddress;  // local address of this daemon
   private final DataNode datanode;
@@ -220,7 +228,8 @@ class DataXceiver extends Receiver implements Runnable {
         opStartTime = now();
         processOp(op);
         ++opsProcessed;
-      } while (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0);
+      } while ((peer != null) &&
+          (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0));
     } catch (Throwable t) {
       LOG.error(datanode.getDisplayName() + ":DataXceiver error processing " +
                 ((op == null) ? "unknown" : op.name()) + " operation " +
@@ -232,15 +241,17 @@ class DataXceiver extends Receiver implements Runnable {
             + datanode.getXceiverCount());
       }
       updateCurrentThreadName("Cleaning up");
-      dataXceiverServer.closePeer(peer);
-      IOUtils.closeStream(in);
+      if (peer != null) {
+        dataXceiverServer.closePeer(peer);
+        IOUtils.closeStream(in);
+      }
     }
   }
 
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> token,
-      int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion) throws IOException {
     updateCurrentThreadName("Passing file descriptors for block " + blk);
     BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
     FileInputStream fis[] = null;
@@ -249,7 +260,17 @@ class DataXceiver extends Receiver implements Runnable {
         throw new IOException("You cannot pass file descriptors over " +
             "anything but a UNIX domain socket.");
       }
-      fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
+      if (slotId != null) {
+        datanode.shortCircuitRegistry.registerSlot(
+            ExtendedBlockId.fromExtendedBlock(blk), slotId);
+      }
+      try {
+        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
+      } finally {
+        if ((fis == null) && (slotId != null)) {
+          datanode.shortCircuitRegistry.unregisterSlot(slotId);
+        }
+      }
       bld.setStatus(SUCCESS);
       bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
     } catch (ShortCircuitFdsVersionException e) {
@@ -293,6 +314,122 @@ class DataXceiver extends Receiver implements Runnable {
     }
   }
 
+  @Override
+  public void releaseShortCircuitFds(SlotId slotId) throws IOException {
+    boolean success = false;
+    try {
+      String error;
+      Status status;
+      try {
+        datanode.shortCircuitRegistry.unregisterSlot(slotId);
+        error = null;
+        status = Status.SUCCESS;
+      } catch (UnsupportedOperationException e) {
+        error = "unsupported operation";
+        status = Status.ERROR_UNSUPPORTED;
+      } catch (Throwable e) {
+        error = e.getMessage();
+        status = Status.ERROR_INVALID;
+      }
+      ReleaseShortCircuitAccessResponseProto.Builder bld =
+          ReleaseShortCircuitAccessResponseProto.newBuilder();
+      bld.setStatus(status);
+      if (error != null) {
+        bld.setError(error);
+      }
+      bld.build().writeDelimitedTo(socketOut);
+      success = true;
+    } finally {
+      if (ClientTraceLog.isInfoEnabled()) {
+        BlockSender.ClientTraceLog.info(String.format(
+            "src: 127.0.0.1, dest: 127.0.0.1, op: RELEASE_SHORT_CIRCUIT_FDS," +
+            " shmId: %016x%016x, slotIdx: %d, srvID: %s, success: %b",
+            slotId.getShmId().getHi(), slotId.getShmId().getLo(),
+            slotId.getSlotIdx(), datanode.getDatanodeUuid(), success));
+      }
+    }
+  }
+
+  private void sendShmErrorResponse(Status status, String error)
+      throws IOException {
+    ShortCircuitShmResponseProto.newBuilder().setStatus(status).
+        setError(error).build().writeDelimitedTo(socketOut);
+  }
+
+  private void sendShmSuccessResponse(DomainSocket sock, NewShmInfo shmInfo)
+      throws IOException {
+    ShortCircuitShmResponseProto.newBuilder().setStatus(SUCCESS).
+        setId(PBHelper.convert(shmInfo.shmId)).build().
+        writeDelimitedTo(socketOut);
+    // Send the file descriptor for the shared memory segment.
+    byte buf[] = new byte[] { (byte)0 };
+    FileDescriptor shmFdArray[] =
+        new FileDescriptor[] { shmInfo.stream.getFD() };
+    sock.sendFileDescriptors(shmFdArray, buf, 0, buf.length);
+  }
+
+  @Override
+  public void requestShortCircuitShm(String clientName) throws IOException {
+    NewShmInfo shmInfo = null;
+    boolean success = false;
+    DomainSocket sock = peer.getDomainSocket();
+    try {
+      if (sock == null) {
+        sendShmErrorResponse(ERROR_INVALID, "Bad request from " +
+            peer + ": must request a shared " +
+            "memory segment over a UNIX domain socket.");
+        return;
+      }
+      try {
+        shmInfo = datanode.shortCircuitRegistry.
+            createNewMemorySegment(clientName, sock);
+        // After calling #{ShortCircuitRegistry#createNewMemorySegment}, the
+        // socket is managed by the DomainSocketWatcher, not the DataXceiver.
+        releaseSocket();
+      } catch (UnsupportedOperationException e) {
+        sendShmErrorResponse(ERROR_UNSUPPORTED, 
+            "This datanode has not been configured to support " +
+            "short-circuit shared memory segments.");
+        return;
+      } catch (IOException e) {
+        sendShmErrorResponse(ERROR,
+            "Failed to create shared file descriptor: " + e.getMessage());
+        return;
+      }
+      sendShmSuccessResponse(sock, shmInfo);
+      success = true;
+    } finally {
+      if (ClientTraceLog.isInfoEnabled()) {
+        if (success) {
+          BlockSender.ClientTraceLog.info(String.format(
+              "cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " +
+              "op: REQUEST_SHORT_CIRCUIT_SHM," +
+              " shmId: %016x%016x, srvID: %s, success: true",
+              clientName, shmInfo.shmId.getHi(), shmInfo.shmId.getLo(),
+              datanode.getDatanodeUuid()));
+        } else {
+          BlockSender.ClientTraceLog.info(String.format(
+              "cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " +
+              "op: REQUEST_SHORT_CIRCUIT_SHM, " +
+              "shmId: n/a, srvID: %s, success: false",
+              clientName, datanode.getDatanodeUuid()));
+        }
+      }
+      if ((!success) && (peer == null)) {
+        // If we failed to pass the shared memory segment to the client,
+        // close the UNIX domain socket now.  This will trigger the 
+        // DomainSocketWatcher callback, cleaning up the segment.
+        IOUtils.cleanup(null, sock);
+      }
+      IOUtils.cleanup(null, shmInfo);
+    }
+  }
+
+  void releaseSocket() {
+    dataXceiverServer.releasePeer(peer);
+    peer = null;
+  }
+
   @Override
   public void readBlock(final ExtendedBlock block,
       final Token<BlockTokenIdentifier> blockToken,

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -248,4 +248,8 @@ class DataXceiverServer implements Runnable {
   synchronized int getNumPeers() {
     return peers.size();
   }
+
+  synchronized void releasePeer(Peer peer) {
+    peers.remove(peer);
+  }
 }

+ 332 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java

@@ -0,0 +1,332 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT;
+
+import java.io.Closeable;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.ShortCircuitShm;
+import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
+import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.DomainSocketWatcher;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.HashMultimap;
+
+/*
+ * Manages client short-circuit memory segments on the DataNode.
+ *
+ * DFSClients request shared memory segments from the DataNode.  The 
+ * ShortCircuitRegistry generates and manages these segments.  Each segment
+ * has a randomly generated 128-bit ID which uniquely identifies it.  The
+ * segments each contain several "slots."
+ *
+ * Before performing a short-circuit read, DFSClients must request a pair of
+ * file descriptors from the DataNode via the REQUEST_SHORT_CIRCUIT_FDS
+ * operation.  As part of this operation, DFSClients pass the ID of the shared
+ * memory segment they would like to use to communicate information about this
+ * replica, as well as the slot number within that segment they would like to
+ * use.  Slot allocation is always done by the client.
+ *
+ * Slots are used to track the state of the block on the both the client and
+ * datanode. When this DataNode mlocks a block, the corresponding slots for the
+ * replicas are marked as "anchorable".  Anchorable blocks can be safely read
+ * without verifying the checksum.  This means that BlockReaderLocal objects
+ * using these replicas can skip checksumming.  It also means that we can do
+ * zero-copy reads on these replicas (the ZCR interface has no way of
+ * verifying checksums.)
+ * 
+ * When a DN needs to munlock a block, it needs to first wait for the block to
+ * be unanchored by clients doing a no-checksum read or a zero-copy read. The 
+ * DN also marks the block's slots as "unanchorable" to prevent additional 
+ * clients from initiating these operations in the future.
+ * 
+ * The counterpart fo this class on the client is {@link DfsClientShmManager}.
+ */
+public class ShortCircuitRegistry {
+  public static final Log LOG = LogFactory.getLog(ShortCircuitRegistry.class);
+
+  private static final int SHM_LENGTH = 8192;
+
+  private static class RegisteredShm extends ShortCircuitShm
+      implements DomainSocketWatcher.Handler {
+    private final ShortCircuitRegistry registry;
+
+    RegisteredShm(ShmId shmId, FileInputStream stream,
+        ShortCircuitRegistry registry) throws IOException {
+      super(shmId, stream);
+      this.registry = registry;
+    }
+
+    @Override
+    public boolean handle(DomainSocket sock) {
+      synchronized (registry) {
+        synchronized (this) {
+          registry.removeShm(this);
+        }
+      }
+      return true;
+    }
+  }
+
+  public synchronized void removeShm(ShortCircuitShm shm) {
+    if (LOG.isTraceEnabled()) {
+      LOG.debug("removing shm " + shm);
+    }
+    // Stop tracking the shmId.
+    RegisteredShm removedShm = segments.remove(shm.getShmId());
+    Preconditions.checkState(removedShm == shm,
+        "failed to remove " + shm.getShmId());
+    // Stop tracking the slots.
+    for (Iterator<Slot> iter = shm.slotIterator(); iter.hasNext(); ) {
+      Slot slot = iter.next();
+      boolean removed = slots.remove(slot.getBlockId(), slot);
+      Preconditions.checkState(removed);
+      slot.makeInvalid();
+    }
+    // De-allocate the memory map and close the shared file. 
+    shm.free();
+  }
+
+  /**
+   * Whether or not the registry is enabled.
+   */
+  private boolean enabled;
+
+  /**
+   * The factory which creates shared file descriptors.
+   */
+  private final SharedFileDescriptorFactory shmFactory;
+  
+  /**
+   * A watcher which sends out callbacks when the UNIX domain socket
+   * associated with a shared memory segment closes.
+   */
+  private final DomainSocketWatcher watcher;
+
+  private final HashMap<ShmId, RegisteredShm> segments =
+      new HashMap<ShmId, RegisteredShm>(0);
+  
+  private final HashMultimap<ExtendedBlockId, Slot> slots =
+      HashMultimap.create(0, 1);
+  
+  public ShortCircuitRegistry(Configuration conf) throws IOException {
+    boolean enabled = false;
+    SharedFileDescriptorFactory shmFactory = null;
+    DomainSocketWatcher watcher = null;
+    try {
+      if (!NativeIO.isAvailable()) {
+        LOG.debug("Disabling ShortCircuitRegistry because NativeIO is " +
+            "not available.");
+        return;
+      }
+      String shmPath = conf.get(DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH,
+          DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH_DEFAULT);
+      if (shmPath.isEmpty()) {
+        LOG.info("Disabling ShortCircuitRegistry because shmPath was not set.");
+        return;
+      }
+      int interruptCheck = conf.getInt(
+          DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
+          DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
+      if (interruptCheck <= 0) {
+        LOG.info("Disabling ShortCircuitRegistry because interruptCheckMs " +
+            "was set to " + interruptCheck);
+        return;
+      }
+      shmFactory = 
+          new SharedFileDescriptorFactory("HadoopShortCircuitShm_", shmPath);
+      watcher = new DomainSocketWatcher(interruptCheck);
+      enabled = true;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("created new ShortCircuitRegistry with interruptCheck=" +
+          interruptCheck + ", shmPath=" + shmPath);
+      }
+    } finally {
+      this.enabled = enabled;
+      this.shmFactory = shmFactory;
+      this.watcher = watcher;
+    }
+  }
+
+  /**
+   * Process a block mlock event from the FsDatasetCache.
+   *
+   * @param blockId    The block that was mlocked.
+   */
+  public synchronized void processBlockMlockEvent(ExtendedBlockId blockId) {
+    if (!enabled) return;
+    Set<Slot> affectedSlots = slots.get(blockId);
+    for (Slot slot : affectedSlots) {
+      slot.makeAnchorable();
+    }
+  }
+
+  /**
+   * Mark any slots associated with this blockId as unanchorable.
+   *
+   * @param blockId        The block ID.
+   * @return               True if we should allow the munlock request.
+   */
+  public synchronized boolean processBlockMunlockRequest(
+      ExtendedBlockId blockId) {
+    if (!enabled) return true;
+    boolean allowMunlock = true;
+    Set<Slot> affectedSlots = slots.get(blockId);
+    for (Slot slot : affectedSlots) {
+      slot.makeUnanchorable();
+      if (slot.isAnchored()) {
+        allowMunlock = false;
+      }
+    }
+    return allowMunlock;
+  }
+  
+  public static class NewShmInfo implements Closeable {
+    public final ShmId shmId;
+    public final FileInputStream stream;
+
+    NewShmInfo(ShmId shmId, FileInputStream stream) {
+      this.shmId = shmId;
+      this.stream = stream;
+    }
+
+    @Override
+    public void close() throws IOException {
+      stream.close();
+    }
+  }
+
+  /**
+   * Handle a DFSClient request to create a new memory segment.
+   *
+   * @param clientName    Client name as reported by the client.
+   * @param sock          The DomainSocket to associate with this memory
+   *                        segment.  When this socket is closed, or the
+   *                        other side writes anything to the socket, the
+   *                        segment will be closed.  This can happen at any
+   *                        time, including right after this function returns.
+   * @return              A NewShmInfo object.  The caller must close the
+   *                        NewShmInfo object once they are done with it.
+   * @throws IOException  If the new memory segment could not be created.
+   */
+  public NewShmInfo createNewMemorySegment(String clientName,
+      DomainSocket sock) throws IOException {
+    NewShmInfo info = null;
+    RegisteredShm shm = null;
+    ShmId shmId = null;
+    synchronized (this) {
+      if (!enabled) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("createNewMemorySegment: ShortCircuitRegistry is " +
+              "not enabled.");
+        }
+        throw new UnsupportedOperationException();
+      }
+      FileInputStream fis = null;
+      try {
+        do {
+          shmId = ShmId.createRandom();
+        } while (segments.containsKey(shmId));
+        fis = shmFactory.createDescriptor(clientName, SHM_LENGTH);
+        shm = new RegisteredShm(shmId, fis, this);
+      } finally {
+        if (shm == null) {
+          IOUtils.closeQuietly(fis);
+        }
+      }
+      info = new NewShmInfo(shmId, fis);
+      segments.put(shmId, shm);
+    }
+    // Drop the registry lock to prevent deadlock.
+    // After this point, RegisteredShm#handle may be called at any time.
+    watcher.add(sock, shm);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("createNewMemorySegment: created " + info.shmId);
+    }
+    return info;
+  }
+  
+  public synchronized void registerSlot(ExtendedBlockId blockId, SlotId slotId)
+      throws InvalidRequestException {
+    if (!enabled) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("registerSlot: ShortCircuitRegistry is " +
+            "not enabled.");
+      }
+      throw new UnsupportedOperationException();
+    }
+    ShmId shmId = slotId.getShmId();
+    RegisteredShm shm = segments.get(shmId);
+    if (shm == null) {
+      throw new InvalidRequestException("there is no shared memory segment " +
+          "registered with shmId " + shmId);
+    }
+    Slot slot = shm.registerSlot(slotId.getSlotIdx(), blockId);
+    boolean added = slots.put(blockId, slot);
+    Preconditions.checkState(added);
+  }
+  
+  public synchronized void unregisterSlot(SlotId slotId)
+      throws InvalidRequestException {
+    if (!enabled) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("unregisterSlot: ShortCircuitRegistry is " +
+            "not enabled.");
+      }
+      throw new UnsupportedOperationException();
+    }
+    ShmId shmId = slotId.getShmId();
+    RegisteredShm shm = segments.get(shmId);
+    if (shm == null) {
+      throw new InvalidRequestException("there is no shared memory segment " +
+          "registered with shmId " + shmId);
+    }
+    Slot slot = shm.getSlot(slotId.getSlotIdx());
+    slot.makeInvalid();
+    shm.unregisterSlot(slotId.getSlotIdx());
+    slots.remove(slot.getBlockId(), slot);
+  }
+  
+  public void shutdown() {
+    synchronized (this) {
+      if (!enabled) return;
+      enabled = false;
+    }
+    IOUtils.closeQuietly(watcher);
+  }
+}

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java

@@ -266,6 +266,15 @@ public class FsDatasetCache {
     ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
     Value prevValue = mappableBlockMap.get(key);
 
+    if (!dataset.datanode.getShortCircuitRegistry().
+            processBlockMunlockRequest(key)) {
+      // TODO: we probably want to forcibly uncache the block (and close the 
+      // shm) after a certain timeout has elapsed.
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(key + " is anchored, and can't be uncached now.");
+      }
+      return;
+    }
     if (prevValue == null) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Block with id " + blockId + ", pool " + bpid + " " +
@@ -380,6 +389,7 @@ public class FsDatasetCache {
           LOG.debug("Successfully cached " + key + ".  We are now caching " +
               newUsedBytes + " bytes in total.");
         }
+        dataset.datanode.getShortCircuitRegistry().processBlockMlockEvent(key);
         numBlocksCached.addAndGet(1);
         success = true;
       } finally {

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java

@@ -47,8 +47,6 @@ import com.google.common.base.Preconditions;
  */
 @InterfaceAudience.Private
 public final class CachePool {
-  public static final Log LOG = LogFactory.getLog(CachePool.class);
-
   @Nonnull
   private final String poolName;
 

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java

@@ -62,6 +62,7 @@ import com.google.common.io.LimitInputStream;
 final class FileDistributionCalculator {
   private final static long MAX_SIZE_DEFAULT = 0x2000000000L; // 1/8 TB = 2^37
   private final static int INTERVAL_DEFAULT = 0x200000; // 2 MB = 2^21
+  private final static int MAX_INTERVALS = 0x8000000; // 128 M = 2^27
 
   private final Configuration conf;
   private final long maxSize;
@@ -82,9 +83,11 @@ final class FileDistributionCalculator {
     this.steps = steps == 0 ? INTERVAL_DEFAULT : steps;
     this.out = out;
     long numIntervals = this.maxSize / this.steps;
+    // avoid OutOfMemoryError when allocating an array
+    Preconditions.checkState(numIntervals <= MAX_INTERVALS,
+        "Too many distribution intervals (maxSize/step): " + numIntervals +
+        ", should be less than " + (MAX_INTERVALS+1) + ".");
     this.distribution = new int[1 + (int) (numIntervals)];
-    Preconditions.checkState(numIntervals < Integer.MAX_VALUE,
-        "Too many distribution intervals");
   }
 
   void visit(RandomAccessFile file) throws IOException {

+ 12 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java

@@ -101,9 +101,8 @@ public class OfflineImageViewerPB {
 
     options.addOption("p", "processor", true, "");
     options.addOption("h", "help", false, "");
-    options.addOption("skipBlocks", false, "");
-    options.addOption("printToScreen", false, "");
-    options.addOption("delimiter", true, "");
+    options.addOption("maxSize", true, "");
+    options.addOption("step", true, "");
 
     return options;
   }
@@ -118,10 +117,15 @@ public class OfflineImageViewerPB {
    * @throws IOException
    */
   public static void main(String[] args) throws IOException {
+    int status = run(args);
+    System.exit(status);
+  }
+
+  public static int run(String[] args) throws IOException {
     Options options = buildOptions();
     if (args.length == 0) {
       printUsage();
-      return;
+      return 0;
     }
 
     CommandLineParser parser = new PosixParser();
@@ -132,12 +136,12 @@ public class OfflineImageViewerPB {
     } catch (ParseException e) {
       System.out.println("Error parsing command-line options: ");
       printUsage();
-      return;
+      return -1;
     }
 
     if (cmd.hasOption("h")) { // print help and exit
       printUsage();
-      return;
+      return 0;
     }
 
     String inputFile = cmd.getOptionValue("i");
@@ -160,6 +164,7 @@ public class OfflineImageViewerPB {
       } else {
         new LsrPBImage(conf, out).visit(new RandomAccessFile(inputFile, "r"));
       }
+      return 0;
     } catch (EOFException e) {
       System.err.println("Input file ended unexpectedly. Exiting");
     } catch (IOException e) {
@@ -167,7 +172,7 @@ public class OfflineImageViewerPB {
     } finally {
       IOUtils.cleanup(null, out);
     }
-
+    return -1;
   }
 
   /**

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto

@@ -128,6 +128,22 @@ message OpBlockChecksumProto {
   required BaseHeaderProto header = 1;
 }
 
+/**
+ * An ID uniquely identifying a shared memory segment.
+ */
+message ShortCircuitShmIdProto { 
+  required int64 hi = 1;
+  required int64 lo = 2;
+}
+
+/**
+ * An ID uniquely identifying a slot within a shared memory segment.
+ */
+message ShortCircuitShmSlotProto {
+  required ShortCircuitShmIdProto shmId = 1;
+  required int32 slotIdx = 2; 
+}
+
 message OpRequestShortCircuitAccessProto { 
   required BaseHeaderProto header = 1;
 
@@ -137,6 +153,32 @@ message OpRequestShortCircuitAccessProto {
    * if the on-disk format changes.
    */
   required uint32 maxVersion = 2;
+
+  /**
+   * The shared memory slot to use, if we are using one.
+   */
+  optional ShortCircuitShmSlotProto slotId = 3;
+}
+
+message ReleaseShortCircuitAccessRequestProto {
+  required ShortCircuitShmSlotProto slotId = 1;
+}
+
+message ReleaseShortCircuitAccessResponseProto {
+  required Status status = 1;
+  optional string error = 2;
+}
+
+message ShortCircuitShmRequestProto { 
+  // The name of the client requesting the shared memory segment.  This is
+  // purely for logging / debugging purposes.
+  required string clientName = 1;
+}
+
+message ShortCircuitShmResponseProto { 
+  required Status status = 1;
+  optional string error = 2;
+  optional ShortCircuitShmIdProto id = 3;
 }
 
 message PacketHeaderProto {

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -1149,6 +1149,27 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.datanode.shared.file.descriptor.path</name>
+  <value>/dev/shm</value>
+  <description>
+    The path to use when creating file descriptors that will be shared
+    between the DataNode and the DFSClient.  Typically we use /dev/shm, so
+    that the file descriptors will not be written to disk.  Systems that
+    don't have /dev/shm should use /tmp.
+  </description>
+</property>
+
+<property>
+  <name>dfs.short.circuit.shared.memory.watcher.interrupt.check.ms</name>
+  <value>60000</value>
+  <description>
+    The length of time in milliseconds that the short-circuit shared memory
+    watcher will go between checking for java interruptions sent from other
+    threads.  This is provided mainly for unit tests.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.kerberos.internal.spnego.principal</name>
   <value>${dfs.web.authentication.kerberos.principal}</value>

+ 151 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java

@@ -17,9 +17,15 @@
  */
 package org.apache.hadoop.fs;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
+
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.IOException;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.util.concurrent.TimeoutException;
@@ -34,6 +40,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.ClientContext;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -42,18 +49,24 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.ShortCircuitCache;
 import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.BeforeClass;
@@ -71,12 +84,28 @@ public class TestEnhancedByteBufferAccess {
   private static final Log LOG =
       LogFactory.getLog(TestEnhancedByteBufferAccess.class.getName());
 
-  static TemporarySocketDirectory sockDir;
+  static private TemporarySocketDirectory sockDir;
+
+  static private CacheManipulator prevCacheManipulator;
 
   @BeforeClass
   public static void init() {
     sockDir = new TemporarySocketDirectory();
     DomainSocket.disableBindPathValidation();
+    prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
+    NativeIO.POSIX.setCacheManipulator(new CacheManipulator() {
+      @Override
+      public void mlock(String identifier,
+          ByteBuffer mmap, long length) throws IOException {
+        LOG.info("mlocking " + identifier);
+      }
+    });
+  }
+
+  @AfterClass
+  public static void teardown() {
+    // Restore the original CacheManipulator
+    NativeIO.POSIX.setCacheManipulator(prevCacheManipulator);
   }
 
   private static byte[] byteBufferToArray(ByteBuffer buf) {
@@ -86,12 +115,14 @@ public class TestEnhancedByteBufferAccess {
     return resultArray;
   }
   
+  private static int BLOCK_SIZE = 4096;
+  
   public static HdfsConfiguration initZeroCopyTest() {
     Assume.assumeTrue(NativeIO.isAvailable());
     Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE, 3);
     conf.setLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS, 100);
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
@@ -99,6 +130,9 @@ public class TestEnhancedByteBufferAccess {
           "TestRequestMmapAccess._PORT.sock").getAbsolutePath());
     conf.setBoolean(DFSConfigKeys.
         DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
+    conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
+    conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000);
     return conf;
   }
 
@@ -549,4 +583,119 @@ public class TestEnhancedByteBufferAccess {
       new File(TEST_PATH).delete();
     }
   }
+
+  /**
+   * Test that we can zero-copy read cached data even without disabling
+   * checksums.
+   */
+  @Test(timeout=120000)
+  public void testZeroCopyReadOfCachedData() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    BlockReaderTestUtil.enableBlockReaderFactoryTracing();
+    BlockReaderTestUtil.enableHdfsCachingTracing();
+
+    final int TEST_FILE_LENGTH = 16385;
+    final Path TEST_PATH = new Path("/a");
+    final int RANDOM_SEED = 23453;
+    HdfsConfiguration conf = initZeroCopyTest();
+    conf.setBoolean(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    final String CONTEXT = "testZeroCopyReadOfCachedData";
+    conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT);
+    conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
+        DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, 4096));
+    MiniDFSCluster cluster = null;
+    ByteBuffer result = null;
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    FsDatasetSpi<?> fsd = cluster.getDataNodes().get(0).getFSDataset();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    DFSTestUtil.createFile(fs, TEST_PATH,
+        TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
+    DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
+    byte original[] = DFSTestUtil.
+        calculateFileContentsFromSeed(RANDOM_SEED, TEST_FILE_LENGTH);
+
+    // Prior to caching, the file can't be read via zero-copy
+    FSDataInputStream fsIn = fs.open(TEST_PATH);
+    try {
+      result = fsIn.read(null, TEST_FILE_LENGTH / 2,
+          EnumSet.noneOf(ReadOption.class));
+      Assert.fail("expected UnsupportedOperationException");
+    } catch (UnsupportedOperationException e) {
+      // expected
+    }
+    // Cache the file
+    fs.addCachePool(new CachePoolInfo("pool1"));
+    long directiveId = fs.addCacheDirective(new CacheDirectiveInfo.Builder().
+        setPath(TEST_PATH).
+        setReplication((short)1).
+        setPool("pool1").
+        build());
+    int numBlocks = (int)Math.ceil((double)TEST_FILE_LENGTH / BLOCK_SIZE);
+    DFSTestUtil.verifyExpectedCacheUsage(
+        DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, BLOCK_SIZE),
+        numBlocks, cluster.getDataNodes().get(0).getFSDataset());
+    try {
+      result = fsIn.read(null, TEST_FILE_LENGTH,
+          EnumSet.noneOf(ReadOption.class));
+    } catch (UnsupportedOperationException e) {
+      Assert.fail("expected to be able to read cached file via zero-copy");
+    }
+    // Verify result
+    Assert.assertArrayEquals(Arrays.copyOfRange(original, 0,
+        BLOCK_SIZE), byteBufferToArray(result));
+    // check that the replica is anchored 
+    final ExtendedBlock firstBlock =
+        DFSTestUtil.getFirstBlock(fs, TEST_PATH);
+    final ShortCircuitCache cache = ClientContext.get(
+        CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache();
+    waitForReplicaAnchorStatus(cache, firstBlock, true, true, 1);
+    // Uncache the replica
+    fs.removeCacheDirective(directiveId);
+    waitForReplicaAnchorStatus(cache, firstBlock, false, true, 1);
+    fsIn.releaseBuffer(result);
+    waitForReplicaAnchorStatus(cache, firstBlock, false, false, 1);
+    DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
+
+    fsIn.close();
+    fs.close();
+    cluster.shutdown();
+  }
+  
+  private void waitForReplicaAnchorStatus(final ShortCircuitCache cache,
+      final ExtendedBlock block, final boolean expectedIsAnchorable,
+        final boolean expectedIsAnchored, final int expectedOutstandingMmaps)
+          throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        final MutableBoolean result = new MutableBoolean(false);
+        cache.accept(new CacheVisitor() {
+          @Override
+          public void visit(int numOutstandingMmaps,
+              Map<ExtendedBlockId, ShortCircuitReplica> replicas,
+              Map<ExtendedBlockId, InvalidToken> failedLoads,
+              Map<Long, ShortCircuitReplica> evictable,
+              Map<Long, ShortCircuitReplica> evictableMmapped) {
+            Assert.assertEquals(expectedOutstandingMmaps, numOutstandingMmaps);
+            ShortCircuitReplica replica =
+                replicas.get(ExtendedBlockId.fromExtendedBlock(block));
+            Assert.assertNotNull(replica);
+            Slot slot = replica.getSlot();
+            if ((expectedIsAnchorable != slot.isAnchorable()) ||
+                (expectedIsAnchored != slot.isAnchored())) {
+              LOG.info("replica " + replica + " has isAnchorable = " +
+                slot.isAnchorable() + ", isAnchored = " + slot.isAnchored() + 
+                ".  Waiting for isAnchorable = " + expectedIsAnchorable + 
+                ", isAnchored = " + expectedIsAnchored);
+              return;
+            }
+            result.setValue(true);
+          }
+        });
+        return result.toBoolean();
+      }
+    }, 10, 60000);
+  }
 }

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java

@@ -31,6 +31,7 @@ import java.util.Random;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.DfsClientShmManager;
 import org.apache.hadoop.hdfs.client.ShortCircuitCache;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
 import org.apache.hadoop.hdfs.net.Peer;
@@ -38,9 +39,13 @@ import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache;
+import org.apache.hadoop.hdfs.server.namenode.CacheManager;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
@@ -206,6 +211,15 @@ public class BlockReaderTestUtil {
     return cluster.getDataNode(ipcport);
   }
   
+  public static void enableHdfsCachingTracing() {
+    LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
+        Level.TRACE);
+    LogManager.getLogger(CacheManager.class.getName()).setLevel(
+        Level.TRACE);
+    LogManager.getLogger(FsDatasetCache.class.getName()).setLevel(
+        Level.TRACE);
+  }
+
   public static void enableBlockReaderFactoryTracing() {
     LogManager.getLogger(BlockReaderFactory.class.getName()).setLevel(
         Level.TRACE);
@@ -213,5 +227,18 @@ public class BlockReaderTestUtil {
         Level.TRACE);
     LogManager.getLogger(ShortCircuitReplica.class.getName()).setLevel(
         Level.TRACE);
+    LogManager.getLogger(BlockReaderLocal.class.getName()).setLevel(
+        Level.TRACE);
+  }
+
+  public static void enableShortCircuitShmTracing() {
+    LogManager.getLogger(DfsClientShmManager.class.getName()).setLevel(
+        Level.TRACE);
+    LogManager.getLogger(ShortCircuitRegistry.class.getName()).setLevel(
+        Level.TRACE);
+    LogManager.getLogger(ShortCircuitShm.class.getName()).setLevel(
+        Level.TRACE);
+    LogManager.getLogger(DataNode.class.getName()).setLevel(
+        Level.TRACE);
   }
 }

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
+import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 
 import org.apache.commons.io.FileUtils;
@@ -49,15 +50,18 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.VersionInfo;
 
 import java.io.*;
@@ -1091,4 +1095,47 @@ public class DFSTestUtil {
     buf.duplicate().get(arr);
     return arr;
   }
+
+  /**
+   * Blocks until cache usage hits the expected new value.
+   */
+  public static long verifyExpectedCacheUsage(final long expectedCacheUsed,
+      final long expectedBlocks, final FsDatasetSpi<?> fsd) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      private int tries = 0;
+      
+      @Override
+      public Boolean get() {
+        long curCacheUsed = fsd.getCacheUsed();
+        long curBlocks = fsd.getNumBlocksCached();
+        if ((curCacheUsed != expectedCacheUsed) ||
+            (curBlocks != expectedBlocks)) {
+          if (tries++ > 10) {
+            LOG.info("verifyExpectedCacheUsage: have " +
+                curCacheUsed + "/" + expectedCacheUsed + " bytes cached; " +
+                curBlocks + "/" + expectedBlocks + " blocks cached. " +
+                "memlock limit = " +
+                NativeIO.POSIX.getCacheManipulator().getMemlockLimit() +
+                ".  Waiting...");
+          }
+          return false;
+        }
+        return true;
+      }
+    }, 100, 60000);
+    return expectedCacheUsed;
+  }
+
+  /**
+   * Round a long value up to a multiple of a factor.
+   *
+   * @param val    The value.
+   * @param factor The factor to round up to.  Must be > 1.
+   * @return       The rounded value.
+   */
+  public static long roundUpToMultiple(long val, int factor) {
+    assert (factor > 1);
+    long c = (val + factor - 1) / factor;
+    return c * factor;
+  }
 }

+ 92 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java

@@ -18,7 +18,9 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -29,8 +31,11 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.DfsClientShmManager.PerDatanodeVisitorInfo;
+import org.apache.hadoop.hdfs.client.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.client.ShortCircuitCache;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -47,6 +52,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
 import static org.hamcrest.CoreMatchers.equalTo;
 
@@ -56,10 +62,6 @@ public class TestBlockReaderFactory {
   @Before
   public void init() {
     DomainSocket.disableBindPathValidation();
-  }
-
-  @Before
-  public void before() {
     Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
   }
 
@@ -69,7 +71,7 @@ public class TestBlockReaderFactory {
     BlockReaderFactory.createShortCircuitReplicaInfoCallback = null;
   }
 
-  private static Configuration createShortCircuitConf(String testName,
+  public static Configuration createShortCircuitConf(String testName,
       TemporarySocketDirectory sockDir) {
     Configuration conf = new Configuration();
     conf.set(DFS_CLIENT_CONTEXT, testName);
@@ -99,6 +101,8 @@ public class TestBlockReaderFactory {
     // the client is.  Both support UNIX domain reads.
     Configuration clientConf = createShortCircuitConf(
         "testFallbackFromShortCircuitToUnixDomainTraffic", sockDir);
+    clientConf.set(DFS_CLIENT_CONTEXT,
+        "testFallbackFromShortCircuitToUnixDomainTraffic_clientContext");
     clientConf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
     Configuration serverConf = new Configuration(clientConf);
     serverConf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
@@ -289,4 +293,87 @@ public class TestBlockReaderFactory {
     sockDir.close();
     Assert.assertFalse(testFailed.get());
   }
+
+   /**
+   * Test that a client which supports short-circuit reads using
+   * shared memory can fall back to not using shared memory when
+   * the server doesn't support it.
+   */
+  @Test
+  public void testShortCircuitReadFromServerWithoutShm() throws Exception {
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration clientConf = createShortCircuitConf(
+        "testShortCircuitReadFromServerWithoutShm", sockDir);
+    Configuration serverConf = new Configuration(clientConf);
+    serverConf.setInt(
+        DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, 0);
+    DFSInputStream.tcpReadsDisabledForTesting = true;
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
+    cluster.waitActive();
+    clientConf.set(DFS_CLIENT_CONTEXT,
+        "testShortCircuitReadFromServerWithoutShm_clientContext");
+    final DistributedFileSystem fs =
+        (DistributedFileSystem)FileSystem.get(cluster.getURI(0), clientConf);
+    final String TEST_FILE = "/test_file";
+    final int TEST_FILE_LEN = 4000;
+    final int SEED = 0xFADEC;
+    DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
+        (short)1, SEED);
+    byte contents[] = DFSTestUtil.readFileBuffer(fs, new Path(TEST_FILE));
+    byte expected[] = DFSTestUtil.
+        calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
+    Assert.assertTrue(Arrays.equals(contents, expected));
+    final ShortCircuitCache cache =
+        fs.dfs.getClientContext().getShortCircuitCache();
+    final DatanodeInfo datanode =
+        new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId());
+    cache.getDfsClientShmManager().visit(new Visitor() {
+      @Override
+      public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
+          throws IOException {
+        Assert.assertEquals(1,  info.size());
+        PerDatanodeVisitorInfo vinfo = info.get(datanode);
+        Assert.assertTrue(vinfo.disabled);
+        Assert.assertEquals(0, vinfo.full.size());
+        Assert.assertEquals(0, vinfo.notFull.size());
+      }
+    });
+    cluster.shutdown();
+  }
+ 
+  /**
+   * Test that a client which does not support short-circuit reads using
+   * shared memory can talk with a server which supports it.
+   */
+  @Test
+  public void testShortCircuitReadFromClientWithoutShm() throws Exception {
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration clientConf = createShortCircuitConf(
+        "testShortCircuitReadWithoutShm", sockDir);
+    Configuration serverConf = new Configuration(clientConf);
+    DFSInputStream.tcpReadsDisabledForTesting = true;
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
+    cluster.waitActive();
+    clientConf.setInt(
+        DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, 0);
+    clientConf.set(DFS_CLIENT_CONTEXT,
+        "testShortCircuitReadFromClientWithoutShm_clientContext");
+    final DistributedFileSystem fs =
+        (DistributedFileSystem)FileSystem.get(cluster.getURI(0), clientConf);
+    final String TEST_FILE = "/test_file";
+    final int TEST_FILE_LEN = 4000;
+    final int SEED = 0xFADEC;
+    DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
+        (short)1, SEED);
+    byte contents[] = DFSTestUtil.readFileBuffer(fs, new Path(TEST_FILE));
+    byte expected[] = DFSTestUtil.
+        calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
+    Assert.assertTrue(Arrays.equals(contents, expected));
+    final ShortCircuitCache cache =
+        fs.dfs.getClientContext().getShortCircuitCache();
+    Assert.assertEquals(null, cache.getDfsClientShmManager());
+    cluster.shutdown();
+  }
 }

+ 25 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java

@@ -23,19 +23,21 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.ShortCircuitCache;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.apache.hadoop.util.Time;
@@ -132,6 +134,8 @@ public class TestBlockReaderLocal {
     byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
     
     FileSystem fs = null;
+    ShortCircuitShm shm = null;
+    RandomAccessFile raf = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster.waitActive();
@@ -156,7 +160,6 @@ public class TestBlockReaderLocal {
       File dataFile = MiniDFSCluster.getBlockFile(0, block);
       File metaFile = MiniDFSCluster.getBlockMetadataFile(0, block);
 
-      DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
       ShortCircuitCache shortCircuitCache =
           ClientContext.getFromConf(conf).getShortCircuitCache();
       cluster.shutdown();
@@ -168,15 +171,23 @@ public class TestBlockReaderLocal {
       };
       dataIn = streams[0];
       metaIn = streams[1];
-      ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
-      ShortCircuitReplica replica = new ShortCircuitReplica(
-          key, dataIn, metaIn, shortCircuitCache, Time.now());
+      ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
+          block.getBlockPoolId());
+      raf = new RandomAccessFile(
+          new File(sockDir.getDir().getAbsolutePath(),
+            UUID.randomUUID().toString()), "rw");
+      raf.setLength(8192);
+      FileInputStream shmStream = new FileInputStream(raf.getFD());
+      shm = new ShortCircuitShm(ShmId.createRandom(), shmStream);
+      ShortCircuitReplica replica = 
+          new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache,
+              Time.now(), shm.allocAndRegisterSlot(
+                  ExtendedBlockId.fromExtendedBlock(block)));
       blockReaderLocal = new BlockReaderLocal.Builder(
               new DFSClient.Conf(conf)).
           setFilename(TEST_PATH.getName()).
           setBlock(block).
           setShortCircuitReplica(replica).
-          setDatanodeID(datanodeID).
           setCachingStrategy(new CachingStrategy(false, readahead)).
           setVerifyChecksum(checksum).
           build();
@@ -193,6 +204,8 @@ public class TestBlockReaderLocal {
       if (dataIn != null) dataIn.close();
       if (metaIn != null) metaIn.close();
       if (blockReaderLocal != null) blockReaderLocal.close();
+      if (shm != null) shm.free();
+      if (raf != null) raf.close();
     }
   }
   
@@ -369,13 +382,13 @@ public class TestBlockReaderLocal {
       assertArrayRegionsEqual(original, 6657,
           DFSTestUtil.asArray(buf), 0,
           1);
-      reader.setMlocked(true);
+      reader.forceAnchorable();
       readFully(reader, buf, 0, 5120);
       buf.flip();
       assertArrayRegionsEqual(original, 6658,
           DFSTestUtil.asArray(buf), 0,
           5120);
-      reader.setMlocked(false);
+      reader.forceUnanchorable();
       readFully(reader, buf, 0, 513);
       buf.flip();
       assertArrayRegionsEqual(original, 11778,
@@ -544,10 +557,10 @@ public class TestBlockReaderLocal {
       assertArrayRegionsEqual(original, 1, buf.array(), 1, 9);
       readFully(reader, buf, 10, 100);
       assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
-      reader.setMlocked(true);
+      reader.forceAnchorable();
       readFully(reader, buf, 110, 700);
       assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
-      reader.setMlocked(false);
+      reader.forceUnanchorable();
       reader.skip(1); // skip from offset 810 to offset 811
       readFully(reader, buf, 811, 5);
       assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
@@ -599,10 +612,10 @@ public class TestBlockReaderLocal {
       assertArrayRegionsEqual(original, 1, buf.array(), 1, 9);
       readFully(reader, buf, 10, 100);
       assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
-      reader.setMlocked(true);
+      reader.forceAnchorable();
       readFully(reader, buf, 110, 700);
       assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
-      reader.setMlocked(false);
+      reader.forceUnanchorable();
       reader.skip(1); // skip from offset 810 to offset 811
       readFully(reader, buf, 811, 5);
       assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);

+ 177 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitCache.java

@@ -20,26 +20,50 @@ package org.apache.hadoop.hdfs;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.client.DfsClientShmManager.PerDatanodeVisitorInfo;
 import org.apache.hadoop.hdfs.client.ShortCircuitCache;
+import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator;
+import org.apache.hadoop.hdfs.client.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
 import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
+import org.apache.hadoop.hdfs.net.DomainPeer;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Test;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 
 import java.io.DataOutputStream;
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
+import static org.hamcrest.CoreMatchers.equalTo;
 
 public class TestShortCircuitCache {
   static final Log LOG = LogFactory.getLog(TestShortCircuitCache.class);
@@ -104,7 +128,7 @@ public class TestShortCircuitCache {
         return new ShortCircuitReplicaInfo(
             new ShortCircuitReplica(key,
                 pair.getFileInputStreams()[0], pair.getFileInputStreams()[1],
-                cache, Time.monotonicNow()));
+                cache, Time.monotonicNow(), null));
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -114,14 +138,14 @@ public class TestShortCircuitCache {
   @Test(timeout=60000)
   public void testCreateAndDestroy() throws Exception {
     ShortCircuitCache cache =
-        new ShortCircuitCache(10, 1, 10, 1, 1, 10000);
+        new ShortCircuitCache(10, 1, 10, 1, 1, 10000, 0);
     cache.close();
   }
   
   @Test(timeout=60000)
   public void testAddAndRetrieve() throws Exception {
     final ShortCircuitCache cache =
-        new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000);
+        new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000, 0);
     final TestFileDescriptorPair pair = new TestFileDescriptorPair();
     ShortCircuitReplicaInfo replicaInfo1 =
       cache.fetchOrCreate(new ExtendedBlockId(123, "test_bp1"),
@@ -170,7 +194,7 @@ public class TestShortCircuitCache {
   @Test(timeout=60000)
   public void testExpiry() throws Exception {
     final ShortCircuitCache cache =
-        new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000);
+        new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000, 0);
     final TestFileDescriptorPair pair = new TestFileDescriptorPair();
     ShortCircuitReplicaInfo replicaInfo1 =
       cache.fetchOrCreate(
@@ -203,7 +227,7 @@ public class TestShortCircuitCache {
   @Test(timeout=60000)
   public void testEviction() throws Exception {
     final ShortCircuitCache cache =
-        new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000);
+        new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000, 0);
     final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] {
       new TestFileDescriptorPair(),
       new TestFileDescriptorPair(),
@@ -269,10 +293,10 @@ public class TestShortCircuitCache {
   }
   
   @Test(timeout=60000)
-  public void testStaleness() throws Exception {
+  public void testTimeBasedStaleness() throws Exception {
     // Set up the cache with a short staleness time.
     final ShortCircuitCache cache =
-        new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10);
+        new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10, 0);
     final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] {
       new TestFileDescriptorPair(),
       new TestFileDescriptorPair(),
@@ -294,7 +318,7 @@ public class TestShortCircuitCache {
                 new ShortCircuitReplica(key,
                     pairs[iVal].getFileInputStreams()[0],
                     pairs[iVal].getFileInputStreams()[1],
-                    cache, Time.monotonicNow() + (iVal * HOUR_IN_MS)));
+                    cache, Time.monotonicNow() + (iVal * HOUR_IN_MS), null));
           } catch (IOException e) {
             throw new RuntimeException(e);
           }
@@ -343,4 +367,149 @@ public class TestShortCircuitCache {
     }
     cache.close();
   }
+
+  private static Configuration createShortCircuitConf(String testName,
+      TemporarySocketDirectory sockDir) {
+    Configuration conf = new Configuration();
+    conf.set(DFS_CLIENT_CONTEXT, testName);
+    conf.setLong(DFS_BLOCK_SIZE_KEY, 4096);
+    conf.set(DFS_DOMAIN_SOCKET_PATH_KEY, new File(sockDir.getDir(),
+        testName).getAbsolutePath());
+    conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+        false);
+    conf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
+    DFSInputStream.tcpReadsDisabledForTesting = true;
+    DomainSocket.disableBindPathValidation();
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+    return conf;
+  }
+  
+  private static DomainPeer getDomainPeerToDn(Configuration conf)
+      throws IOException {
+    DomainSocket sock =
+        DomainSocket.connect(conf.get(DFS_DOMAIN_SOCKET_PATH_KEY));
+    return new DomainPeer(sock);
+  }
+  
+  @Test(timeout=60000)
+  public void testAllocShm() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration conf = createShortCircuitConf("testAllocShm", sockDir);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    final ShortCircuitCache cache =
+        fs.dfs.getClientContext().getShortCircuitCache();
+    cache.getDfsClientShmManager().visit(new Visitor() {
+      @Override
+      public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
+          throws IOException {
+        // The ClientShmManager starts off empty
+        Assert.assertEquals(0,  info.size());
+      }
+    });
+    DomainPeer peer = getDomainPeerToDn(conf);
+    MutableBoolean usedPeer = new MutableBoolean(false);
+    ExtendedBlockId blockId = new ExtendedBlockId(123, "xyz");
+    final DatanodeInfo datanode =
+        new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId());
+    // Allocating the first shm slot requires using up a peer.
+    Slot slot = cache.allocShmSlot(datanode, peer, usedPeer,
+                    blockId, "testAllocShm_client");
+    Assert.assertNotNull(slot);
+    Assert.assertTrue(usedPeer.booleanValue());
+    cache.getDfsClientShmManager().visit(new Visitor() {
+      @Override
+      public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
+          throws IOException {
+        // The ClientShmManager starts off empty
+        Assert.assertEquals(1,  info.size());
+        PerDatanodeVisitorInfo vinfo = info.get(datanode);
+        Assert.assertFalse(vinfo.disabled);
+        Assert.assertEquals(0, vinfo.full.size());
+        Assert.assertEquals(1, vinfo.notFull.size());
+      }
+    });
+    cache.scheduleSlotReleaser(slot);
+    // Wait for the slot to be released, and the shared memory area to be
+    // closed.  Since we didn't register this shared memory segment on the
+    // server, it will also be a test of how well the server deals with
+    // bogus client behavior.
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        final MutableBoolean done = new MutableBoolean(false);
+        try {
+          cache.getDfsClientShmManager().visit(new Visitor() {
+            @Override
+            public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
+                throws IOException {
+              done.setValue(info.get(datanode).full.isEmpty() &&
+                  info.get(datanode).notFull.isEmpty());
+            }
+          });
+        } catch (IOException e) {
+          LOG.error("error running visitor", e);
+        }
+        return done.booleanValue();
+      }
+    }, 10, 60000);
+    cluster.shutdown();
+  }
+
+  @Test(timeout=60000)
+  public void testShmBasedStaleness() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration conf = createShortCircuitConf("testShmBasedStaleness", sockDir);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    final ShortCircuitCache cache =
+        fs.dfs.getClientContext().getShortCircuitCache();
+    String TEST_FILE = "/test_file";
+    final int TEST_FILE_LEN = 8193;
+    final int SEED = 0xFADED;
+    DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
+        (short)1, SEED);
+    FSDataInputStream fis = fs.open(new Path(TEST_FILE));
+    int first = fis.read();
+    final ExtendedBlock block =
+        DFSTestUtil.getFirstBlock(fs, new Path(TEST_FILE));
+    Assert.assertTrue(first != -1);
+    cache.accept(new CacheVisitor() {
+      @Override
+      public void visit(int numOutstandingMmaps,
+          Map<ExtendedBlockId, ShortCircuitReplica> replicas,
+          Map<ExtendedBlockId, InvalidToken> failedLoads,
+          Map<Long, ShortCircuitReplica> evictable,
+          Map<Long, ShortCircuitReplica> evictableMmapped) {
+        ShortCircuitReplica replica = replicas.get(
+            ExtendedBlockId.fromExtendedBlock(block));
+        Assert.assertNotNull(replica);
+        Assert.assertTrue(replica.getSlot().isValid());
+      }
+    });
+    // Stop the Namenode.  This will close the socket keeping the client's
+    // shared memory segment alive, and make it stale.
+    cluster.getDataNodes().get(0).shutdown();
+    cache.accept(new CacheVisitor() {
+      @Override
+      public void visit(int numOutstandingMmaps,
+          Map<ExtendedBlockId, ShortCircuitReplica> replicas,
+          Map<ExtendedBlockId, InvalidToken> failedLoads,
+          Map<Long, ShortCircuitReplica> evictable,
+          Map<Long, ShortCircuitReplica> evictableMmapped) {
+        ShortCircuitReplica replica = replicas.get(
+            ExtendedBlockId.fromExtendedBlock(block));
+        Assert.assertNotNull(replica);
+        Assert.assertFalse(replica.getSlot().isValid());
+      }
+    });
+    cluster.shutdown();
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java

@@ -420,7 +420,7 @@ public class TestShortCircuitLocalRead {
     }
   }
 
-  @Test
+  @Test(timeout=120000)
   public void testHandleTruncatedBlockFile() throws IOException {
     MiniDFSCluster cluster = null;
     HdfsConfiguration conf = new HdfsConfiguration();

+ 0 - 104
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitSharedMemorySegment.java

@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.client;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.util.ArrayList;
-
-import org.apache.commons.lang.SystemUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.io.nativeio.NativeIO;
-import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
-import org.apache.hadoop.hdfs.client.ShortCircuitSharedMemorySegment.Slot;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.Assert;
-
-public class TestShortCircuitSharedMemorySegment {
-  public static final Log LOG =
-      LogFactory.getLog(TestShortCircuitSharedMemorySegment.class);
-  
-  private static final File TEST_BASE =
-      new File(System.getProperty("test.build.data", "/tmp"));
-
-  @Before
-  public void before() {
-    Assume.assumeTrue(NativeIO.isAvailable());
-    Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
-  }
-
-  @Test(timeout=60000)
-  public void testStartupShutdown() throws Exception {
-    File path = new File(TEST_BASE, "testStartupShutdown");
-    path.mkdirs();
-    SharedFileDescriptorFactory factory =
-        new SharedFileDescriptorFactory("shm_", path.getAbsolutePath());
-    FileInputStream stream = factory.createDescriptor(4096);
-    ShortCircuitSharedMemorySegment shm = 
-        new ShortCircuitSharedMemorySegment(stream);
-    shm.close();
-    stream.close();
-    FileUtil.fullyDelete(path);
-  }
-
-  @Test(timeout=60000)
-  public void testAllocateSlots() throws Exception {
-    File path = new File(TEST_BASE, "testAllocateSlots");
-    path.mkdirs();
-    SharedFileDescriptorFactory factory =
-        new SharedFileDescriptorFactory("shm_", path.getAbsolutePath());
-    FileInputStream stream = factory.createDescriptor(4096);
-    ShortCircuitSharedMemorySegment shm = 
-        new ShortCircuitSharedMemorySegment(stream);
-    int numSlots = 0;
-    ArrayList<Slot> slots = new ArrayList<Slot>();
-    while (true) {
-      Slot slot = shm.allocateNextSlot();
-      if (slot == null) {
-        LOG.info("allocated " + numSlots + " slots before running out.");
-        break;
-      }
-      slots.add(slot);
-      numSlots++;
-    }
-    int slotIdx = 0;
-    for (Slot slot : slots) {
-      Assert.assertFalse(slot.addAnchor());
-      Assert.assertEquals(slotIdx++, slot.getIndex());
-    }
-    for (Slot slot : slots) {
-      slot.makeAnchorable();
-    }
-    for (Slot slot : slots) {
-      Assert.assertTrue(slot.addAnchor());
-    }
-    for (Slot slot : slots) {
-      slot.removeAnchor();
-    }
-    shm.close();
-    for (Slot slot : slots) {
-      slot.close();
-    }
-    stream.close();
-    FileUtil.fullyDelete(path);
-  }
-}

+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitShm.java


+ 16 - 43
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java

@@ -208,41 +208,11 @@ public class TestFsDatasetCache {
     return sizes;
   }
 
-  /**
-   * Blocks until cache usage hits the expected new value.
-   */
-  private long verifyExpectedCacheUsage(final long expectedCacheUsed,
-      final long expectedBlocks) throws Exception {
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      private int tries = 0;
-      
-      @Override
-      public Boolean get() {
-        long curCacheUsed = fsd.getCacheUsed();
-        long curBlocks = fsd.getNumBlocksCached();
-        if ((curCacheUsed != expectedCacheUsed) ||
-            (curBlocks != expectedBlocks)) {
-          if (tries++ > 10) {
-            LOG.info("verifyExpectedCacheUsage: have " +
-                curCacheUsed + "/" + expectedCacheUsed + " bytes cached; " +
-                curBlocks + "/" + expectedBlocks + " blocks cached. " +
-                "memlock limit = " +
-                NativeIO.POSIX.getCacheManipulator().getMemlockLimit() +
-                ".  Waiting...");
-          }
-          return false;
-        }
-        return true;
-      }
-    }, 100, 60000);
-    return expectedCacheUsed;
-  }
-
   private void testCacheAndUncacheBlock() throws Exception {
     LOG.info("beginning testCacheAndUncacheBlock");
     final int NUM_BLOCKS = 5;
 
-    verifyExpectedCacheUsage(0, 0);
+    DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
     assertEquals(0, fsd.getNumBlocksCached());
 
     // Write a test file
@@ -270,7 +240,8 @@ public class TestFsDatasetCache {
     // Cache each block in succession, checking each time
     for (int i=0; i<NUM_BLOCKS; i++) {
       setHeartbeatResponse(cacheBlock(locs[i]));
-      current = verifyExpectedCacheUsage(current + blockSizes[i], i + 1);
+      current = DFSTestUtil.verifyExpectedCacheUsage(
+          current + blockSizes[i], i + 1, fsd);
       dnMetrics = getMetrics(dn.getMetrics().name());
       long cmds = MetricsAsserts.getLongCounter("BlocksCached", dnMetrics);
       assertTrue("Expected more cache requests from the NN ("
@@ -282,8 +253,9 @@ public class TestFsDatasetCache {
     // Uncache each block in succession, again checking each time
     for (int i=0; i<NUM_BLOCKS; i++) {
       setHeartbeatResponse(uncacheBlock(locs[i]));
-      current = verifyExpectedCacheUsage(current - blockSizes[i],
-          NUM_BLOCKS - 1 - i);
+      current = DFSTestUtil.
+          verifyExpectedCacheUsage(current - blockSizes[i],
+              NUM_BLOCKS - 1 - i, fsd);
       dnMetrics = getMetrics(dn.getMetrics().name());
       long cmds = MetricsAsserts.getLongCounter("BlocksUncached", dnMetrics);
       assertTrue("Expected more uncache requests from the NN",
@@ -350,11 +322,11 @@ public class TestFsDatasetCache {
 
     // Cache the first n-1 files
     long total = 0;
-    verifyExpectedCacheUsage(0, 0);
+    DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
     for (int i=0; i<numFiles-1; i++) {
       setHeartbeatResponse(cacheBlocks(fileLocs[i]));
-      total = verifyExpectedCacheUsage(
-          rounder.round(total + fileSizes[i]), 4 * (i + 1));
+      total = DFSTestUtil.verifyExpectedCacheUsage(
+          rounder.round(total + fileSizes[i]), 4 * (i + 1), fsd);
     }
 
     // nth file should hit a capacity exception
@@ -380,7 +352,7 @@ public class TestFsDatasetCache {
     for (int i=0; i<numFiles-1; i++) {
       setHeartbeatResponse(uncacheBlocks(fileLocs[i]));
       total -= rounder.round(fileSizes[i]);
-      verifyExpectedCacheUsage(total, 4 * (numFiles - 2 - i));
+      DFSTestUtil.verifyExpectedCacheUsage(total, 4 * (numFiles - 2 - i), fsd);
     }
     LOG.info("finishing testFilesExceedMaxLockedMemory");
   }
@@ -390,7 +362,7 @@ public class TestFsDatasetCache {
     LOG.info("beginning testUncachingBlocksBeforeCachingFinishes");
     final int NUM_BLOCKS = 5;
 
-    verifyExpectedCacheUsage(0, 0);
+    DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
 
     // Write a test file
     final Path testFile = new Path("/testCacheBlock");
@@ -426,7 +398,8 @@ public class TestFsDatasetCache {
     // should increase, even though caching doesn't complete on any of them.
     for (int i=0; i<NUM_BLOCKS; i++) {
       setHeartbeatResponse(cacheBlock(locs[i]));
-      current = verifyExpectedCacheUsage(current + blockSizes[i], i + 1);
+      current = DFSTestUtil.verifyExpectedCacheUsage(
+          current + blockSizes[i], i + 1, fsd);
     }
     
     setHeartbeatResponse(new DatanodeCommand[] {
@@ -434,7 +407,7 @@ public class TestFsDatasetCache {
     });
 
     // wait until all caching jobs are finished cancelling.
-    current = verifyExpectedCacheUsage(0, 0);
+    current = DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
     LOG.info("finishing testUncachingBlocksBeforeCachingFinishes");
   }
 
@@ -475,10 +448,10 @@ public class TestFsDatasetCache {
         fileName, 0, fileLen);
     // Cache the file and check the sizes match the page size
     setHeartbeatResponse(cacheBlocks(locs));
-    verifyExpectedCacheUsage(PAGE_SIZE * numBlocks, numBlocks);
+    DFSTestUtil.verifyExpectedCacheUsage(PAGE_SIZE * numBlocks, numBlocks, fsd);
     // Uncache and check that it decrements by the page size too
     setHeartbeatResponse(uncacheBlocks(locs));
-    verifyExpectedCacheUsage(0, 0);
+    DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
   }
 
   @Test(timeout=60000)

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -141,10 +142,7 @@ public class TestCacheDirectives {
     namenode = cluster.getNameNode();
     prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
     NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
-    LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
-        Level.TRACE);
-    LogManager.getLogger(CacheManager.class.getName()).setLevel(
-        Level.TRACE);
+    BlockReaderTestUtil.enableHdfsCachingTracing();
   }
 
   @After

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -277,6 +277,14 @@ public class TestOfflineImageViewer {
     assertEquals(maxFile.getLen(), Long.parseLong(matcher.group(1)));
   }
 
+  @Test
+  public void testFileDistributionCalculatorWithOptions() throws IOException {
+    int status = OfflineImageViewerPB.run(new String[] {"-i",
+        originalFsimage.getAbsolutePath(), "-o", "-", "-p", "FileDistribution",
+        "-maxSize", "512", "-step", "8"});
+    assertEquals(0, status);
+  }
+
   @Test
   public void testPBImageXmlWriter() throws IOException, SAXException,
       ParserConfigurationException {

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

@@ -185,6 +185,9 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5766. Moved ping messages from TaskAttempts to be at DEBUG level
     inside the ApplicationMaster log. (Jian He via vinodkv)
 
+    MAPREDUCE-5773. Provide dedicated MRAppMaster syslog length limit (Gera
+    Shegalov via jlowe)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 5 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -414,9 +414,13 @@ public interface MRJobConfig {
     MR_AM_PREFIX+"log.level";
   public static final String DEFAULT_MR_AM_LOG_LEVEL = "INFO";
 
+  public static final String MR_AM_LOG_KB =
+      MR_AM_PREFIX + "container.log.limit.kb";
+  public static final int DEFAULT_MR_AM_LOG_KB = 0; // don't roll
+
   public static final String MR_AM_LOG_BACKUPS =
       MR_AM_PREFIX + "container.log.backups";
-  public static final int DEFAULT_MR_AM_LOG_BACKUPS = 0; // don't roll
+  public static final int DEFAULT_MR_AM_LOG_BACKUPS = 0;
 
   /**The number of splits when reporting progress in MR*/
   public static final String MR_AM_NUM_PROGRESS_SPLITS = 

+ 8 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -510,6 +510,14 @@
   </description>
 </property>
 
+<property>
+  <name>yarn.app.mapreduce.am.container.log.limit.kb</name>
+  <value>0</value>
+  <description>The maximum size of the MRAppMaster attempt container logs in KB.
+    0 disables the cap.
+  </description>
+</property>
+
 <property>
   <name>yarn.app.mapreduce.task.container.log.backups</name>
   <value>0</value>

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -391,7 +391,8 @@ public class YARNRunner implements ClientProtocol {
     vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
 
     // TODO: why do we use 'conf' some places and 'jobConf' others?
-    long logSize = TaskLog.getTaskLogLength(new JobConf(conf));
+    long logSize = jobConf.getLong(MRJobConfig.MR_AM_LOG_KB,
+        MRJobConfig.DEFAULT_MR_AM_LOG_KB) << 10;
     String logLevel = jobConf.get(
         MRJobConfig.MR_AM_LOG_LEVEL, MRJobConfig.DEFAULT_MR_AM_LOG_LEVEL);
     int numBackups = jobConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS,

+ 21 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java

@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.io.StringReader;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
@@ -442,9 +443,12 @@ public class TestMRJobs {
     final SleepJob sleepJob = new SleepJob();
     final JobConf sleepConf = new JobConf(mrCluster.getConfig());
     sleepConf.set(MRJobConfig.MAP_LOG_LEVEL, Level.ALL.toString());
-    sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, Level.ALL.toString());
-    sleepConf.setLong(MRJobConfig.TASK_USERLOG_LIMIT, 1);
+    final long userLogKb = 4;
+    sleepConf.setLong(MRJobConfig.TASK_USERLOG_LIMIT, userLogKb);
     sleepConf.setInt(MRJobConfig.TASK_LOG_BACKUPS, 3);
+    sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, Level.ALL.toString());
+    final long amLogKb = 7;
+    sleepConf.setLong(MRJobConfig.MR_AM_LOG_KB, amLogKb);
     sleepConf.setInt(MRJobConfig.MR_AM_LOG_BACKUPS, 7);
     sleepJob.setConf(sleepConf);
 
@@ -503,6 +507,8 @@ public class TestMRJobs {
 
           final FileStatus[] sysSiblings = localFs.globStatus(new Path(
               containerPathComponent, TaskLog.LogName.SYSLOG + "*"));
+          // sort to ensure for i > 0 sysSiblings[i] == "syslog.i"
+          Arrays.sort(sysSiblings);
 
           if (foundAppMaster) {
             numAppMasters++;
@@ -510,11 +516,19 @@ public class TestMRJobs {
             numMapTasks++;
           }
 
-          Assert.assertSame("Number of sylog* files",
-              foundAppMaster
-                ? sleepConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS, 0) + 1
-                : sleepConf.getInt(MRJobConfig.TASK_LOG_BACKUPS, 0) + 1,
-              sysSiblings.length);
+          if (foundAppMaster) {
+            Assert.assertSame("Unexpected number of AM sylog* files",
+                sleepConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS, 0) + 1,
+                sysSiblings.length);
+            Assert.assertTrue("AM syslog.1 length kb should be >= " + amLogKb,
+                sysSiblings[1].getLen() >= amLogKb * 1024);
+          } else {
+            Assert.assertSame("Unexpected number of MR task sylog* files",
+                sleepConf.getInt(MRJobConfig.TASK_LOG_BACKUPS, 0) + 1,
+                sysSiblings.length);
+            Assert.assertTrue("MR syslog.1 length kb should be >= " + userLogKb,
+                sysSiblings[1].getLen() >= userLogKb * 1024);
+          }
         }
       }
     }

+ 13 - 0
hadoop-yarn-project/CHANGES.txt

@@ -243,6 +243,12 @@ Release 2.4.0 - UNRELEASED
 
     YARN-1528. Allow setting auth for ZK connections. (kasha)
 
+    YARN-1704. Modified LICENSE and NOTICE files to reflect newly used levelDB
+    related libraries. (Billie Rinaldi via vinodkv)
+
+    YARN-1765. Added test cases to verify that killApplication API works across
+    ResourceManager failover. (Xuan Gong via vinodkv) 
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -368,6 +374,13 @@ Release 2.4.0 - UNRELEASED
 
     YARN-1760. TestRMAdminService assumes CapacityScheduler. (kasha)
 
+    YARN-1758. Fixed ResourceManager to not mandate the presence of site specific
+    configuration files and thus fix failures in downstream tests. (Xuan Gong via
+    vinodkv)
+
+    YARN-1748. Excluded core-site.xml from hadoop-yarn-server-tests package's jar
+    and thus avoid breaking downstream tests. (Sravya Tirukkovalur via vinodkv)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 97 - 0
hadoop-yarn-project/LICENSE.txt

@@ -242,3 +242,100 @@ For the org.apache.hadoop.util.bloom.* classes:
  * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
  * POSSIBILITY OF SUCH DAMAGE.
  */
+
+The binary distribution of this product bundles binaries of leveldbjni
+(https://github.com/fusesource/leveldbjni), which is available under the
+following license:
+
+Copyright (c) 2011 FuseSource Corp. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of FuseSource Corp. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles binaries of leveldb
+(http://code.google.com/p/leveldb/), which is available under the following
+license:
+
+Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles binaries of snappy
+(http://code.google.com/p/snappy/), which is available under the following
+license:
+
+Copyright 2011, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

+ 15 - 0
hadoop-yarn-project/NOTICE.txt

@@ -1,2 +1,17 @@
 This product includes software developed by The Apache Software
 Foundation (http://www.apache.org/).
+
+The binary distribution of this product bundles binaries of
+org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the
+following notices:
+* Copyright 2011 Dain Sundstrom <dain@iq80.com>
+* Copyright 2011 FuseSource Corp. http://fusesource.com
+
+The binary distribution of this product bundles binaries of
+org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni),
+which has the following notices:
+* This product includes software developed by FuseSource Corp.
+  http://fusesource.com
+* This product includes software developed at
+  Progress Software Corporation and/or its  subsidiaries or affiliates.
+* This product includes software developed by IBM Corporation and others.

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/FileSystemBasedConfigurationProvider.java

@@ -54,13 +54,14 @@ public class FileSystemBasedConfigurationProvider
     if (YarnConfiguration.RM_CONFIGURATION_FILES.contains(name)) {
       filePath = new Path(this.configDir, name);
       if (!fs.exists(filePath)) {
-        throw new YarnException("Can not find Configuration: " + name + " in "
-            + configDir);
+        LOG.info(filePath + " not found");
+        return null;
       }
     } else {
       filePath = new Path(name);
       if (!fs.exists(filePath)) {
-        throw new YarnException("Can not find file: " + name);
+        LOG.info(filePath + " not found");
+        return null;
       }
     }
     return fs.open(filePath);

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.Set;
@@ -538,8 +539,11 @@ public class AdminService extends CompositeService implements
 
   private synchronized Configuration getConfiguration(Configuration conf,
       String confFileName) throws YarnException, IOException {
-    conf.addResource(this.rmContext.getConfigurationProvider()
-        .getConfigurationInputStream(conf, confFileName));
+    InputStream confFileInputStream = this.rmContext.getConfigurationProvider()
+        .getConfigurationInputStream(conf, confFileName);
+    if (confFileInputStream != null) {
+      conf.addResource(confFileInputStream);
+    }
     return conf;
   }
 

+ 8 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
@@ -142,9 +143,13 @@ public class ApplicationMasterService extends AbstractService implements
     if (conf.getBoolean(
         CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, 
         false)) {
-      conf.addResource(this.rmContext.getConfigurationProvider()
-          .getConfigurationInputStream(conf,
-              YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE));
+      InputStream inputStream =
+          this.rmContext.getConfigurationProvider()
+              .getConfigurationInputStream(conf,
+                  YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
+      if (inputStream != null) {
+        conf.addResource(inputStream);
+      }
       refreshServiceAcls(conf, RMPolicyProvider.getInstance());
     }
     

+ 8 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.security.AccessControlException;
 import java.util.ArrayList;
@@ -170,9 +171,13 @@ public class ClientRMService extends AbstractService implements
     if (conf.getBoolean(
         CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, 
         false)) {
-      conf.addResource(this.rmContext.getConfigurationProvider()
-          .getConfigurationInputStream(conf,
-              YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE));
+      InputStream inputStream =
+          this.rmContext.getConfigurationProvider()
+              .getConfigurationInputStream(conf,
+                  YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
+      if (inputStream != null) {
+        conf.addResource(inputStream);
+      }
       refreshServiceAcls(conf, RMPolicyProvider.getInstance());
     }
     

+ 14 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
@@ -190,13 +191,20 @@ public class ResourceManager extends CompositeService implements Recoverable {
     rmContext.setConfigurationProvider(configurationProvider);
 
     // load yarn-site.xml
-    this.conf.addResource(this.configurationProvider
-        .getConfigurationInputStream(this.conf,
-            YarnConfiguration.YARN_SITE_CONFIGURATION_FILE));
+    InputStream yarnSiteXMLInputStream =
+        this.configurationProvider.getConfigurationInputStream(this.conf,
+            YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
+    if (yarnSiteXMLInputStream != null) {
+      this.conf.addResource(yarnSiteXMLInputStream);
+    }
     // load core-site.xml
-    this.conf.addResource(this.configurationProvider
-        .getConfigurationInputStream(this.conf,
-            YarnConfiguration.CORE_SITE_CONFIGURATION_FILE));
+    InputStream coreSiteXMLInputStream =
+        this.configurationProvider.getConfigurationInputStream(this.conf,
+            YarnConfiguration.CORE_SITE_CONFIGURATION_FILE);
+    if (coreSiteXMLInputStream != null) {
+      this.conf.addResource(coreSiteXMLInputStream);
+    }
+
     // Do refreshUserToGroupsMappings with loaded core-site.xml
     Groups.getUserToGroupsMappingServiceWithLoadedConfiguration(this.conf)
         .refresh();

+ 8 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetSocketAddress;
 
 import org.apache.commons.logging.Log;
@@ -163,9 +164,13 @@ public class ResourceTrackerService extends AbstractService implements
     if (conf.getBoolean(
         CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, 
         false)) {
-      conf.addResource(this.rmContext.getConfigurationProvider()
-          .getConfigurationInputStream(conf,
-              YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE));
+      InputStream inputStream =
+          this.rmContext.getConfigurationProvider()
+              .getConfigurationInputStream(conf,
+                  YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
+      if (inputStream != null) {
+        conf.addResource(inputStream);
+      }
       refreshServiceAcls(conf, RMPolicyProvider.getInstance());
     }
 

+ 10 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -1024,10 +1025,15 @@ public class CapacityScheduler extends AbstractYarnScheduler
   private CapacitySchedulerConfiguration loadCapacitySchedulerConfiguration(
       Configuration configuration) throws IOException {
     try {
-      configuration.addResource(this.rmContext.getConfigurationProvider()
-          .getConfigurationInputStream(configuration,
-              YarnConfiguration.CS_CONFIGURATION_FILE));
-      return new CapacitySchedulerConfiguration(configuration, false);
+      InputStream CSInputStream =
+          this.rmContext.getConfigurationProvider()
+              .getConfigurationInputStream(configuration,
+                  YarnConfiguration.CS_CONFIGURATION_FILE);
+      if (CSInputStream != null) {
+        configuration.addResource(CSInputStream);
+        return new CapacitySchedulerConfiguration(configuration, false);
+      }
+      return new CapacitySchedulerConfiguration(configuration, true);
     } catch (Exception e) {
       throw new IOException(e);
     }

+ 416 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java

@@ -0,0 +1,416 @@
+/*
+ * 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.yarn.server.resourcemanager;
+
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.ClientBaseWithFixes;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.conf.HAUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestKillApplicationWithRMHA extends ClientBaseWithFixes{
+
+  public static final Log LOG = LogFactory
+      .getLog(TestKillApplicationWithRMHA.class);
+  private static final int ZK_TIMEOUT_MS = 5000;
+  private static StateChangeRequestInfo requestInfo =
+      new StateChangeRequestInfo(
+          HAServiceProtocol.RequestSource.REQUEST_BY_USER);
+  private Configuration configuration = new YarnConfiguration();
+  static MockRM rm1 = null;
+  static MockRM rm2 = null;
+  Configuration confForRM1;
+  Configuration confForRM2;
+
+  @Before
+  public void setup() throws Exception {
+    configuration.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    configuration.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2");
+    configuration.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    configuration.set(YarnConfiguration.RM_STORE,
+        ZKRMStateStore.class.getName());
+    configuration.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
+    configuration.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, ZK_TIMEOUT_MS);
+    configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
+    configuration.set(YarnConfiguration.RM_CLUSTER_ID, "test-yarn-cluster");
+    int base = 100;
+    for (String confKey : YarnConfiguration
+        .getServiceAddressConfKeys(configuration)) {
+      configuration.set(HAUtil.addSuffix(confKey, "rm1"), "0.0.0.0:"
+          + (base + 20));
+      configuration.set(HAUtil.addSuffix(confKey, "rm2"), "0.0.0.0:"
+          + (base + 40));
+      base = base * 2;
+    }
+    confForRM1 = new Configuration(configuration);
+    confForRM1.set(YarnConfiguration.RM_HA_ID, "rm1");
+    confForRM2 = new Configuration(configuration);
+    confForRM2.set(YarnConfiguration.RM_HA_ID, "rm2");
+  }
+
+  @After
+  public void teardown() {
+    if (rm1 != null) {
+      rm1.stop();
+    }
+    if (rm2 != null) {
+      rm2.stop();
+    }
+  }
+
+  @Test (timeout = 20000)
+  public void testKillAppWhenFailoverHappensAtNewState()
+      throws Exception {
+    // create a customized RMAppManager
+    // During the process of Application submission,
+    // the RMAppState will always be NEW.
+    // The ApplicationState will not be saved in RMStateStore.
+    startRMsWithCustomizedRMAppManager();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
+    nm1.registerNode();
+
+    // Submit the application
+    RMApp app0 =
+        rm1.submitApp(200, "", UserGroupInformation
+            .getCurrentUser().getShortUserName(), null, false, null,
+            configuration.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+                YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null,
+            false, false);
+
+    // failover and kill application
+    // When FailOver happens, the state of this application is NEW,
+    // and ApplicationState is not saved in RMStateStore. The active RM
+    // can not load the ApplicationState of this application.
+    // Expected to get ApplicationNotFoundException
+    // when receives the KillApplicationRequest
+    try {
+      failOverAndKillApp(app0.getApplicationId(), RMAppState.NEW);
+      fail("Should get an exception here");
+    } catch (ApplicationNotFoundException ex) {
+      Assert.assertTrue(ex.getMessage().contains(
+          "Trying to kill an absent application " + app0.getApplicationId()));
+    }
+  }
+
+  @Test (timeout = 20000)
+  public void testKillAppWhenFailoverHappensAtRunningState()
+      throws Exception {
+    startRMs();
+    MockNM nm1 = new MockNM("127.0.0.1:1234", 15120,
+        rm1.getResourceTrackerService());
+    nm1.registerNode();
+
+    // create app and launch the AM
+    RMApp app0 = rm1.submitApp(200);
+    MockAM am0 = launchAM(app0, rm1, nm1);
+
+    // failover and kill application
+    // The application is at RUNNING State when failOver happens.
+    // Since RMStateStore has already saved ApplicationState, the active RM
+    // will load the ApplicationState. After that, the application will be at
+    // ACCEPTED State. Because the application is not at Final State,
+    // KillApplicationResponse.getIsKillCompleted is expected to return false.
+    failOverAndKillApp(app0.getApplicationId(),
+        am0.getApplicationAttemptId(), RMAppState.RUNNING,
+        RMAppAttemptState.RUNNING, RMAppState.ACCEPTED);
+  }
+
+  @Test (timeout = 20000)
+  public void testKillAppWhenFailoverHappensAtFinalState()
+      throws Exception {
+    startRMs();
+    MockNM nm1 = new MockNM("127.0.0.1:1234", 15120,
+        rm1.getResourceTrackerService());
+    nm1.registerNode();
+
+    // create app and launch the AM
+    RMApp app0 = rm1.submitApp(200);
+    MockAM am0 = launchAM(app0, rm1, nm1);
+
+    // kill the app.
+    rm1.killApp(app0.getApplicationId());
+    rm1.waitForState(app0.getApplicationId(), RMAppState.KILLED);
+    rm1.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.KILLED);
+
+    // failover and kill application
+    // The application is at Killed State and RMStateStore has already
+    // saved this applicationState. After failover happens, the current
+    // active RM will load the ApplicationState whose RMAppState is killed.
+    // Because this application is at Final State,
+    // KillApplicationResponse.getIsKillCompleted is expected to return true.
+    failOverAndKillApp(app0.getApplicationId(),
+        am0.getApplicationAttemptId(), RMAppState.KILLED,
+        RMAppAttemptState.KILLED, RMAppState.KILLED);
+  }
+
+  @Test (timeout = 20000)
+  public void testKillAppWhenFailOverHappensDuringApplicationKill()
+      throws Exception {
+    // create a customized ClientRMService
+    // When receives the killApplicationRequest, simply return the response
+    // and make sure the application will not be KILLED State
+    startRMsWithCustomizedClientRMService();
+    MockNM nm1 = new MockNM("127.0.0.1:1234", 15120,
+        rm1.getResourceTrackerService());
+    nm1.registerNode();
+
+    // create app and launch the AM
+    RMApp app0 = rm1.submitApp(200);
+    MockAM am0 = launchAM(app0, rm1, nm1);
+
+    // ensure that the app is in running state
+    Assert.assertEquals(app0.getState(), RMAppState.RUNNING);
+
+    // kill the app.
+    rm1.killApp(app0.getApplicationId());
+
+    // failover happens before this application goes to final state.
+    // The RMAppState that will be loaded by the active rm
+    // should be ACCEPTED.
+    failOverAndKillApp(app0.getApplicationId(),
+        am0.getApplicationAttemptId(), RMAppState.RUNNING,
+        RMAppAttemptState.RUNNING, RMAppState.ACCEPTED);
+
+  }
+
+  private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
+      throws Exception {
+    RMAppAttempt attempt = app.getCurrentAppAttempt();
+    nm.nodeHeartbeat(true);
+    MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
+    am.registerAppAttempt();
+    rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
+    rm.waitForState(app.getCurrentAppAttempt().getAppAttemptId(),
+        RMAppAttemptState.RUNNING);
+    return am;
+  }
+
+  private void failOverAndKillApp(ApplicationId appId,
+      ApplicationAttemptId appAttemptId, RMAppState initialRMAppState,
+      RMAppAttemptState initialRMAppAttemptState,
+      RMAppState expectedAppStateBeforeKillApp) throws Exception {
+    Assert.assertEquals(initialRMAppState,
+        rm1.getRMContext().getRMApps().get(appId).getState());
+    Assert.assertEquals(initialRMAppAttemptState, rm1.getRMContext()
+        .getRMApps().get(appId).getAppAttempts().get(appAttemptId).getState());
+    explicitFailover();
+    Assert.assertEquals(expectedAppStateBeforeKillApp,
+        rm2.getRMContext().getRMApps().get(appId).getState());
+    killApplication(rm2, appId, appAttemptId, initialRMAppState);
+  }
+
+  private void failOverAndKillApp(ApplicationId appId,
+      RMAppState initialRMAppState) throws Exception {
+    Assert.assertEquals(initialRMAppState,
+        rm1.getRMContext().getRMApps().get(appId).getState());
+    explicitFailover();
+    Assert.assertTrue(rm2.getRMContext().getRMApps().get(appId) == null);
+    killApplication(rm2, appId, null, initialRMAppState);
+  }
+
+  private void startRMs() throws IOException {
+    rm1 = new MockRM(confForRM1);
+    rm2 = new MockRM(confForRM2);
+    startRMs(rm1, confForRM1, rm2, confForRM2);
+
+  }
+
+  private void startRMsWithCustomizedRMAppManager() throws IOException {
+    final Configuration conf1 = new Configuration(confForRM1);
+
+    rm1 = new MockRM(conf1) {
+      @Override
+      protected RMAppManager createRMAppManager() {
+        return new MyRMAppManager(this.rmContext, this.scheduler,
+            this.masterService, this.applicationACLsManager, conf1);
+      }
+    };
+
+    rm2 = new MockRM(confForRM2);
+
+    startRMs(rm1, conf1, rm2, confForRM2);
+  }
+
+  private void startRMsWithCustomizedClientRMService() throws IOException {
+    final Configuration conf1 = new Configuration(confForRM1);
+
+    rm1 = new MockRM(conf1) {
+      @Override
+      protected ClientRMService createClientRMService() {
+        return new MyClientRMService(this.rmContext, this.scheduler,
+            this.rmAppManager, this.applicationACLsManager,
+            this.queueACLsManager, getRMDTSecretManager());
+      }
+    };
+
+    rm2 = new MockRM(confForRM2);
+
+    startRMs(rm1, conf1, rm2, confForRM2);
+  }
+
+  private static class MyRMAppManager extends RMAppManager {
+
+    private Configuration conf;
+    private RMContext rmContext;
+
+    public MyRMAppManager(RMContext context, YarnScheduler scheduler,
+        ApplicationMasterService masterService,
+        ApplicationACLsManager applicationACLsManager, Configuration conf) {
+      super(context, scheduler, masterService, applicationACLsManager, conf);
+      this.conf = conf;
+      this.rmContext = context;
+    }
+
+    @Override
+    protected void submitApplication(
+        ApplicationSubmissionContext submissionContext, long submitTime,
+        String user, boolean isRecovered, RMState state) throws YarnException {
+      //Do nothing, just add the application to RMContext
+      RMAppImpl application =
+          new RMAppImpl(submissionContext.getApplicationId(), this.rmContext,
+              this.conf, submissionContext.getApplicationName(), user,
+              submissionContext.getQueue(), submissionContext,
+              this.rmContext.getScheduler(),
+              this.rmContext.getApplicationMasterService(),
+              submitTime, submissionContext.getApplicationType(),
+              submissionContext.getApplicationTags());
+      this.rmContext.getRMApps().put(submissionContext.getApplicationId(),
+          application);
+      //Do not send RMAppEventType.START event
+      //so the state of Application will not reach to NEW_SAVING state.
+    }
+  }
+
+  private static class MyClientRMService extends ClientRMService {
+
+    private RMContext rmContext;
+
+    public MyClientRMService(RMContext rmContext, YarnScheduler scheduler,
+        RMAppManager rmAppManager,
+        ApplicationACLsManager applicationACLsManager,
+        QueueACLsManager queueACLsManager,
+        RMDelegationTokenSecretManager rmDTSecretManager) {
+      super(rmContext, scheduler, rmAppManager, applicationACLsManager,
+          queueACLsManager, rmDTSecretManager);
+      this.rmContext = rmContext;
+    }
+
+    @Override
+    protected void serviceStart() {
+      // override to not start rpc handler
+    }
+
+    @Override
+    protected void serviceStop() {
+      // don't do anything
+    }
+
+    @Override
+    public KillApplicationResponse forceKillApplication(
+        KillApplicationRequest request) throws YarnException {
+      ApplicationId applicationId = request.getApplicationId();
+      RMApp application = this.rmContext.getRMApps().get(applicationId);
+      if (application.isAppSafeToTerminate()) {
+        return KillApplicationResponse.newInstance(true);
+      } else {
+        return KillApplicationResponse.newInstance(false);
+      }
+    }
+  }
+
+  private boolean isFinalState(RMAppState state) {
+    return state.equals(RMAppState.FINISHING)
+    || state.equals(RMAppState.FINISHED) || state.equals(RMAppState.FAILED)
+    || state.equals(RMAppState.KILLED);
+  }
+
+  private void explicitFailover() throws IOException {
+    rm1.adminService.transitionToStandby(requestInfo);
+    rm2.adminService.transitionToActive(requestInfo);
+    Assert.assertTrue(rm1.getRMContext().getHAServiceState()
+        == HAServiceState.STANDBY);
+    Assert.assertTrue(rm2.getRMContext().getHAServiceState()
+        == HAServiceState.ACTIVE);
+  }
+
+  private void killApplication(MockRM rm, ApplicationId appId,
+      ApplicationAttemptId appAttemptId, RMAppState rmAppState)
+      throws Exception {
+    KillApplicationResponse response = rm.killApp(appId);
+    Assert
+        .assertTrue(response.getIsKillCompleted() == isFinalState(rmAppState));
+    RMApp loadedApp0 =
+        rm.getRMContext().getRMApps().get(appId);
+    rm.waitForState(appId, RMAppState.KILLED);
+    if (appAttemptId != null) {
+      rm.waitForState(appAttemptId, RMAppAttemptState.KILLED);
+    }
+    // no new attempt is created.
+    Assert.assertEquals(1, loadedApp0.getAppAttempts().size());
+  }
+
+  private void startRMs(MockRM rm1, Configuration confForRM1, MockRM rm2,
+      Configuration confForRM2) throws IOException {
+    rm1.init(confForRM1);
+    rm1.start();
+    Assert.assertTrue(rm1.getRMContext().getHAServiceState()
+        == HAServiceState.STANDBY);
+
+    rm2.init(confForRM2);
+    rm2.start();
+    Assert.assertTrue(rm2.getRMContext().getHAServiceState()
+        == HAServiceState.STANDBY);
+
+    rm1.adminService.transitionToActive(requestInfo);
+    Assert.assertTrue(rm1.getRMContext().getHAServiceState()
+        == HAServiceState.ACTIVE);
+  }
+}

+ 28 - 56
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java

@@ -68,9 +68,16 @@ public class TestRMAdminService {
   private Path workingPath;
   private Path tmpDir;
 
+  static {
+    YarnConfiguration.addDefaultResource(
+        YarnConfiguration.CS_CONFIGURATION_FILE);
+  }
+
   @Before
   public void setup() throws IOException {
     configuration = new YarnConfiguration();
+    configuration.set(YarnConfiguration.RM_SCHEDULER,
+        CapacityScheduler.class.getCanonicalName());
     fs = FileSystem.get(configuration);
     workingPath =
         new Path(new File("target", this.getClass().getSimpleName()
@@ -94,16 +101,9 @@ public class TestRMAdminService {
     fs.delete(tmpDir, true);
   }
 
-  private void useCapacityScheduler() {
-    configuration.set(YarnConfiguration.RM_SCHEDULER,
-        CapacityScheduler.class.getCanonicalName());
-    configuration.addResource(YarnConfiguration.CS_CONFIGURATION_FILE);
-  }
-
   @Test
   public void testAdminRefreshQueuesWithLocalConfigurationProvider()
       throws IOException, YarnException {
-    useCapacityScheduler();
     rm = new MockRM(configuration);
     rm.init(configuration);
     rm.start();
@@ -126,15 +126,6 @@ public class TestRMAdminService {
       throws IOException, YarnException {
     configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
         "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
-    useCapacityScheduler();
-    try {
-      rm = new MockRM(configuration);
-      rm.init(configuration);
-      rm.start();
-      fail("Should throw an exception");
-    } catch(Exception ex) {
-      // Expect exception here
-    }
 
     //upload default configurations
     uploadDefaultConfiguration();
@@ -181,14 +172,6 @@ public class TestRMAdminService {
       throws IOException, YarnException {
     configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
         "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
-    try {
-      rm = new MockRM(configuration);
-      rm.init(configuration);
-      rm.start();
-      fail("Should throw an exception");
-    } catch(Exception ex) {
-      // Expect exception here
-    }
 
     //upload default configurations
     uploadDefaultConfiguration();
@@ -247,14 +230,6 @@ public class TestRMAdminService {
         "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
     ResourceManager resourceManager = null;
     try {
-      try {
-        resourceManager = new ResourceManager();
-        resourceManager.init(configuration);
-        resourceManager.start();
-        fail("Should throw an exception");
-      } catch (Exception ex) {
-        // expect to get an exception here
-      }
 
       //upload default configurations
       uploadDefaultConfiguration();
@@ -351,14 +326,6 @@ public class TestRMAdminService {
       throws IOException, YarnException {
     configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
         "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
-    try {
-      rm = new MockRM(configuration);
-      rm.init(configuration);
-      rm.start();
-      fail("Should throw an exception");
-    } catch(Exception ex) {
-      // Expect exception here
-    }
 
     //upload default configurations
     uploadDefaultConfiguration();
@@ -409,14 +376,6 @@ public class TestRMAdminService {
           throws IOException, YarnException {
     configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
         "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
-    try {
-      rm = new MockRM(configuration);
-      rm.init(configuration);
-      rm.start();
-      fail("Should throw an exception");
-    } catch (Exception ex) {
-      // Expect exception here
-    }
 
     String user = UserGroupInformation.getCurrentUser().getUserName();
     List<String> groupWithInit =
@@ -485,14 +444,6 @@ public class TestRMAdminService {
       throws IOException, YarnException {
     configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
         "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
-    try {
-      rm = new MockRM(configuration);
-      rm.init(configuration);
-      rm.start();
-      fail("Should throw an exception");
-    } catch (Exception ex) {
-      // Expect exception here
-    }
 
     // upload default configurations
     uploadDefaultConfiguration();
@@ -618,6 +569,27 @@ public class TestRMAdminService {
     }
   }
 
+  @Test
+  public void testRMStartsWithoutConfigurationFilesProvided() {
+    // enable FileSystemBasedConfigurationProvider without uploading
+    // any configuration files into Remote File System.
+    configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
+        "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
+
+    // The configurationProvider will return NULL instead of
+    // throwing out Exceptions, if there are no configuration files provided.
+    // RM will not load the remote Configuration files,
+    // and should start successfully.
+    try {
+      rm = new MockRM(configuration);
+      rm.init(configuration);
+      rm.start();
+    } catch (Exception ex) {
+      fail("Should not get any exceptions");
+    }
+
+  }
+
   private String writeConfigurationXML(Configuration conf, String confXMLName)
       throws IOException {
     DataOutputStream output = null;

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml

@@ -138,6 +138,11 @@
             <goals>
               <goal>test-jar</goal>
             </goals>
+            <configuration>
+              <excludes>
+                <exclude>**/core-site.xml</exclude>
+              </excludes>
+            </configuration>
             <phase>test-compile</phase>
           </execution>
         </executions>