Prechádzať zdrojové kódy

Merging r1466653 through r1467712 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1467713 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 rokov pred
rodič
commit
bf807063bc
100 zmenil súbory, kde vykonal 9678 pridanie a 1709 odobranie
  1. 0 4
      hadoop-client/pom.xml
  2. 32 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 1 0
      hadoop-common-project/hadoop-common/pom.xml
  4. 2 1
      hadoop-common-project/hadoop-common/src/CMakeLists.txt
  5. 1 1
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
  6. 16 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  7. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  8. 3 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketInputStream.java
  9. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketOutputStream.java
  10. 704 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocket.java
  11. 22 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
  12. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java
  13. 109 0
      hadoop-common-project/hadoop-common/src/main/native/src/exception.c
  14. 82 0
      hadoop-common-project/hadoop-common/src/main/native/src/exception.h
  15. 944 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c
  16. 4 0
      hadoop-common-project/hadoop-common/src/main/native/src/org_apache_hadoop.h
  17. 524 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/CompressDecompressTester.java
  18. 101 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCompressorDecompressor.java
  19. 316 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/lz4/TestLz4CompressorDecompressor.java
  20. 362 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java
  21. 58 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TemporarySocketDirectory.java
  22. 706 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java
  23. 79 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  24. 8 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  25. 1 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  26. 23 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
  27. 189 51
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
  28. 95 271
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  29. 704 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
  30. 45 45
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  31. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  32. 148 87
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  33. 142 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DomainSocketFactory.java
  34. 264 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/FileInputStreamCache.java
  35. 112 90
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java
  36. 34 50
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
  37. 46 66
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  38. 128 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
  39. 117 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DomainPeer.java
  40. 89 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DomainPeerServer.java
  41. 142 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
  42. 131 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
  43. 115 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/Peer.java
  44. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/PeerServer.java
  45. 156 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/TcpPeerServer.java
  46. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
  47. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java
  48. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
  49. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  50. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
  51. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  52. 147 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  53. 110 49
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  54. 28 32
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
  55. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  56. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  57. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  58. 138 140
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  59. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  60. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
  61. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
  62. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  63. 68 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ShortCircuitLocalReads.apt.vm
  64. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
  65. 34 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  66. 302 59
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
  67. 154 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
  68. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java
  69. 29 160
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
  70. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  71. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
  72. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java
  73. 127 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileInputStreamCache.java
  74. 11 34
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java
  75. 35 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
  76. 9 31
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitLegacyRead.java
  77. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitRead.java
  78. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadNoChecksum.java
  79. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java
  80. 58 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelUnixDomainRead.java
  81. 288 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java
  82. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
  83. 163 56
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
  84. 0 171
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSocketCache.java
  85. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
  86. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  87. 7 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
  88. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  89. 69 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
  90. 197 55
      hadoop-mapreduce-project/CHANGES.txt
  91. 116 58
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  92. 23 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobStartEvent.java
  93. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptEventType.java
  94. 50 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptRecoverEvent.java
  95. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskEventType.java
  96. 50 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskRecoverEvent.java
  97. 24 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  98. 2 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/MapTaskImpl.java
  99. 2 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/ReduceTaskImpl.java
  100. 180 56
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

+ 0 - 4
hadoop-client/pom.xml

@@ -143,10 +143,6 @@
           <groupId>org.mortbay.jetty</groupId>
           <artifactId>jetty</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty-util</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>com.sun.jersey</groupId>
           <artifactId>jersey-core</artifactId>

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

@@ -617,6 +617,17 @@ Release 2.0.5-beta - UNRELEASED
     HADOOP-9429. TestConfiguration fails with IBM JAVA. (Amir Sanjar via
     suresh)
 
+    HADOOP-9222. Cover package with org.apache.hadoop.io.lz4 unit tests (Vadim 
+    Bondarev via jlowe)
+
+    HADOOP-9233. Cover package org.apache.hadoop.io.compress.zlib with unit
+    tests (Vadim Bondarev via jlowe)
+
+    HADOOP-9211. Set default max heap size in HADOOP_CLIENT_OPTS to 512m
+    in order to avoid OOME. (Plamen Jeliazkov via shv)
+
+    HADOOP-9473. Typo in FileUtil copy() method. (Glen Mazza via suresh)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -643,6 +654,9 @@ Release 2.0.4-alpha - UNRELEASED
     HADOOP-9444. Modify hadoop-policy.xml to replace unexpanded variables to a
     default value of '*'. (Roman Shaposhnik via vinodkv)
 
+    HADOOP-9471. hadoop-client wrongfully excludes jetty-util JAR, 
+    breaking webhdfs. (tucu)
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES
@@ -1606,6 +1620,24 @@ Release 2.0.0-alpha - 05-23-2012
     HADOOP-8655. Fix TextInputFormat for large deliminators. (Gelesh via
     bobby) 
 
+Release 0.23.8 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HADOOP-9222. Cover package with org.apache.hadoop.io.lz4 unit tests (Vadim 
+    Bondarev via jlowe)
+
+    HADOOP-9233. Cover package org.apache.hadoop.io.compress.zlib with unit
+    tests (Vadim Bondarev via jlowe)
+
 Release 0.23.7 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 1 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -522,6 +522,7 @@
                     <javahClassName>org.apache.hadoop.io.compress.lz4.Lz4Compressor</javahClassName>
                     <javahClassName>org.apache.hadoop.io.compress.lz4.Lz4Decompressor</javahClassName>
                     <javahClassName>org.apache.hadoop.util.NativeCrc32</javahClassName>
+                    <javahClassName>org.apache.hadoop.net.unix.DomainSocket</javahClassName>
                   </javahClassNames>
                   <javahOutputDirectory>${project.build.directory}/native/javah</javahOutputDirectory>
                 </configuration>

+ 2 - 1
hadoop-common-project/hadoop-common/src/CMakeLists.txt

@@ -163,10 +163,10 @@ add_executable(test_bulk_crc32
     ${D}/util/bulk_crc32.c
     ${T}/util/test_bulk_crc32.c
 )
-set_property(SOURCE main.cpp PROPERTY INCLUDE_DIRECTORIES "\"-Werror\" \"-Wall\"")
 
 SET(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
 add_dual_library(hadoop
+    main/native/src/exception.c
     ${D}/io/compress/lz4/Lz4Compressor.c
     ${D}/io/compress/lz4/Lz4Decompressor.c
     ${D}/io/compress/lz4/lz4.c
@@ -177,6 +177,7 @@ add_dual_library(hadoop
     ${D}/io/nativeio/NativeIO.c
     ${D}/io/nativeio/errno_enum.c
     ${D}/io/nativeio/file_descriptor.c
+    ${D}/net/unix/DomainSocket.c
     ${D}/security/JniBasedUnixGroupsMapping.c
     ${D}/security/JniBasedUnixGroupsNetgroupMapping.c
     ${D}/security/getGroup.c

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh

@@ -62,7 +62,7 @@ export HADOOP_DATANODE_OPTS="-Dhadoop.security.logger=ERROR,RFAS $HADOOP_DATANOD
 export HADOOP_SECONDARYNAMENODE_OPTS="-Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,RFAS} -Dhdfs.audit.logger=${HDFS_AUDIT_LOGGER:-INFO,NullAppender} $HADOOP_SECONDARYNAMENODE_OPTS"
 
 # The following applies to multiple commands (fs, dfs, fsck, distcp etc)
-export HADOOP_CLIENT_OPTS="-Xmx128m $HADOOP_CLIENT_OPTS"
+export HADOOP_CLIENT_OPTS="-Xmx512m $HADOOP_CLIENT_OPTS"
 #HADOOP_JAVA_PLATFORM_OPTS="-XX:-UsePerfData $HADOOP_JAVA_PLATFORM_OPTS"
 
 # On secure datanodes, user to run the datanode as after dropping privileges

+ 16 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -743,6 +743,21 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       return value.trim();
     }
   }
+  
+  /**
+   * Get the value of the <code>name</code> property as a trimmed <code>String</code>, 
+   * <code>defaultValue</code> if no such property exists. 
+   * See @{Configuration#getTrimmed} for more details.
+   * 
+   * @param name          the property name.
+   * @param defaultValue  the property default value.
+   * @return              the value of the <code>name</code> or defaultValue
+   *                      if it is not set.
+   */
+  public String getTrimmed(String name, String defaultValue) {
+    String ret = getTrimmed(name);
+    return ret == null ? defaultValue : ret;
+  }
 
   /**
    * Get the value of the <code>name</code> property, without doing
@@ -877,7 +892,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
     return result;
   }
-    
+
   /** 
    * Get the value of the <code>name</code> property as an <code>int</code>.
    *   

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

@@ -284,7 +284,7 @@ public class FileUtil {
     // Check if dest is directory
     if (!dstFS.exists(dst)) {
       throw new IOException("`" + dst +"': specified destination directory " +
-                            "doest not exist");
+                            "does not exist");
     } else {
       FileStatus sdst = dstFS.getFileStatus(dst);
       if (!sdst.isDirectory()) 

+ 3 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketInputStream.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.net;
 
 import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
 import java.io.InputStream;
 import java.net.Socket;
 import java.net.SocketTimeoutException;
@@ -37,7 +38,8 @@ import java.nio.channels.SelectionKey;
  * IllegalBlockingModeException. 
  * Please use {@link SocketOutputStream} for writing.
  */
-class SocketInputStream extends InputStream
+@InterfaceAudience.LimitedPrivate("HDFS")
+public class SocketInputStream extends InputStream
                                implements ReadableByteChannel {
 
   private Reader reader;

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketOutputStream.java

@@ -260,4 +260,8 @@ public class SocketOutputStream extends OutputStream
       throws IOException {
     transferToFully(fileCh, position, count, null, null);
   }
+
+  public void setTimeout(int timeoutMs) {
+    writer.setTimeout(timeoutMs);
+  }
 }

+ 704 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocket.java

@@ -0,0 +1,704 @@
+/**
+ * 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.net.unix;
+
+import java.io.Closeable;
+import org.apache.hadoop.classification.InterfaceAudience;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.SocketException;
+import java.nio.channels.AsynchronousCloseException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.lang.SystemUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.NativeCodeLoader;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * The implementation of UNIX domain sockets in Java.
+ * 
+ * See {@link DomainSocket} for more information about UNIX domain sockets.
+ */
+@InterfaceAudience.LimitedPrivate("HDFS")
+public class DomainSocket implements Closeable {
+  static {
+    if (SystemUtils.IS_OS_WINDOWS) {
+      loadingFailureReason = "UNIX Domain sockets are not available on Windows.";
+    } else if (!NativeCodeLoader.isNativeCodeLoaded()) {
+      loadingFailureReason = "libhadoop cannot be loaded.";
+    } else {
+      String problem;
+      try {
+        anchorNative();
+        problem = null;
+      } catch (Throwable t) {
+        problem = "DomainSocket#anchorNative got error: " + t.getMessage();
+      }
+      loadingFailureReason = problem;
+    }
+  }
+
+  static Log LOG = LogFactory.getLog(DomainSocket.class);
+
+  /**
+   * True only if we should validate the paths used in {@link DomainSocket#bind()}
+   */
+  private static boolean validateBindPaths = true;
+
+  /**
+   * The reason why DomainSocket is not available, or null if it is available.
+   */
+  private final static String loadingFailureReason;
+
+  /**
+   * Initialize the native library code.
+   */
+  private static native void anchorNative();
+
+  /**
+   * This function is designed to validate that the path chosen for a UNIX
+   * domain socket is secure.  A socket path is secure if it doesn't allow
+   * unprivileged users to perform a man-in-the-middle attack against it.
+   * For example, one way to perform a man-in-the-middle attack would be for
+   * a malicious user to move the server socket out of the way and create his
+   * own socket in the same place.  Not good.
+   * 
+   * Note that we only check the path once.  It's possible that the
+   * permissions on the path could change, perhaps to something more relaxed,
+   * immediately after the path passes our validation test-- hence creating a
+   * security hole.  However, the purpose of this check is to spot common
+   * misconfigurations.  System administrators do not commonly change
+   * permissions on these paths while the server is running.
+   *
+   * @param path             the path to validate
+   * @param skipComponents   the number of starting path components to skip 
+   *                         validation for (used only for testing)
+   */
+  @VisibleForTesting
+  native static void validateSocketPathSecurity0(String path,
+      int skipComponents) throws IOException;
+
+  /**
+   * Return true only if UNIX domain sockets are available.
+   */
+  public static String getLoadingFailureReason() {
+    return loadingFailureReason;
+  }
+
+  /**
+   * Disable validation of the server bind paths.
+   */
+  @VisibleForTesting
+  public static void disableBindPathValidation() {
+    validateBindPaths = false;
+  }
+
+  /**
+   * Given a path and a port, compute the effective path by replacing
+   * occurrences of _PORT with the port.  This is mainly to make it 
+   * possible to run multiple DataNodes locally for testing purposes.
+   *
+   * @param path            The source path
+   * @param port            Port number to use
+   *
+   * @return                The effective path
+   */
+  public static String getEffectivePath(String path, int port) {
+    return path.replace("_PORT", String.valueOf(port));
+  }
+
+  /**
+   * Tracks the reference count of the file descriptor, and also whether it is
+   * open or closed.
+   */
+  private static class Status {
+    /**
+     * Bit mask representing a closed domain socket. 
+     */
+    private static final int STATUS_CLOSED_MASK = 1 << 30;
+    
+    /**
+     * Status bits
+     * 
+     * Bit 30: 0 = DomainSocket open, 1 = DomainSocket closed
+     * Bits 29 to 0: the reference count.
+     */
+    private final AtomicInteger bits = new AtomicInteger(0);
+
+    Status() { }
+
+    /**
+     * Increment the reference count of the underlying file descriptor.
+     *
+     * @throws ClosedChannelException      If the file descriptor is closed.
+     */
+    void reference() throws ClosedChannelException {
+      int curBits = bits.incrementAndGet();
+      if ((curBits & STATUS_CLOSED_MASK) != 0) {
+        bits.decrementAndGet();
+        throw new ClosedChannelException();
+      }
+    }
+
+    /**
+     * Decrement the reference count of the underlying file descriptor.
+     *
+     * @param checkClosed        Whether to throw an exception if the file
+     *                           descriptor is closed.
+     *
+     * @throws AsynchronousCloseException  If the file descriptor is closed and
+     *                                     checkClosed is set.
+     */
+    void unreference(boolean checkClosed) throws AsynchronousCloseException {
+      int newCount = bits.decrementAndGet();
+      assert (newCount & ~STATUS_CLOSED_MASK) >= 0;
+      if (checkClosed && ((newCount & STATUS_CLOSED_MASK) != 0)) {
+        throw new AsynchronousCloseException();
+      }
+    }
+
+    /**
+     * Return true if the file descriptor is currently open.
+     * 
+     * @return                 True if the file descriptor is currently open.
+     */
+    boolean isOpen() {
+      return ((bits.get() & STATUS_CLOSED_MASK) == 0);
+    }
+
+    /**
+     * Mark the file descriptor as closed.
+     *
+     * Once the file descriptor is closed, it cannot be reopened.
+     *
+     * @return                         The current reference count.
+     * @throws ClosedChannelException  If someone else closes the file 
+     *                                 descriptor before we do.
+     */
+    int setClosed() throws ClosedChannelException {
+      while (true) {
+        int curBits = bits.get();
+        if ((curBits & STATUS_CLOSED_MASK) != 0) {
+          throw new ClosedChannelException();
+        }
+        if (bits.compareAndSet(curBits, curBits | STATUS_CLOSED_MASK)) {
+          return curBits & (~STATUS_CLOSED_MASK);
+        }
+      }
+    }
+
+    /**
+     * Get the current reference count.
+     *
+     * @return                 The current reference count.
+     */
+    int getReferenceCount() {
+      return bits.get() & (~STATUS_CLOSED_MASK);
+    }
+  }
+
+  /**
+   * The socket status.
+   */
+  private final Status status;
+
+  /**
+   * The file descriptor associated with this UNIX domain socket.
+   */
+  private final int fd;
+
+  /**
+   * The path associated with this UNIX domain socket.
+   */
+  private final String path;
+
+  /**
+   * The InputStream associated with this socket.
+   */
+  private final DomainInputStream inputStream = new DomainInputStream();
+
+  /**
+   * The OutputStream associated with this socket.
+   */
+  private final DomainOutputStream outputStream = new DomainOutputStream();
+
+  /**
+   * The Channel associated with this socket.
+   */
+  private final DomainChannel channel = new DomainChannel();
+
+  private DomainSocket(String path, int fd) {
+    this.status = new Status();
+    this.fd = fd;
+    this.path = path;
+  }
+
+  private static native int bind0(String path) throws IOException;
+
+  /**
+   * Create a new DomainSocket listening on the given path.
+   *
+   * @param path         The path to bind and listen on.
+   * @return             The new DomainSocket.
+   */
+  public static DomainSocket bindAndListen(String path) throws IOException {
+    if (loadingFailureReason != null) {
+      throw new UnsupportedOperationException(loadingFailureReason);
+    }
+    if (validateBindPaths) {
+      validateSocketPathSecurity0(path, 0);
+    }
+    int fd = bind0(path);
+    return new DomainSocket(path, fd);
+  }
+
+  private static native int accept0(int fd) throws IOException;
+
+  /**
+   * Accept a new UNIX domain connection.
+   *
+   * This method can only be used on sockets that were bound with bind().
+   *
+   * @return                              The new connection.
+   * @throws IOException                  If there was an I/O error
+   *                                      performing the accept-- such as the
+   *                                      socket being closed from under us.
+   * @throws SocketTimeoutException       If the accept timed out.
+   */
+  public DomainSocket accept() throws IOException {
+    status.reference();
+    boolean exc = true;
+    try {
+      DomainSocket ret = new DomainSocket(path, accept0(fd));
+      exc = false;
+      return ret;
+    } finally {
+      status.unreference(exc);
+    }
+  }
+
+  private static native int connect0(String path);
+
+  /**
+   * Create a new DomainSocket connected to the given path.
+   *
+   * @param path         The path to connect to.
+   * @return             The new DomainSocket.
+   */
+  public static DomainSocket connect(String path) throws IOException {
+    if (loadingFailureReason != null) {
+      throw new UnsupportedOperationException(loadingFailureReason);
+    }
+    int fd = connect0(path);
+    return new DomainSocket(path, fd);
+  }
+
+ /**
+  * Return true if the file descriptor is currently open.
+  *
+  * @return                 True if the file descriptor is currently open.
+  */
+ public boolean isOpen() {
+   return status.isOpen();
+ }
+
+  /**
+   * @return                 The socket path.
+   */
+  public String getPath() {
+    return path;
+  }
+
+  /**
+   * @return                 The socket InputStream
+   */
+  public DomainInputStream getInputStream() {
+    return inputStream;
+  }
+
+  /**
+   * @return                 The socket OutputStream
+   */
+  public DomainOutputStream getOutputStream() {
+    return outputStream;
+  }
+
+  /**
+   * @return                 The socket Channel
+   */
+  public DomainChannel getChannel() {
+    return channel;
+  }
+
+  public static final int SEND_BUFFER_SIZE = 1;
+  public static final int RECEIVE_BUFFER_SIZE = 2;
+  public static final int SEND_TIMEOUT = 3;
+  public static final int RECEIVE_TIMEOUT = 4;
+
+  private static native void setAttribute0(int fd, int type, int val)
+      throws IOException;
+
+  public void setAttribute(int type, int size) throws IOException {
+    status.reference();
+    boolean exc = true;
+    try {
+      setAttribute0(fd, type, size);
+      exc = false;
+    } finally {
+      status.unreference(exc);
+    }
+  }
+
+  private native int getAttribute0(int fd, int type) throws IOException;
+
+  public int getAttribute(int type) throws IOException {
+    status.reference();
+    int attribute;
+    boolean exc = true;
+    try {
+      attribute = getAttribute0(fd, type);
+      exc = false;
+      return attribute;
+    } finally {
+      status.unreference(exc);
+    }
+  }
+
+  private static native void close0(int fd) throws IOException;
+
+  private static native void closeFileDescriptor0(FileDescriptor fd)
+      throws IOException;
+
+  private static native void shutdown0(int fd) throws IOException;
+
+  /**
+   * Close the Socket.
+   */
+  @Override
+  public void close() throws IOException {
+    // Set the closed bit on this DomainSocket
+    int refCount;
+    try {
+      refCount = status.setClosed();
+    } catch (ClosedChannelException e) {
+      // Someone else already closed the DomainSocket.
+      return;
+    }
+    // Wait for all references to go away
+    boolean didShutdown = false;
+    boolean interrupted = false;
+    while (refCount > 0) {
+      if (!didShutdown) {
+        try {
+          // Calling shutdown on the socket will interrupt blocking system
+          // calls like accept, write, and read that are going on in a
+          // different thread.
+          shutdown0(fd);
+        } catch (IOException e) {
+          LOG.error("shutdown error: ", e);
+        }
+        didShutdown = true;
+      }
+      try {
+        Thread.sleep(10);
+      } catch (InterruptedException e) {
+        interrupted = true;
+      }
+      refCount = status.getReferenceCount();
+    }
+
+    // At this point, nobody has a reference to the file descriptor, 
+    // and nobody will be able to get one in the future either.
+    // We now call close(2) on the file descriptor.
+    // After this point, the file descriptor number will be reused by 
+    // something else.  Although this DomainSocket object continues to hold 
+    // the old file descriptor number (it's a final field), we never use it 
+    // again because this DomainSocket is closed.
+    close0(fd);
+    if (interrupted) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  private native static void sendFileDescriptors0(int fd,
+      FileDescriptor descriptors[],
+      byte jbuf[], int offset, int length) throws IOException;
+
+  /**
+   * Send some FileDescriptor objects to the process on the other side of this
+   * socket.
+   * 
+   * @param descriptors       The file descriptors to send.
+   * @param jbuf              Some bytes to send.  You must send at least
+   *                          one byte.
+   * @param offset            The offset in the jbuf array to start at.
+   * @param length            Length of the jbuf array to use.
+   */
+  public void sendFileDescriptors(FileDescriptor descriptors[],
+      byte jbuf[], int offset, int length) throws IOException {
+    status.reference();
+    boolean exc = true;
+    try {
+      sendFileDescriptors0(fd, descriptors, jbuf, offset, length);
+      exc = false;
+    } finally {
+      status.unreference(exc);
+    }
+  }
+
+  private static native int receiveFileDescriptors0(int fd,
+      FileDescriptor[] descriptors,
+      byte jbuf[], int offset, int length) throws IOException;
+
+  /**
+   * Receive some FileDescriptor objects from the process on the other side of
+   * this socket.
+   *
+   * @param descriptors       (output parameter) Array of FileDescriptors.
+   *                          We will fill as many slots as possible with file
+   *                          descriptors passed from the remote process.  The
+   *                          other slots will contain NULL.
+   * @param jbuf              (output parameter) Buffer to read into.
+   *                          The UNIX domain sockets API requires you to read
+   *                          at least one byte from the remote process, even
+   *                          if all you care about is the file descriptors
+   *                          you will receive.
+   * @param offset            Offset into the byte buffer to load data
+   * @param length            Length of the byte buffer to use for data
+   *
+   * @return                  The number of bytes read.  This will be -1 if we
+   *                          reached EOF (similar to SocketInputStream);
+   *                          otherwise, it will be positive.
+   * @throws                  IOException if there was an I/O error.
+   */
+  public int receiveFileDescriptors(FileDescriptor[] descriptors,
+      byte jbuf[], int offset, int length) throws IOException {
+    status.reference();
+    boolean exc = true;
+    try {
+      int nBytes = receiveFileDescriptors0(fd, descriptors, jbuf, offset, length);
+      exc = false;
+      return nBytes;
+    } finally {
+      status.unreference(exc);
+    }
+  }
+
+  /**
+   * Receive some FileDescriptor objects from the process on the other side of
+   * this socket, and wrap them in FileInputStream objects.
+   *
+   * See {@link DomainSocket#recvFileInputStreams(ByteBuffer)}
+   */
+  public int recvFileInputStreams(FileInputStream[] streams, byte buf[],
+        int offset, int length) throws IOException {
+    FileDescriptor descriptors[] = new FileDescriptor[streams.length];
+    boolean success = false;
+    for (int i = 0; i < streams.length; i++) {
+      streams[i] = null;
+    }
+    status.reference();
+    try {
+      int ret = receiveFileDescriptors0(fd, descriptors, buf, offset, length);
+      for (int i = 0, j = 0; i < descriptors.length; i++) {
+        if (descriptors[i] != null) {
+          streams[j++] = new FileInputStream(descriptors[i]);
+          descriptors[i] = null;
+        }
+      }
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        for (int i = 0; i < descriptors.length; i++) {
+          if (descriptors[i] != null) {
+            try {
+              closeFileDescriptor0(descriptors[i]);
+            } catch (Throwable t) {
+              LOG.warn(t);
+            }
+          } else if (streams[i] != null) {
+            try {
+              streams[i].close();
+            } catch (Throwable t) {
+              LOG.warn(t);
+            } finally {
+              streams[i] = null; }
+          }
+        }
+      }
+      status.unreference(!success);
+    }
+  }
+
+  private native static int readArray0(int fd, byte b[], int off, int len)
+      throws IOException;
+  
+  private native static int available0(int fd) throws IOException;
+
+  private static native void write0(int fd, int b) throws IOException;
+
+  private static native void writeArray0(int fd, byte b[], int offset, int length)
+      throws IOException;
+
+  private native static int readByteBufferDirect0(int fd, ByteBuffer dst,
+      int position, int remaining) throws IOException;
+
+  /**
+   * Input stream for UNIX domain sockets.
+   */
+  @InterfaceAudience.LimitedPrivate("HDFS")
+  public class DomainInputStream extends InputStream {
+    @Override
+    public int read() throws IOException {
+      status.reference();
+      boolean exc = true;
+      try {
+        byte b[] = new byte[1];
+        int ret = DomainSocket.readArray0(DomainSocket.this.fd, b, 0, 1);
+        exc = false;
+        return (ret >= 0) ? b[0] : -1;
+      } finally {
+        status.unreference(exc);
+      }
+    }
+    
+    @Override
+    public int read(byte b[], int off, int len) throws IOException {
+      status.reference();
+      boolean exc = true;
+      try {
+        int nRead = DomainSocket.readArray0(DomainSocket.this.fd, b, off, len);
+        exc = false;
+        return nRead;
+      } finally {
+        status.unreference(exc);
+      }
+    }
+
+    @Override
+    public int available() throws IOException {
+      status.reference();
+      boolean exc = true;
+      try {
+        int nAvailable = DomainSocket.available0(DomainSocket.this.fd);
+        exc = false;
+        return nAvailable;
+      } finally {
+        status.unreference(exc);
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      DomainSocket.this.close();
+    }
+  }
+
+  /**
+   * Output stream for UNIX domain sockets.
+   */
+  @InterfaceAudience.LimitedPrivate("HDFS")
+  public class DomainOutputStream extends OutputStream {
+    @Override
+    public void close() throws IOException {
+      DomainSocket.this.close();
+    }
+
+    @Override
+    public void write(int val) throws IOException {
+      status.reference();
+      boolean exc = true;
+      try {
+        byte b[] = new byte[1];
+        b[0] = (byte)val;
+        DomainSocket.writeArray0(DomainSocket.this.fd, b, 0, 1);
+        exc = false;
+      } finally {
+        status.unreference(exc);
+      }
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      status.reference();
+        boolean exc = true;
+      try {
+        DomainSocket.writeArray0(DomainSocket.this.fd, b, off, len);
+        exc = false;
+      } finally {
+        status.unreference(exc);
+      }
+    }
+  }
+
+  @InterfaceAudience.LimitedPrivate("HDFS")
+  public class DomainChannel implements ReadableByteChannel {
+    @Override
+    public boolean isOpen() {
+      return DomainSocket.this.isOpen();
+    }
+
+    @Override
+    public void close() throws IOException {
+      DomainSocket.this.close();
+    }
+
+    @Override
+    public int read(ByteBuffer dst) throws IOException {
+      status.reference();
+      boolean exc = true;
+      try {
+        int nread = 0;
+        if (dst.isDirect()) {
+          nread = DomainSocket.readByteBufferDirect0(DomainSocket.this.fd,
+              dst, dst.position(), dst.remaining());
+        } else if (dst.hasArray()) {
+          nread = DomainSocket.readArray0(DomainSocket.this.fd,
+              dst.array(), dst.position() + dst.arrayOffset(),
+              dst.remaining());
+        } else {
+          throw new AssertionError("we don't support " +
+              "using ByteBuffers that aren't either direct or backed by " +
+              "arrays");
+        }
+        if (nread > 0) {
+          dst.position(dst.position() + nread);
+        }
+        exc = false;
+        return nread;
+      } finally {
+        status.unreference(exc);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return String.format("DomainSocket(fd=%d,path=%s)", fd, path);
+  }
+}

+ 22 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java

@@ -27,8 +27,10 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Set;
 
 import javax.crypto.SecretKey;
 
@@ -144,6 +146,10 @@ extends AbstractDelegationTokenIdentifier>
     return;
   }
   
+  protected void logExpireToken(TokenIdent ident) throws IOException {
+    return;
+  }
+
   /** 
    * Update the current master key 
    * This is called once by startThreads before tokenRemoverThread is created, 
@@ -363,15 +369,25 @@ extends AbstractDelegationTokenIdentifier>
   }
   
   /** Remove expired delegation tokens from cache */
-  private synchronized void removeExpiredToken() {
+  private void removeExpiredToken() throws IOException {
     long now = Time.now();
-    Iterator<DelegationTokenInformation> i = currentTokens.values().iterator();
-    while (i.hasNext()) {
-      long renewDate = i.next().getRenewDate();
-      if (now > renewDate) {
-        i.remove();
+    Set<TokenIdent> expiredTokens = new HashSet<TokenIdent>();
+    synchronized (this) {
+      Iterator<Map.Entry<TokenIdent, DelegationTokenInformation>> i =
+          currentTokens.entrySet().iterator();
+      while (i.hasNext()) {
+        Map.Entry<TokenIdent, DelegationTokenInformation> entry = i.next();
+        long renewDate = entry.getValue().getRenewDate();
+        if (renewDate < now) {
+          expiredTokens.add(entry.getKey());
+          i.remove();
+        }
       }
     }
+    // don't hold lock on 'this' to avoid edit log updates blocking token ops
+    for (TokenIdent ident : expiredTokens) {
+      logExpireToken(ident);
+    }
   }
 
   public void stopThreads() {

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java

@@ -104,7 +104,7 @@ public class DataChecksum implements Checksum {
                            ( (bytes[offset+2] & 0xff) << 16 ) |
                            ( (bytes[offset+3] & 0xff) << 8 )  |
                            ( (bytes[offset+4] & 0xff) );
-    return newDataChecksum( Type.valueOf(bytes[0]), bytesPerChecksum );
+    return newDataChecksum( Type.valueOf(bytes[offset]), bytesPerChecksum );
   }
   
   /**

+ 109 - 0
hadoop-common-project/hadoop-common/src/main/native/src/exception.c

@@ -0,0 +1,109 @@
+/**
+ * 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.
+ */
+
+#include "exception.h"
+
+#include <jni.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+jthrowable newExceptionV(JNIEnv* env, const char *name,
+                         const char *fmt, va_list ap)
+{
+  int need;
+  char buf[1], *msg = NULL;
+  va_list ap2;
+  jstring jstr = NULL;
+  jthrowable jthr;
+  jclass clazz;
+  jmethodID excCtor;
+
+  va_copy(ap2, ap);
+  clazz = (*env)->FindClass(env, name);
+  if (!clazz) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  excCtor = (*env)->GetMethodID(env,
+        clazz, "<init>", "(Ljava/lang/String;)V");
+  if (!excCtor) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  need = vsnprintf(buf, sizeof(buf), fmt, ap);
+  if (need < 0) {
+    fmt = "vsnprintf error";
+    need = strlen(fmt);
+  }
+  msg = malloc(need + 1);
+  vsnprintf(msg, need + 1, fmt, ap2);
+  jstr = (*env)->NewStringUTF(env, msg);
+  if (!jstr) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  jthr = (*env)->NewObject(env, clazz, excCtor, jstr);
+  if (!jthr) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+
+done:
+  free(msg);
+  va_end(ap2);
+  (*env)->DeleteLocalRef(env, jstr);
+  return jthr;
+}
+
+jthrowable newException(JNIEnv* env, const char *name, const char *fmt, ...)
+{
+  va_list ap;
+  jthrowable jthr;
+
+  va_start(ap, fmt);
+  jthr = newExceptionV(env, name, fmt, ap);
+  va_end(ap);
+  return jthr;
+}
+
+jthrowable newRuntimeException(JNIEnv* env, const char *fmt, ...)
+{
+  va_list ap;
+  jthrowable jthr;
+
+  va_start(ap, fmt);
+  jthr = newExceptionV(env, "java/lang/RuntimeException", fmt, ap);
+  va_end(ap);
+  return jthr;
+}
+
+jthrowable newIOException(JNIEnv* env, const char *fmt, ...)
+{
+  va_list ap;
+  jthrowable jthr;
+
+  va_start(ap, fmt);
+  jthr = newExceptionV(env, "java/io/IOException", fmt, ap);
+  va_end(ap);
+  return jthr;
+}

+ 82 - 0
hadoop-common-project/hadoop-common/src/main/native/src/exception.h

@@ -0,0 +1,82 @@
+/*
+ *  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.
+ */
+#ifndef HADOOP_MAIN_NATIVE_SRC_EXCEPTION_H
+#define HADOOP_MAIN_NATIVE_SRC_EXCEPTION_H
+
+#include <jni.h> /* for jthrowable */
+#include <stdarg.h> /* for va_list */
+
+/**
+ * Create a new Exception.
+ *
+ * No exceptions will be pending on return.
+ *
+ * @param env           The JNI environment
+ * @param name          full name of the Java exception class
+ * @param fmt           printf-style format string
+ * @param ap            printf-style arguments
+ *
+ * @return              The RuntimeException
+ */
+jthrowable newExceptionV(JNIEnv* env, const char *name,
+                         const char *fmt, va_list ap);
+
+/**
+ * Create a new Exception.
+ *
+ * No exceptions will be pending on return.
+ *
+ * @param env           The JNI environment
+ * @param name          full name of the Java exception class
+ * @param fmt           printf-style format string
+ * @param ...           printf-style arguments
+ *
+ * @return              The RuntimeException
+ */
+jthrowable newException(JNIEnv* env, const char *name, const char *fmt, ...)
+    __attribute__((format(printf, 3, 4)));
+
+/**
+ * Create a new RuntimeException.
+ *
+ * No exceptions will be pending on return.
+ *
+ * @param env           The JNI environment
+ * @param fmt           printf-style format string
+ * @param ...           printf-style arguments
+ *
+ * @return              The RuntimeException
+ */
+jthrowable newRuntimeException(JNIEnv* env, const char *fmt, ...)
+    __attribute__((format(printf, 2, 3)));
+
+/**
+ * Create a new IOException.
+ *
+ * No exceptions will be pending on return.
+ *
+ * @param env           The JNI environment
+ * @param fmt           printf-style format string
+ * @param ...           printf-style arguments
+ *
+ * @return              The IOException, or another exception if we failed
+ *                      to create the NativeIOException.
+ */
+jthrowable newIOException(JNIEnv* env, const char *fmt, ...)
+    __attribute__((format(printf, 2, 3)));
+
+#endif

+ 944 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c

@@ -0,0 +1,944 @@
+/*
+ * 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.
+ */
+
+#define _GNU_SOURCE
+
+#include "exception.h"
+#include "org/apache/hadoop/io/nativeio/file_descriptor.h"
+#include "org_apache_hadoop.h"
+#include "org_apache_hadoop_net_unix_DomainSocket.h"
+
+#include <errno.h>
+#include <fcntl.h>
+#include <inttypes.h>
+#include <jni.h>
+#include <limits.h>
+#include <stdint.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <sys/ioctl.h> /* for FIONREAD */
+#include <sys/socket.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+#include <sys/un.h>
+#include <unistd.h>
+
+#define SEND_BUFFER_SIZE org_apache_hadoop_net_unix_DomainSocket_SEND_BUFFER_SIZE
+#define RECEIVE_BUFFER_SIZE org_apache_hadoop_net_unix_DomainSocket_RECEIVE_BUFFER_SIZE
+#define SEND_TIMEOUT org_apache_hadoop_net_unix_DomainSocket_SEND_TIMEOUT
+#define RECEIVE_TIMEOUT org_apache_hadoop_net_unix_DomainSocket_RECEIVE_TIMEOUT
+
+#define DEFAULT_RECEIVE_TIMEOUT 120000
+#define DEFAULT_SEND_TIMEOUT 120000
+#define LISTEN_BACKLOG 128
+
+/**
+ * Can't pass more than this number of file descriptors in a single message.
+ */
+#define MAX_PASSED_FDS 16
+
+static jthrowable setAttribute0(JNIEnv *env, jint fd, jint type, jint val);
+
+/**
+ * Convert an errno to a socket exception name.
+ *
+ * Note: we assume that all of these exceptions have a one-argument constructor
+ * that takes a string.
+ *
+ * @return               The exception class name
+ */
+static const char *errnoToSocketExceptionName(int errnum)
+{
+  switch (errnum) {
+  case EAGAIN:
+    /* accept(2) returns EAGAIN when a socket timeout has been set, and that
+     * timeout elapses without an incoming connection.  This error code is also
+     * used in non-blocking I/O, but we don't support that. */
+  case ETIMEDOUT:
+    return "java/net/SocketTimeoutException";
+  case EHOSTDOWN:
+  case EHOSTUNREACH:
+  case ECONNREFUSED:
+    return "java/net/NoRouteToHostException";
+  case ENOTSUP:
+    return "java/lang/UnsupportedOperationException";
+  default:
+    return "java/net/SocketException";
+  }
+}
+
+static jthrowable newSocketException(JNIEnv *env, int errnum,
+                                     const char *fmt, ...)
+    __attribute__((format(printf, 3, 4)));
+
+static jthrowable newSocketException(JNIEnv *env, int errnum,
+                                     const char *fmt, ...)
+{
+  va_list ap;
+  jthrowable jthr;
+
+  va_start(ap, fmt);
+  jthr = newExceptionV(env, errnoToSocketExceptionName(errnum), fmt, ap);
+  va_end(ap);
+  return jthr;
+}
+
+static const char* terror(int errnum)
+{
+  if ((errnum < 0) || (errnum >= sys_nerr)) {
+    return "unknown error.";
+  }
+  return sys_errlist[errnum];
+}
+
+/**
+ * Flexible buffer that will try to fit data on the stack, and fall back
+ * to the heap if necessary.
+ */
+struct flexibleBuffer {
+  int8_t *curBuf;
+  int8_t *allocBuf;
+  int8_t stackBuf[8196];
+};
+
+static jthrowable flexBufInit(JNIEnv *env, struct flexibleBuffer *flexBuf, jint length)
+{
+  flexBuf->curBuf = flexBuf->allocBuf = NULL;
+  if (length < sizeof(flexBuf->stackBuf)) {
+    flexBuf->curBuf = flexBuf->stackBuf;
+    return NULL;
+  }
+  flexBuf->allocBuf = malloc(length);
+  if (!flexBuf->allocBuf) {
+    return newException(env, "java/lang/OutOfMemoryError",
+        "OOM allocating space for %d bytes of data.", length);
+  }
+  flexBuf->curBuf = flexBuf->allocBuf;
+  return NULL;
+}
+
+static void flexBufFree(struct flexibleBuffer *flexBuf)
+{
+  free(flexBuf->allocBuf);
+}
+
+static jthrowable setup(JNIEnv *env, int *ofd, jobject jpath, int doConnect)
+{
+  const char *cpath = NULL;
+  struct sockaddr_un addr;
+  jthrowable jthr = NULL;
+  int fd = -1, ret;
+
+  fd = socket(PF_UNIX, SOCK_STREAM, 0);
+  if (fd < 0) {
+    ret = errno;
+    jthr = newSocketException(env, ret,
+        "error creating UNIX domain socket with SOCK_STREAM: %s",
+        terror(ret));
+    goto done;
+  }
+  memset(&addr, 0, sizeof(&addr));
+  addr.sun_family = AF_UNIX;
+  cpath = (*env)->GetStringUTFChars(env, jpath, NULL);
+  if (!cpath) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  ret = snprintf(addr.sun_path, sizeof(addr.sun_path),
+                 "%s", cpath);
+  if (ret < 0) {
+    ret = errno;
+    jthr = newSocketException(env, EIO,
+        "error computing UNIX domain socket path: error %d (%s)",
+        ret, terror(ret));
+    goto done;
+  }
+  if (ret >= sizeof(addr.sun_path)) {
+    jthr = newSocketException(env, ENAMETOOLONG,
+        "error computing UNIX domain socket path: path too long.  "
+        "The longest UNIX domain socket path possible on this host "
+        "is %zd bytes.", sizeof(addr.sun_path) - 1);
+    goto done;
+  }
+  if (doConnect) {
+    RETRY_ON_EINTR(ret, connect(fd, 
+        (struct sockaddr*)&addr, sizeof(addr)));
+    if (ret < 0) {
+      ret = errno;
+      jthr = newException(env, "java/net/ConnectException",
+              "connect(2) error: %s when trying to connect to '%s'",
+              terror(ret), addr.sun_path);
+      goto done;
+    }
+  } else {
+    RETRY_ON_EINTR(ret, unlink(addr.sun_path));
+    RETRY_ON_EINTR(ret, bind(fd, (struct sockaddr*)&addr, sizeof(addr)));
+    if (ret < 0) {
+      ret = errno;
+      jthr = newException(env, "java/net/BindException",
+              "bind(2) error: %s when trying to bind to '%s'",
+              terror(ret), addr.sun_path);
+      goto done;
+    }
+    /* We need to make the socket readable and writable for all users in the
+     * system.
+     *
+     * If the system administrator doesn't want the socket to be accessible to
+     * all users, he can simply adjust the +x permissions on one of the socket's
+     * parent directories.
+     *
+     * See HDFS-4485 for more discussion.
+     */
+    if (chmod(addr.sun_path, 0666)) {
+      ret = errno;
+      jthr = newException(env, "java/net/BindException",
+              "chmod(%s, 0666) failed: %s", addr.sun_path, terror(ret));
+      goto done;
+    }
+    if (listen(fd, LISTEN_BACKLOG) < 0) {
+      ret = errno;
+      jthr = newException(env, "java/net/BindException",
+              "listen(2) error: %s when trying to listen to '%s'",
+              terror(ret), addr.sun_path);
+      goto done;
+    }
+  }
+
+done:
+  if (cpath) {
+    (*env)->ReleaseStringUTFChars(env, jpath, cpath);
+  }
+  if (jthr) {
+    if (fd > 0) {
+      RETRY_ON_EINTR(ret, close(fd));
+      fd = -1;
+    }
+  } else {
+    *ofd = fd;
+  }
+  return jthr;
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_anchorNative(
+JNIEnv *env, jclass clazz)
+{
+  fd_init(env); // for fd_get, fd_create, etc.
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_validateSocketPathSecurity0(
+JNIEnv *env, jclass clazz, jobject jstr, jint skipComponents)
+{
+  jint utfLength;
+  char path[PATH_MAX], check[PATH_MAX], *token, *rest;
+  struct stat st;
+  int ret, mode, strlenPath;
+  uid_t uid;
+  jthrowable jthr = NULL;
+
+  utfLength = (*env)->GetStringUTFLength(env, jstr);
+  if (utfLength > sizeof(path)) {
+    jthr = newIOException(env, "path is too long!  We expected a path "
+        "no longer than %zd UTF-8 bytes.", sizeof(path));
+    goto done;
+  }
+  (*env)->GetStringUTFRegion(env, jstr, 0, utfLength, path);
+  jthr = (*env)->ExceptionOccurred(env);
+  if (jthr) {
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  uid = geteuid();
+  strlenPath = strlen(path);
+  if (strlenPath == 0) {
+    jthr = newIOException(env, "socket path is empty.");
+    goto done;
+  }
+  if (path[strlenPath - 1] == '/') {
+    /* It makes no sense to have a socket path that ends in a slash, since
+     * sockets are not directories. */
+    jthr = newIOException(env, "bad socket path '%s'.  The socket path "
+          "must not end in a slash.", path);
+    goto done;
+  }
+  // This loop iterates through all of the path components except for the very
+  // last one.  We don't validate the last component, since it's not supposed to
+  // be a directory.  (If it is a directory, we will fail to create the socket
+  // later with EISDIR or similar.)
+  for (check[0] = '/', check[1] = '\0', rest = path, token = "";
+       token && rest[0];
+       token = strtok_r(rest, "/", &rest)) {
+    if (strcmp(check, "/") != 0) {
+      // If the previous directory we checked was '/', we skip appending another
+      // slash to the end because it would be unncessary.  Otherwise we do it.
+      strcat(check, "/");
+    }
+    // These strcats are safe because the length of 'check' is the same as the
+    // length of 'path' and we never add more slashes than were in the original
+    // path.
+    strcat(check, token);
+    if (skipComponents > 0) {
+      skipComponents--;
+      continue;
+    }
+    if (stat(check, &st) < 0) {
+      ret = errno;
+      jthr = newIOException(env, "failed to stat a path component: '%s'.  "
+          "error code %d (%s)", check, ret, terror(ret));
+      goto done;
+    }
+    mode = st.st_mode & 0777;
+    if (mode & 0002) {
+      jthr = newIOException(env, "the path component: '%s' is "
+        "world-writable.  Its permissions are 0%03o.  Please fix "
+        "this or select a different socket path.", check, mode);
+      goto done;
+    }
+    if ((mode & 0020) && (st.st_gid != 0)) {
+      jthr = newIOException(env, "the path component: '%s' is "
+        "group-writable, and the group is not root.  Its permissions are "
+        "0%03o, and it is owned by gid %d.  Please fix this or "
+        "select a different socket path.", check, mode, st.st_gid);
+      goto done;
+    }
+    if ((mode & 0200) && (st.st_uid != 0) &&
+        (st.st_uid != uid)) {
+      jthr = newIOException(env, "the path component: '%s' is "
+        "owned by a user who is not root and not you.  Your effective user "
+        "id is %d; the path is owned by user id %d, and its permissions are "
+        "0%03o.  Please fix this or select a different socket path.",
+        check, uid, st.st_uid, mode);
+        goto done;
+      goto done;
+    }
+  }
+done:
+  if (jthr) {
+    (*env)->Throw(env, jthr);
+  }
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_bind0(
+JNIEnv *env, jclass clazz, jstring path)
+{
+  int fd;
+  jthrowable jthr = NULL;
+
+  jthr = setup(env, &fd, path, 0);
+  if (jthr) {
+    (*env)->Throw(env, jthr);
+  }
+  return fd;
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_accept0(
+JNIEnv *env, jclass clazz, jint fd)
+{
+  int ret, newFd = -1;
+  socklen_t slen;
+  struct sockaddr_un remote;
+  jthrowable jthr = NULL;
+
+  slen = sizeof(remote);
+  do {
+    newFd = accept(fd, (struct sockaddr*)&remote, &slen);
+  } while ((newFd < 0) && (errno == EINTR));
+  if (newFd < 0) {
+    ret = errno;
+    jthr = newSocketException(env, ret, "accept(2) error: %s", terror(ret));
+    goto done;
+  }
+
+done:
+  if (jthr) {
+    if (newFd > 0) {
+      RETRY_ON_EINTR(ret, close(newFd));
+      newFd = -1;
+    }
+    (*env)->Throw(env, jthr);
+  }
+  return newFd;
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_connect0(
+JNIEnv *env, jclass clazz, jstring path)
+{
+  int ret, fd;
+  jthrowable jthr = NULL;
+
+  jthr = setup(env, &fd, path, 1);
+  if (jthr) {
+    (*env)->Throw(env, jthr);
+    return -1;
+  }
+  if (((jthr = setAttribute0(env, fd, SEND_TIMEOUT, DEFAULT_SEND_TIMEOUT))) || 
+      ((jthr = setAttribute0(env, fd, RECEIVE_TIMEOUT, DEFAULT_RECEIVE_TIMEOUT)))) {
+    RETRY_ON_EINTR(ret, close(fd));
+    (*env)->Throw(env, jthr);
+    return -1;
+  }
+  return fd;
+}
+
+static void javaMillisToTimeVal(int javaMillis, struct timeval *tv)
+{
+  tv->tv_sec = javaMillis / 1000;
+  tv->tv_usec = (javaMillis - (tv->tv_sec * 1000)) * 1000;
+}
+
+static jthrowable setAttribute0(JNIEnv *env, jint fd, jint type, jint val)
+{
+  struct timeval tv;
+  int ret, buf;
+
+  switch (type) {
+  case SEND_BUFFER_SIZE:
+    buf = val;
+    if (setsockopt(fd, SOL_SOCKET, SO_SNDBUF, &buf, sizeof(buf))) {
+      ret = errno;
+      return newSocketException(env, ret,
+          "setsockopt(SO_SNDBUF) error: %s", terror(ret));
+    }
+    return NULL;
+  case RECEIVE_BUFFER_SIZE:
+    buf = val;
+    if (setsockopt(fd, SOL_SOCKET, SO_RCVBUF, &buf, sizeof(buf))) {
+      ret = errno;
+      return newSocketException(env, ret,
+          "setsockopt(SO_RCVBUF) error: %s", terror(ret));
+    }
+    return NULL;
+  case SEND_TIMEOUT:
+    javaMillisToTimeVal(val, &tv);
+    if (setsockopt(fd, SOL_SOCKET, SO_SNDTIMEO, (struct timeval *)&tv,
+               sizeof(tv))) {
+      ret = errno;
+      return newSocketException(env, ret,
+          "setsockopt(SO_SNDTIMEO) error: %s", terror(ret));
+    }
+    return NULL;
+  case RECEIVE_TIMEOUT:
+    javaMillisToTimeVal(val, &tv);
+    if (setsockopt(fd, SOL_SOCKET, SO_RCVTIMEO, (struct timeval *)&tv,
+               sizeof(tv))) {
+      ret = errno;
+      return newSocketException(env, ret,
+          "setsockopt(SO_RCVTIMEO) error: %s", terror(ret));
+    }
+    return NULL;
+  default:
+    break;
+  }
+  return newRuntimeException(env, "Invalid attribute type %d.", type);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_setAttribute0(
+JNIEnv *env, jclass clazz, jint fd, jint type, jint val)
+{
+  jthrowable jthr = setAttribute0(env, fd, type, val);
+  if (jthr) {
+    (*env)->Throw(env, jthr);
+  }
+}
+
+static jint getSockOptBufSizeToJavaBufSize(int size)
+{
+#ifdef __linux__
+  // Linux always doubles the value that you set with setsockopt.
+  // We cut it in half here so that programs can at least read back the same
+  // value they set.
+  size /= 2;
+#endif
+  return size;
+}
+
+static int timeValToJavaMillis(const struct timeval *tv)
+{
+  return (tv->tv_sec * 1000) + (tv->tv_usec / 1000);
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_getAttribute0(
+JNIEnv *env, jclass clazz, jint fd, jint type)
+{
+  struct timeval tv;
+  socklen_t len;
+  int ret, rval = 0;
+
+  switch (type) {
+  case SEND_BUFFER_SIZE:
+    len = sizeof(rval);
+    if (getsockopt(fd, SOL_SOCKET, SO_SNDBUF, &rval, &len)) {
+      ret = errno;
+      (*env)->Throw(env, newSocketException(env, ret,
+          "getsockopt(SO_SNDBUF) error: %s", terror(ret)));
+      return -1;
+    }
+    return getSockOptBufSizeToJavaBufSize(rval);
+  case RECEIVE_BUFFER_SIZE:
+    len = sizeof(rval);
+    if (getsockopt(fd, SOL_SOCKET, SO_RCVBUF, &rval, &len)) {
+      ret = errno;
+      (*env)->Throw(env, newSocketException(env, ret,
+          "getsockopt(SO_RCVBUF) error: %s", terror(ret)));
+      return -1;
+    }
+    return getSockOptBufSizeToJavaBufSize(rval);
+  case SEND_TIMEOUT:
+    memset(&tv, 0, sizeof(tv));
+    len = sizeof(struct timeval);
+    if (getsockopt(fd, SOL_SOCKET, SO_SNDTIMEO, &tv, &len)) {
+      ret = errno;
+      (*env)->Throw(env, newSocketException(env, ret,
+          "getsockopt(SO_SNDTIMEO) error: %s", terror(ret)));
+      return -1;
+    }
+    return timeValToJavaMillis(&tv);
+  case RECEIVE_TIMEOUT:
+    memset(&tv, 0, sizeof(tv));
+    len = sizeof(struct timeval);
+    if (getsockopt(fd, SOL_SOCKET, SO_RCVTIMEO, &tv, &len)) {
+      ret = errno;
+      (*env)->Throw(env, newSocketException(env, ret,
+          "getsockopt(SO_RCVTIMEO) error: %s", terror(ret)));
+      return -1;
+    }
+    return timeValToJavaMillis(&tv);
+  default:
+    (*env)->Throw(env, newRuntimeException(env,
+          "Invalid attribute type %d.", type));
+    return -1;
+  }
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_close0(
+JNIEnv *env, jclass clazz, jint fd)
+{
+  int ret;
+
+  RETRY_ON_EINTR(ret, close(fd));
+  if (ret) {
+    ret = errno;
+    (*env)->Throw(env, newSocketException(env, ret,
+          "close(2) error: %s", terror(ret)));
+  }
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_closeFileDescriptor0(
+JNIEnv *env, jclass clazz, jobject jfd)
+{
+  Java_org_apache_hadoop_net_unix_DomainSocket_close0(
+      env, clazz, fd_get(env, jfd));
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_shutdown0(
+JNIEnv *env, jclass clazz, jint fd)
+{
+  int ret;
+
+  RETRY_ON_EINTR(ret, shutdown(fd, SHUT_RDWR));
+  if (ret) {
+    ret = errno;
+    (*env)->Throw(env, newSocketException(env, ret,
+          "shutdown(2) error: %s", terror(ret)));
+  }
+}
+
+/**
+ * Write an entire buffer to a file descriptor.
+ *
+ * @param env            The JNI environment.
+ * @param fd             The fd to write to.
+ * @param buf            The buffer to write
+ * @param amt            The length of the buffer to write.
+ * @return               NULL on success; or the unraised exception representing
+ *                       the problem.
+ */
+static jthrowable write_fully(JNIEnv *env, int fd, int8_t *buf, int amt)
+{
+  int err, res;
+
+  while (amt > 0) {
+    res = write(fd, buf, amt);
+    if (res < 0) {
+      err = errno;
+      if (err == EINTR) {
+        continue;
+      }
+      return newSocketException(env, err, "write(2) error: %s", terror(err));
+    }
+    amt -= res;
+    buf += res;
+  }
+  return NULL;
+}
+
+/**
+ * Our auxillary data setup.
+ *
+ * See man 3 cmsg for more information about auxillary socket data on UNIX.
+ *
+ * We use __attribute__((packed)) to ensure that the compiler doesn't insert any
+ * padding between 'hdr' and 'fds'.
+ * We use __attribute__((aligned(8)) to ensure that the compiler puts the start
+ * of the structure at an address which is a multiple of 8.  If we did not do
+ * this, the attribute((packed)) would cause the compiler to generate a lot of
+ * slow code for accessing unaligned memory.
+ */
+struct cmsghdr_with_fds {
+  struct cmsghdr hdr;
+  int fds[MAX_PASSED_FDS];
+}  __attribute__((packed,aligned(8)));
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_sendFileDescriptors0(
+JNIEnv *env, jclass clazz, jint fd, jobject jfds, jobject jbuf,
+jint offset, jint length)
+{
+  struct iovec vec[1];
+  struct flexibleBuffer flexBuf;
+  struct cmsghdr_with_fds aux;
+  jint jfdsLen;
+  int i, ret = -1, auxLen;
+  struct msghdr socketMsg;
+  jthrowable jthr = NULL;
+
+  jthr = flexBufInit(env, &flexBuf, length);
+  if (jthr) {
+    goto done;
+  }
+  if (length <= 0) {
+    jthr = newException(env, "java/lang/IllegalArgumentException",
+        "You must write at least one byte.");
+    goto done;
+  }
+  jfdsLen = (*env)->GetArrayLength(env, jfds);
+  if (jfdsLen <= 0) {
+    jthr = newException(env, "java/lang/IllegalArgumentException",
+        "Called sendFileDescriptors with no file descriptors.");
+    goto done;
+  } else if (jfdsLen > MAX_PASSED_FDS) {
+    jfdsLen = 0;
+    jthr = newException(env, "java/lang/IllegalArgumentException",
+          "Called sendFileDescriptors with an array of %d length.  "
+          "The maximum is %d.", jfdsLen, MAX_PASSED_FDS);
+    goto done;
+  }
+  (*env)->GetByteArrayRegion(env, jbuf, offset, length, flexBuf.curBuf); 
+  jthr = (*env)->ExceptionOccurred(env);
+  if (jthr) {
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  memset(&vec, 0, sizeof(vec));
+  vec[0].iov_base = flexBuf.curBuf;
+  vec[0].iov_len = length;
+  auxLen = CMSG_LEN(jfdsLen * sizeof(int));
+  memset(&aux, 0, auxLen);
+  memset(&socketMsg, 0, sizeof(socketMsg));
+  socketMsg.msg_iov = vec;
+  socketMsg.msg_iovlen = 1;
+  socketMsg.msg_control = &aux;
+  socketMsg.msg_controllen = auxLen;
+  aux.hdr.cmsg_len = auxLen;
+  aux.hdr.cmsg_level = SOL_SOCKET;
+  aux.hdr.cmsg_type = SCM_RIGHTS;
+  for (i = 0; i < jfdsLen; i++) {
+    jobject jfd = (*env)->GetObjectArrayElement(env, jfds, i);
+    if (!jfd) {
+      jthr = (*env)->ExceptionOccurred(env);
+      if (jthr) {
+        (*env)->ExceptionClear(env);
+        goto done;
+      }
+      jthr = newException(env, "java/lang/NullPointerException",
+            "element %d of jfds was NULL.", i);
+      goto done;
+    }
+    aux.fds[i] = fd_get(env, jfd);
+    (*env)->DeleteLocalRef(env, jfd);
+    if (jthr) {
+      goto done;
+    }
+  }
+  RETRY_ON_EINTR(ret, sendmsg(fd, &socketMsg, 0));
+  if (ret < 0) {
+    ret = errno;
+    jthr = newSocketException(env, ret, "sendmsg(2) error: %s", terror(ret));
+    goto done;
+  }
+  length -= ret;
+  if (length > 0) {
+    // Write the rest of the bytes we were asked to send.
+    // This time, no fds will be attached.
+    jthr = write_fully(env, fd, flexBuf.curBuf + ret, length);
+    if (jthr) {
+      goto done;
+    }
+  }
+
+done:
+  flexBufFree(&flexBuf);
+  if (jthr) {
+    (*env)->Throw(env, jthr);
+  }
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_receiveFileDescriptors0(
+JNIEnv *env, jclass clazz, jint fd, jarray jfds, jarray jbuf,
+jint offset, jint length)
+{
+  struct iovec vec[1];
+  struct flexibleBuffer flexBuf;
+  struct cmsghdr_with_fds aux;
+  int i, jRecvFdsLen = 0, auxLen;
+  jint jfdsLen = 0;
+  struct msghdr socketMsg;
+  ssize_t bytesRead = -1;
+  jobject fdObj;
+  jthrowable jthr = NULL;
+
+  jthr = flexBufInit(env, &flexBuf, length);
+  if (jthr) {
+    goto done;
+  }
+  if (length <= 0) {
+    jthr = newRuntimeException(env, "You must read at least one byte.");
+    goto done;
+  }
+  jfdsLen = (*env)->GetArrayLength(env, jfds);
+  if (jfdsLen <= 0) {
+    jthr = newException(env, "java/lang/IllegalArgumentException",
+        "Called receiveFileDescriptors with an array of %d length.  "
+        "You must pass at least one fd.", jfdsLen);
+    goto done;
+  } else if (jfdsLen > MAX_PASSED_FDS) {
+    jfdsLen = 0;
+    jthr = newException(env, "java/lang/IllegalArgumentException",
+          "Called receiveFileDescriptors with an array of %d length.  "
+          "The maximum is %d.", jfdsLen, MAX_PASSED_FDS);
+    goto done;
+  }
+  for (i = 0; i < jfdsLen; i++) {
+    (*env)->SetObjectArrayElement(env, jfds, i, NULL);
+  }
+  vec[0].iov_base = flexBuf.curBuf;
+  vec[0].iov_len = length;
+  auxLen = CMSG_LEN(jfdsLen * sizeof(int));
+  memset(&aux, 0, auxLen);
+  memset(&socketMsg, 0, auxLen);
+  socketMsg.msg_iov = vec;
+  socketMsg.msg_iovlen = 1;
+  socketMsg.msg_control = &aux;
+  socketMsg.msg_controllen = auxLen;
+  aux.hdr.cmsg_len = auxLen;
+  aux.hdr.cmsg_level = SOL_SOCKET;
+  aux.hdr.cmsg_type = SCM_RIGHTS;
+  RETRY_ON_EINTR(bytesRead, recvmsg(fd, &socketMsg, 0));
+  if (bytesRead < 0) {
+    int ret = errno;
+    if (ret == ECONNABORTED) {
+      // The remote peer disconnected on us.  Treat this as an EOF.
+      bytesRead = -1;
+      goto done;
+    }
+    jthr = newSocketException(env, ret, "recvmsg(2) failed: %s",
+                              terror(ret));
+    goto done;
+  } else if (bytesRead == 0) {
+    bytesRead = -1;
+    goto done;
+  }
+  jRecvFdsLen = (aux.hdr.cmsg_len - sizeof(struct cmsghdr)) / sizeof(int);
+  for (i = 0; i < jRecvFdsLen; i++) {
+    fdObj = fd_create(env, aux.fds[i]);
+    if (!fdObj) {
+      jthr = (*env)->ExceptionOccurred(env);
+      (*env)->ExceptionClear(env);
+      goto done;
+    }
+    // Make this -1 so we don't attempt to close it twice in an error path.
+    aux.fds[i] = -1;
+    (*env)->SetObjectArrayElement(env, jfds, i, fdObj);
+    // There is no point keeping around a local reference to the fdObj.
+    // The array continues to reference it.
+    (*env)->DeleteLocalRef(env, fdObj);
+  }
+  (*env)->SetByteArrayRegion(env, jbuf, offset, length, flexBuf.curBuf);
+  jthr = (*env)->ExceptionOccurred(env);
+  if (jthr) {
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+done:
+  flexBufFree(&flexBuf);
+  if (jthr) {
+    // Free any FileDescriptor references we may have created,
+    // or file descriptors we may have been passed.
+    for (i = 0; i < jRecvFdsLen; i++) {
+      if (aux.fds[i] >= 0) {
+        RETRY_ON_EINTR(i, close(aux.fds[i]));
+        aux.fds[i] = -1;
+      }
+      fdObj = (*env)->GetObjectArrayElement(env, jfds, i);
+      if (fdObj) {
+        int ret, afd = fd_get(env, fdObj);
+        if (afd >= 0) {
+          RETRY_ON_EINTR(ret, close(afd));
+        }
+        (*env)->SetObjectArrayElement(env, jfds, i, NULL);
+        (*env)->DeleteLocalRef(env, fdObj);
+      }
+    }
+    (*env)->Throw(env, jthr);
+  }
+  return bytesRead;
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_readArray0(
+JNIEnv *env, jclass clazz, jint fd, jarray b, jint offset, jint length)
+{
+  int ret = -1;
+  struct flexibleBuffer flexBuf;
+  jthrowable jthr;
+
+  jthr = flexBufInit(env, &flexBuf, length);
+  if (jthr) {
+    goto done;
+  }
+  RETRY_ON_EINTR(ret, read(fd, flexBuf.curBuf, length));
+  if (ret < 0) {
+    ret = errno;
+    if (ret == ECONNABORTED) {
+      // The remote peer disconnected on us.  Treat this as an EOF.
+      ret = -1;
+      goto done;
+    }
+    jthr = newSocketException(env, ret, "read(2) error: %s", 
+                              terror(ret));
+    goto done;
+  }
+  if (ret == 0) {
+    goto done;
+  }
+  (*env)->SetByteArrayRegion(env, b, offset, ret, flexBuf.curBuf);
+  jthr = (*env)->ExceptionOccurred(env);
+  if (jthr) {
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+done:
+  flexBufFree(&flexBuf);
+  if (jthr) { 
+    (*env)->Throw(env, jthr);
+  }
+  return ret == 0 ? -1 : ret; // Java wants -1 on EOF
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_available0(
+JNIEnv *env, jclass clazz, jint fd)
+{
+  int ret, avail = 0;
+  jthrowable jthr = NULL;
+
+  RETRY_ON_EINTR(ret, ioctl(fd, FIONREAD, &avail));
+  if (ret < 0) {
+    ret = errno;
+    jthr = newSocketException(env, ret,
+              "ioctl(%d, FIONREAD) error: %s", fd, terror(ret));
+    goto done;
+  }
+done:
+  if (jthr) {
+    (*env)->Throw(env, jthr);
+  }
+  return avail;
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_writeArray0(
+JNIEnv *env, jclass clazz, jint fd, jarray b, jint offset, jint length)
+{
+  struct flexibleBuffer flexBuf;
+  jthrowable jthr;
+
+  jthr = flexBufInit(env, &flexBuf, length);
+  if (jthr) {
+    goto done;
+  }
+  (*env)->GetByteArrayRegion(env, b, offset, length, flexBuf.curBuf);
+  jthr = (*env)->ExceptionOccurred(env);
+  if (jthr) {
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  jthr = write_fully(env, fd, flexBuf.curBuf, length);
+  if (jthr) {
+    goto done;
+  }
+
+done:
+  flexBufFree(&flexBuf);
+  if (jthr) { 
+    (*env)->Throw(env, jthr);
+  }
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_readByteBufferDirect0(
+JNIEnv *env, jclass clazz, jint fd, jobject dst, jint position, jint remaining)
+{
+  uint8_t *buf;
+  jthrowable jthr = NULL;
+  int res = -1;
+
+  buf = (*env)->GetDirectBufferAddress(env, dst);
+  if (!buf) {
+    jthr = newRuntimeException(env, "GetDirectBufferAddress failed.");
+    goto done;
+  }
+  RETRY_ON_EINTR(res, read(fd, buf + position, remaining));
+  if (res < 0) {
+    res = errno;
+    if (res != ECONNABORTED) {
+      jthr = newSocketException(env, res, "read(2) error: %s", 
+                                terror(res));
+      goto done;
+    } else {
+      // The remote peer disconnected on us.  Treat this as an EOF.
+      res = -1;
+    }
+  }
+done:
+  if (jthr) {
+    (*env)->Throw(env, jthr);
+  }
+  return res;
+}

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org_apache_hadoop.h

@@ -180,6 +180,10 @@ static FARPROC WINAPI do_dlsym(JNIEnv *env, HMODULE handle, LPCSTR symbol) {
     THROW(env, "java/lang/InternalError", exception_msg); \
   }
 
+#define RETRY_ON_EINTR(ret, expr) do { \
+  ret = expr; \
+} while ((ret == -1) && (errno == EINTR));
+
 #endif
 
 //vim: sw=2: ts=2: et

+ 524 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/CompressDecompressTester.java

@@ -0,0 +1,524 @@
+/*
+ * 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.io.compress;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.lz4.Lz4Compressor;
+import org.apache.hadoop.io.compress.snappy.SnappyCompressor;
+import org.apache.hadoop.io.compress.zlib.BuiltInZlibDeflater;
+import org.apache.hadoop.io.compress.zlib.ZlibCompressor;
+import org.apache.hadoop.io.compress.zlib.ZlibFactory;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.log4j.Logger;
+import org.junit.Assert;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import static org.junit.Assert.*;
+
+public class CompressDecompressTester<T extends Compressor, E extends Decompressor> {
+
+  private static final Logger logger = Logger
+      .getLogger(CompressDecompressTester.class);
+
+  private final byte[] originalRawData;
+
+  private ImmutableList<TesterPair<T, E>> pairs = ImmutableList.of();
+  private ImmutableList.Builder<TesterPair<T, E>> builder = ImmutableList.builder();     
+
+  private ImmutableSet<CompressionTestStrategy> stateges = ImmutableSet.of();
+
+  private PreAssertionTester<T, E> assertionDelegate;
+  
+  public CompressDecompressTester(byte[] originalRawData) {
+    this.originalRawData = Arrays.copyOf(originalRawData,
+        originalRawData.length);
+    this.assertionDelegate = new PreAssertionTester<T, E>() {
+
+      @Override
+      public ImmutableList<TesterPair<T, E>> filterOnAssumeWhat(
+          ImmutableList<TesterPair<T, E>> pairs) {
+        ImmutableList.Builder<TesterPair<T, E>> builder = ImmutableList
+            .builder();
+
+        for (TesterPair<T, E> pair : pairs) {
+          if (isAvailable(pair))
+            builder.add(pair);
+        }
+        return builder.build();
+      }
+    };
+  }
+
+  private static boolean isNativeSnappyLoadable() {
+    boolean snappyAvailable = false;
+    boolean loaded = false;
+    try {
+      System.loadLibrary("snappy");
+      logger.warn("Snappy native library is available");
+      snappyAvailable = true;
+      boolean hadoopNativeAvailable = NativeCodeLoader.isNativeCodeLoaded();
+      loaded = snappyAvailable && hadoopNativeAvailable;
+      if (loaded) {
+        logger.info("Snappy native library loaded");
+      } else {
+        logger.warn("Snappy native library not loaded");
+      }
+    } catch (Throwable t) {
+      logger.warn("Failed to load snappy: ", t);
+      return false;
+    }
+    return loaded;
+  }
+
+  public static <T extends Compressor, E extends Decompressor> CompressDecompressTester<T, E> of(
+      byte[] rawData) {
+    return new CompressDecompressTester<T, E>(rawData);
+  }
+  
+
+  public CompressDecompressTester<T, E> withCompressDecompressPair(
+      T compressor, E decompressor) {
+    addPair(
+        compressor,
+        decompressor,
+        Joiner.on("_").join(compressor.getClass().getCanonicalName(),
+            decompressor.getClass().getCanonicalName()));
+    return this;
+  }
+  
+  public CompressDecompressTester<T, E> withTestCases(
+      ImmutableSet<CompressionTestStrategy> stateges) {
+    this.stateges = ImmutableSet.copyOf(stateges);
+    return this;
+  }
+
+  private void addPair(T compressor, E decompressor, String name) {
+    builder.add(new TesterPair<T, E>(name, compressor, decompressor));
+  }
+
+  public void test() throws InstantiationException, IllegalAccessException {
+    pairs = builder.build();
+    pairs = assertionDelegate.filterOnAssumeWhat(pairs);
+
+    for (TesterPair<T, E> pair : pairs) {
+      for (CompressionTestStrategy strategy : stateges) {
+        strategy.getTesterStrategy().assertCompression(pair.getName(),
+            pair.getCompressor(), pair.getDecompressor(),
+            Arrays.copyOf(originalRawData, originalRawData.length));
+      }
+    }
+    endAll(pairs);
+  }
+
+  private void endAll(ImmutableList<TesterPair<T, E>> pairs) {
+    for (TesterPair<T, E> pair : pairs)
+      pair.end();
+  }
+
+  interface PreAssertionTester<T extends Compressor, E extends Decompressor> {
+    ImmutableList<TesterPair<T, E>> filterOnAssumeWhat(
+        ImmutableList<TesterPair<T, E>> pairs);
+  }
+
+  public enum CompressionTestStrategy {
+
+    COMPRESS_DECOMPRESS_ERRORS(new TesterCompressionStrategy() {
+      private final Joiner joiner = Joiner.on("- ");
+
+      @Override
+      public void assertCompression(String name, Compressor compressor,
+          Decompressor decompressor, byte[] rawData) {
+        assertTrue(checkSetInputNullPointerException(compressor));
+        assertTrue(checkSetInputNullPointerException(decompressor));
+
+        assertTrue(checkCompressArrayIndexOutOfBoundsException(compressor,
+            rawData));
+        assertTrue(checkCompressArrayIndexOutOfBoundsException(decompressor,
+            rawData));
+
+        assertTrue(checkCompressNullPointerException(compressor, rawData));
+        assertTrue(checkCompressNullPointerException(decompressor, rawData));
+
+        assertTrue(checkSetInputArrayIndexOutOfBoundsException(compressor));
+        assertTrue(checkSetInputArrayIndexOutOfBoundsException(decompressor));
+      }
+
+      private boolean checkSetInputNullPointerException(Compressor compressor) {
+        try {
+          compressor.setInput(null, 0, 1);
+        } catch (NullPointerException npe) {
+          return true;
+        } catch (Exception ex) {
+          logger.error(joiner.join(compressor.getClass().getCanonicalName(),
+              "checkSetInputNullPointerException error !!!"));
+        }
+        return false;
+      }
+
+      private boolean checkCompressNullPointerException(Compressor compressor,
+          byte[] rawData) {
+        try {
+          compressor.setInput(rawData, 0, rawData.length);
+          compressor.compress(null, 0, 1);
+        } catch (NullPointerException npe) {
+          return true;
+        } catch (Exception ex) {
+          logger.error(joiner.join(compressor.getClass().getCanonicalName(),
+              "checkCompressNullPointerException error !!!"));
+        }
+        return false;
+      }
+
+      private boolean checkCompressNullPointerException(
+          Decompressor decompressor, byte[] rawData) {
+        try {
+          decompressor.setInput(rawData, 0, rawData.length);
+          decompressor.decompress(null, 0, 1);
+        } catch (NullPointerException npe) {
+          return true;
+        } catch (Exception ex) {
+          logger.error(joiner.join(decompressor.getClass().getCanonicalName(),
+              "checkCompressNullPointerException error !!!"));
+        }
+        return false;
+      }
+
+      private boolean checkSetInputNullPointerException(
+          Decompressor decompressor) {
+        try {
+          decompressor.setInput(null, 0, 1);
+        } catch (NullPointerException npe) {
+          return true;
+        } catch (Exception ex) {
+          logger.error(joiner.join(decompressor.getClass().getCanonicalName(),
+              "checkSetInputNullPointerException error !!!"));
+        }
+        return false;
+      }
+
+      private boolean checkSetInputArrayIndexOutOfBoundsException(
+          Compressor compressor) {
+        try {
+          compressor.setInput(new byte[] { (byte) 0 }, 0, -1);
+        } catch (ArrayIndexOutOfBoundsException e) {
+          return true;
+        } catch (Exception e) {
+          logger.error(joiner.join(compressor.getClass().getCanonicalName(),
+              "checkSetInputArrayIndexOutOfBoundsException error !!!"));
+        }
+        return false;
+      }
+
+      private boolean checkCompressArrayIndexOutOfBoundsException(
+          Compressor compressor, byte[] rawData) {
+        try {
+          compressor.setInput(rawData, 0, rawData.length);
+          compressor.compress(new byte[rawData.length], 0, -1);
+        } catch (ArrayIndexOutOfBoundsException e) {
+          return true;
+        } catch (Exception e) {
+          logger.error(joiner.join(compressor.getClass().getCanonicalName(),
+              "checkCompressArrayIndexOutOfBoundsException error !!!"));
+        }
+        return false;
+      }
+
+      private boolean checkCompressArrayIndexOutOfBoundsException(
+          Decompressor decompressor, byte[] rawData) {
+        try {
+          decompressor.setInput(rawData, 0, rawData.length);
+          decompressor.decompress(new byte[rawData.length], 0, -1);
+        } catch (ArrayIndexOutOfBoundsException e) {
+          return true;
+        } catch (Exception e) {
+          logger.error(joiner.join(decompressor.getClass().getCanonicalName(),
+              "checkCompressArrayIndexOutOfBoundsException error !!!"));
+        }
+        return false;
+      }
+
+      private boolean checkSetInputArrayIndexOutOfBoundsException(
+          Decompressor decompressor) {
+        try {
+          decompressor.setInput(new byte[] { (byte) 0 }, 0, -1);
+        } catch (ArrayIndexOutOfBoundsException e) {
+          return true;
+        } catch (Exception e) {
+          logger.error(joiner.join(decompressor.getClass().getCanonicalName(),
+              "checkNullPointerException error !!!"));
+        }
+        return false;
+      }
+
+    }),
+
+    COMPRESS_DECOMPRESS_SINGLE_BLOCK(new TesterCompressionStrategy() {
+      final Joiner joiner = Joiner.on("- ");
+
+      @Override
+      public void assertCompression(String name, Compressor compressor,
+          Decompressor decompressor, byte[] rawData) {
+
+        int cSize = 0;
+        int decompressedSize = 0;
+        byte[] compressedResult = new byte[rawData.length];
+        byte[] decompressedBytes = new byte[rawData.length];
+        try {
+          assertTrue(
+              joiner.join(name, "compressor.needsInput before error !!!"),
+              compressor.needsInput());
+          assertTrue(
+              joiner.join(name, "compressor.getBytesWritten before error !!!"),
+              compressor.getBytesWritten() == 0);
+          compressor.setInput(rawData, 0, rawData.length);
+          compressor.finish();
+          while (!compressor.finished()) {
+            cSize += compressor.compress(compressedResult, 0,
+                compressedResult.length);
+          }
+          compressor.reset();
+
+          assertTrue(
+              joiner.join(name, "decompressor.needsInput() before error !!!"),
+              decompressor.needsInput());
+          decompressor.setInput(compressedResult, 0, cSize);
+          assertFalse(
+              joiner.join(name, "decompressor.needsInput() after error !!!"),
+              decompressor.needsInput());
+          while (!decompressor.finished()) {
+            decompressedSize = decompressor.decompress(decompressedBytes, 0,
+                decompressedBytes.length);
+          }
+          decompressor.reset();
+          assertTrue(joiner.join(name, " byte size not equals error !!!"),
+              decompressedSize == rawData.length);
+          assertArrayEquals(
+              joiner.join(name, " byte arrays not equals error !!!"), rawData,
+              decompressedBytes);
+        } catch (Exception ex) {
+          fail(joiner.join(name, ex.getMessage()));
+        }
+      }
+    }),
+
+    COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM(new TesterCompressionStrategy() {
+      final Joiner joiner = Joiner.on("- ");
+      final ImmutableMap<Class<? extends Compressor>, Integer> emptySize = ImmutableMap
+          .of(Lz4Compressor.class, 4, ZlibCompressor.class, 16,
+              SnappyCompressor.class, 4, BuiltInZlibDeflater.class, 16);
+
+      @Override
+      void assertCompression(String name, Compressor compressor,
+          Decompressor decompressor, byte[] originalRawData) {
+        byte[] buf = null;
+        ByteArrayInputStream bytesIn = null;
+        BlockDecompressorStream blockDecompressorStream = null;
+        ByteArrayOutputStream bytesOut = new ByteArrayOutputStream();
+        // close without write
+        try {
+          compressor.reset();
+          // decompressor.end();
+          BlockCompressorStream blockCompressorStream = new BlockCompressorStream(
+              bytesOut, compressor, 1024, 0);
+          blockCompressorStream.close();
+          // check compressed output
+          buf = bytesOut.toByteArray();
+          int emSize = emptySize.get(compressor.getClass());
+          Assert.assertEquals(
+              joiner.join(name, "empty stream compressed output size != "
+                  + emSize), emSize, buf.length);
+          // use compressed output as input for decompression
+          bytesIn = new ByteArrayInputStream(buf);
+          // create decompression stream
+          blockDecompressorStream = new BlockDecompressorStream(bytesIn,
+              decompressor, 1024);
+          // no byte is available because stream was closed
+          assertEquals(joiner.join(name, " return value is not -1"), -1,
+              blockDecompressorStream.read());
+        } catch (IOException e) {
+          fail(joiner.join(name, e.getMessage()));
+        } finally {
+          if (blockDecompressorStream != null)
+            try {
+              bytesOut.close();
+              blockDecompressorStream.close();
+              bytesIn.close();
+              blockDecompressorStream.close();
+            } catch (IOException e) {
+            }
+        }
+      }
+
+    }),
+
+    COMPRESS_DECOMPRESS_BLOCK(new TesterCompressionStrategy() {
+      private final Joiner joiner = Joiner.on("- ");
+      private static final int BLOCK_SIZE = 512;
+      private final byte[] operationBlock = new byte[BLOCK_SIZE];
+      // Use default of 512 as bufferSize and compressionOverhead of
+      // (1% of bufferSize + 12 bytes) = 18 bytes (zlib algorithm).
+      private static final int overheadSpace = BLOCK_SIZE / 100 + 12;
+
+      @Override
+      public void assertCompression(String name, Compressor compressor,
+          Decompressor decompressor, byte[] originalRawData) {
+        int off = 0;
+        int len = originalRawData.length;
+        int maxSize = BLOCK_SIZE - overheadSpace;
+        int compresSize = 0;
+        List<Integer> blockLabels = new ArrayList<Integer>();
+        ByteArrayOutputStream compressedOut = new ByteArrayOutputStream();
+        ByteArrayOutputStream decompressOut = new ByteArrayOutputStream();
+        try {
+          if (originalRawData.length > maxSize) {
+            do {
+              int bufLen = Math.min(len, maxSize);
+              compressor.setInput(originalRawData, off, bufLen);
+              compressor.finish();
+              while (!compressor.finished()) {
+                compresSize = compressor.compress(operationBlock, 0,
+                    operationBlock.length);
+                compressedOut.write(operationBlock, 0, compresSize);
+                blockLabels.add(compresSize);
+              }
+              compressor.reset();
+              off += bufLen;
+              len -= bufLen;
+            } while (len > 0);
+          }
+
+          off = 0;
+          // compressed bytes
+          byte[] compressedBytes = compressedOut.toByteArray();
+          for (Integer step : blockLabels) {
+            decompressor.setInput(compressedBytes, off, step);
+            while (!decompressor.finished()) {
+              int dSize = decompressor.decompress(operationBlock, 0,
+                  operationBlock.length);
+              decompressOut.write(operationBlock, 0, dSize);
+            }
+            decompressor.reset();
+            off = off + step;
+          }
+          assertArrayEquals(
+              joiner.join(name, "byte arrays not equals error !!!"),
+              originalRawData, decompressOut.toByteArray());
+        } catch (Exception ex) {
+          fail(joiner.join(name, ex.getMessage()));
+        } finally {
+          try {
+            compressedOut.close();
+          } catch (IOException e) {
+          }
+          try {
+            decompressOut.close();
+          } catch (IOException e) {
+          }
+        }
+      }
+    });
+
+    private final TesterCompressionStrategy testerStrategy;
+
+    CompressionTestStrategy(TesterCompressionStrategy testStrategy) {
+      this.testerStrategy = testStrategy;
+    }
+
+    public TesterCompressionStrategy getTesterStrategy() {
+      return testerStrategy;
+    }
+  }
+
+  static final class TesterPair<T extends Compressor, E extends Decompressor> {
+    private final T compressor;
+    private final E decompressor;
+    private final String name;
+
+    TesterPair(String name, T compressor, E decompressor) {
+      this.compressor = compressor;
+      this.decompressor = decompressor;
+      this.name = name;
+    }
+
+    public void end() {
+      Configuration cfg = new Configuration();
+      compressor.reinit(cfg);
+      compressor.end();
+      decompressor.end();
+    }
+
+    public T getCompressor() {
+      return compressor;
+    }
+
+    public E getDecompressor() {
+      return decompressor;
+    }
+
+    public String getName() {
+      return name;
+    }
+  }
+  
+  /**
+   * Method for compressor availability check
+   */
+  private static <T extends Compressor, E extends Decompressor> boolean isAvailable(TesterPair<T, E> pair) {
+    Compressor compressor = pair.compressor;
+
+    if (compressor.getClass().isAssignableFrom(Lz4Compressor.class)
+            && (NativeCodeLoader.isNativeCodeLoaded()))
+      return true;
+
+    else if (compressor.getClass().isAssignableFrom(BuiltInZlibDeflater.class)
+            && NativeCodeLoader.isNativeCodeLoaded())
+      return true;
+
+    else if (compressor.getClass().isAssignableFrom(ZlibCompressor.class)) {
+      return ZlibFactory.isNativeZlibLoaded(new Configuration());
+    }              
+    else if (compressor.getClass().isAssignableFrom(SnappyCompressor.class)
+            && isNativeSnappyLoadable())
+      return true;
+    
+    return false;      
+  }
+  
+  abstract static class TesterCompressionStrategy {
+
+    protected final Logger logger = Logger.getLogger(getClass());
+
+    abstract void assertCompression(String name, Compressor compressor,
+        Decompressor decompressor, byte[] originalRawData);
+  }
+}

+ 101 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCompressorDecompressor.java

@@ -0,0 +1,101 @@
+/*
+ * 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.io.compress;
+
+import static org.junit.Assert.fail;
+import java.util.Random;
+import org.apache.hadoop.io.compress.CompressDecompressTester.CompressionTestStrategy;
+import org.apache.hadoop.io.compress.lz4.Lz4Compressor;
+import org.apache.hadoop.io.compress.lz4.Lz4Decompressor;
+import org.apache.hadoop.io.compress.snappy.SnappyCompressor;
+import org.apache.hadoop.io.compress.snappy.SnappyDecompressor;
+import org.apache.hadoop.io.compress.zlib.BuiltInZlibDeflater;
+import org.apache.hadoop.io.compress.zlib.BuiltInZlibInflater;
+import org.junit.Test;
+import com.google.common.collect.ImmutableSet;
+
+/** 
+ * Test for pairs:
+ * <pre>
+ * SnappyCompressor/SnappyDecompressor
+ * Lz4Compressor/Lz4Decompressor
+ * BuiltInZlibDeflater/new BuiltInZlibInflater
+ *
+ *
+ * Note: we can't use ZlibCompressor/ZlibDecompressor here 
+ * because his constructor can throw exception (if native libraries not found)
+ * For ZlibCompressor/ZlibDecompressor pair testing used {@code TestZlibCompressorDecompressor}   
+ *
+ * </pre>
+ *
+ */
+public class TestCompressorDecompressor {
+  
+  private static final Random rnd = new Random(12345L);
+  
+  @Test
+  public void testCompressorDecompressor() {
+    // no more for this data
+    int SIZE = 44 * 1024;
+    
+    byte[] rawData = generate(SIZE);
+    try {
+      CompressDecompressTester.of(rawData)
+          .withCompressDecompressPair(new SnappyCompressor(), new SnappyDecompressor())
+          .withCompressDecompressPair(new Lz4Compressor(), new Lz4Decompressor())
+          .withCompressDecompressPair(new BuiltInZlibDeflater(), new BuiltInZlibInflater())
+          .withTestCases(ImmutableSet.of(CompressionTestStrategy.COMPRESS_DECOMPRESS_SINGLE_BLOCK,
+                      CompressionTestStrategy.COMPRESS_DECOMPRESS_BLOCK,
+                      CompressionTestStrategy.COMPRESS_DECOMPRESS_ERRORS,
+                      CompressionTestStrategy.COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM))
+          .test();
+
+    } catch (Exception ex) {
+      fail("testCompressorDecompressor error !!!" + ex);
+    }
+  }
+  
+  @Test
+  public void testCompressorDecompressorWithExeedBufferLimit() {
+    int BYTE_SIZE = 100 * 1024;
+    byte[] rawData = generate(BYTE_SIZE);
+    try {
+      CompressDecompressTester.of(rawData)
+          .withCompressDecompressPair(
+              new SnappyCompressor(BYTE_SIZE + BYTE_SIZE / 2),
+              new SnappyDecompressor(BYTE_SIZE + BYTE_SIZE / 2))
+          .withCompressDecompressPair(new Lz4Compressor(BYTE_SIZE),
+              new Lz4Decompressor(BYTE_SIZE))
+          .withTestCases(ImmutableSet.of(CompressionTestStrategy.COMPRESS_DECOMPRESS_SINGLE_BLOCK,
+                      CompressionTestStrategy.COMPRESS_DECOMPRESS_BLOCK,
+                      CompressionTestStrategy.COMPRESS_DECOMPRESS_ERRORS,
+                      CompressionTestStrategy.COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM))
+          .test();
+
+    } catch (Exception ex) {
+      fail("testCompressorDecompressorWithExeedBufferLimit error !!!" + ex);
+    }
+  }
+       
+  public static byte[] generate(int size) {
+    byte[] array = new byte[size];
+    for (int i = 0; i < size; i++)
+      array[i] = (byte) rnd.nextInt(16);
+    return array;
+  }
+}

+ 316 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/lz4/TestLz4CompressorDecompressor.java

@@ -0,0 +1,316 @@
+/**
+ * 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.io.compress.lz4;
+
+import static org.junit.Assert.*;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.compress.BlockCompressorStream;
+import org.apache.hadoop.io.compress.BlockDecompressorStream;
+import org.apache.hadoop.io.compress.CompressionInputStream;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Lz4Codec;
+import org.apache.hadoop.io.compress.lz4.Lz4Compressor;
+import org.apache.hadoop.io.compress.lz4.Lz4Decompressor;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assume.*;
+
+public class TestLz4CompressorDecompressor {
+  
+  private static final Random rnd = new Random(12345l);
+
+  @Before
+  public void before() {
+    assumeTrue(Lz4Codec.isNativeCodeLoaded());
+  }
+
+  //test on NullPointerException in {@code compressor.setInput()} 
+  @Test
+  public void testCompressorSetInputNullPointerException() {
+    try {
+      Lz4Compressor compressor = new Lz4Compressor();
+      compressor.setInput(null, 0, 10);
+      fail("testCompressorSetInputNullPointerException error !!!");
+    } catch (NullPointerException ex) {
+      // expected
+    } catch (Exception e) {
+      fail("testCompressorSetInputNullPointerException ex error !!!");
+    }
+  }
+
+  //test on NullPointerException in {@code decompressor.setInput()}
+  @Test
+  public void testDecompressorSetInputNullPointerException() {
+    try {
+      Lz4Decompressor decompressor = new Lz4Decompressor();
+      decompressor.setInput(null, 0, 10);
+      fail("testDecompressorSetInputNullPointerException error !!!");
+    } catch (NullPointerException ex) {
+      // expected
+    } catch (Exception e) {
+      fail("testDecompressorSetInputNullPointerException ex error !!!");
+    }
+  }
+  
+  //test on ArrayIndexOutOfBoundsException in {@code compressor.setInput()}
+  @Test
+  public void testCompressorSetInputAIOBException() {
+    try {
+      Lz4Compressor compressor = new Lz4Compressor();
+      compressor.setInput(new byte[] {}, -5, 10);
+      fail("testCompressorSetInputAIOBException error !!!");
+    } catch (ArrayIndexOutOfBoundsException ex) {
+      // expected
+    } catch (Exception ex) {
+      fail("testCompressorSetInputAIOBException ex error !!!");
+    }
+  }
+
+  //test on ArrayIndexOutOfBoundsException in {@code decompressor.setInput()}
+  @Test
+  public void testDecompressorSetInputAIOUBException() {
+    try {
+      Lz4Decompressor decompressor = new Lz4Decompressor();
+      decompressor.setInput(new byte[] {}, -5, 10);
+      fail("testDecompressorSetInputAIOBException error !!!");
+    } catch (ArrayIndexOutOfBoundsException ex) {
+      // expected
+    } catch (Exception e) {
+      fail("testDecompressorSetInputAIOBException ex error !!!");
+    }
+  }
+
+  //test on NullPointerException in {@code compressor.compress()}  
+  @Test
+  public void testCompressorCompressNullPointerException() {
+    try {
+      Lz4Compressor compressor = new Lz4Compressor();
+      byte[] bytes = generate(1024 * 6);
+      compressor.setInput(bytes, 0, bytes.length);
+      compressor.compress(null, 0, 0);
+      fail("testCompressorCompressNullPointerException error !!!");
+    } catch (NullPointerException ex) {
+      // expected
+    } catch (Exception e) {
+      fail("testCompressorCompressNullPointerException ex error !!!");
+    }
+  }
+
+  //test on NullPointerException in {@code decompressor.decompress()}  
+  @Test
+  public void testDecompressorCompressNullPointerException() {
+    try {
+      Lz4Decompressor decompressor = new Lz4Decompressor();
+      byte[] bytes = generate(1024 * 6);
+      decompressor.setInput(bytes, 0, bytes.length);
+      decompressor.decompress(null, 0, 0);
+      fail("testDecompressorCompressNullPointerException error !!!");
+    } catch (NullPointerException ex) {
+      // expected
+    } catch (Exception e) {
+      fail("testDecompressorCompressNullPointerException ex error !!!");
+    }
+  }
+
+  //test on ArrayIndexOutOfBoundsException in {@code compressor.compress()}  
+  @Test
+  public void testCompressorCompressAIOBException() {
+    try {
+      Lz4Compressor compressor = new Lz4Compressor();
+      byte[] bytes = generate(1024 * 6);
+      compressor.setInput(bytes, 0, bytes.length);
+      compressor.compress(new byte[] {}, 0, -1);
+      fail("testCompressorCompressAIOBException error !!!");
+    } catch (ArrayIndexOutOfBoundsException ex) {
+      // expected
+    } catch (Exception e) {
+      fail("testCompressorCompressAIOBException ex error !!!");
+    }
+  }
+
+  //test on ArrayIndexOutOfBoundsException in decompressor.decompress()  
+  @Test
+  public void testDecompressorCompressAIOBException() {
+    try {
+      Lz4Decompressor decompressor = new Lz4Decompressor();
+      byte[] bytes = generate(1024 * 6);
+      decompressor.setInput(bytes, 0, bytes.length);
+      decompressor.decompress(new byte[] {}, 0, -1);
+      fail("testDecompressorCompressAIOBException error !!!");
+    } catch (ArrayIndexOutOfBoundsException ex) {
+      // expected
+    } catch (Exception e) {
+      fail("testDecompressorCompressAIOBException ex error !!!");
+    }
+  }
+  
+  // test Lz4Compressor compressor.compress()  
+  @Test
+  public void testSetInputWithBytesSizeMoreThenDefaultLz4CompressorByfferSize() {
+    int BYTES_SIZE = 1024 * 64 + 1;
+    try {
+      Lz4Compressor compressor = new Lz4Compressor();
+      byte[] bytes = generate(BYTES_SIZE);
+      assertTrue("needsInput error !!!", compressor.needsInput());
+      compressor.setInput(bytes, 0, bytes.length);
+      byte[] emptyBytes = new byte[BYTES_SIZE];
+      int csize = compressor.compress(emptyBytes, 0, bytes.length);
+      assertTrue(
+          "testSetInputWithBytesSizeMoreThenDefaultLz4CompressorByfferSize error !!!",
+          csize != 0);
+    } catch (Exception ex) {
+      fail("testSetInputWithBytesSizeMoreThenDefaultLz4CompressorByfferSize ex error !!!");
+    }
+  }
+
+  // test compress/decompress process 
+  @Test
+  public void testCompressDecompress() {
+    int BYTE_SIZE = 1024 * 54;
+    byte[] bytes = generate(BYTE_SIZE);
+    Lz4Compressor compressor = new Lz4Compressor();
+    try {
+      compressor.setInput(bytes, 0, bytes.length);
+      assertTrue("Lz4CompressDecompress getBytesRead error !!!",
+          compressor.getBytesRead() > 0);
+      assertTrue(
+          "Lz4CompressDecompress getBytesWritten before compress error !!!",
+          compressor.getBytesWritten() == 0);
+
+      byte[] compressed = new byte[BYTE_SIZE];
+      int cSize = compressor.compress(compressed, 0, compressed.length);
+      assertTrue(
+          "Lz4CompressDecompress getBytesWritten after compress error !!!",
+          compressor.getBytesWritten() > 0);
+      Lz4Decompressor decompressor = new Lz4Decompressor();
+      // set as input for decompressor only compressed data indicated with cSize
+      decompressor.setInput(compressed, 0, cSize);
+      byte[] decompressed = new byte[BYTE_SIZE];
+      decompressor.decompress(decompressed, 0, decompressed.length);
+
+      assertTrue("testLz4CompressDecompress finished error !!!", decompressor.finished());      
+      assertArrayEquals(bytes, decompressed);
+      compressor.reset();
+      decompressor.reset();
+      assertTrue("decompressor getRemaining error !!!",decompressor.getRemaining() == 0);
+    } catch (Exception e) {
+      fail("testLz4CompressDecompress ex error!!!");
+    }
+  }
+
+  // test compress/decompress with empty stream
+  @Test
+  public void testCompressorDecompressorEmptyStreamLogic() {
+    ByteArrayInputStream bytesIn = null;
+    ByteArrayOutputStream bytesOut = null;
+    byte[] buf = null;
+    BlockDecompressorStream blockDecompressorStream = null;
+    try {
+      // compress empty stream
+      bytesOut = new ByteArrayOutputStream();
+      BlockCompressorStream blockCompressorStream = new BlockCompressorStream(
+          bytesOut, new Lz4Compressor(), 1024, 0);
+      // close without write
+      blockCompressorStream.close();
+      // check compressed output
+      buf = bytesOut.toByteArray();
+      assertEquals("empty stream compressed output size != 4", 4, buf.length);
+      // use compressed output as input for decompression
+      bytesIn = new ByteArrayInputStream(buf);
+      // create decompression stream
+      blockDecompressorStream = new BlockDecompressorStream(bytesIn,
+          new Lz4Decompressor(), 1024);
+      // no byte is available because stream was closed
+      assertEquals("return value is not -1", -1, blockDecompressorStream.read());
+    } catch (Exception e) {
+      fail("testCompressorDecompressorEmptyStreamLogic ex error !!!"
+          + e.getMessage());
+    } finally {
+      if (blockDecompressorStream != null)
+        try {
+          bytesIn.close();
+          bytesOut.close();
+          blockDecompressorStream.close();
+        } catch (IOException e) {
+        }
+    }
+  }
+  
+  // test compress/decompress process through CompressionOutputStream/CompressionInputStream api 
+  @Test
+  public void testCompressorDecopressorLogicWithCompressionStreams() {
+    DataOutputStream deflateOut = null;
+    DataInputStream inflateIn = null;
+    int BYTE_SIZE = 1024 * 100;
+    byte[] bytes = generate(BYTE_SIZE);
+    int bufferSize = 262144;
+    int compressionOverhead = (bufferSize / 6) + 32;
+    try {
+      DataOutputBuffer compressedDataBuffer = new DataOutputBuffer();
+      CompressionOutputStream deflateFilter = new BlockCompressorStream(
+          compressedDataBuffer, new Lz4Compressor(bufferSize), bufferSize,
+          compressionOverhead);
+      deflateOut = new DataOutputStream(new BufferedOutputStream(deflateFilter));
+      deflateOut.write(bytes, 0, bytes.length);
+      deflateOut.flush();
+      deflateFilter.finish();
+
+      DataInputBuffer deCompressedDataBuffer = new DataInputBuffer();
+      deCompressedDataBuffer.reset(compressedDataBuffer.getData(), 0,
+          compressedDataBuffer.getLength());
+
+      CompressionInputStream inflateFilter = new BlockDecompressorStream(
+          deCompressedDataBuffer, new Lz4Decompressor(bufferSize), bufferSize);
+
+      inflateIn = new DataInputStream(new BufferedInputStream(inflateFilter));
+
+      byte[] result = new byte[BYTE_SIZE];
+      inflateIn.read(result);
+
+      assertArrayEquals("original array not equals compress/decompressed array", result,
+          bytes);
+    } catch (IOException e) {
+      fail("testLz4CompressorDecopressorLogicWithCompressionStreams ex error !!!");
+    } finally {
+      try {
+        if (deflateOut != null)
+          deflateOut.close();
+        if (inflateIn != null)
+          inflateIn.close();
+      } catch (Exception e) {
+      }
+    }
+  }  
+
+  public static byte[] generate(int size) {
+    byte[] array = new byte[size];
+    for (int i = 0; i < size; i++)
+      array[i] = (byte)rnd.nextInt(16);
+    return array;
+  }
+}

+ 362 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java

@@ -0,0 +1,362 @@
+/**
+ * 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.io.compress.zlib;
+
+import static org.junit.Assert.*;
+import static org.junit.Assume.*;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.compress.CompressDecompressTester;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DecompressorStream;
+import org.apache.hadoop.io.compress.CompressDecompressTester.CompressionTestStrategy;
+import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
+import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
+import org.junit.Before;
+import org.junit.Test;
+import com.google.common.collect.ImmutableSet;
+
+public class TestZlibCompressorDecompressor {
+
+  private static final Random random = new Random(12345L);
+
+  @Before
+  public void before() {
+    assumeTrue(ZlibFactory.isNativeZlibLoaded(new Configuration()));
+  }  
+  
+  @Test
+  public void testZlibCompressorDecompressor() {
+    try {
+      int SIZE = 44 * 1024;
+      byte[] rawData = generate(SIZE);
+      
+      CompressDecompressTester.of(rawData)
+        .withCompressDecompressPair(new ZlibCompressor(), new ZlibDecompressor())
+        .withTestCases(ImmutableSet.of(CompressionTestStrategy.COMPRESS_DECOMPRESS_SINGLE_BLOCK,
+           CompressionTestStrategy.COMPRESS_DECOMPRESS_BLOCK,
+           CompressionTestStrategy.COMPRESS_DECOMPRESS_ERRORS,
+           CompressionTestStrategy.COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM))
+         .test();
+    } catch (Exception ex) {
+      fail("testCompressorDecompressor error !!!" + ex);
+    }
+  }
+  
+  @Test
+  public void testCompressorDecompressorWithExeedBufferLimit() {
+    int BYTE_SIZE = 100 * 1024;
+    byte[] rawData = generate(BYTE_SIZE);
+    try {
+      CompressDecompressTester.of(rawData)
+      .withCompressDecompressPair(
+        new ZlibCompressor(
+            org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel.BEST_COMPRESSION,
+            CompressionStrategy.DEFAULT_STRATEGY,
+            org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionHeader.DEFAULT_HEADER,
+            BYTE_SIZE),
+         new ZlibDecompressor(
+            org.apache.hadoop.io.compress.zlib.ZlibDecompressor.CompressionHeader.DEFAULT_HEADER,
+            BYTE_SIZE))
+         .withTestCases(ImmutableSet.of(CompressionTestStrategy.COMPRESS_DECOMPRESS_SINGLE_BLOCK,
+            CompressionTestStrategy.COMPRESS_DECOMPRESS_BLOCK,
+            CompressionTestStrategy.COMPRESS_DECOMPRESS_ERRORS,
+            CompressionTestStrategy.COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM))
+          .test();
+    } catch (Exception ex) {
+      fail("testCompressorDecompressorWithExeedBufferLimit error !!!" + ex);
+    } 
+  }
+  
+  
+  @Test
+  public void testZlibCompressorDecompressorWithConfiguration() {
+    Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, true);
+    if (ZlibFactory.isNativeZlibLoaded(conf)) {
+      byte[] rawData;
+      int tryNumber = 5;
+      int BYTE_SIZE = 10 * 1024;
+      Compressor zlibCompressor = ZlibFactory.getZlibCompressor(conf);
+      Decompressor zlibDecompressor = ZlibFactory.getZlibDecompressor(conf);
+      rawData = generate(BYTE_SIZE);
+      try {
+        for (int i = 0; i < tryNumber; i++)
+          compressDecompressZlib(rawData, (ZlibCompressor) zlibCompressor,
+              (ZlibDecompressor) zlibDecompressor);
+        zlibCompressor.reinit(conf);
+      } catch (Exception ex) {
+        fail("testZlibCompressorDecompressorWithConfiguration ex error " + ex);
+      }
+    } else {
+      assertTrue("ZlibFactory is using native libs against request",
+          ZlibFactory.isNativeZlibLoaded(conf));
+    }
+  }
+
+  @Test
+  public void testZlibCompressDecompress() {
+    byte[] rawData = null;
+    int rawDataSize = 0;
+    rawDataSize = 1024 * 64;
+    rawData = generate(rawDataSize);
+    try {
+      ZlibCompressor compressor = new ZlibCompressor();
+      ZlibDecompressor decompressor = new ZlibDecompressor();
+      assertFalse("testZlibCompressDecompress finished error",
+          compressor.finished());
+      compressor.setInput(rawData, 0, rawData.length);
+      assertTrue("testZlibCompressDecompress getBytesRead before error",
+          compressor.getBytesRead() == 0);
+      compressor.finish();
+
+      byte[] compressedResult = new byte[rawDataSize];
+      int cSize = compressor.compress(compressedResult, 0, rawDataSize);
+      assertTrue("testZlibCompressDecompress getBytesRead ather error",
+          compressor.getBytesRead() == rawDataSize);
+      assertTrue(
+          "testZlibCompressDecompress compressed size no less then original size",
+          cSize < rawDataSize);
+      decompressor.setInput(compressedResult, 0, cSize);
+      byte[] decompressedBytes = new byte[rawDataSize];
+      decompressor.decompress(decompressedBytes, 0, decompressedBytes.length);
+      assertArrayEquals("testZlibCompressDecompress arrays not equals ",
+          rawData, decompressedBytes);
+      compressor.reset();
+      decompressor.reset();
+    } catch (IOException ex) {
+      fail("testZlibCompressDecompress ex !!!" + ex);
+    }
+  }
+  
+  @Test
+  public void testZlibCompressorDecompressorSetDictionary() {
+    Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, true);
+    if (ZlibFactory.isNativeZlibLoaded(conf)) {
+      Compressor zlibCompressor = ZlibFactory.getZlibCompressor(conf);
+      Decompressor zlibDecompressor = ZlibFactory.getZlibDecompressor(conf);
+
+      checkSetDictionaryNullPointerException(zlibCompressor);
+      checkSetDictionaryNullPointerException(zlibDecompressor);
+
+      checkSetDictionaryArrayIndexOutOfBoundsException(zlibDecompressor);
+      checkSetDictionaryArrayIndexOutOfBoundsException(zlibCompressor);
+    } else {
+      assertTrue("ZlibFactory is using native libs against request",
+          ZlibFactory.isNativeZlibLoaded(conf));
+    }
+  }
+
+  @Test
+  public void testZlibFactory() {
+    Configuration cfg = new Configuration();
+
+    assertTrue("testZlibFactory compression level error !!!",
+        CompressionLevel.DEFAULT_COMPRESSION == ZlibFactory
+            .getCompressionLevel(cfg));
+
+    assertTrue("testZlibFactory compression strategy error !!!",
+        CompressionStrategy.DEFAULT_STRATEGY == ZlibFactory
+            .getCompressionStrategy(cfg));
+
+    ZlibFactory.setCompressionLevel(cfg, CompressionLevel.BEST_COMPRESSION);
+    assertTrue("testZlibFactory compression strategy error !!!",
+        CompressionLevel.BEST_COMPRESSION == ZlibFactory
+            .getCompressionLevel(cfg));
+
+    ZlibFactory.setCompressionStrategy(cfg, CompressionStrategy.FILTERED);
+    assertTrue("testZlibFactory compression strategy error !!!",
+        CompressionStrategy.FILTERED == ZlibFactory.getCompressionStrategy(cfg));
+  }
+  
+
+  private boolean checkSetDictionaryNullPointerException(
+      Decompressor decompressor) {
+    try {
+      decompressor.setDictionary(null, 0, 1);
+    } catch (NullPointerException ex) {
+      return true;
+    } catch (Exception ex) {
+    }
+    return false;
+  }
+
+  private boolean checkSetDictionaryNullPointerException(Compressor compressor) {
+    try {
+      compressor.setDictionary(null, 0, 1);
+    } catch (NullPointerException ex) {
+      return true;
+    } catch (Exception ex) {
+    }
+    return false;
+  }
+
+  private boolean checkSetDictionaryArrayIndexOutOfBoundsException(
+      Compressor compressor) {
+    try {
+      compressor.setDictionary(new byte[] { (byte) 0 }, 0, -1);
+    } catch (ArrayIndexOutOfBoundsException e) {
+      return true;
+    } catch (Exception e) {
+    }
+    return false;
+  }
+
+  private boolean checkSetDictionaryArrayIndexOutOfBoundsException(
+      Decompressor decompressor) {
+    try {
+      decompressor.setDictionary(new byte[] { (byte) 0 }, 0, -1);
+    } catch (ArrayIndexOutOfBoundsException e) {
+      return true;
+    } catch (Exception e) {
+    }
+    return false;
+  }
+
+  private byte[] compressDecompressZlib(byte[] rawData,
+      ZlibCompressor zlibCompressor, ZlibDecompressor zlibDecompressor)
+      throws IOException {
+    int cSize = 0;
+    byte[] compressedByte = new byte[rawData.length];
+    byte[] decompressedRawData = new byte[rawData.length];
+    zlibCompressor.setInput(rawData, 0, rawData.length);
+    zlibCompressor.finish();
+    while (!zlibCompressor.finished()) {
+      cSize = zlibCompressor.compress(compressedByte, 0, compressedByte.length);
+    }
+    zlibCompressor.reset();
+
+    assertTrue(zlibDecompressor.getBytesWritten() == 0);
+    assertTrue(zlibDecompressor.getBytesRead() == 0);
+    assertTrue(zlibDecompressor.needsInput());
+    zlibDecompressor.setInput(compressedByte, 0, cSize);
+    assertFalse(zlibDecompressor.needsInput());
+    while (!zlibDecompressor.finished()) {
+      zlibDecompressor.decompress(decompressedRawData, 0,
+          decompressedRawData.length);
+    }
+    assertTrue(zlibDecompressor.getBytesWritten() == rawData.length);
+    assertTrue(zlibDecompressor.getBytesRead() == cSize);
+    zlibDecompressor.reset();
+    assertTrue(zlibDecompressor.getRemaining() == 0);
+    assertArrayEquals(
+        "testZlibCompressorDecompressorWithConfiguration array equals error",
+        rawData, decompressedRawData);
+
+    return decompressedRawData;
+  }
+
+  @Test
+  public void testBuiltInGzipDecompressorExceptions() {
+    BuiltInGzipDecompressor decompresser = new BuiltInGzipDecompressor();
+    try {
+      decompresser.setInput(null, 0, 1);
+    } catch (NullPointerException ex) {
+      // expected
+    } catch (Exception ex) {
+      fail("testBuiltInGzipDecompressorExceptions npe error " + ex);
+    }
+
+    try {
+      decompresser.setInput(new byte[] { 0 }, 0, -1);
+    } catch (ArrayIndexOutOfBoundsException ex) {
+      // expected
+    } catch (Exception ex) {
+      fail("testBuiltInGzipDecompressorExceptions aioob error" + ex);
+    }        
+    
+    assertTrue("decompresser.getBytesRead error",
+        decompresser.getBytesRead() == 0);
+    assertTrue("decompresser.getRemaining error",
+        decompresser.getRemaining() == 0);
+    decompresser.reset();
+    decompresser.end();
+
+    InputStream decompStream = null;
+    try {
+      // invalid 0 and 1 bytes , must be 31, -117
+      int buffSize = 1 * 1024;
+      byte buffer[] = new byte[buffSize];
+      Decompressor decompressor = new BuiltInGzipDecompressor();
+      DataInputBuffer gzbuf = new DataInputBuffer();
+      decompStream = new DecompressorStream(gzbuf, decompressor);
+      gzbuf.reset(new byte[] { 0, 0, 1, 1, 1, 1, 11, 1, 1, 1, 1 }, 11);
+      decompStream.read(buffer);
+    } catch (IOException ioex) {
+      // expected
+    } catch (Exception ex) {
+      fail("invalid 0 and 1 byte in gzip stream" + ex);
+    }
+
+    // invalid 2 byte, must be 8
+    try {
+      int buffSize = 1 * 1024;
+      byte buffer[] = new byte[buffSize];
+      Decompressor decompressor = new BuiltInGzipDecompressor();
+      DataInputBuffer gzbuf = new DataInputBuffer();
+      decompStream = new DecompressorStream(gzbuf, decompressor);
+      gzbuf.reset(new byte[] { 31, -117, 7, 1, 1, 1, 1, 11, 1, 1, 1, 1 }, 11);
+      decompStream.read(buffer);
+    } catch (IOException ioex) {
+      // expected
+    } catch (Exception ex) {
+      fail("invalid 2 byte in gzip stream" + ex);
+    }
+
+    try {
+      int buffSize = 1 * 1024;
+      byte buffer[] = new byte[buffSize];
+      Decompressor decompressor = new BuiltInGzipDecompressor();
+      DataInputBuffer gzbuf = new DataInputBuffer();
+      decompStream = new DecompressorStream(gzbuf, decompressor);
+      gzbuf.reset(new byte[] { 31, -117, 8, -32, 1, 1, 1, 11, 1, 1, 1, 1 }, 11);
+      decompStream.read(buffer);
+    } catch (IOException ioex) {
+      // expected
+    } catch (Exception ex) {
+      fail("invalid 3 byte in gzip stream" + ex);
+    }
+    try {
+      int buffSize = 1 * 1024;
+      byte buffer[] = new byte[buffSize];
+      Decompressor decompressor = new BuiltInGzipDecompressor();
+      DataInputBuffer gzbuf = new DataInputBuffer();
+      decompStream = new DecompressorStream(gzbuf, decompressor);
+      gzbuf.reset(new byte[] { 31, -117, 8, 4, 1, 1, 1, 11, 1, 1, 1, 1 }, 11);
+      decompStream.read(buffer);
+    } catch (IOException ioex) {
+      // expected
+    } catch (Exception ex) {
+      fail("invalid 3 byte make hasExtraField" + ex);
+    }
+  }
+  
+  public static byte[] generate(int size) {
+    byte[] data = new byte[size];
+    for (int i = 0; i < size; i++)
+      data[i] = (byte)random.nextInt(16);
+    return data;
+  }
+}

+ 58 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TemporarySocketDirectory.java

@@ -0,0 +1,58 @@
+/**
+ * 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.net.unix;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.commons.io.FileUtils;
+
+/**
+ * Create a temporary directory in which sockets can be created.
+ * When creating a UNIX domain socket, the name
+ * must be fairly short (around 110 bytes on most platforms).
+ */
+public class TemporarySocketDirectory implements Closeable {
+  private File dir;
+
+  public TemporarySocketDirectory() {
+    String tmp = System.getProperty("java.io.tmpdir", "/tmp");
+    dir = new File(tmp, "socks." + (System.currentTimeMillis() +
+        "." + (new Random().nextInt())));
+    dir.mkdirs();
+    dir.setWritable(true, true);
+  }
+
+  public File getDir() {
+    return dir;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (dir != null) {
+      FileUtils.deleteDirectory(dir);
+      dir = null;
+    }
+  }
+
+  protected void finalize() throws IOException {
+    close();
+  }
+}

+ 706 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java

@@ -0,0 +1,706 @@
+/**
+ * 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.net.unix;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.SocketTimeoutException;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousCloseException;
+import java.nio.channels.ClosedChannelException;
+import java.util.Arrays;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.unix.DomainSocket.DomainChannel;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Shell;
+
+import com.google.common.io.Files;
+
+public class TestDomainSocket {
+  private static TemporarySocketDirectory sockDir;
+
+  @BeforeClass
+  public static void init() {
+    sockDir = new TemporarySocketDirectory();
+    DomainSocket.disableBindPathValidation();
+  }
+
+  @AfterClass
+  public static void shutdown() throws IOException {
+    sockDir.close();
+  }
+  
+  @Before
+  public void before() {
+    Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null);
+  }
+    
+  /**
+   * Test that we can create a socket and close it, even if it hasn't been
+   * opened.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=180000)
+  public void testSocketCreateAndClose() throws IOException {
+    DomainSocket serv = DomainSocket.bindAndListen(
+      new File(sockDir.getDir(), "test_sock_create_and_close").
+        getAbsolutePath());
+    serv.close();
+  }
+
+  /**
+   * Test DomainSocket path setting and getting.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=180000)
+  public void testSocketPathSetGet() throws IOException {
+    Assert.assertEquals("/var/run/hdfs/sock.100",
+        DomainSocket.getEffectivePath("/var/run/hdfs/sock._PORT", 100));
+  }
+
+  /**
+   * Test that we get a read result of -1 on EOF.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=180000)
+  public void testSocketReadEof() throws Exception {
+    final String TEST_PATH = new File(sockDir.getDir(),
+        "testSocketReadEof").getAbsolutePath();
+    final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
+    ExecutorService exeServ = Executors.newSingleThreadExecutor();
+    Callable<Void> callable = new Callable<Void>() {
+      public Void call(){
+        DomainSocket conn;
+        try {
+          conn = serv.accept();
+        } catch (IOException e) {
+          throw new RuntimeException("unexpected IOException", e);
+        }
+        byte buf[] = new byte[100];
+        for (int i = 0; i < buf.length; i++) {
+          buf[i] = 0;
+        }
+        try {
+          Assert.assertEquals(-1, conn.getInputStream().read());
+        } catch (IOException e) {
+          throw new RuntimeException("unexpected IOException", e);
+        }
+        return null;
+      }
+    };
+    Future<Void> future = exeServ.submit(callable);
+    DomainSocket conn = DomainSocket.connect(serv.getPath());
+    Thread.sleep(50);
+    conn.close();
+    serv.close();
+    future.get(2, TimeUnit.MINUTES);
+  }
+
+  /**
+   * Test that if one thread is blocking in a read or write operation, another
+   * thread can close the socket and stop the accept.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=180000)
+  public void testSocketAcceptAndClose() throws Exception {
+    final String TEST_PATH =
+        new File(sockDir.getDir(), "test_sock_accept_and_close").getAbsolutePath();
+    final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
+    ExecutorService exeServ = Executors.newSingleThreadExecutor();
+    Callable<Void> callable = new Callable<Void>() {
+      public Void call(){
+        try {
+          serv.accept();
+          throw new RuntimeException("expected the accept() to be " +
+              "interrupted and fail");
+        } catch (AsynchronousCloseException e) {
+          return null;
+        } catch (IOException e) {
+          throw new RuntimeException("unexpected IOException", e);
+        }
+      }
+    };
+    Future<Void> future = exeServ.submit(callable);
+    Thread.sleep(500);
+    serv.close();
+    future.get(2, TimeUnit.MINUTES);
+  }
+
+  /**
+   * Test that we get an AsynchronousCloseException when the DomainSocket
+   * we're using is closed during a read or write operation.
+   *
+   * @throws IOException
+   */
+  private void testAsyncCloseDuringIO(final boolean closeDuringWrite)
+      throws Exception {
+    final String TEST_PATH = new File(sockDir.getDir(),
+        "testAsyncCloseDuringIO(" + closeDuringWrite + ")").getAbsolutePath();
+    final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
+    ExecutorService exeServ = Executors.newFixedThreadPool(2);
+    Callable<Void> serverCallable = new Callable<Void>() {
+      public Void call() {
+        DomainSocket serverConn = null;
+        try {
+          serverConn = serv.accept();
+          byte buf[] = new byte[100];
+          for (int i = 0; i < buf.length; i++) {
+            buf[i] = 0;
+          }
+          // The server just continues either writing or reading until someone
+          // asynchronously closes the client's socket.  At that point, all our
+          // reads return EOF, and writes get a socket error.
+          if (closeDuringWrite) {
+            try {
+              while (true) {
+                serverConn.getOutputStream().write(buf);
+              }
+            } catch (IOException e) {
+            }
+          } else {
+            do { ; } while 
+              (serverConn.getInputStream().read(buf, 0, buf.length) != -1);
+          }
+        } catch (IOException e) {
+          throw new RuntimeException("unexpected IOException", e);
+        } finally {
+          IOUtils.cleanup(DomainSocket.LOG, serverConn);
+        }
+        return null;
+      }
+    };
+    Future<Void> serverFuture = exeServ.submit(serverCallable);
+    final DomainSocket clientConn = DomainSocket.connect(serv.getPath());
+    Callable<Void> clientCallable = new Callable<Void>() {
+      public Void call(){
+        // The client writes or reads until another thread
+        // asynchronously closes the socket.  At that point, we should
+        // get ClosedChannelException, or possibly its subclass
+        // AsynchronousCloseException.
+        byte buf[] = new byte[100];
+        for (int i = 0; i < buf.length; i++) {
+          buf[i] = 0;
+        }
+        try {
+          if (closeDuringWrite) {
+            while (true) {
+              clientConn.getOutputStream().write(buf);
+            }
+          } else {
+            while (true) {
+              clientConn.getInputStream().read(buf, 0, buf.length);
+            }
+          }
+        } catch (ClosedChannelException e) {
+          return null;
+        } catch (IOException e) {
+          throw new RuntimeException("unexpected IOException", e);
+        }
+      }
+    };
+    Future<Void> clientFuture = exeServ.submit(clientCallable);
+    Thread.sleep(500);
+    clientConn.close();
+    serv.close();
+    clientFuture.get(2, TimeUnit.MINUTES);
+    serverFuture.get(2, TimeUnit.MINUTES);
+  }
+  
+  @Test(timeout=180000)
+  public void testAsyncCloseDuringWrite() throws Exception {
+    testAsyncCloseDuringIO(true);
+  }
+  
+  @Test(timeout=180000)
+  public void testAsyncCloseDuringRead() throws Exception {
+    testAsyncCloseDuringIO(false);
+  }
+  
+  /**
+   * Test that attempting to connect to an invalid path doesn't work.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=180000)
+  public void testInvalidOperations() throws IOException {
+    try {
+      DomainSocket.connect(
+        new File(sockDir.getDir(), "test_sock_invalid_operation").
+          getAbsolutePath());
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("connect(2) error: ", e);
+    }
+  }
+
+  /**
+   * Test setting some server options.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=180000)
+  public void testServerOptions() throws Exception {
+    final String TEST_PATH = new File(sockDir.getDir(),
+        "test_sock_server_options").getAbsolutePath();
+    DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
+    try {
+      // Let's set a new receive buffer size
+      int bufSize = serv.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE);
+      int newBufSize = bufSize / 2;
+      serv.setAttribute(DomainSocket.RECEIVE_BUFFER_SIZE, newBufSize);
+      int nextBufSize = serv.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE);
+      Assert.assertEquals(newBufSize, nextBufSize);
+      // Let's set a server timeout
+      int newTimeout = 1000;
+      serv.setAttribute(DomainSocket.RECEIVE_TIMEOUT, newTimeout);
+      int nextTimeout = serv.getAttribute(DomainSocket.RECEIVE_TIMEOUT);
+      Assert.assertEquals(newTimeout, nextTimeout);
+      try {
+        serv.accept();
+        Assert.fail("expected the accept() to time out and fail");
+      } catch (SocketTimeoutException e) {
+        GenericTestUtils.assertExceptionContains("accept(2) error: ", e);
+      }
+    } finally {
+      serv.close();
+      Assert.assertFalse(serv.isOpen());
+    }
+  }
+  
+  /**
+   * A Throwable representing success.
+   *
+   * We can't use null to represent this, because you cannot insert null into
+   * ArrayBlockingQueue.
+   */
+  static class Success extends Throwable {
+    private static final long serialVersionUID = 1L;
+  }
+  
+  static interface WriteStrategy {
+    /**
+     * Initialize a WriteStrategy object from a Socket.
+     */
+    public void init(DomainSocket s) throws IOException;
+    
+    /**
+     * Write some bytes.
+     */
+    public void write(byte b[]) throws IOException;
+  }
+  
+  static class OutputStreamWriteStrategy implements WriteStrategy {
+    private OutputStream outs = null;
+    
+    public void init(DomainSocket s) throws IOException {
+      outs = s.getOutputStream();
+    }
+    
+    public void write(byte b[]) throws IOException {
+      outs.write(b);
+    }
+  }
+  
+  abstract static class ReadStrategy {
+    /**
+     * Initialize a ReadStrategy object from a DomainSocket.
+     */
+    public abstract void init(DomainSocket s) throws IOException;
+    
+    /**
+     * Read some bytes.
+     */
+    public abstract int read(byte b[], int off, int length) throws IOException;
+    
+    public void readFully(byte buf[], int off, int len) throws IOException {
+      int toRead = len;
+      while (toRead > 0) {
+        int ret = read(buf, off, toRead);
+        if (ret < 0) {
+          throw new IOException( "Premature EOF from inputStream");
+        }
+        toRead -= ret;
+        off += ret;
+      }
+    }
+  }
+  
+  static class InputStreamReadStrategy extends ReadStrategy {
+    private InputStream ins = null;
+    
+    @Override
+    public void init(DomainSocket s) throws IOException {
+      ins = s.getInputStream();
+    }
+
+    @Override
+    public int read(byte b[], int off, int length) throws IOException {
+      return ins.read(b, off, length);
+    }
+  }
+  
+  static class DirectByteBufferReadStrategy extends ReadStrategy {
+    private DomainChannel ch = null;
+
+    @Override
+    public void init(DomainSocket s) throws IOException {
+      ch = s.getChannel();
+    }
+    
+    @Override
+    public int read(byte b[], int off, int length) throws IOException {
+      ByteBuffer buf = ByteBuffer.allocateDirect(b.length);
+      int nread = ch.read(buf);
+      if (nread < 0) return nread;
+      buf.flip();
+      buf.get(b, off, nread);
+      return nread;
+    }
+  }
+
+  static class ArrayBackedByteBufferReadStrategy extends ReadStrategy {
+    private DomainChannel ch = null;
+    
+    @Override
+    public void init(DomainSocket s) throws IOException {
+      ch = s.getChannel();
+    }
+    
+    @Override
+    public int read(byte b[], int off, int length) throws IOException {
+      ByteBuffer buf = ByteBuffer.wrap(b);
+      int nread = ch.read(buf);
+      if (nread < 0) return nread;
+      buf.flip();
+      buf.get(b, off, nread);
+      return nread;
+    }
+  }
+  
+  /**
+   * Test a simple client/server interaction.
+   *
+   * @throws IOException
+   */
+  void testClientServer1(final Class<? extends WriteStrategy> writeStrategyClass,
+      final Class<? extends ReadStrategy> readStrategyClass) throws Exception {
+    final String TEST_PATH = new File(sockDir.getDir(),
+        "test_sock_client_server1").getAbsolutePath();
+    final byte clientMsg1[] = new byte[] { 0x1, 0x2, 0x3, 0x4, 0x5, 0x6 };
+    final byte serverMsg1[] = new byte[] { 0x9, 0x8, 0x7, 0x6, 0x5 };
+    final byte clientMsg2 = 0x45;
+    final ArrayBlockingQueue<Throwable> threadResults =
+        new ArrayBlockingQueue<Throwable>(2);
+    final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
+    Thread serverThread = new Thread() {
+      public void run(){
+        // Run server
+        DomainSocket conn = null;
+        try {
+          conn = serv.accept();
+          byte in1[] = new byte[clientMsg1.length];
+          ReadStrategy reader = readStrategyClass.newInstance();
+          reader.init(conn);
+          reader.readFully(in1, 0, in1.length);
+          Assert.assertTrue(Arrays.equals(clientMsg1, in1));
+          WriteStrategy writer = writeStrategyClass.newInstance();
+          writer.init(conn);
+          writer.write(serverMsg1);
+          InputStream connInputStream = conn.getInputStream();
+          int in2 = connInputStream.read();
+          Assert.assertEquals((int)clientMsg2, in2);
+          conn.close();
+        } catch (Throwable e) {
+          threadResults.add(e);
+          Assert.fail(e.getMessage());
+        }
+        threadResults.add(new Success());
+      }
+    };
+    serverThread.start();
+    
+    Thread clientThread = new Thread() {
+      public void run(){
+        try {
+          DomainSocket client = DomainSocket.connect(TEST_PATH);
+          WriteStrategy writer = writeStrategyClass.newInstance();
+          writer.init(client);
+          writer.write(clientMsg1);
+          ReadStrategy reader = readStrategyClass.newInstance();
+          reader.init(client);
+          byte in1[] = new byte[serverMsg1.length];
+          reader.readFully(in1, 0, in1.length);
+          Assert.assertTrue(Arrays.equals(serverMsg1, in1));
+          OutputStream clientOutputStream = client.getOutputStream();
+          clientOutputStream.write(clientMsg2);
+          client.close();
+        } catch (Throwable e) {
+          threadResults.add(e);
+        }
+        threadResults.add(new Success());
+      }
+    };
+    clientThread.start();
+    
+    for (int i = 0; i < 2; i++) {
+      Throwable t = threadResults.take();
+      if (!(t instanceof Success)) {
+        Assert.fail(t.getMessage() + ExceptionUtils.getStackTrace(t));
+      }
+    }
+    serverThread.join(120000);
+    clientThread.join(120000);
+    serv.close();
+  }
+
+  @Test(timeout=180000)
+  public void testClientServerOutStreamInStream() throws Exception {
+    testClientServer1(OutputStreamWriteStrategy.class,
+        InputStreamReadStrategy.class);
+  }
+
+  @Test(timeout=180000)
+  public void testClientServerOutStreamInDbb() throws Exception {
+    testClientServer1(OutputStreamWriteStrategy.class,
+        DirectByteBufferReadStrategy.class);
+  }
+
+  @Test(timeout=180000)
+  public void testClientServerOutStreamInAbb() throws Exception {
+    testClientServer1(OutputStreamWriteStrategy.class,
+        ArrayBackedByteBufferReadStrategy.class);
+  }
+
+  static private class PassedFile {
+    private final int idx;
+    private final byte[] contents;
+    private FileInputStream fis;
+    
+    public PassedFile(int idx) throws IOException {
+      this.idx = idx;
+      this.contents = new byte[] { (byte)(idx % 127) };
+      Files.write(contents, new File(getPath()));
+      this.fis = new FileInputStream(getPath());
+    }
+
+    public String getPath() {
+      return new File(sockDir.getDir(), "passed_file" + idx).getAbsolutePath();
+    }
+
+    public FileInputStream getInputStream() throws IOException {
+      return fis;
+    }
+    
+    public void cleanup() throws IOException {
+      new File(getPath()).delete();
+      fis.close();
+    }
+
+    public void checkInputStream(FileInputStream fis) throws IOException {
+      byte buf[] = new byte[contents.length];
+      IOUtils.readFully(fis, buf, 0, buf.length);
+      Arrays.equals(contents, buf);
+    }
+    
+    protected void finalize() {
+      try {
+        cleanup();
+      } catch(Throwable t) {
+        // ignore
+      }
+    }
+  }
+
+  /**
+   * Test file descriptor passing.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=180000)
+  public void testFdPassing() throws Exception {
+    final String TEST_PATH =
+        new File(sockDir.getDir(), "test_sock").getAbsolutePath();
+    final byte clientMsg1[] = new byte[] { 0x11, 0x22, 0x33, 0x44, 0x55, 0x66 };
+    final byte serverMsg1[] = new byte[] { 0x31, 0x30, 0x32, 0x34, 0x31, 0x33,
+          0x44, 0x1, 0x1, 0x1, 0x1, 0x1 };
+    final ArrayBlockingQueue<Throwable> threadResults =
+        new ArrayBlockingQueue<Throwable>(2);
+    final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
+    final PassedFile passedFiles[] =
+        new PassedFile[] { new PassedFile(1), new PassedFile(2) };
+    final FileDescriptor passedFds[] = new FileDescriptor[passedFiles.length];
+    for (int i = 0; i < passedFiles.length; i++) {
+      passedFds[i] = passedFiles[i].getInputStream().getFD();
+    }
+    Thread serverThread = new Thread() {
+      public void run(){
+        // Run server
+        DomainSocket conn = null;
+        try {
+          conn = serv.accept();
+          byte in1[] = new byte[clientMsg1.length];
+          InputStream connInputStream = conn.getInputStream();
+          IOUtils.readFully(connInputStream, in1, 0, in1.length);
+          Assert.assertTrue(Arrays.equals(clientMsg1, in1));
+          DomainSocket domainConn = (DomainSocket)conn;
+          domainConn.sendFileDescriptors(passedFds, serverMsg1, 0,
+              serverMsg1.length);
+          conn.close();
+        } catch (Throwable e) {
+          threadResults.add(e);
+          Assert.fail(e.getMessage());
+        }
+        threadResults.add(new Success());
+      }
+    };
+    serverThread.start();
+
+    Thread clientThread = new Thread() {
+      public void run(){
+        try {
+          DomainSocket client = DomainSocket.connect(TEST_PATH);
+          OutputStream clientOutputStream = client.getOutputStream();
+          InputStream clientInputStream = client.getInputStream();
+          clientOutputStream.write(clientMsg1);
+          DomainSocket domainConn = (DomainSocket)client;
+          byte in1[] = new byte[serverMsg1.length];
+          FileInputStream recvFis[] = new FileInputStream[passedFds.length];
+          int r = domainConn.
+              recvFileInputStreams(recvFis, in1, 0, in1.length - 1);
+          Assert.assertTrue(r > 0);
+          IOUtils.readFully(clientInputStream, in1, r, in1.length - r);
+          Assert.assertTrue(Arrays.equals(serverMsg1, in1));
+          for (int i = 0; i < passedFds.length; i++) {
+            Assert.assertNotNull(recvFis[i]);
+            passedFiles[i].checkInputStream(recvFis[i]);
+          }
+          for (FileInputStream fis : recvFis) {
+            fis.close();
+          }
+          client.close();
+        } catch (Throwable e) {
+          threadResults.add(e);
+        }
+        threadResults.add(new Success());
+      }
+    };
+    clientThread.start();
+    
+    for (int i = 0; i < 2; i++) {
+      Throwable t = threadResults.take();
+      if (!(t instanceof Success)) {
+        Assert.fail(t.getMessage() + ExceptionUtils.getStackTrace(t));
+      }
+    }
+    serverThread.join(120000);
+    clientThread.join(120000);
+    serv.close();
+    for (PassedFile pf : passedFiles) {
+      pf.cleanup();
+    }
+  }
+  
+  /**
+   * Run validateSocketPathSecurity
+   *
+   * @param str            The path to validate
+   * @param prefix         A prefix to skip validation for
+   * @throws IOException
+   */
+  private static void testValidateSocketPath(String str, String prefix)
+      throws IOException {
+    int skipComponents = 1;
+    File prefixFile = new File(prefix);
+    while (true) {
+      prefixFile = prefixFile.getParentFile();
+      if (prefixFile == null) {
+        break;
+      }
+      skipComponents++;
+    }
+    DomainSocket.validateSocketPathSecurity0(str,
+        skipComponents);
+  }
+  
+  /**
+   * Test file descriptor path security.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=180000)
+  public void testFdPassingPathSecurity() throws Exception {
+    TemporarySocketDirectory tmp = new TemporarySocketDirectory();
+    try {
+      String prefix = tmp.getDir().getAbsolutePath();
+      Shell.execCommand(new String [] {
+          "mkdir", "-p", prefix + "/foo/bar/baz" });
+      Shell.execCommand(new String [] {
+          "chmod", "0700", prefix + "/foo/bar/baz" });
+      Shell.execCommand(new String [] {
+          "chmod", "0700", prefix + "/foo/bar" });
+      Shell.execCommand(new String [] {
+          "chmod", "0707", prefix + "/foo" });
+      Shell.execCommand(new String [] {
+          "mkdir", "-p", prefix + "/q1/q2" });
+      Shell.execCommand(new String [] {
+          "chmod", "0700", prefix + "/q1" });
+      Shell.execCommand(new String [] {
+          "chmod", "0700", prefix + "/q1/q2" });
+      testValidateSocketPath(prefix + "/q1/q2", prefix);
+      try {
+        testValidateSocketPath(prefix + "/foo/bar/baz", prefix);
+      } catch (IOException e) {
+        GenericTestUtils.assertExceptionContains("/foo' is world-writable.  " +
+            "Its permissions are 0707.  Please fix this or select a " +
+            "different socket path.", e);
+      }
+      try {
+        testValidateSocketPath(prefix + "/nope", prefix);
+      } catch (IOException e) {
+        GenericTestUtils.assertExceptionContains("failed to stat a path " +
+            "component: ", e);
+      }
+      // Root should be secure
+      DomainSocket.validateSocketPathSecurity0("/foo", 1);
+    } finally {
+      tmp.close();
+    }
+  }
+}

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

@@ -187,6 +187,9 @@ Trunk (Unreleased)
     HDFS-4633 TestDFSClientExcludedNodes fails sporadically if excluded nodes
     cache expires too quickly  (Chris Nauroth via Sanjay)
 
+    HDFS-347. DFS read performance suboptimal when client co-located on nodes
+    with data. (Colin Patrick McCabe via todd and atm)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -355,6 +358,62 @@ Trunk (Unreleased)
     HDFS-4674. TestBPOfferService fails on Windows due to failure parsing 
     datanode data directory as URI. (Chris Nauroth via suresh)
 
+  BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
+
+    HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
+    (Colin Patrick McCabe via todd)
+    
+    HDFS-4354. Create DomainSocket and DomainPeer and associated unit tests.
+    (Colin Patrick McCabe via todd)
+    
+    HDFS-4356. BlockReaderLocal should use passed file descriptors rather than paths.
+    (Colin Patrick McCabe via todd)
+    
+    HDFS-4388. DomainSocket should throw AsynchronousCloseException when appropriate.
+    (Colin Patrick McCabe via todd)
+    
+    HDFS-4390. Bypass UNIX domain socket unit tests when they cannot be run.
+    (Colin Patrick McCabe via todd)
+    
+    HDFS-4400. DFSInputStream#getBlockReader: last retries should ignore the cache
+    (Colin Patrick McCabe via todd)
+    
+    HDFS-4401. Fix bug in DomainSocket path validation
+    (Colin Patrick McCabe via todd)
+    
+    HDFS-4402. Some small DomainSocket fixes: avoid findbugs warning, change
+    log level, etc. (Colin Patrick McCabe via todd)
+    
+    HDFS-4418. increase default FileInputStreamCache size (todd)
+    
+    HDFS-4416. Rename dfs.datanode.domain.socket.path to dfs.domain.socket.path
+    (Colin Patrick McCabe via todd)
+    
+    HDFS-4417. Fix case where local reads get disabled incorrectly
+    (Colin Patrick McCabe and todd via todd)
+    
+    HDFS-4433. Make TestPeerCache not flaky (Colin Patrick McCabe via todd)
+    
+    HDFS-4438. TestDomainSocket fails when system umask is set to 0002. (Colin
+    Patrick McCabe via atm)
+    
+    HDFS-4440. Avoid annoying log message when dfs.domain.socket.path is not
+    set. (Colin Patrick McCabe via atm)
+    
+    HDFS-4473. Don't create domain socket unless we need it. (Colin Patrick McCabe via atm)
+    
+    HDFS-4485. DN should chmod socket path a+w. (Colin Patrick McCabe via atm)
+    
+    HDFS-4453. Make a simple doc to describe the usage and design of the
+    shortcircuit read feature. (Colin Patrick McCabe via atm)
+    
+    HDFS-4496. DFSClient: don't create a domain socket unless we need it (Colin
+    Patrick McCabe via todd)
+    
+    HDFS-347: style cleanups (Colin Patrick McCabe via atm)
+    
+    HDFS-4538. Allow use of legacy blockreader (Colin Patrick McCabe via todd)
+
 Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -399,6 +458,9 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-3940. Add Gset#clear method and clear the block map when namenode is
     shutdown. (suresh)
 
+    HDFS-4679. Namenode operation checks should be done in a consistent
+    manner. (suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -500,6 +562,9 @@ Release 2.0.5-beta - UNRELEASED
 
     HDFS-4643. Fix flakiness in TestQuorumJournalManager. (todd)
 
+    HDFS-4639. startFileInternal() should not increment generation stamp.
+    (Plamen Jeliazkov via shv)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -2473,6 +2538,20 @@ Release 2.0.0-alpha - 05-23-2012
     
     HDFS-3039. Address findbugs and javadoc warnings on branch. (todd via atm)
 
+Release 0.23.8 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HDFS-4477. Secondary namenode may retain old tokens (daryn via kihwal)
+
 Release 0.23.7 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -290,6 +290,14 @@
        <Method name="persistPaxosData" />
        <Bug pattern="OS_OPEN_STREAM" />
      </Match>
+
+     <!-- getShortCircuitFdsForRead is supposed to return open streams. -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl" />
+       <Method name="getShortCircuitFdsForRead" />
+       <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
+     </Match>
+
      <!-- Don't complain about LocalDatanodeInfo's anonymous class -->
      <Match>
        <Class name="org.apache.hadoop.hdfs.BlockReaderLocal$LocalDatanodeInfo$1" />

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -515,6 +515,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
           <excludes>
             <exclude>CHANGES.txt</exclude>
             <exclude>CHANGES.HDFS-1623.txt</exclude>
+            <exclude>CHANGES.HDFS-347.txt</exclude>
             <exclude>.idea/**</exclude>
             <exclude>src/main/conf/*</exclude>
             <exclude>src/main/docs/**</exclude>

+ 23 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java

@@ -18,10 +18,8 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.net.Socket;
 
 import org.apache.hadoop.fs.ByteBufferReadable;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 
 /**
  * A BlockReader is responsible for reading a single block
@@ -43,7 +41,29 @@ public interface BlockReader extends ByteBufferReadable {
    */
   long skip(long n) throws IOException;
 
-  void close() throws IOException;
+  /**
+   * Returns an estimate of the number of bytes that can be read
+   * (or skipped over) from this input stream without performing
+   * network I/O.
+   */
+  int available() throws IOException;
+
+  /**
+   * Close the block reader.
+   *
+   * @param peerCache      The PeerCache to put the Peer we're using back
+   *                       into, or null if we should simply close the Peer
+   *                       we're using (along with its Socket).
+   *                       Ignored by Readers that don't maintain Peers.
+   * @param fisCache       The FileInputStreamCache to put our FileInputStreams
+   *                       back into, or null if we should simply close them.
+   *                       Ignored by Readers that don't maintain
+   *                       FileInputStreams.
+   *
+   * @throws IOException
+   */
+  void close(PeerCache peerCache, FileInputStreamCache fisCache)
+      throws IOException;
 
   /**
    * Read exactly the given amount of data, throwing an exception
@@ -60,20 +80,4 @@ public interface BlockReader extends ByteBufferReadable {
    * filled or the next call will return EOF.
    */
   int readAll(byte[] buf, int offset, int len) throws IOException;
-
-  /**
-   * Take the socket used to talk to the DN.
-   */
-  Socket takeSocket();
-
-  /**
-   * Whether the BlockReader has reached the end of its input stream
-   * and successfully sent a status code back to the datanode.
-   */
-  boolean hasSentStatusCode();
-
-  /**
-   * @return a reference to the streams this block reader is using.
-   */
-  IOStreamPair getStreams();
 }

+ 189 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -17,20 +17,31 @@
  */
 package org.apache.hadoop.hdfs;
 
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.net.Socket;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSClient.Conf;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 
 
@@ -39,75 +50,182 @@ import org.apache.hadoop.security.token.Token;
  */
 @InterfaceAudience.Private
 public class BlockReaderFactory {
-  /**
-   * @see #newBlockReader(Conf, Socket, String, ExtendedBlock, Token, long, long, int, boolean, String)
-   */
-  public static BlockReader newBlockReader(
-      Configuration conf,
-      Socket sock, String file,
-      ExtendedBlock block, Token<BlockTokenIdentifier> blockToken, 
-      long startOffset, long len, DataEncryptionKey encryptionKey)
-          throws IOException {
-    int bufferSize = conf.getInt(DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY,
-        DFSConfigKeys.IO_FILE_BUFFER_SIZE_DEFAULT);
-    return newBlockReader(new Conf(conf),
-        sock, file, block, blockToken, startOffset,
-        len, bufferSize, true, "", encryptionKey, null);
-  }
-
   /**
    * Create a new BlockReader specifically to satisfy a read.
    * This method also sends the OP_READ_BLOCK request.
    * 
    * @param conf the DFSClient configuration
-   * @param sock  An established Socket to the DN. The BlockReader will not close it normally
    * @param file  File location
    * @param block  The block object
    * @param blockToken  The block token for security
    * @param startOffset  The read offset, relative to block head
-   * @param len  The number of bytes to read
+   * @param len  The number of bytes to read, or -1 to read as many as
+   *             possible.
    * @param bufferSize  The IO buffer size (not the client buffer size)
+   *                    Ignored except on the legacy BlockReader.
    * @param verifyChecksum  Whether to verify checksum
-   * @param clientName  Client name
-   * @return New BlockReader instance, or null on error.
+   * @param clientName  Client name.  Used for log messages.
+   * @param peer  The peer
+   * @param datanodeID  The datanode that the Peer is connected to
+   * @param domainSocketFactory  The DomainSocketFactory to notify if the Peer
+   *                             is a DomainPeer which turns out to be faulty.
+   *                             If null, no factory will be notified in this
+   *                             case.
+   * @param allowShortCircuitLocalReads  True if short-circuit local reads
+   *                                     should be allowed.
+   * @return New BlockReader instance
    */
   @SuppressWarnings("deprecation")
   public static BlockReader newBlockReader(
-                                     Conf conf,
-                                     Socket sock, String file,
+                                     Configuration conf,
+                                     String file,
                                      ExtendedBlock block, 
                                      Token<BlockTokenIdentifier> blockToken,
                                      long startOffset, long len,
-                                     int bufferSize, boolean verifyChecksum,
+                                     boolean verifyChecksum,
                                      String clientName,
-                                     DataEncryptionKey encryptionKey,
-                                     IOStreamPair ioStreams)
-                                     throws IOException {
-    
-    if (conf.useLegacyBlockReader) {
-      if (encryptionKey != null) {
-        throw new RuntimeException("Encryption is not supported with the legacy block reader.");
+                                     Peer peer,
+                                     DatanodeID datanodeID,
+                                     DomainSocketFactory domSockFactory,
+                                     boolean allowShortCircuitLocalReads)
+  throws IOException {
+    peer.setReadTimeout(conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
+        HdfsServerConstants.READ_TIMEOUT));
+    peer.setWriteTimeout(HdfsServerConstants.WRITE_TIMEOUT);
+
+    if (peer.getDomainSocket() != null) {
+      if (allowShortCircuitLocalReads &&
+         (!conf.getBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
+            DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT))) {
+        // If this is a domain socket, and short-circuit local reads are 
+        // enabled, try to set up a BlockReaderLocal.
+        BlockReader reader = newShortCircuitBlockReader(conf, file,
+            block, blockToken, startOffset, len, peer, datanodeID,
+            domSockFactory, verifyChecksum);
+        if (reader != null) {
+          // One we've constructed the short-circuit block reader, we don't
+          // need the socket any more.  So let's return it to the cache.
+          PeerCache peerCache = PeerCache.getInstance(
+              conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 
+                DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT),
+              conf.getLong(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, 
+                DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT));
+          peerCache.put(datanodeID, peer);
+          return reader;
+        }
+      }
+      // If this is a domain socket and we couldn't (or didn't want to) set
+      // up a BlockReaderLocal, check that we are allowed to pass data traffic
+      // over the socket before proceeding.
+      if (!conf.getBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+            DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
+        throw new IOException("Because we can't do short-circuit access, " +
+          "and data traffic over domain sockets is disabled, " +
+          "we cannot use this socket to talk to " + datanodeID);
       }
-      return RemoteBlockReader.newBlockReader(
-          sock, file, block, blockToken, startOffset, len, bufferSize, verifyChecksum, clientName);
+    }
+
+    if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER,
+        DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT)) {
+      return RemoteBlockReader.newBlockReader(file,
+          block, blockToken, startOffset, len,
+          conf.getInt(DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY,
+              DFSConfigKeys.IO_FILE_BUFFER_SIZE_DEFAULT),
+          verifyChecksum, clientName, peer, datanodeID);
     } else {
-      if (ioStreams == null) {
-        ioStreams = new IOStreamPair(NetUtils.getInputStream(sock),
-            NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT));
-        if (encryptionKey != null) {
-          IOStreamPair encryptedStreams =
-              DataTransferEncryptor.getEncryptedStreams(
-                  ioStreams.out, ioStreams.in, encryptionKey);
-          ioStreams = encryptedStreams;
+      return RemoteBlockReader2.newBlockReader(
+          file, block, blockToken, startOffset, len,
+          verifyChecksum, clientName, peer, datanodeID);
+    }
+  }
+
+  /**
+   * Create a new short-circuit BlockReader.
+   * 
+   * Here, we ask the DataNode to pass us file descriptors over our
+   * DomainSocket.  If the DataNode declines to do so, we'll return null here;
+   * otherwise, we'll return the BlockReaderLocal.  If the DataNode declines,
+   * this function will inform the DomainSocketFactory that short-circuit local
+   * reads are disabled for this DataNode, so that we don't ask again.
+   * 
+   * @param conf               the configuration.
+   * @param file               the file name. Used in log messages.
+   * @param block              The block object.
+   * @param blockToken         The block token for security.
+   * @param startOffset        The read offset, relative to block head.
+   * @param len                The number of bytes to read, or -1 to read 
+   *                           as many as possible.
+   * @param peer               The peer to use.
+   * @param datanodeID         The datanode that the Peer is connected to.
+   * @param domSockFactory     The DomainSocketFactory to notify if the Peer
+   *                           is a DomainPeer which turns out to be faulty.
+   *                           If null, no factory will be notified in this
+   *                           case.
+   * @param verifyChecksum     True if we should verify the checksums.
+   *                           Note: even if this is true, when
+   *                           DFS_CLIENT_READ_CHECKSUM_SKIP_CHECKSUM_KEY is
+   *                           set, we will skip checksums.
+   *
+   * @return                   The BlockReaderLocal, or null if the
+   *                           DataNode declined to provide short-circuit
+   *                           access.
+   * @throws IOException       If there was a communication error.
+   */
+  private static BlockReaderLocal newShortCircuitBlockReader(
+      Configuration conf, String file, ExtendedBlock block,
+      Token<BlockTokenIdentifier> blockToken, long startOffset,
+      long len, Peer peer, DatanodeID datanodeID,
+      DomainSocketFactory domSockFactory, boolean verifyChecksum)
+          throws IOException {
+    final DataOutputStream out =
+        new DataOutputStream(new BufferedOutputStream(
+          peer.getOutputStream()));
+    new Sender(out).requestShortCircuitFds(block, blockToken, 1);
+    DataInputStream in =
+        new DataInputStream(peer.getInputStream());
+    BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
+        PBHelper.vintPrefixed(in));
+    DomainSocket sock = peer.getDomainSocket();
+    switch (resp.getStatus()) {
+    case SUCCESS:
+      BlockReaderLocal reader = null;
+      byte buf[] = new byte[1];
+      FileInputStream fis[] = new FileInputStream[2];
+      sock.recvFileInputStreams(fis, buf, 0, buf.length);
+      try {
+        reader = new BlockReaderLocal(conf, file, block,
+            startOffset, len, fis[0], fis[1], datanodeID, verifyChecksum);
+      } finally {
+        if (reader == null) {
+          IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
         }
       }
-      
-      return RemoteBlockReader2.newBlockReader(
-          sock, file, block, blockToken, startOffset, len, bufferSize,
-          verifyChecksum, clientName, encryptionKey, ioStreams);
+      return reader;
+    case ERROR_UNSUPPORTED:
+      if (!resp.hasShortCircuitAccessVersion()) {
+        DFSClient.LOG.warn("short-circuit read access is disabled for " +
+            "DataNode " + datanodeID + ".  reason: " + resp.getMessage());
+        domSockFactory.disableShortCircuitForPath(sock.getPath());
+      } else {
+        DFSClient.LOG.warn("short-circuit read access for the file " +
+            file + " is disabled for DataNode " + datanodeID +
+            ".  reason: " + resp.getMessage());
+      }
+      return null;
+    case ERROR_ACCESS_TOKEN:
+      String msg = "access control error while " +
+          "attempting to set up short-circuit access to " +
+          file + resp.getMessage();
+      DFSClient.LOG.debug(msg);
+      throw new InvalidBlockTokenException(msg);
+    default:
+      DFSClient.LOG.warn("error while attempting to set up short-circuit " +
+          "access to " + file + ": " + resp.getMessage());
+      domSockFactory.disableShortCircuitForPath(sock.getPath());
+      return null;
     }
   }
-  
+
   /**
    * File name to print when accessing a block directly (from servlets)
    * @param s Address of the block location
@@ -119,4 +237,24 @@ public class BlockReaderFactory {
       final String poolId, final long blockId) {
     return s.toString() + ":" + poolId + ":" + blockId;
   }
+
+  /**
+   * Get {@link BlockReaderLocalLegacy} for short circuited local reads.
+   * This block reader implements the path-based style of local reads
+   * first introduced in HDFS-2246.
+   */
+  static BlockReader getLegacyBlockReaderLocal(UserGroupInformation ugi,
+      Configuration conf, String src, ExtendedBlock blk,
+      Token<BlockTokenIdentifier> accessToken, DatanodeInfo chosenNode,
+      int socketTimeout, long offsetIntoBlock,
+      boolean connectToDnViaHostname) throws InvalidToken, IOException {
+    try {
+      return BlockReaderLocalLegacy.newBlockReader(ugi, conf, src,
+          blk, accessToken, chosenNode, socketTimeout, offsetIntoBlock,
+          blk.getNumBytes() - offsetIntoBlock, connectToDnViaHostname);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(InvalidToken.class,
+          AccessControlException.class);
+    }
+  }
 }

+ 95 - 271
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -18,33 +18,20 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.DataInputStream;
-import java.io.File;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.BufferedInputStream;
 import java.io.FileInputStream;
 import java.io.IOException;
-import java.net.Socket;
 import java.nio.ByteBuffer;
-import java.security.PrivilegedExceptionAction;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.util.DirectBufferPool;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
 /**
@@ -56,84 +43,19 @@ import org.apache.hadoop.util.DataChecksum;
  * <ul>
  * <li>The client performing short circuit reads must be configured at the
  * datanode.</li>
- * <li>The client gets the path to the file where block is stored using
- * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
- * RPC call</li>
- * <li>Client uses kerberos authentication to connect to the datanode over RPC,
- * if security is enabled.</li>
+ * <li>The client gets the file descriptors for the metadata file and the data 
+ * file for the block using
+ * {@link org.apache.hadoop.hdfs.server.datanode.DataXceiver#requestShortCircuitFds}.
+ * </li>
+ * <li>The client reads the file descriptors.</li>
  * </ul>
  */
 class BlockReaderLocal implements BlockReader {
-  private static final Log LOG = LogFactory.getLog(DFSClient.class);
-
-  //Stores the cache and proxy for a local datanode.
-  private static class LocalDatanodeInfo {
-    private ClientDatanodeProtocol proxy = null;
-    private final Map<ExtendedBlock, BlockLocalPathInfo> cache;
-
-    LocalDatanodeInfo() {
-      final int cacheSize = 10000;
-      final float hashTableLoadFactor = 0.75f;
-      int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
-      cache = Collections
-          .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
-              hashTableCapacity, hashTableLoadFactor, true) {
-            private static final long serialVersionUID = 1;
-
-            @Override
-            protected boolean removeEldestEntry(
-                Map.Entry<ExtendedBlock, BlockLocalPathInfo> eldest) {
-              return size() > cacheSize;
-            }
-          });
-    }
-
-    private synchronized ClientDatanodeProtocol getDatanodeProxy(
-        UserGroupInformation ugi, final DatanodeInfo node,
-        final Configuration conf, final int socketTimeout,
-        final boolean connectToDnViaHostname) throws IOException {
-      if (proxy == null) {
-        try {
-          proxy = ugi.doAs(new PrivilegedExceptionAction<ClientDatanodeProtocol>() {
-            @Override
-            public ClientDatanodeProtocol run() throws Exception {
-              return DFSUtil.createClientDatanodeProtocolProxy(node, conf,
-                  socketTimeout, connectToDnViaHostname);
-            }
-          });
-        } catch (InterruptedException e) {
-          LOG.warn("encountered exception ", e);
-        }
-      }
-      return proxy;
-    }
-    
-    private synchronized void resetDatanodeProxy() {
-      if (null != proxy) {
-        RPC.stopProxy(proxy);
-        proxy = null;
-      }
-    }
-
-    private BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
-      return cache.get(b);
-    }
-
-    private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
-      cache.put(b, info);
-    }
-
-    private void removeBlockLocalPathInfo(ExtendedBlock b) {
-      cache.remove(b);
-    }
-  }
-  
-  // Multiple datanodes could be running on the local machine. Store proxies in
-  // a map keyed by the ipc port of the datanode.
-  private static Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
+  static final Log LOG = LogFactory.getLog(BlockReaderLocal.class);
 
   private final FileInputStream dataIn; // reader for the data file
   private final FileInputStream checksumIn;   // reader for the checksum file
+  private final boolean verifyChecksum;
 
   /**
    * Offset from the most recent chunk boundary at which the next read should
@@ -153,7 +75,6 @@ class BlockReaderLocal implements BlockReader {
   private ByteBuffer slowReadBuff = null;
   private ByteBuffer checksumBuff = null;
   private DataChecksum checksum;
-  private final boolean verifyChecksum;
 
   private static DirectBufferPool bufferPool = new DirectBufferPool();
 
@@ -163,187 +84,92 @@ class BlockReaderLocal implements BlockReader {
   /** offset in block where reader wants to actually read */
   private long startOffset;
   private final String filename;
-  
-  /**
-   * The only way this object can be instantiated.
-   */
-  static BlockReaderLocal newBlockReader(UserGroupInformation ugi,
-      Configuration conf, String file, ExtendedBlock blk,
-      Token<BlockTokenIdentifier> token, DatanodeInfo node, int socketTimeout,
-      long startOffset, long length, boolean connectToDnViaHostname)
-      throws IOException {
-
-    LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
-        .getIpcPort());
-    // check the cache first
-    BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
-    if (pathinfo == null) {
-      pathinfo = getBlockPathInfo(ugi, blk, node, conf, socketTimeout, token,
-          connectToDnViaHostname);
-    }
-
-    // check to see if the file exists. It may so happen that the
-    // HDFS file has been deleted and this block-lookup is occurring
-    // on behalf of a new HDFS file. This time, the block file could
-    // be residing in a different portion of the fs.data.dir directory.
-    // In this case, we remove this entry from the cache. The next
-    // call to this method will re-populate the cache.
-    FileInputStream dataIn = null;
-    FileInputStream checksumIn = null;
-    BlockReaderLocal localBlockReader = null;
-    boolean skipChecksumCheck = skipChecksumCheck(conf);
-    try {
-      // get a local file system
-      File blkfile = new File(pathinfo.getBlockPath());
-      dataIn = new FileInputStream(blkfile);
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("New BlockReaderLocal for file " + blkfile + " of size "
-            + blkfile.length() + " startOffset " + startOffset + " length "
-            + length + " short circuit checksum " + !skipChecksumCheck);
-      }
 
-      if (!skipChecksumCheck) {
-        // get the metadata file
-        File metafile = new File(pathinfo.getMetaPath());
-        checksumIn = new FileInputStream(metafile);
-
-        // read and handle the common header here. For now just a version
-        BlockMetadataHeader header = BlockMetadataHeader
-            .readHeader(new DataInputStream(checksumIn));
-        short version = header.getVersion();
-        if (version != BlockMetadataHeader.VERSION) {
-          LOG.warn("Wrong version (" + version + ") for metadata file for "
-              + blk + " ignoring ...");
-        }
-        DataChecksum checksum = header.getChecksum();
-        long firstChunkOffset = startOffset
-            - (startOffset % checksum.getBytesPerChecksum());
-        localBlockReader = new BlockReaderLocal(conf, file, blk, token,
-            startOffset, length, pathinfo, checksum, true, dataIn,
-            firstChunkOffset, checksumIn);
-      } else {
-        localBlockReader = new BlockReaderLocal(conf, file, blk, token,
-            startOffset, length, pathinfo, dataIn);
-      }
-    } catch (IOException e) {
-      // remove from cache
-      localDatanodeInfo.removeBlockLocalPathInfo(blk);
-      DFSClient.LOG.warn("BlockReaderLocal: Removing " + blk
-          + " from cache because local file " + pathinfo.getBlockPath()
-          + " could not be opened.");
-      throw e;
-    } finally {
-      if (localBlockReader == null) {
-        if (dataIn != null) {
-          dataIn.close();
-        }
-        if (checksumIn != null) {
-          checksumIn.close();
-        }
-      }
-    }
-    return localBlockReader;
-  }
+  private final DatanodeID datanodeID;
+  private final ExtendedBlock block;
   
-  private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
-    LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
-    if (ldInfo == null) {
-      ldInfo = new LocalDatanodeInfo();
-      localDatanodeInfoMap.put(port, ldInfo);
-    }
-    return ldInfo;
-  }
-  
-  private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
-      ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
-      Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname)
-      throws IOException {
-    LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
-    BlockLocalPathInfo pathinfo = null;
-    ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
-        conf, timeout, connectToDnViaHostname);
-    try {
-      // make RPC to local datanode to find local pathnames of blocks
-      pathinfo = proxy.getBlockLocalPathInfo(blk, token);
-      if (pathinfo != null) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Cached location of block " + blk + " as " + pathinfo);
-        }
-        localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
-      }
-    } catch (IOException e) {
-      localDatanodeInfo.resetDatanodeProxy(); // Reset proxy on error
-      throw e;
-    }
-    return pathinfo;
-  }
-  
-  private static boolean skipChecksumCheck(Configuration conf) {
-    return conf.getBoolean(
-        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
-        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
-  }
-  
-  private static int getSlowReadBufferNumChunks(Configuration conf, int bytesPerChecksum) {
-    int bufferSizeBytes = conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
-        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
+  private static int getSlowReadBufferNumChunks(Configuration conf,
+      int bytesPerChecksum) {
 
-    if (bufferSizeBytes < bytesPerChecksum) {
-      throw new IllegalArgumentException("Configured BlockReaderLocal buffer size (" + bufferSizeBytes + ") " +
-          "is not large enough to hold a single chunk (" + bytesPerChecksum +  "). Please configure " +
+    int bufSize =
+        conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
+            DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
+
+    if (bufSize < bytesPerChecksum) {
+      throw new IllegalArgumentException("Configured BlockReaderLocal buffer size (" +
+          bufSize + ") is not large enough to hold a single chunk (" +
+          bytesPerChecksum +  "). Please configure " +
           DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY + " appropriately");
     }
 
     // Round down to nearest chunk size
-    return bufferSizeBytes / bytesPerChecksum;
-  }
-
-  private BlockReaderLocal(Configuration conf, String hdfsfile,
-      ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
-      long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
-      throws IOException {
-    this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
-        DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
-        dataIn, startOffset, null);
+    return bufSize / bytesPerChecksum;
   }
 
-  private BlockReaderLocal(Configuration conf, String hdfsfile,
-      ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
-      long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
-      boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
-      FileInputStream checksumIn) throws IOException {
-    this.filename = hdfsfile;
-    this.checksum = checksum;
-    this.verifyChecksum = verifyChecksum;
-    this.startOffset = Math.max(startOffset, 0);
-
-    bytesPerChecksum = this.checksum.getBytesPerChecksum();
-    checksumSize = this.checksum.getChecksumSize();
-
+  public BlockReaderLocal(Configuration conf, String filename,
+      ExtendedBlock block, long startOffset, long length,
+      FileInputStream dataIn, FileInputStream checksumIn,
+      DatanodeID datanodeID, boolean verifyChecksum) throws IOException {
     this.dataIn = dataIn;
     this.checksumIn = checksumIn;
-    this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
-
-    int chunksPerChecksumRead = getSlowReadBufferNumChunks(conf, bytesPerChecksum);
-    slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
-    checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
-    // Initially the buffers have nothing to read.
-    slowReadBuff.flip();
-    checksumBuff.flip();
+    this.startOffset = Math.max(startOffset, 0);
+    this.filename = filename;
+    this.datanodeID = datanodeID;
+    this.block = block;
+
+    // read and handle the common header here. For now just a version
+    checksumIn.getChannel().position(0);
+    BlockMetadataHeader header = BlockMetadataHeader
+        .readHeader(new DataInputStream(
+            new BufferedInputStream(checksumIn,
+                BlockMetadataHeader.getHeaderSize())));
+    short version = header.getVersion();
+    if (version != BlockMetadataHeader.VERSION) {
+      throw new IOException("Wrong version (" + version + ") of the " +
+          "metadata file for " + filename + ".");
+    }
+    if (!verifyChecksum) {
+      this.verifyChecksum = false; 
+    } else {
+      this.verifyChecksum = !conf.getBoolean(DFSConfigKeys.
+          DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, 
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
+    }
+    long firstChunkOffset;
+    if (this.verifyChecksum) {
+      this.checksum = header.getChecksum();
+      this.bytesPerChecksum = this.checksum.getBytesPerChecksum();
+      this.checksumSize = this.checksum.getChecksumSize();
+      firstChunkOffset = startOffset
+          - (startOffset % checksum.getBytesPerChecksum());
+      this.offsetFromChunkBoundary = (int) (startOffset - firstChunkOffset);
+
+      int chunksPerChecksumRead = getSlowReadBufferNumChunks(conf, bytesPerChecksum);
+      slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
+      checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
+      // Initially the buffers have nothing to read.
+      slowReadBuff.flip();
+      checksumBuff.flip();
+      long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
+      IOUtils.skipFully(checksumIn, checkSumOffset);
+    } else {
+      firstChunkOffset = startOffset;
+      this.checksum = null;
+      this.bytesPerChecksum = 0;
+      this.checksumSize = 0;
+      this.offsetFromChunkBoundary = 0;
+    }
+    
     boolean success = false;
     try {
-      // Skip both input streams to beginning of the chunk containing startOffset
-      IOUtils.skipFully(dataIn, firstChunkOffset);
-      if (checksumIn != null) {
-        long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
-        IOUtils.skipFully(checksumIn, checkSumOffset);
-      }
+      // Reposition both input streams to the beginning of the chunk
+      // containing startOffset
+      this.dataIn.getChannel().position(firstChunkOffset);
       success = true;
     } finally {
       if (!success) {
-        bufferPool.returnBuffer(slowReadBuff);
-        bufferPool.returnBuffer(checksumBuff);
+        if (slowReadBuff != null) bufferPool.returnBuffer(slowReadBuff);
+        if (checksumBuff != null) bufferPool.returnBuffer(checksumBuff);
       }
     }
   }
@@ -663,10 +489,17 @@ class BlockReaderLocal implements BlockReader {
   }
 
   @Override
-  public synchronized void close() throws IOException {
-    dataIn.close();
-    if (checksumIn != null) {
-      checksumIn.close();
+  public synchronized void close(PeerCache peerCache,
+      FileInputStreamCache fisCache) throws IOException {
+    if (fisCache != null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("putting FileInputStream for " + filename +
+            " back into FileInputStreamCache");
+      }
+      fisCache.put(datanodeID, block, new FileInputStream[] {dataIn, checksumIn});
+    } else {
+      LOG.debug("closing FileInputStream for " + filename);
+      IOUtils.cleanup(LOG, dataIn, checksumIn);
     }
     if (slowReadBuff != null) {
       bufferPool.returnBuffer(slowReadBuff);
@@ -691,17 +524,8 @@ class BlockReaderLocal implements BlockReader {
   }
 
   @Override
-  public Socket takeSocket() {
-    return null;
-  }
-
-  @Override
-  public boolean hasSentStatusCode() {
-    return false;
-  }
-
-  @Override
-  public IOStreamPair getStreams() {
-    return null;
+  public int available() throws IOException {
+    // We never do network I/O in BlockReaderLocal.
+    return Integer.MAX_VALUE;
   }
 }

+ 704 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java

@@ -0,0 +1,704 @@
+/**
+ * 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.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.util.DirectBufferPool;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
+
+/**
+ * BlockReaderLocalLegacy enables local short circuited reads. If the DFS client is on
+ * the same machine as the datanode, then the client can read files directly
+ * from the local file system rather than going through the datanode for better
+ * performance. <br>
+ *
+ * This is the legacy implementation based on HDFS-2246, which requires
+ * permissions on the datanode to be set so that clients can directly access the
+ * blocks. The new implementation based on HDFS-347 should be preferred on UNIX
+ * systems where the required native code has been implemented.<br>
+ *
+ * {@link BlockReaderLocalLegacy} works as follows:
+ * <ul>
+ * <li>The client performing short circuit reads must be configured at the
+ * datanode.</li>
+ * <li>The client gets the path to the file where block is stored using
+ * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
+ * RPC call</li>
+ * <li>Client uses kerberos authentication to connect to the datanode over RPC,
+ * if security is enabled.</li>
+ * </ul>
+ */
+class BlockReaderLocalLegacy implements BlockReader {
+  private static final Log LOG = LogFactory.getLog(DFSClient.class);
+
+  //Stores the cache and proxy for a local datanode.
+  private static class LocalDatanodeInfo {
+    private ClientDatanodeProtocol proxy = null;
+    private final Map<ExtendedBlock, BlockLocalPathInfo> cache;
+
+    LocalDatanodeInfo() {
+      final int cacheSize = 10000;
+      final float hashTableLoadFactor = 0.75f;
+      int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
+      cache = Collections
+          .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
+              hashTableCapacity, hashTableLoadFactor, true) {
+            private static final long serialVersionUID = 1;
+
+            @Override
+            protected boolean removeEldestEntry(
+                Map.Entry<ExtendedBlock, BlockLocalPathInfo> eldest) {
+              return size() > cacheSize;
+            }
+          });
+    }
+
+    private synchronized ClientDatanodeProtocol getDatanodeProxy(
+        UserGroupInformation ugi, final DatanodeInfo node,
+        final Configuration conf, final int socketTimeout,
+        final boolean connectToDnViaHostname) throws IOException {
+      if (proxy == null) {
+        try {
+          proxy = ugi.doAs(new PrivilegedExceptionAction<ClientDatanodeProtocol>() {
+            @Override
+            public ClientDatanodeProtocol run() throws Exception {
+              return DFSUtil.createClientDatanodeProtocolProxy(node, conf,
+                  socketTimeout, connectToDnViaHostname);
+            }
+          });
+        } catch (InterruptedException e) {
+          LOG.warn("encountered exception ", e);
+        }
+      }
+      return proxy;
+    }
+    
+    private synchronized void resetDatanodeProxy() {
+      if (null != proxy) {
+        RPC.stopProxy(proxy);
+        proxy = null;
+      }
+    }
+
+    private BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
+      return cache.get(b);
+    }
+
+    private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
+      cache.put(b, info);
+    }
+
+    private void removeBlockLocalPathInfo(ExtendedBlock b) {
+      cache.remove(b);
+    }
+  }
+  
+  // Multiple datanodes could be running on the local machine. Store proxies in
+  // a map keyed by the ipc port of the datanode.
+  private static Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
+
+  private final FileInputStream dataIn; // reader for the data file
+  private final FileInputStream checksumIn;   // reader for the checksum file
+
+  /**
+   * Offset from the most recent chunk boundary at which the next read should
+   * take place. Is only set to non-zero at construction time, and is
+   * decremented (usually to 0) by subsequent reads. This avoids having to do a
+   * checksum read at construction to position the read cursor correctly.
+   */
+  private int offsetFromChunkBoundary;
+  
+  private byte[] skipBuf = null;
+
+  /**
+   * Used for checksummed reads that need to be staged before copying to their
+   * output buffer because they are either a) smaller than the checksum chunk
+   * size or b) issued by the slower read(byte[]...) path
+   */
+  private ByteBuffer slowReadBuff = null;
+  private ByteBuffer checksumBuff = null;
+  private DataChecksum checksum;
+  private final boolean verifyChecksum;
+
+  private static DirectBufferPool bufferPool = new DirectBufferPool();
+
+  private final int bytesPerChecksum;
+  private final int checksumSize;
+
+  /** offset in block where reader wants to actually read */
+  private long startOffset;
+  private final String filename;
+  
+  /**
+   * The only way this object can be instantiated.
+   */
+  static BlockReaderLocalLegacy newBlockReader(UserGroupInformation ugi,
+      Configuration conf, String file, ExtendedBlock blk,
+      Token<BlockTokenIdentifier> token, DatanodeInfo node, int socketTimeout,
+      long startOffset, long length, boolean connectToDnViaHostname)
+      throws IOException {
+
+    LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
+        .getIpcPort());
+    // check the cache first
+    BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
+    if (pathinfo == null) {
+      pathinfo = getBlockPathInfo(ugi, blk, node, conf, socketTimeout, token,
+          connectToDnViaHostname);
+    }
+
+    // check to see if the file exists. It may so happen that the
+    // HDFS file has been deleted and this block-lookup is occurring
+    // on behalf of a new HDFS file. This time, the block file could
+    // be residing in a different portion of the fs.data.dir directory.
+    // In this case, we remove this entry from the cache. The next
+    // call to this method will re-populate the cache.
+    FileInputStream dataIn = null;
+    FileInputStream checksumIn = null;
+    BlockReaderLocalLegacy localBlockReader = null;
+    boolean skipChecksumCheck = skipChecksumCheck(conf);
+    try {
+      // get a local file system
+      File blkfile = new File(pathinfo.getBlockPath());
+      dataIn = new FileInputStream(blkfile);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("New BlockReaderLocalLegacy for file " + blkfile + " of size "
+            + blkfile.length() + " startOffset " + startOffset + " length "
+            + length + " short circuit checksum " + !skipChecksumCheck);
+      }
+
+      if (!skipChecksumCheck) {
+        // get the metadata file
+        File metafile = new File(pathinfo.getMetaPath());
+        checksumIn = new FileInputStream(metafile);
+
+        // read and handle the common header here. For now just a version
+        BlockMetadataHeader header = BlockMetadataHeader
+            .readHeader(new DataInputStream(checksumIn));
+        short version = header.getVersion();
+        if (version != BlockMetadataHeader.VERSION) {
+          LOG.warn("Wrong version (" + version + ") for metadata file for "
+              + blk + " ignoring ...");
+        }
+        DataChecksum checksum = header.getChecksum();
+        long firstChunkOffset = startOffset
+            - (startOffset % checksum.getBytesPerChecksum());
+        localBlockReader = new BlockReaderLocalLegacy(conf, file, blk, token,
+            startOffset, length, pathinfo, checksum, true, dataIn,
+            firstChunkOffset, checksumIn);
+      } else {
+        localBlockReader = new BlockReaderLocalLegacy(conf, file, blk, token,
+            startOffset, length, pathinfo, dataIn);
+      }
+    } catch (IOException e) {
+      // remove from cache
+      localDatanodeInfo.removeBlockLocalPathInfo(blk);
+      DFSClient.LOG.warn("BlockReaderLocalLegacy: Removing " + blk
+          + " from cache because local file " + pathinfo.getBlockPath()
+          + " could not be opened.");
+      throw e;
+    } finally {
+      if (localBlockReader == null) {
+        if (dataIn != null) {
+          dataIn.close();
+        }
+        if (checksumIn != null) {
+          checksumIn.close();
+        }
+      }
+    }
+    return localBlockReader;
+  }
+  
+  private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
+    LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
+    if (ldInfo == null) {
+      ldInfo = new LocalDatanodeInfo();
+      localDatanodeInfoMap.put(port, ldInfo);
+    }
+    return ldInfo;
+  }
+  
+  private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
+      ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
+      Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname)
+      throws IOException {
+    LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
+    BlockLocalPathInfo pathinfo = null;
+    ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
+        conf, timeout, connectToDnViaHostname);
+    try {
+      // make RPC to local datanode to find local pathnames of blocks
+      pathinfo = proxy.getBlockLocalPathInfo(blk, token);
+      if (pathinfo != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Cached location of block " + blk + " as " + pathinfo);
+        }
+        localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
+      }
+    } catch (IOException e) {
+      localDatanodeInfo.resetDatanodeProxy(); // Reset proxy on error
+      throw e;
+    }
+    return pathinfo;
+  }
+  
+  private static boolean skipChecksumCheck(Configuration conf) {
+    return conf.getBoolean(
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
+  }
+  
+  private static int getSlowReadBufferNumChunks(Configuration conf, int bytesPerChecksum) {
+    int bufferSizeBytes = conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
+
+    if (bufferSizeBytes < bytesPerChecksum) {
+      throw new IllegalArgumentException("Configured BlockReaderLocalLegacy " +
+          "buffer size (" + bufferSizeBytes + ") is not large enough to hold " +
+          "a single chunk (" + bytesPerChecksum +  "). Please configure " +
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY +
+          " appropriately");
+    }
+
+    // Round down to nearest chunk size
+    return bufferSizeBytes / bytesPerChecksum;
+  }
+
+  private BlockReaderLocalLegacy(Configuration conf, String hdfsfile,
+      ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
+      long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
+      throws IOException {
+    this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
+        DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
+        dataIn, startOffset, null);
+  }
+
+  private BlockReaderLocalLegacy(Configuration conf, String hdfsfile,
+      ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
+      long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
+      boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
+      FileInputStream checksumIn) throws IOException {
+    this.filename = hdfsfile;
+    this.checksum = checksum;
+    this.verifyChecksum = verifyChecksum;
+    this.startOffset = Math.max(startOffset, 0);
+
+    bytesPerChecksum = this.checksum.getBytesPerChecksum();
+    checksumSize = this.checksum.getChecksumSize();
+
+    this.dataIn = dataIn;
+    this.checksumIn = checksumIn;
+    this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
+
+    int chunksPerChecksumRead = getSlowReadBufferNumChunks(conf, bytesPerChecksum);
+    slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
+    checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
+    // Initially the buffers have nothing to read.
+    slowReadBuff.flip();
+    checksumBuff.flip();
+    boolean success = false;
+    try {
+      // Skip both input streams to beginning of the chunk containing startOffset
+      IOUtils.skipFully(dataIn, firstChunkOffset);
+      if (checksumIn != null) {
+        long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
+        IOUtils.skipFully(checksumIn, checkSumOffset);
+      }
+      success = true;
+    } finally {
+      if (!success) {
+        bufferPool.returnBuffer(slowReadBuff);
+        bufferPool.returnBuffer(checksumBuff);
+      }
+    }
+  }
+
+  /**
+   * Reads bytes into a buffer until EOF or the buffer's limit is reached
+   */
+  private int fillBuffer(FileInputStream stream, ByteBuffer buf)
+      throws IOException {
+    int bytesRead = stream.getChannel().read(buf);
+    if (bytesRead < 0) {
+      //EOF
+      return bytesRead;
+    }
+    while (buf.remaining() > 0) {
+      int n = stream.getChannel().read(buf);
+      if (n < 0) {
+        //EOF
+        return bytesRead;
+      }
+      bytesRead += n;
+    }
+    return bytesRead;
+  }
+  
+  /**
+   * Utility method used by read(ByteBuffer) to partially copy a ByteBuffer into
+   * another.
+   */
+  private void writeSlice(ByteBuffer from, ByteBuffer to, int length) {
+    int oldLimit = from.limit();
+    from.limit(from.position() + length);
+    try {
+      to.put(from);
+    } finally {
+      from.limit(oldLimit);
+    }
+  }
+
+  @Override
+  public synchronized int read(ByteBuffer buf) throws IOException {
+    int nRead = 0;
+    if (verifyChecksum) {
+      // A 'direct' read actually has three phases. The first drains any
+      // remaining bytes from the slow read buffer. After this the read is
+      // guaranteed to be on a checksum chunk boundary. If there are still bytes
+      // to read, the fast direct path is used for as many remaining bytes as
+      // possible, up to a multiple of the checksum chunk size. Finally, any
+      // 'odd' bytes remaining at the end of the read cause another slow read to
+      // be issued, which involves an extra copy.
+
+      // Every 'slow' read tries to fill the slow read buffer in one go for
+      // efficiency's sake. As described above, all non-checksum-chunk-aligned
+      // reads will be served from the slower read path.
+
+      if (slowReadBuff.hasRemaining()) {
+        // There are remaining bytes from a small read available. This usually
+        // means this read is unaligned, which falls back to the slow path.
+        int fromSlowReadBuff = Math.min(buf.remaining(), slowReadBuff.remaining());
+        writeSlice(slowReadBuff, buf, fromSlowReadBuff);
+        nRead += fromSlowReadBuff;
+      }
+
+      if (buf.remaining() >= bytesPerChecksum && offsetFromChunkBoundary == 0) {
+        // Since we have drained the 'small read' buffer, we are guaranteed to
+        // be chunk-aligned
+        int len = buf.remaining() - (buf.remaining() % bytesPerChecksum);
+
+        // There's only enough checksum buffer space available to checksum one
+        // entire slow read buffer. This saves keeping the number of checksum
+        // chunks around.
+        len = Math.min(len, slowReadBuff.capacity());
+        int oldlimit = buf.limit();
+        buf.limit(buf.position() + len);
+        int readResult = 0;
+        try {
+          readResult = doByteBufferRead(buf);
+        } finally {
+          buf.limit(oldlimit);
+        }
+        if (readResult == -1) {
+          return nRead;
+        } else {
+          nRead += readResult;
+          buf.position(buf.position() + readResult);
+        }
+      }
+
+      // offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
+      // until chunk boundary
+      if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) || offsetFromChunkBoundary > 0) {
+        int toRead = Math.min(buf.remaining(), bytesPerChecksum - offsetFromChunkBoundary);
+        int readResult = fillSlowReadBuffer(toRead);
+        if (readResult == -1) {
+          return nRead;
+        } else {
+          int fromSlowReadBuff = Math.min(readResult, buf.remaining());
+          writeSlice(slowReadBuff, buf, fromSlowReadBuff);
+          nRead += fromSlowReadBuff;
+        }
+      }
+    } else {
+      // Non-checksummed reads are much easier; we can just fill the buffer directly.
+      nRead = doByteBufferRead(buf);
+      if (nRead > 0) {
+        buf.position(buf.position() + nRead);
+      }
+    }
+    return nRead;
+  }
+
+  /**
+   * Tries to read as many bytes as possible into supplied buffer, checksumming
+   * each chunk if needed.
+   *
+   * <b>Preconditions:</b>
+   * <ul>
+   * <li>
+   * If checksumming is enabled, buf.remaining must be a multiple of
+   * bytesPerChecksum. Note that this is not a requirement for clients of
+   * read(ByteBuffer) - in the case of non-checksum-sized read requests,
+   * read(ByteBuffer) will substitute a suitably sized buffer to pass to this
+   * method.
+   * </li>
+   * </ul>
+   * <b>Postconditions:</b>
+   * <ul>
+   * <li>buf.limit and buf.mark are unchanged.</li>
+   * <li>buf.position += min(offsetFromChunkBoundary, totalBytesRead) - so the
+   * requested bytes can be read straight from the buffer</li>
+   * </ul>
+   *
+   * @param buf
+   *          byte buffer to write bytes to. If checksums are not required, buf
+   *          can have any number of bytes remaining, otherwise there must be a
+   *          multiple of the checksum chunk size remaining.
+   * @return <tt>max(min(totalBytesRead, len) - offsetFromChunkBoundary, 0)</tt>
+   *         that is, the the number of useful bytes (up to the amount
+   *         requested) readable from the buffer by the client.
+   */
+  private synchronized int doByteBufferRead(ByteBuffer buf) throws IOException {
+    if (verifyChecksum) {
+      assert buf.remaining() % bytesPerChecksum == 0;
+    }
+    int dataRead = -1;
+
+    int oldpos = buf.position();
+    // Read as much as we can into the buffer.
+    dataRead = fillBuffer(dataIn, buf);
+
+    if (dataRead == -1) {
+      return -1;
+    }
+
+    if (verifyChecksum) {
+      ByteBuffer toChecksum = buf.duplicate();
+      toChecksum.position(oldpos);
+      toChecksum.limit(oldpos + dataRead);
+
+      checksumBuff.clear();
+      // Equivalent to (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
+      int numChunks =
+        (toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
+      checksumBuff.limit(checksumSize * numChunks);
+
+      fillBuffer(checksumIn, checksumBuff);
+      checksumBuff.flip();
+
+      checksum.verifyChunkedSums(toChecksum, checksumBuff, filename,
+          this.startOffset);
+    }
+
+    if (dataRead >= 0) {
+      buf.position(oldpos + Math.min(offsetFromChunkBoundary, dataRead));
+    }
+
+    if (dataRead < offsetFromChunkBoundary) {
+      // yikes, didn't even get enough bytes to honour offset. This can happen
+      // even if we are verifying checksums if we are at EOF.
+      offsetFromChunkBoundary -= dataRead;
+      dataRead = 0;
+    } else {
+      dataRead -= offsetFromChunkBoundary;
+      offsetFromChunkBoundary = 0;
+    }
+
+    return dataRead;
+  }
+
+  /**
+   * Ensures that up to len bytes are available and checksummed in the slow read
+   * buffer. The number of bytes available to read is returned. If the buffer is
+   * not already empty, the number of remaining bytes is returned and no actual
+   * read happens.
+   *
+   * @param len
+   *          the maximum number of bytes to make available. After len bytes
+   *          are read, the underlying bytestream <b>must</b> be at a checksum
+   *          boundary, or EOF. That is, (len + currentPosition) %
+   *          bytesPerChecksum == 0.
+   * @return the number of bytes available to read, or -1 if EOF.
+   */
+  private synchronized int fillSlowReadBuffer(int len) throws IOException {
+    int nRead = -1;
+    if (slowReadBuff.hasRemaining()) {
+      // Already got data, good to go.
+      nRead = Math.min(len, slowReadBuff.remaining());
+    } else {
+      // Round a complete read of len bytes (plus any implicit offset) to the
+      // next chunk boundary, since we try and read in multiples of a chunk
+      int nextChunk = len + offsetFromChunkBoundary +
+          (bytesPerChecksum - ((len + offsetFromChunkBoundary) % bytesPerChecksum));
+      int limit = Math.min(nextChunk, slowReadBuff.capacity());
+      assert limit % bytesPerChecksum == 0;
+
+      slowReadBuff.clear();
+      slowReadBuff.limit(limit);
+
+      nRead = doByteBufferRead(slowReadBuff);
+
+      if (nRead > 0) {
+        // So that next time we call slowReadBuff.hasRemaining(), we don't get a
+        // false positive.
+        slowReadBuff.limit(nRead + slowReadBuff.position());
+      }
+    }
+    return nRead;
+  }
+
+  @Override
+  public synchronized int read(byte[] buf, int off, int len) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read off " + off + " len " + len);
+    }
+    if (!verifyChecksum) {
+      return dataIn.read(buf, off, len);
+    }
+
+    int nRead = fillSlowReadBuffer(slowReadBuff.capacity());
+
+    if (nRead > 0) {
+      // Possible that buffer is filled with a larger read than we need, since
+      // we tried to read as much as possible at once
+      nRead = Math.min(len, nRead);
+      slowReadBuff.get(buf, off, nRead);
+    }
+
+    return nRead;
+  }
+
+  @Override
+  public synchronized long skip(long n) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("skip " + n);
+    }
+    if (n <= 0) {
+      return 0;
+    }
+    if (!verifyChecksum) {
+      return dataIn.skip(n);
+    }
+  
+    // caller made sure newPosition is not beyond EOF.
+    int remaining = slowReadBuff.remaining();
+    int position = slowReadBuff.position();
+    int newPosition = position + (int)n;
+  
+    // if the new offset is already read into dataBuff, just reposition
+    if (n <= remaining) {
+      assert offsetFromChunkBoundary == 0;
+      slowReadBuff.position(newPosition);
+      return n;
+    }
+  
+    // for small gap, read through to keep the data/checksum in sync
+    if (n - remaining <= bytesPerChecksum) {
+      slowReadBuff.position(position + remaining);
+      if (skipBuf == null) {
+        skipBuf = new byte[bytesPerChecksum];
+      }
+      int ret = read(skipBuf, 0, (int)(n - remaining));
+      return ret;
+    }
+  
+    // optimize for big gap: discard the current buffer, skip to
+    // the beginning of the appropriate checksum chunk and then
+    // read to the middle of that chunk to be in sync with checksums.
+  
+    // We can't use this.offsetFromChunkBoundary because we need to know how
+    // many bytes of the offset were really read. Calling read(..) with a
+    // positive this.offsetFromChunkBoundary causes that many bytes to get
+    // silently skipped.
+    int myOffsetFromChunkBoundary = newPosition % bytesPerChecksum;
+    long toskip = n - remaining - myOffsetFromChunkBoundary;
+
+    slowReadBuff.position(slowReadBuff.limit());
+    checksumBuff.position(checksumBuff.limit());
+  
+    IOUtils.skipFully(dataIn, toskip);
+    long checkSumOffset = (toskip / bytesPerChecksum) * checksumSize;
+    IOUtils.skipFully(checksumIn, checkSumOffset);
+
+    // read into the middle of the chunk
+    if (skipBuf == null) {
+      skipBuf = new byte[bytesPerChecksum];
+    }
+    assert skipBuf.length == bytesPerChecksum;
+    assert myOffsetFromChunkBoundary < bytesPerChecksum;
+
+    int ret = read(skipBuf, 0, myOffsetFromChunkBoundary);
+
+    if (ret == -1) {  // EOS
+      return toskip;
+    } else {
+      return (toskip + ret);
+    }
+  }
+
+  @Override
+  public synchronized void close(PeerCache peerCache,
+      FileInputStreamCache fisCache) throws IOException {
+    IOUtils.cleanup(LOG, dataIn, checksumIn);
+    if (slowReadBuff != null) {
+      bufferPool.returnBuffer(slowReadBuff);
+      slowReadBuff = null;
+    }
+    if (checksumBuff != null) {
+      bufferPool.returnBuffer(checksumBuff);
+      checksumBuff = null;
+    }
+    startOffset = -1;
+    checksum = null;
+  }
+
+  @Override
+  public int readAll(byte[] buf, int offset, int len) throws IOException {
+    return BlockReaderUtil.readAll(this, buf, offset, len);
+  }
+
+  @Override
+  public void readFully(byte[] buf, int off, int len) throws IOException {
+    BlockReaderUtil.readFully(this, buf, off, len);
+  }
+
+  @Override
+  public int available() throws IOException {
+    // We never do network I/O in BlockReaderLocalLegacy.
+    return Integer.MAX_VALUE;
+  }
+}

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

@@ -131,7 +131,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 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;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -196,12 +195,13 @@ public class DFSClient implements java.io.Closeable {
   final FileSystem.Statistics stats;
   final int hdfsTimeout;    // timeout value for a DFS operation.
   private final String authority;
-  final SocketCache socketCache;
+  final PeerCache peerCache;
   final Conf dfsClientConf;
   private Random r = new Random();
   private SocketAddress[] localInterfaceAddrs;
   private DataEncryptionKey encryptionKey;
-
+  private boolean shouldUseLegacyBlockReaderLocal;
+  
   /**
    * DFSClient configuration 
    */
@@ -228,11 +228,16 @@ public class DFSClient implements java.io.Closeable {
     final short defaultReplication;
     final String taskId;
     final FsPermission uMask;
-    final boolean useLegacyBlockReader;
+    final boolean useLegacyBlockReaderLocal;
     final boolean connectToDnViaHostname;
     final boolean getHdfsBlocksMetadataEnabled;
     final int getFileBlockStorageLocationsNumThreads;
     final int getFileBlockStorageLocationsTimeout;
+    final String domainSocketPath;
+    final boolean skipShortCircuitChecksums;
+    final int shortCircuitBufferSize;
+    final boolean shortCircuitLocalReads;
+    final boolean domainSocketDataTraffic;
 
     Conf(Configuration conf) {
       maxFailoverAttempts = conf.getInt(
@@ -283,9 +288,9 @@ public class DFSClient implements java.io.Closeable {
           .getInt(DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
               DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
       uMask = FsPermission.getUMask(conf);
-      useLegacyBlockReader = conf.getBoolean(
-          DFS_CLIENT_USE_LEGACY_BLOCKREADER,
-          DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT);
+      useLegacyBlockReaderLocal = conf.getBoolean(
+          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
+          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT);
       connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
           DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
       getHdfsBlocksMetadataEnabled = conf.getBoolean(
@@ -297,6 +302,20 @@ public class DFSClient implements java.io.Closeable {
       getFileBlockStorageLocationsTimeout = conf.getInt(
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT,
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_DEFAULT);
+      domainSocketPath = conf.getTrimmed(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+          DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT);
+      skipShortCircuitChecksums = conf.getBoolean(
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
+      shortCircuitBufferSize = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
+      shortCircuitLocalReads = conf.getBoolean(
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
+      domainSocketDataTraffic = conf.getBoolean(
+        DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+        DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT);
     }
 
     private DataChecksum.Type getChecksumType(Configuration conf) {
@@ -354,7 +373,7 @@ public class DFSClient implements java.io.Closeable {
   private final Map<String, DFSOutputStream> filesBeingWritten
       = new HashMap<String, DFSOutputStream>();
 
-  private boolean shortCircuitLocalReads;
+  private final DomainSocketFactory domainSocketFactory;
   
   /**
    * Same as this(NameNode.getAddress(conf), conf);
@@ -398,6 +417,11 @@ public class DFSClient implements java.io.Closeable {
     throws IOException {
     // Copy only the required DFSClient configuration
     this.dfsClientConf = new Conf(conf);
+    this.shouldUseLegacyBlockReaderLocal = 
+        this.dfsClientConf.useLegacyBlockReaderLocal;
+    if (this.dfsClientConf.useLegacyBlockReaderLocal) {
+      LOG.debug("Using legacy short-circuit local reads.");
+    }
     this.conf = conf;
     this.stats = stats;
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
@@ -427,12 +451,8 @@ public class DFSClient implements java.io.Closeable {
     }
 
     // read directly from the block file if configured.
-    this.shortCircuitLocalReads = conf.getBoolean(
-        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
-        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Short circuit read is " + shortCircuitLocalReads);
-    }
+    this.domainSocketFactory = new DomainSocketFactory(dfsClientConf);
+
     String localInterfaces[] =
       conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
     localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
@@ -442,7 +462,7 @@ public class DFSClient implements java.io.Closeable {
       Joiner.on(',').join(localInterfaceAddrs) + "]");
     }
     
-    this.socketCache = SocketCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
+    this.peerCache = PeerCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
   }
 
   /**
@@ -797,29 +817,11 @@ public class DFSClient implements java.io.Closeable {
                                      AccessControlException.class);
     }
   }
-
-  /**
-   * Get {@link BlockReader} for short circuited local reads.
-   */
-  static BlockReader getLocalBlockReader(UserGroupInformation ugi,
-      Configuration conf, String src, ExtendedBlock blk,
-      Token<BlockTokenIdentifier> accessToken, DatanodeInfo chosenNode,
-      int socketTimeout, long offsetIntoBlock, boolean connectToDnViaHostname)
-      throws InvalidToken, IOException {
-    try {
-      return BlockReaderLocal.newBlockReader(ugi, conf, src, blk, accessToken,
-          chosenNode, socketTimeout, offsetIntoBlock, blk.getNumBytes()
-              - offsetIntoBlock, connectToDnViaHostname);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(InvalidToken.class,
-          AccessControlException.class);
-    }
-  }
   
   private static Map<String, Boolean> localAddrMap = Collections
       .synchronizedMap(new HashMap<String, Boolean>());
   
-  private static boolean isLocalAddress(InetSocketAddress targetAddr) {
+  static boolean isLocalAddress(InetSocketAddress targetAddr) {
     InetAddress addr = targetAddr.getAddress();
     Boolean cached = localAddrMap.get(addr.getHostAddress());
     if (cached != null) {
@@ -2319,10 +2321,6 @@ public class DFSClient implements java.io.Closeable {
       super(in);
     }
   }
-  
-  boolean shouldTryShortCircuitRead(InetSocketAddress targetAddr) {
-    return shortCircuitLocalReads && isLocalAddress(targetAddr);
-  }
 
   void reportChecksumFailure(String file, ExtendedBlock blk, DatanodeInfo dn) {
     DatanodeInfo [] dnArr = { dn };
@@ -2346,13 +2344,15 @@ public class DFSClient implements java.io.Closeable {
         + ", ugi=" + ugi + "]"; 
   }
 
-  void disableShortCircuit() {
-    LOG.info("Short circuit is disabled");
-    shortCircuitLocalReads = false;
+  public DomainSocketFactory getDomainSocketFactory() {
+    return domainSocketFactory;
   }
-  
-  @VisibleForTesting
-  boolean getShortCircuitLocalReads() {
-    return shortCircuitLocalReads;
+
+  public void disableLegacyBlockReaderLocal() {
+    shouldUseLegacyBlockReaderLocal = false;
+  }
+
+  public boolean useLegacyBlockReaderLocal() {
+    return shouldUseLegacyBlockReaderLocal;
   }
 }

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

@@ -265,6 +265,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT = 3;
   public static final String  DFS_CLIENT_USE_LEGACY_BLOCKREADER = "dfs.client.use.legacy.blockreader";
   public static final boolean DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT = false;
+  public static final String  DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL = "dfs.client.use.legacy.blockreader.local";
+  public static final boolean DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT = false;
   public static final String  DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth";
   public static final long    DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
@@ -347,7 +349,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY = "dfs.client.read.shortcircuit.skip.checksum";
   public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT = false;
   public static final String DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY = "dfs.client.read.shortcircuit.buffer.size";
+  public static final String DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY = "dfs.client.read.shortcircuit.streams.cache.size";
+  public static final int DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT = 100;
+  public static final String DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY = "dfs.client.read.shortcircuit.streams.cache.expiry.ms";
+  public static final long DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT = 5000;
   public static final int DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT = 1024 * 1024;
+  public static final String DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC = "dfs.client.domain.socket.data.traffic";
+  public static final boolean DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT = false;
 
   // property for fsimage compression
   public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress";
@@ -404,6 +412,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_MAX_OP_SIZE_DEFAULT = 50 * 1024 * 1024;
   
   public static final String DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY = "dfs.block.local-path-access.user";
+  public static final String DFS_DOMAIN_SOCKET_PATH_KEY = "dfs.domain.socket.path";
+  public static final String DFS_DOMAIN_SOCKET_PATH_DEFAULT = "";
 
   // HA related configuration
   public static final String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";

+ 148 - 87
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
@@ -32,18 +33,20 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.SocketCache.SocketAndStreams;
+import org.apache.hadoop.hdfs.net.DomainPeer;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 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.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
@@ -51,20 +54,23 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /****************************************************************
  * DFSInputStream provides bytes from a named file.  It handles 
  * negotiation of the namenode and various datanodes as necessary.
  ****************************************************************/
 @InterfaceAudience.Private
 public class DFSInputStream extends FSInputStream implements ByteBufferReadable {
-  private final SocketCache socketCache;
-
+  @VisibleForTesting
+  static boolean tcpReadsDisabledForTesting = false;
+  private final PeerCache peerCache;
   private final DFSClient dfsClient;
   private boolean closed = false;
-
   private final String src;
   private final long prefetchSize;
   private BlockReader blockReader = null;
@@ -76,6 +82,8 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
   private long pos = 0;
   private long blockEnd = -1;
 
+  private final FileInputStreamCache fileInputStreamCache;
+
   /**
    * This variable tracks the number of failures since the start of the
    * most recent user-facing operation. That is to say, it should be reset
@@ -110,7 +118,14 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
     this.verifyChecksum = verifyChecksum;
     this.buffersize = buffersize;
     this.src = src;
-    this.socketCache = dfsClient.socketCache;
+    this.peerCache = dfsClient.peerCache;
+    this.fileInputStreamCache = new FileInputStreamCache(
+      dfsClient.conf.getInt(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY,
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT),
+      dfsClient.conf.getLong(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT));
     prefetchSize = dfsClient.getConf().prefetchSize;
     timeWindow = dfsClient.getConf().timeWindow;
     nCachedConnRetry = dfsClient.getConf().nCachedConnRetry;
@@ -243,7 +258,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
         locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
   }
 
-  private synchronized boolean blockUnderConstruction() {
+  // Short circuit local reads are forbidden for files that are
+  // under construction.  See HDFS-2757.
+  synchronized boolean shortCircuitForbidden() {
     return locatedBlocks.isUnderConstruction();
   }
 
@@ -424,7 +441,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
 
     // Will be getting a new BlockReader.
     if (blockReader != null) {
-      closeBlockReader(blockReader);
+      blockReader.close(peerCache, fileInputStreamCache);
       blockReader = null;
     }
 
@@ -462,7 +479,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
         return chosenNode;
       } catch (AccessControlException ex) {
         DFSClient.LOG.warn("Short circuit access failed " + ex);
-        dfsClient.disableShortCircuit();
+        dfsClient.disableLegacyBlockReaderLocal();
         continue;
       } catch (IOException ex) {
         if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
@@ -510,10 +527,11 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
     dfsClient.checkOpen();
 
     if (blockReader != null) {
-      closeBlockReader(blockReader);
+      blockReader.close(peerCache, fileInputStreamCache);
       blockReader = null;
     }
     super.close();
+    fileInputStreamCache.close();
     closed = true;
   }
 
@@ -811,7 +829,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
         addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
       } catch (AccessControlException ex) {
         DFSClient.LOG.warn("Short circuit access failed " + ex);
-        dfsClient.disableShortCircuit();
+        dfsClient.disableLegacyBlockReaderLocal();
         continue;
       } catch (IOException e) {
         if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
@@ -837,7 +855,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
         }
       } finally {
         if (reader != null) {
-          closeBlockReader(reader);
+          reader.close(peerCache, fileInputStreamCache);
         }
       }
       // Put chosen node into dead list, continue
@@ -845,22 +863,34 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
     }
   }
 
-  /**
-   * Close the given BlockReader and cache its socket.
-   */
-  private void closeBlockReader(BlockReader reader) throws IOException {
-    if (reader.hasSentStatusCode()) {
-      IOStreamPair ioStreams = reader.getStreams();
-      Socket oldSock = reader.takeSocket();
-      socketCache.put(oldSock, ioStreams);
+  private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
+    Peer peer = null;
+    boolean success = false;
+    Socket sock = null;
+    try {
+      sock = dfsClient.socketFactory.createSocket();
+      NetUtils.connect(sock, addr,
+        dfsClient.getRandomLocalInterfaceAddr(),
+        dfsClient.getConf().socketTimeout);
+      peer = TcpPeerServer.peerFromSocketAndKey(sock, 
+          dfsClient.getDataEncryptionKey());
+      success = true;
+      return peer;
+    } finally {
+      if (!success) {
+        IOUtils.closeQuietly(peer);
+        IOUtils.closeQuietly(sock);
+      }
     }
-    reader.close();
   }
 
   /**
    * Retrieve a BlockReader suitable for reading.
    * This method will reuse the cached connection to the DN if appropriate.
    * Otherwise, it will create a new connection.
+   * Throwing an IOException from this method is basically equivalent to 
+   * declaring the DataNode bad, so we try to connect a lot of different ways
+   * before doing that.
    *
    * @param dnAddr  Address of the datanode
    * @param chosenNode Chosen datanode information
@@ -885,82 +915,113 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
                                        boolean verifyChecksum,
                                        String clientName)
       throws IOException {
-    
-    // Can't local read a block under construction, see HDFS-2757
-    if (dfsClient.shouldTryShortCircuitRead(dnAddr) &&
-        !blockUnderConstruction()) {
-      return DFSClient.getLocalBlockReader(dfsClient.ugi, dfsClient.conf,
-          src, block, blockToken, chosenNode, dfsClient.hdfsTimeout,
-          startOffset, dfsClient.connectToDnViaHostname());
+    // Firstly, we check to see if we have cached any file descriptors for
+    // local blocks.  If so, we can just re-use those file descriptors.
+    FileInputStream fis[] = fileInputStreamCache.get(chosenNode, block);
+    if (fis != null) {
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("got FileInputStreams for " + block + " from " +
+            "the FileInputStreamCache.");
+      }
+      return new BlockReaderLocal(dfsClient.conf, file,
+        block, startOffset, len, fis[0], fis[1], chosenNode, verifyChecksum);
     }
     
-    IOException err = null;
-    boolean fromCache = true;
-
-    // Allow retry since there is no way of knowing whether the cached socket
-    // is good until we actually use it.
-    for (int retries = 0; retries <= nCachedConnRetry && fromCache; ++retries) {
-      SocketAndStreams sockAndStreams = null;
-      // Don't use the cache on the last attempt - it's possible that there
-      // are arbitrarily many unusable sockets in the cache, but we don't
-      // want to fail the read.
-      if (retries < nCachedConnRetry) {
-        sockAndStreams = socketCache.get(dnAddr);
+    // If the legacy local block reader is enabled and we are reading a local
+    // block, try to create a BlockReaderLocalLegacy.  The legacy local block
+    // reader implements local reads in the style first introduced by HDFS-2246.
+    if ((dfsClient.useLegacyBlockReaderLocal()) &&
+        DFSClient.isLocalAddress(dnAddr) &&
+        (!shortCircuitForbidden())) {
+      try {
+        return BlockReaderFactory.getLegacyBlockReaderLocal(dfsClient.ugi,
+            dfsClient.conf, clientName, block, blockToken, chosenNode,
+            dfsClient.hdfsTimeout, startOffset,dfsClient.connectToDnViaHostname());
+      } catch (IOException e) {
+        DFSClient.LOG.warn("error creating legacy BlockReaderLocal.  " +
+            "Disabling legacy local reads.", e);
+        dfsClient.disableLegacyBlockReaderLocal();
       }
-      Socket sock;
-      if (sockAndStreams == null) {
-        fromCache = false;
+    }
 
-        sock = dfsClient.socketFactory.createSocket();
-        
-        // TCP_NODELAY is crucial here because of bad interactions between
-        // Nagle's Algorithm and Delayed ACKs. With connection keepalive
-        // between the client and DN, the conversation looks like:
-        //   1. Client -> DN: Read block X
-        //   2. DN -> Client: data for block X
-        //   3. Client -> DN: Status OK (successful read)
-        //   4. Client -> DN: Read block Y
-        // The fact that step #3 and #4 are both in the client->DN direction
-        // triggers Nagling. If the DN is using delayed ACKs, this results
-        // in a delay of 40ms or more.
-        //
-        // TCP_NODELAY disables nagling and thus avoids this performance
-        // disaster.
-        sock.setTcpNoDelay(true);
-
-        NetUtils.connect(sock, dnAddr,
-            dfsClient.getRandomLocalInterfaceAddr(),
-            dfsClient.getConf().socketTimeout);
-        sock.setSoTimeout(dfsClient.getConf().socketTimeout);
-      } else {
-        sock = sockAndStreams.sock;
+    // Look for cached domain peers.
+    int cacheTries = 0;
+    DomainSocketFactory dsFactory = dfsClient.getDomainSocketFactory();
+    BlockReader reader = null;
+    for (; cacheTries < nCachedConnRetry; ++cacheTries) {
+      Peer peer = peerCache.get(chosenNode, true);
+      if (peer == null) break;
+      try {
+        boolean allowShortCircuitLocalReads = dfsClient.getConf().
+            shortCircuitLocalReads && (!shortCircuitForbidden());
+        reader = BlockReaderFactory.newBlockReader(
+            dfsClient.conf, file, block, blockToken, startOffset,
+            len, verifyChecksum, clientName, peer, chosenNode, 
+            dsFactory, allowShortCircuitLocalReads);
+        return reader;
+      } catch (IOException ex) {
+        DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
+            "Closing stale " + peer, ex);
+      } finally {
+        if (reader == null) {
+          IOUtils.closeQuietly(peer);
+        }
       }
+    }
 
+    // Try to create a DomainPeer.
+    DomainSocket domSock = dsFactory.create(dnAddr, this);
+    if (domSock != null) {
+      Peer peer = new DomainPeer(domSock);
       try {
-        // The OP_READ_BLOCK request is sent as we make the BlockReader
-        BlockReader reader =
-            BlockReaderFactory.newBlockReader(dfsClient.getConf(),
-                                       sock, file, block,
-                                       blockToken,
-                                       startOffset, len,
-                                       bufferSize, verifyChecksum,
-                                       clientName,
-                                       dfsClient.getDataEncryptionKey(),
-                                       sockAndStreams == null ? null : sockAndStreams.ioStreams);
+        boolean allowShortCircuitLocalReads = dfsClient.getConf().
+            shortCircuitLocalReads && (!shortCircuitForbidden());
+        reader = BlockReaderFactory.newBlockReader(
+            dfsClient.conf, file, block, blockToken, startOffset,
+            len, verifyChecksum, clientName, peer, chosenNode, 
+            dsFactory, allowShortCircuitLocalReads);
         return reader;
-      } catch (IOException ex) {
-        // Our socket is no good.
-        DFSClient.LOG.debug("Error making BlockReader. Closing stale " + sock, ex);
-        if (sockAndStreams != null) {
-          sockAndStreams.close();
-        } else {
-          sock.close();
+      } catch (IOException e) {
+        DFSClient.LOG.warn("failed to connect to " + domSock, e);
+      } finally {
+        if (reader == null) {
+         // If the Peer that we got the error from was a DomainPeer,
+         // mark the socket path as bad, so that newDataSocket will not try 
+         // to re-open this socket for a while.
+         dsFactory.disableDomainSocketPath(domSock.getPath());
+         IOUtils.closeQuietly(peer);
         }
-        err = ex;
       }
     }
 
-    throw err;
+    // Look for cached peers.
+    for (; cacheTries < nCachedConnRetry; ++cacheTries) {
+      Peer peer = peerCache.get(chosenNode, false);
+      if (peer == null) break;
+      try {
+        reader = BlockReaderFactory.newBlockReader(
+            dfsClient.conf, file, block, blockToken, startOffset,
+            len, verifyChecksum, clientName, peer, chosenNode, 
+            dsFactory, false);
+        return reader;
+      } catch (IOException ex) {
+        DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
+          peer, ex);
+      } finally {
+        if (reader == null) {
+          IOUtils.closeQuietly(peer);
+        }
+      }
+    }
+    if (tcpReadsDisabledForTesting) {
+      throw new IOException("TCP reads are disabled.");
+    }
+    // Try to create a new remote peer.
+    Peer peer = newTcpPeer(dnAddr);
+    return BlockReaderFactory.newBlockReader(
+        dfsClient.conf, file, block, blockToken, startOffset,
+        len, verifyChecksum, clientName, peer, chosenNode, 
+        dsFactory, false);
   }
 
 
@@ -1094,7 +1155,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
       // the TCP buffer, then just eat up the intervening data.
       //
       int diff = (int)(targetPos - pos);
-      if (diff <= DFSClient.TCP_WINDOW_SIZE) {
+      if (diff <= blockReader.available()) {
         try {
           pos += blockReader.skip(diff);
           if (pos == targetPos) {

+ 142 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DomainSocketFactory.java

@@ -0,0 +1,142 @@
+/**
+ * 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.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.DFSClient.Conf;
+
+import org.apache.hadoop.net.unix.DomainSocket;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+
+class DomainSocketFactory {
+  public static final Log LOG = LogFactory.getLog(DomainSocketFactory.class);
+  private final Conf conf;
+
+  enum PathStatus {
+    UNUSABLE,
+    SHORT_CIRCUIT_DISABLED,
+  }
+
+  /**
+   * Information about domain socket paths.
+   */
+  Cache<String, PathStatus> pathInfo =
+      CacheBuilder.newBuilder()
+      .expireAfterWrite(10, TimeUnit.MINUTES)
+      .build();
+
+  public DomainSocketFactory(Conf conf) {
+    this.conf = conf;
+
+    String feature = null;
+    if (conf.shortCircuitLocalReads && (!conf.useLegacyBlockReaderLocal)) {
+      feature = "The short-circuit local reads feature";
+    } else if (conf.domainSocketDataTraffic) {
+      feature = "UNIX domain socket data traffic";
+    }
+    if (feature != null) {
+      if (conf.domainSocketPath.isEmpty()) {
+        LOG.warn(feature + " is disabled because you have not set " +
+            DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY);
+      } else if (DomainSocket.getLoadingFailureReason() != null) {
+        LOG.warn(feature + " is disabled because " +
+              DomainSocket.getLoadingFailureReason());
+      } else {
+        LOG.debug(feature + "is enabled.");
+      }
+    }
+  }
+
+  /**
+   * Create a DomainSocket.
+   * 
+   * @param addr        The address of the DataNode
+   * @param stream      The DFSInputStream the socket will be created for.
+   *
+   * @return            null if the socket could not be created; the
+   *                    socket otherwise.  If there was an error while
+   *                    creating the socket, we will add the socket path
+   *                    to our list of failed domain socket paths.
+   */
+  DomainSocket create(InetSocketAddress addr, DFSInputStream stream) {
+    // If there is no domain socket path configured, we can't use domain
+    // sockets.
+    if (conf.domainSocketPath.isEmpty()) return null;
+    // If we can't do anything with the domain socket, don't create it.
+    if ((conf.domainSocketDataTraffic == false) &&
+        ((!conf.shortCircuitLocalReads) || conf.useLegacyBlockReaderLocal)) {
+      return null;
+    }
+    // UNIX domain sockets can only be used to talk to local peers
+    if (!DFSClient.isLocalAddress(addr)) return null;
+    // If the DomainSocket code is not loaded, we can't create
+    // DomainSocket objects.
+    if (DomainSocket.getLoadingFailureReason() != null) return null;
+    String escapedPath = DomainSocket.
+        getEffectivePath(conf.domainSocketPath, addr.getPort());
+    PathStatus info = pathInfo.getIfPresent(escapedPath);
+    if (info == PathStatus.UNUSABLE) {
+      // We tried to connect to this domain socket before, and it was totally
+      // unusable.
+      return null;
+    }
+    if ((!conf.domainSocketDataTraffic) &&
+        ((info == PathStatus.SHORT_CIRCUIT_DISABLED) || 
+            stream.shortCircuitForbidden())) {
+      // If we don't want to pass data over domain sockets, and we don't want
+      // to pass file descriptors over them either, we have no use for domain
+      // sockets.
+      return null;
+    }
+    boolean success = false;
+    DomainSocket sock = null;
+    try {
+      sock = DomainSocket.connect(escapedPath);
+      sock.setAttribute(DomainSocket.RECEIVE_TIMEOUT, conf.socketTimeout);
+      success = true;
+    } catch (IOException e) {
+      LOG.warn("error creating DomainSocket", e);
+      // fall through
+    } finally {
+      if (!success) {
+        if (sock != null) {
+          IOUtils.closeQuietly(sock);
+        }
+        pathInfo.put(escapedPath, PathStatus.UNUSABLE);
+        sock = null;
+      }
+    }
+    return sock;
+  }
+
+  public void disableShortCircuitForPath(String path) {
+    pathInfo.put(path, PathStatus.SHORT_CIRCUIT_DISABLED);
+  }
+
+  public void disableDomainSocketPath(String path) {
+    pathInfo.put(path, PathStatus.UNUSABLE);
+  }
+}

+ 264 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/FileInputStreamCache.java

@@ -0,0 +1,264 @@
+/**
+ * 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.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Time;
+
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * FileInputStream cache is used to cache FileInputStream objects that we
+ * have received from the DataNode.
+ */
+class FileInputStreamCache {
+  private final static Log LOG = LogFactory.getLog(FileInputStreamCache.class);
+
+  /**
+   * The executor service that runs the cacheCleaner.  There is only one of
+   * these per VM.
+   */
+  private final static ScheduledThreadPoolExecutor executor
+      = new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
+          setDaemon(true).setNameFormat("FileInputStreamCache Cleaner").
+          build());
+  
+  /**
+   * The CacheCleaner for this FileInputStreamCache.  We don't create this
+   * and schedule it until it becomes necessary.
+   */
+  private CacheCleaner cacheCleaner;
+  
+  /**
+   * Maximum number of entries to allow in the cache.
+   */
+  private final int maxCacheSize;
+  
+  /**
+   * The minimum time in milliseconds to preserve an element in the cache.
+   */
+  private final long expiryTimeMs;
+  
+  /**
+   * True if the FileInputStreamCache is closed.
+   */
+  private boolean closed = false;
+  
+  /**
+   * Cache entries.
+   */
+  private final LinkedListMultimap<Key, Value> map = LinkedListMultimap.create();
+
+  /**
+   * Expiry thread which makes sure that the file descriptors get closed
+   * after a while.
+   */
+  class CacheCleaner implements Runnable {
+    @Override
+    public void run() {
+      synchronized(FileInputStreamCache.this) {
+        if (closed) return;
+        long curTime = Time.monotonicNow();
+        for (Iterator<Entry<Key, Value>> iter = map.entries().iterator();
+              iter.hasNext();
+              iter = map.entries().iterator()) {
+          Entry<Key, Value> entry = iter.next();
+          if (entry.getValue().getTime() + expiryTimeMs >= curTime) {
+            break;
+          }
+          entry.getValue().close();
+          iter.remove();
+        }
+      }
+    }
+  }
+
+  /**
+   * The key identifying a FileInputStream array.
+   */
+  static class Key {
+    private final DatanodeID datanodeID;
+    private final ExtendedBlock block;
+    
+    public Key(DatanodeID datanodeID, ExtendedBlock block) {
+      this.datanodeID = datanodeID;
+      this.block = block;
+    }
+    
+    @Override
+    public boolean equals(Object other) {
+      if (!(other instanceof FileInputStreamCache.Key)) {
+        return false;
+      }
+      FileInputStreamCache.Key otherKey = (FileInputStreamCache.Key)other;
+      return (block.equals(otherKey.block) & 
+          (block.getGenerationStamp() == otherKey.block.getGenerationStamp()) &
+          datanodeID.equals(otherKey.datanodeID));
+    }
+
+    @Override
+    public int hashCode() {
+      return block.hashCode();
+    }
+  }
+
+  /**
+   * The value containing a FileInputStream array and the time it was added to
+   * the cache.
+   */
+  static class Value {
+    private final FileInputStream fis[];
+    private final long time;
+    
+    public Value (FileInputStream fis[]) {
+      this.fis = fis;
+      this.time = Time.monotonicNow();
+    }
+
+    public FileInputStream[] getFileInputStreams() {
+      return fis;
+    }
+
+    public long getTime() {
+      return time;
+    }
+    
+    public void close() {
+      IOUtils.cleanup(LOG, fis);
+    }
+  }
+  
+  /**
+   * Create a new FileInputStream
+   *
+   * @param maxCacheSize         The maximum number of elements to allow in 
+   *                             the cache.
+   * @param expiryTimeMs         The minimum time in milliseconds to preserve
+   *                             elements in the cache.
+   */
+  public FileInputStreamCache(int maxCacheSize, long expiryTimeMs) {
+    this.maxCacheSize = maxCacheSize;
+    this.expiryTimeMs = expiryTimeMs;
+  }
+  
+  /**
+   * Put an array of FileInputStream objects into the cache.
+   *
+   * @param datanodeID          The DatanodeID to store the streams under.
+   * @param block               The Block to store the streams under.
+   * @param fis                 The streams.
+   */
+  public void put(DatanodeID datanodeID, ExtendedBlock block,
+      FileInputStream fis[]) {
+    boolean inserted = false;
+    try {
+      synchronized(this) {
+        if (closed) return;
+        if (map.size() + 1 > maxCacheSize) {
+          Iterator<Entry<Key, Value>> iter = map.entries().iterator();
+          if (!iter.hasNext()) return;
+          Entry<Key, Value> entry = iter.next();
+          entry.getValue().close();
+          iter.remove();
+        }
+        if (cacheCleaner == null) {
+          cacheCleaner = new CacheCleaner();
+          executor.scheduleAtFixedRate(cacheCleaner, expiryTimeMs, expiryTimeMs, 
+              TimeUnit.MILLISECONDS);
+        }
+        map.put(new Key(datanodeID, block), new Value(fis));
+        inserted = true;
+      }
+    } finally {
+      if (!inserted) {
+        IOUtils.cleanup(LOG, fis);
+      }
+    }
+  }
+  
+  /**
+   * Find and remove an array of FileInputStream objects from the cache.
+   *
+   * @param datanodeID          The DatanodeID to search for.
+   * @param block               The Block to search for.
+   *
+   * @return                    null if no streams can be found; the
+   *                            array otherwise.  If this is non-null, the
+   *                            array will have been removed from the cache.
+   */
+  public synchronized FileInputStream[] get(DatanodeID datanodeID,
+      ExtendedBlock block) {
+    Key key = new Key(datanodeID, block);
+    List<Value> ret = map.get(key);
+    if (ret.isEmpty()) return null;
+    Value val = ret.get(0);
+    map.remove(key, val);
+    return val.getFileInputStreams();
+  }
+  
+  /**
+   * Close the cache and free all associated resources.
+   */
+  public synchronized void close() {
+    if (closed) return;
+    closed = true;
+    if (cacheCleaner != null) {
+      executor.remove(cacheCleaner);
+    }
+    for (Iterator<Entry<Key, Value>> iter = map.entries().iterator();
+          iter.hasNext();
+          iter = map.entries().iterator()) {
+      Entry<Key, Value> entry = iter.next();
+      entry.getValue().close();
+      iter.remove();
+    }
+  }
+  
+  public synchronized String toString() {
+    StringBuilder bld = new StringBuilder();
+    bld.append("FileInputStreamCache(");
+    String prefix = "";
+    for (Entry<Key, Value> entry : map.entries()) {
+      bld.append(prefix);
+      bld.append(entry.getKey());
+      prefix = ", ";
+    }
+    bld.append(")");
+    return bld.toString();
+  }
+  
+  public long getExpiryTimeMs() {
+    return expiryTimeMs;
+  }
+  
+  public int getMaxCacheSize() {
+    return maxCacheSize;
+  }
+}

+ 112 - 90
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/SocketCache.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java

@@ -18,92 +18,104 @@
 
 package org.apache.hadoop.hdfs;
 
-import java.io.Closeable;
-import java.net.Socket;
-import java.net.SocketAddress;
-
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 
-import java.io.IOException;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.LinkedListMultimap;
 import org.apache.commons.logging.Log;
-import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 
 /**
  * A cache of input stream sockets to Data Node.
  */
-class SocketCache {
-  private static final Log LOG = LogFactory.getLog(SocketCache.class);
-
-  @InterfaceAudience.Private
-  static class SocketAndStreams implements Closeable {
-    public final Socket sock;
-    public final IOStreamPair ioStreams;
-    long createTime;
+class PeerCache {
+  private static final Log LOG = LogFactory.getLog(PeerCache.class);
+  
+  private static class Key {
+    final DatanodeID dnID;
+    final boolean isDomain;
     
-    public SocketAndStreams(Socket s, IOStreamPair ioStreams) {
-      this.sock = s;
-      this.ioStreams = ioStreams;
-      this.createTime = Time.monotonicNow();
+    Key(DatanodeID dnID, boolean isDomain) {
+      this.dnID = dnID;
+      this.isDomain = isDomain;
     }
     
     @Override
-    public void close() {
-      if (ioStreams != null) { 
-        IOUtils.closeStream(ioStreams.in);
-        IOUtils.closeStream(ioStreams.out);
+    public boolean equals(Object o) {
+      if (!(o instanceof Key)) {
+        return false;
       }
-      IOUtils.closeSocket(sock);
+      Key other = (Key)o;
+      return dnID.equals(other.dnID) && isDomain == other.isDomain;
+    }
+
+    @Override
+    public int hashCode() {
+      return dnID.hashCode() ^ (isDomain ? 1 : 0);
+    }
+  }
+  
+  private static class Value {
+    private final Peer peer;
+    private final long time;
+
+    Value(Peer peer, long time) {
+      this.peer = peer;
+      this.time = time;
+    }
+
+    Peer getPeer() {
+      return peer;
     }
 
-    public long getCreateTime() {
-      return this.createTime;
+    long getTime() {
+      return time;
     }
   }
 
   private Daemon daemon;
   /** A map for per user per datanode. */
-  private static LinkedListMultimap<SocketAddress, SocketAndStreams> multimap =
+  private final LinkedListMultimap<Key, Value> multimap =
     LinkedListMultimap.create();
-  private static int capacity;
-  private static long expiryPeriod;
-  private static SocketCache scInstance = new SocketCache();
-  private static boolean isInitedOnce = false;
+  private final int capacity;
+  private final long expiryPeriod;
+  private static PeerCache instance = null;
+  
+  @VisibleForTesting
+  PeerCache(int c, long e) {
+    this.capacity = c;
+    this.expiryPeriod = e;
+
+    if (capacity == 0 ) {
+      LOG.info("SocketCache disabled.");
+    }
+    else if (expiryPeriod == 0) {
+      throw new IllegalStateException("Cannot initialize expiryPeriod to " +
+         expiryPeriod + "when cache is enabled.");
+    }
+  }
  
-  public static synchronized SocketCache getInstance(int c, long e) {
+  public static synchronized PeerCache getInstance(int c, long e) {
     // capacity is only initialized once
-    if (isInitedOnce == false) {
-      capacity = c;
-      expiryPeriod = e;
-
-      if (capacity == 0 ) {
-        LOG.info("SocketCache disabled.");
-      }
-      else if (expiryPeriod == 0) {
-        throw new IllegalStateException("Cannot initialize expiryPeriod to " +
-           expiryPeriod + "when cache is enabled.");
-      }
-      isInitedOnce = true;
+    if (instance == null) {
+      instance = new PeerCache(c, e);
     } else { //already initialized once
-      if (capacity != c || expiryPeriod != e) {
-        LOG.info("capacity and expiry periods already set to " + capacity + 
-          " and " + expiryPeriod + " respectively. Cannot set it to " + c + 
-          " and " + e);
+      if (instance.capacity != c || instance.expiryPeriod != e) {
+        LOG.info("capacity and expiry periods already set to " +
+          instance.capacity + " and " + instance.expiryPeriod +
+          " respectively. Cannot set it to " + c + " and " + e);
       }
     }
 
-    return scInstance;
+    return instance;
   }
 
   private boolean isDaemonStarted() {
@@ -120,44 +132,47 @@ class SocketCache {
       @Override
       public void run() {
         try {
-          SocketCache.this.run();
+          PeerCache.this.run();
         } catch(InterruptedException e) {
           //noop
         } finally {
-          SocketCache.this.clear();
+          PeerCache.this.clear();
         }
       }
 
       @Override
       public String toString() {
-        return String.valueOf(SocketCache.this);
+        return String.valueOf(PeerCache.this);
       }
     });
     daemon.start();
   }
 
   /**
-   * Get a cached socket to the given address.
-   * @param remote  Remote address the socket is connected to.
-   * @return  A socket with unknown state, possibly closed underneath. Or null.
+   * Get a cached peer connected to the given DataNode.
+   * @param dnId         The DataNode to get a Peer for.
+   * @param isDomain     Whether to retrieve a DomainPeer or not.
+   *
+   * @return             An open Peer connected to the DN, or null if none
+   *                     was found. 
    */
-  public synchronized SocketAndStreams get(SocketAddress remote) {
+  public synchronized Peer get(DatanodeID dnId, boolean isDomain) {
 
     if (capacity <= 0) { // disabled
       return null;
     }
 
-    List<SocketAndStreams> sockStreamList = multimap.get(remote);
+    List<Value> sockStreamList = multimap.get(new Key(dnId, isDomain));
     if (sockStreamList == null) {
       return null;
     }
 
-    Iterator<SocketAndStreams> iter = sockStreamList.iterator();
+    Iterator<Value> iter = sockStreamList.iterator();
     while (iter.hasNext()) {
-      SocketAndStreams candidate = iter.next();
+      Value candidate = iter.next();
       iter.remove();
-      if (!candidate.sock.isClosed()) {
-        return candidate;
+      if (!candidate.getPeer().isClosed()) {
+        return candidate.getPeer();
       }
     }
     return null;
@@ -167,30 +182,23 @@ class SocketCache {
    * Give an unused socket to the cache.
    * @param sock socket not used by anyone.
    */
-  public synchronized void put(Socket sock, IOStreamPair ioStreams) {
-
-    Preconditions.checkNotNull(sock);
-    SocketAndStreams s = new SocketAndStreams(sock, ioStreams);
+  public synchronized void put(DatanodeID dnId, Peer peer) {
+    Preconditions.checkNotNull(dnId);
+    Preconditions.checkNotNull(peer);
+    if (peer.isClosed()) return;
     if (capacity <= 0) {
       // Cache disabled.
-      s.close();
+      IOUtils.cleanup(LOG, peer);
       return;
     }
  
     startExpiryDaemon();
 
-    SocketAddress remoteAddr = sock.getRemoteSocketAddress();
-    if (remoteAddr == null) {
-      LOG.warn("Cannot cache (unconnected) socket with no remote address: " +
-               sock);
-      IOUtils.closeSocket(sock);
-      return;
-    }
-
     if (capacity == multimap.size()) {
       evictOldest();
     }
-    multimap.put(remoteAddr, s);
+    multimap.put(new Key(dnId, peer.getDomainSocket() != null),
+        new Value(peer, Time.monotonicNow()));
   }
 
   public synchronized int size() {
@@ -202,18 +210,17 @@ class SocketCache {
    */
   private synchronized void evictExpired(long expiryPeriod) {
     while (multimap.size() != 0) {
-      Iterator<Entry<SocketAddress, SocketAndStreams>> iter =
+      Iterator<Entry<Key, Value>> iter =
         multimap.entries().iterator();
-      Entry<SocketAddress, SocketAndStreams> entry = iter.next();
+      Entry<Key, Value> entry = iter.next();
       // if oldest socket expired, remove it
       if (entry == null || 
-        Time.monotonicNow() - entry.getValue().getCreateTime() < 
+        Time.monotonicNow() - entry.getValue().getTime() <
         expiryPeriod) {
         break;
       }
+      IOUtils.cleanup(LOG, entry.getValue().getPeer());
       iter.remove();
-      SocketAndStreams s = entry.getValue();
-      s.close();
     }
   }
 
@@ -221,16 +228,18 @@ class SocketCache {
    * Evict the oldest entry in the cache.
    */
   private synchronized void evictOldest() {
-    Iterator<Entry<SocketAddress, SocketAndStreams>> iter =
+    // We can get the oldest element immediately, because of an interesting
+    // property of LinkedListMultimap: its iterator traverses entries in the
+    // order that they were added.
+    Iterator<Entry<Key, Value>> iter =
       multimap.entries().iterator();
     if (!iter.hasNext()) {
       throw new IllegalStateException("Cannot evict from empty cache! " +
         "capacity: " + capacity);
     }
-    Entry<SocketAddress, SocketAndStreams> entry = iter.next();
+    Entry<Key, Value> entry = iter.next();
+    IOUtils.cleanup(LOG, entry.getValue().getPeer());
     iter.remove();
-    SocketAndStreams s = entry.getValue();
-    s.close();
   }
 
   /**
@@ -255,11 +264,24 @@ class SocketCache {
    * Empty the cache, and close all sockets.
    */
   @VisibleForTesting
-  protected synchronized void clear() {
-    for (SocketAndStreams sockAndStream : multimap.values()) {
-      sockAndStream.close();
+  synchronized void clear() {
+    for (Value value : multimap.values()) {
+      IOUtils.cleanup(LOG, value.getPeer());
     }
     multimap.clear();
   }
-
+  
+  @VisibleForTesting
+  void close() {
+    clear();
+    if (daemon != null) {
+      daemon.interrupt();
+      try {
+        daemon.join();
+      } catch (InterruptedException e) {
+        throw new RuntimeException("failed to join thread");
+      }
+    }
+    daemon = null;
+  }
 }

+ 34 - 50
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java

@@ -29,6 +29,8 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSInputChecker;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
@@ -54,8 +56,8 @@ import org.apache.hadoop.util.DataChecksum;
 @InterfaceAudience.Private
 @Deprecated
 public class RemoteBlockReader extends FSInputChecker implements BlockReader {
-
-  Socket dnSock; //for now just sending the status code (e.g. checksumOk) after the read.
+  private final Peer peer;
+  private final DatanodeID datanodeID;
   private final DataInputStream in;
   private DataChecksum checksum;
 
@@ -125,9 +127,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     // if eos was set in the previous read, send a status code to the DN
     if (eos && !eosBefore && nRead >= 0) {
       if (needChecksum()) {
-        sendReadResult(dnSock, Status.CHECKSUM_OK);
+        sendReadResult(peer, Status.CHECKSUM_OK);
       } else {
-        sendReadResult(dnSock, Status.SUCCESS);
+        sendReadResult(peer, Status.SUCCESS);
       }
     }
     return nRead;
@@ -321,7 +323,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   
   private RemoteBlockReader(String file, String bpid, long blockId,
       DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
-      long startOffset, long firstChunkOffset, long bytesToRead, Socket dnSock) {
+      long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
+      DatanodeID datanodeID) {
     // Path is used only for printing block and file information in debug
     super(new Path("/blk_" + blockId + ":" + bpid + ":of:"+ file)/*too non path-like?*/,
           1, verifyChecksum,
@@ -329,7 +332,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
           checksum.getBytesPerChecksum(),
           checksum.getChecksumSize());
     
-    this.dnSock = dnSock;
+    this.peer = peer;
+    this.datanodeID = datanodeID;
     this.in = in;
     this.checksum = checksum;
     this.startOffset = Math.max( startOffset, 0 );
@@ -348,13 +352,6 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     checksumSize = this.checksum.getChecksumSize();
   }
 
-  public static RemoteBlockReader newBlockReader(Socket sock, String file,
-      ExtendedBlock block, Token<BlockTokenIdentifier> blockToken, 
-      long startOffset, long len, int bufferSize) throws IOException {
-    return newBlockReader(sock, file, block, blockToken, startOffset,
-        len, bufferSize, true, "");
-  }
-
   /**
    * Create a new BlockReader specifically to satisfy a read.
    * This method also sends the OP_READ_BLOCK request.
@@ -370,16 +367,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
    * @param clientName  Client name
    * @return New BlockReader instance, or null on error.
    */
-  public static RemoteBlockReader newBlockReader( Socket sock, String file,
+  public static RemoteBlockReader newBlockReader(String file,
                                      ExtendedBlock block, 
                                      Token<BlockTokenIdentifier> blockToken,
                                      long startOffset, long len,
                                      int bufferSize, boolean verifyChecksum,
-                                     String clientName)
+                                     String clientName, Peer peer,
+                                     DatanodeID datanodeID)
                                      throws IOException {
     // in and out will be closed when sock is closed (by the caller)
-    final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-          NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT)));
+    final DataOutputStream out =
+        new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
         verifyChecksum);
     
@@ -388,12 +386,11 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     //
 
     DataInputStream in = new DataInputStream(
-        new BufferedInputStream(NetUtils.getInputStream(sock), 
-                                bufferSize));
+        new BufferedInputStream(peer.getInputStream(), bufferSize));
     
     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
-    RemoteBlockReader2.checkSuccess(status, sock, block, file);
+    RemoteBlockReader2.checkSuccess(status, peer, block, file);
     ReadOpChecksumInfoProto checksumInfo =
       status.getReadOpChecksumInfo();
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
@@ -411,15 +408,19 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     }
 
     return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
-        in, checksum, verifyChecksum, startOffset, firstChunkOffset, len, sock);
+        in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
+        peer, datanodeID);
   }
 
   @Override
-  public synchronized void close() throws IOException {
+  public synchronized void close(PeerCache peerCache,
+      FileInputStreamCache fisCache) throws IOException {
     startOffset = -1;
     checksum = null;
-    if (dnSock != null) {
-      dnSock.close();
+    if (peerCache != null & sentStatusCode) {
+      peerCache.put(datanodeID, peer);
+    } else {
+      peer.close();
     }
 
     // in will be closed when its Socket is closed.
@@ -436,37 +437,21 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     return readFully(this, buf, offset, len);
   }
 
-  @Override
-  public Socket takeSocket() {
-    assert hasSentStatusCode() :
-      "BlockReader shouldn't give back sockets mid-read";
-    Socket res = dnSock;
-    dnSock = null;
-    return res;
-  }
-
-  @Override
-  public boolean hasSentStatusCode() {
-    return sentStatusCode;
-  }
-
   /**
    * When the reader reaches end of the read, it sends a status response
    * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
    * closing our connection (which we will re-open), but won't affect
    * data correctness.
    */
-  void sendReadResult(Socket sock, Status statusCode) {
-    assert !sentStatusCode : "already sent status code to " + sock;
+  void sendReadResult(Peer peer, Status statusCode) {
+    assert !sentStatusCode : "already sent status code to " + peer;
     try {
-      RemoteBlockReader2.writeReadResult(
-          NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT),
-          statusCode);
+      RemoteBlockReader2.writeReadResult(peer.getOutputStream(), statusCode);
       sentStatusCode = true;
     } catch (IOException e) {
       // It's ok not to be able to send this. But something is probably wrong.
       LOG.info("Could not send read status (" + statusCode + ") to datanode " +
-               sock.getInetAddress() + ": " + e.getMessage());
+               peer.getRemoteAddressString() + ": " + e.getMessage());
     }
   }
   
@@ -486,12 +471,11 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   public int read(ByteBuffer buf) throws IOException {
     throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
   }
-
+  
   @Override
-  public IOStreamPair getStreams() {
-    // This class doesn't support encryption, which is the only thing this
-    // method is used for. See HDFS-3637.
-    return null;
+  public int available() throws IOException {
+    // An optimistic estimate of how much data is available
+    // to us without doing network I/O.
+    return DFSClient.TCP_WINDOW_SIZE;
   }
-
 }

+ 46 - 66
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java

@@ -23,16 +23,16 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.InetSocketAddress;
-import java.net.Socket;
 import java.nio.ByteBuffer;
 import java.nio.channels.ReadableByteChannel;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -44,10 +44,11 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.net.SocketInputWrapper;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This is a wrapper around connection to datanode
  * and understands checksum, offset etc.
@@ -79,9 +80,8 @@ public class RemoteBlockReader2  implements BlockReader {
 
   static final Log LOG = LogFactory.getLog(RemoteBlockReader2.class);
   
-  Socket dnSock;
-  // for now just sending the status code (e.g. checksumOk) after the read.
-  private IOStreamPair ioStreams;
+  final private Peer peer;
+  final private DatanodeID datanodeID;
   private final ReadableByteChannel in;
   private DataChecksum checksum;
   
@@ -114,6 +114,11 @@ public class RemoteBlockReader2  implements BlockReader {
   /** Amount of unread data in the current received packet */
   int dataLeft = 0;
   
+  @VisibleForTesting
+  public Peer getPeer() {
+    return peer;
+  }
+  
   @Override
   public synchronized int read(byte[] buf, int off, int len) 
                                throws IOException {
@@ -246,13 +251,13 @@ public class RemoteBlockReader2  implements BlockReader {
   }
 
   protected RemoteBlockReader2(String file, String bpid, long blockId,
-      ReadableByteChannel in, DataChecksum checksum, boolean verifyChecksum,
-      long startOffset, long firstChunkOffset, long bytesToRead, Socket dnSock,
-      IOStreamPair ioStreams) {
+      DataChecksum checksum, boolean verifyChecksum,
+      long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
+      DatanodeID datanodeID) {
     // Path is used only for printing block and file information in debug
-    this.dnSock = dnSock;
-    this.ioStreams = ioStreams;
-    this.in = in;
+    this.peer = peer;
+    this.datanodeID = datanodeID;
+    this.in = peer.getInputStreamChannel();
     this.checksum = checksum;
     this.verifyChecksum = verifyChecksum;
     this.startOffset = Math.max( startOffset, 0 );
@@ -269,39 +274,20 @@ public class RemoteBlockReader2  implements BlockReader {
 
 
   @Override
-  public synchronized void close() throws IOException {
+  public synchronized void close(PeerCache peerCache,
+      FileInputStreamCache fisCache) throws IOException {
     packetReceiver.close();
-    
     startOffset = -1;
     checksum = null;
-    if (dnSock != null) {
-      dnSock.close();
+    if (peerCache != null && sentStatusCode) {
+      peerCache.put(datanodeID, peer);
+    } else {
+      peer.close();
     }
 
     // in will be closed when its Socket is closed.
   }
   
-  /**
-   * Take the socket used to talk to the DN.
-   */
-  @Override
-  public Socket takeSocket() {
-    assert hasSentStatusCode() :
-      "BlockReader shouldn't give back sockets mid-read";
-    Socket res = dnSock;
-    dnSock = null;
-    return res;
-  }
-
-  /**
-   * Whether the BlockReader has reached the end of its input stream
-   * and successfully sent a status code back to the datanode.
-   */
-  @Override
-  public boolean hasSentStatusCode() {
-    return sentStatusCode;
-  }
-
   /**
    * When the reader reaches end of the read, it sends a status response
    * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
@@ -309,14 +295,14 @@ public class RemoteBlockReader2  implements BlockReader {
    * data correctness.
    */
   void sendReadResult(Status statusCode) {
-    assert !sentStatusCode : "already sent status code to " + dnSock;
+    assert !sentStatusCode : "already sent status code to " + peer;
     try {
-      writeReadResult(ioStreams.out, statusCode);
+      writeReadResult(peer.getOutputStream(), statusCode);
       sentStatusCode = true;
     } catch (IOException e) {
       // It's ok not to be able to send this. But something is probably wrong.
       LOG.info("Could not send read status (" + statusCode + ") to datanode " +
-               dnSock.getInetAddress() + ": " + e.getMessage());
+               peer.getRemoteAddressString() + ": " + e.getMessage());
     }
   }
 
@@ -367,42 +353,34 @@ public class RemoteBlockReader2  implements BlockReader {
    * @param blockToken  The block token for security
    * @param startOffset  The read offset, relative to block head
    * @param len  The number of bytes to read
-   * @param bufferSize  The IO buffer size (not the client buffer size)
    * @param verifyChecksum  Whether to verify checksum
    * @param clientName  Client name
+   * @param peer  The Peer to use
+   * @param datanodeID  The DatanodeID this peer is connected to
    * @return New BlockReader instance, or null on error.
    */
-  public static BlockReader newBlockReader(Socket sock, String file,
+  public static BlockReader newBlockReader(String file,
                                      ExtendedBlock block,
                                      Token<BlockTokenIdentifier> blockToken,
                                      long startOffset, long len,
-                                     int bufferSize, boolean verifyChecksum,
+                                     boolean verifyChecksum,
                                      String clientName,
-                                     DataEncryptionKey encryptionKey,
-                                     IOStreamPair ioStreams)
+                                     Peer peer, DatanodeID datanodeID)
                                      throws IOException {
-    
-    ReadableByteChannel ch;
-    if (ioStreams.in instanceof SocketInputWrapper) {
-      ch = ((SocketInputWrapper)ioStreams.in).getReadableByteChannel();
-    } else {
-      ch = (ReadableByteChannel) ioStreams.in;
-    }
-    
     // in and out will be closed when sock is closed (by the caller)
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-          ioStreams.out));
+          peer.getOutputStream()));
     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
         verifyChecksum);
 
     //
     // Get bytes in block
     //
-    DataInputStream in = new DataInputStream(ioStreams.in);
+    DataInputStream in = new DataInputStream(peer.getInputStream());
 
     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
-    checkSuccess(status, sock, block, file);
+    checkSuccess(status, peer, block, file);
     ReadOpChecksumInfoProto checksumInfo =
       status.getReadOpChecksumInfo();
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
@@ -420,34 +398,36 @@ public class RemoteBlockReader2  implements BlockReader {
     }
 
     return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
-        ch, checksum, verifyChecksum, startOffset, firstChunkOffset, len, sock,
-        ioStreams);
+        checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
+        datanodeID);
   }
 
   static void checkSuccess(
-      BlockOpResponseProto status, Socket sock,
+      BlockOpResponseProto status, Peer peer,
       ExtendedBlock block, String file)
       throws IOException {
     if (status.getStatus() != Status.SUCCESS) {
       if (status.getStatus() == Status.ERROR_ACCESS_TOKEN) {
         throw new InvalidBlockTokenException(
             "Got access token error for OP_READ_BLOCK, self="
-                + sock.getLocalSocketAddress() + ", remote="
-                + sock.getRemoteSocketAddress() + ", for file " + file
+                + peer.getLocalAddressString() + ", remote="
+                + peer.getRemoteAddressString() + ", for file " + file
                 + ", for pool " + block.getBlockPoolId() + " block " 
                 + block.getBlockId() + "_" + block.getGenerationStamp());
       } else {
         throw new IOException("Got error for OP_READ_BLOCK, self="
-            + sock.getLocalSocketAddress() + ", remote="
-            + sock.getRemoteSocketAddress() + ", for file " + file
+            + peer.getLocalAddressString() + ", remote="
+            + peer.getRemoteAddressString() + ", for file " + file
             + ", for pool " + block.getBlockPoolId() + " block " 
             + block.getBlockId() + "_" + block.getGenerationStamp());
       }
     }
   }
-
+  
   @Override
-  public IOStreamPair getStreams() {
-    return ioStreams;
+  public int available() throws IOException {
+    // An optimistic estimate of how much data is available
+    // to us without doing network I/O.
+    return DFSClient.TCP_WINDOW_SIZE;
   }
 }

+ 128 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java

@@ -0,0 +1,128 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.nio.channels.ReadableByteChannel;
+
+import org.apache.hadoop.net.unix.DomainSocket;
+
+/**
+ * Represents a peer that we communicate with by using a basic Socket
+ * that has no associated Channel.
+ *
+ */
+class BasicInetPeer implements Peer {
+  private final Socket socket;
+  private final OutputStream out;
+  private final InputStream in;
+  private final boolean isLocal;
+
+  public BasicInetPeer(Socket socket) throws IOException {
+    this.socket = socket;
+    this.out = socket.getOutputStream();
+    this.in = socket.getInputStream();
+    this.isLocal = socket.getInetAddress().equals(socket.getLocalAddress());
+  }
+
+  @Override
+  public ReadableByteChannel getInputStreamChannel() {
+    /*
+     * This Socket has no channel, so there's nothing to return here.
+     */
+    return null;
+  }
+
+  @Override
+  public void setReadTimeout(int timeoutMs) throws IOException {
+    socket.setSoTimeout(timeoutMs);
+  }
+
+  @Override
+  public int getReceiveBufferSize() throws IOException {
+    return socket.getReceiveBufferSize();
+  }
+
+  @Override
+  public boolean getTcpNoDelay() throws IOException {
+    return socket.getTcpNoDelay();
+  }
+
+  @Override
+  public void setWriteTimeout(int timeoutMs) {
+   /* 
+    * We can't implement write timeouts. :(
+    * 
+    * Java provides no facility to set a blocking write timeout on a Socket.
+    * You can simulate a blocking write with a timeout by using
+    * non-blocking I/O.  However, we can't use nio here, because this Socket
+    * doesn't have an associated Channel.
+    * 
+    * See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4031100 for
+    * more details.
+    */
+  }
+
+  @Override
+  public boolean isClosed() {
+    return socket.isClosed();
+  }
+
+  @Override
+  public void close() throws IOException {
+    socket.close();
+  }
+
+  @Override
+  public String getRemoteAddressString() {
+    return socket.getRemoteSocketAddress().toString();
+  }
+
+  @Override
+  public String getLocalAddressString() {
+    return socket.getLocalSocketAddress().toString();
+  }
+  
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return in;
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return out;
+  }
+
+  @Override
+  public boolean isLocal() {
+    return isLocal;
+  }
+
+  @Override
+  public String toString() {
+    return "BasicInetPeer(" + socket.toString() + ")";
+  }
+
+  @Override
+  public DomainSocket getDomainSocket() {
+    return null;
+  }
+}

+ 117 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DomainPeer.java

@@ -0,0 +1,117 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.channels.ReadableByteChannel;
+
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Represents a peer that we communicate with by using blocking I/O 
+ * on a UNIX domain socket.
+ */
+@InterfaceAudience.Private
+public class DomainPeer implements Peer {
+  private final DomainSocket socket;
+  private final OutputStream out;
+  private final InputStream in;
+  private final ReadableByteChannel channel;
+
+  public DomainPeer(DomainSocket socket) {
+    this.socket = socket;
+    this.out = socket.getOutputStream();
+    this.in = socket.getInputStream();
+    this.channel = socket.getChannel();
+  }
+
+  @Override
+  public ReadableByteChannel getInputStreamChannel() {
+    return channel;
+  }
+
+  @Override
+  public void setReadTimeout(int timeoutMs) throws IOException {
+    socket.setAttribute(DomainSocket.RECEIVE_TIMEOUT, timeoutMs);
+  }
+
+  @Override
+  public int getReceiveBufferSize() throws IOException {
+    return socket.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE);
+  }
+
+  @Override
+  public boolean getTcpNoDelay() throws IOException {
+    /* No TCP, no TCP_NODELAY. */
+    return false;
+  }
+
+  @Override
+  public void setWriteTimeout(int timeoutMs) throws IOException {
+    socket.setAttribute(DomainSocket.SEND_TIMEOUT, timeoutMs);
+  }
+
+  @Override
+  public boolean isClosed() {
+    return !socket.isOpen();
+  }
+
+  @Override
+  public void close() throws IOException {
+    socket.close();
+  }
+
+  @Override
+  public String getRemoteAddressString() {
+    return "unix:" + socket.getPath();
+  }
+
+  @Override
+  public String getLocalAddressString() {
+    return "<local>";
+  }
+
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return in;
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return out;
+  }
+
+  @Override
+  public boolean isLocal() {
+    /* UNIX domain sockets can only be used for local communication. */
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "DomainPeer(" + getRemoteAddressString() + ")";
+  }
+
+  @Override
+  public DomainSocket getDomainSocket() {
+    return socket;
+  }
+}

+ 89 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DomainPeerServer.java

@@ -0,0 +1,89 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.net.PeerServer;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class DomainPeerServer implements PeerServer {
+  static Log LOG = LogFactory.getLog(DomainPeerServer.class);
+  private final DomainSocket sock;
+
+  DomainPeerServer(DomainSocket sock) {
+    this.sock = sock;
+  }
+
+  public DomainPeerServer(String path, int port) 
+      throws IOException {
+    this(DomainSocket.bindAndListen(DomainSocket.getEffectivePath(path, port)));
+  }
+  
+  public String getBindPath() {
+    return sock.getPath();
+  }
+
+  @Override
+  public void setReceiveBufferSize(int size) throws IOException {
+    sock.setAttribute(DomainSocket.RECEIVE_BUFFER_SIZE, size);
+  }
+
+  @Override
+  public Peer accept() throws IOException, SocketTimeoutException {
+    DomainSocket connSock = sock.accept();
+    Peer peer = null;
+    boolean success = false;
+    try {
+      peer = new DomainPeer(connSock);
+      success = true;
+      return peer;
+    } finally {
+      if (!success) {
+        if (peer != null) peer.close();
+        connSock.close();
+      }
+    }
+  }
+
+  @Override
+  public String getListeningString() {
+    return "unix:" + sock.getPath();
+  }
+  
+  @Override
+  public void close() throws IOException {
+    try {
+      sock.close();
+    } catch (IOException e) {
+      LOG.error("error closing DomainPeerServer: ", e);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "DomainPeerServer(" + getListeningString() + ")";
+  }
+}

+ 142 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java

@@ -0,0 +1,142 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.net.unix.DomainSocket;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.channels.ReadableByteChannel;
+
+/**
+ * Represents a peer that we communicate with by using an encrypted
+ * communications medium.
+ */
+@InterfaceAudience.Private
+public class EncryptedPeer implements Peer {
+  private final Peer enclosedPeer;
+
+  /**
+   * An encrypted InputStream.
+   */
+  private final InputStream in;
+  
+  /**
+   * An encrypted OutputStream.
+   */
+  private final OutputStream out;
+  
+  /**
+   * An encrypted ReadableByteChannel.
+   */
+  private final ReadableByteChannel channel;
+
+  public EncryptedPeer(Peer enclosedPeer, DataEncryptionKey key)
+      throws IOException {
+    this.enclosedPeer = enclosedPeer;
+    IOStreamPair ios = DataTransferEncryptor.getEncryptedStreams(
+        enclosedPeer.getOutputStream(), enclosedPeer.getInputStream(), key);
+    this.in = ios.in;
+    this.out = ios.out;
+    this.channel = ios.in instanceof ReadableByteChannel ? 
+        (ReadableByteChannel)ios.in : null;
+  }
+
+  @Override
+  public ReadableByteChannel getInputStreamChannel() {
+    return channel;
+  }
+
+  @Override
+  public void setReadTimeout(int timeoutMs) throws IOException {
+    enclosedPeer.setReadTimeout(timeoutMs);
+  }
+
+  @Override
+  public int getReceiveBufferSize() throws IOException {
+    return enclosedPeer.getReceiveBufferSize();
+  }
+
+  @Override
+  public boolean getTcpNoDelay() throws IOException {
+    return enclosedPeer.getTcpNoDelay();
+  }
+
+  @Override
+  public void setWriteTimeout(int timeoutMs) throws IOException {
+    enclosedPeer.setWriteTimeout(timeoutMs);
+  }
+
+  @Override
+  public boolean isClosed() {
+    return enclosedPeer.isClosed();
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      in.close();
+    } finally {
+      try {
+        out.close();
+      } finally {
+        enclosedPeer.close();
+      }
+    }
+  }
+
+  @Override
+  public String getRemoteAddressString() {
+    return enclosedPeer.getRemoteAddressString();
+  }
+
+  @Override
+  public String getLocalAddressString() {
+    return enclosedPeer.getLocalAddressString();
+  }
+
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return in;
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return out;
+  }
+
+  @Override
+  public boolean isLocal() {
+    return enclosedPeer.isLocal();
+  }
+
+  @Override
+  public String toString() {
+    return "EncryptedPeer(" + enclosedPeer + ")";
+  }
+
+  @Override
+  public DomainSocket getDomainSocket() {
+    return enclosedPeer.getDomainSocket();
+  }
+}

+ 131 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java

@@ -0,0 +1,131 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.nio.channels.ReadableByteChannel;
+
+import org.apache.hadoop.net.SocketInputStream;
+import org.apache.hadoop.net.SocketOutputStream;
+import org.apache.hadoop.net.unix.DomainSocket;
+
+/**
+ * Represents a peer that we communicate with by using non-blocking I/O 
+ * on a Socket.
+ */
+class NioInetPeer implements Peer {
+  private final Socket socket;
+
+  /**
+   * An InputStream which simulates blocking I/O with timeouts using NIO.
+   */
+  private final SocketInputStream in;
+  
+  /**
+   * An OutputStream which simulates blocking I/O with timeouts using NIO.
+   */
+  private final SocketOutputStream out;
+
+  private final boolean isLocal;
+
+  NioInetPeer(Socket socket) throws IOException {
+    this.socket = socket;
+    this.in = new SocketInputStream(socket.getChannel(), 0);
+    this.out = new SocketOutputStream(socket.getChannel(), 0);
+    this.isLocal = socket.getInetAddress().equals(socket.getLocalAddress());
+  }
+
+  @Override
+  public ReadableByteChannel getInputStreamChannel() {
+    return in;
+  }
+
+  @Override
+  public void setReadTimeout(int timeoutMs) throws IOException {
+    in.setTimeout(timeoutMs);
+  }
+
+  @Override
+  public int getReceiveBufferSize() throws IOException {
+    return socket.getReceiveBufferSize();
+  }
+
+  @Override
+  public boolean getTcpNoDelay() throws IOException {
+    return socket.getTcpNoDelay();
+  }
+
+  @Override
+  public void setWriteTimeout(int timeoutMs) throws IOException {
+    out.setTimeout(timeoutMs);
+  }
+
+  @Override
+  public boolean isClosed() {
+    return socket.isClosed();
+  }
+
+  @Override
+  public void close() throws IOException {
+    // We always close the outermost streams-- in this case, 'in' and 'out'
+    // Closing either one of these will also close the Socket.
+    try {
+      in.close();
+    } finally {
+      out.close();
+    }
+  }
+
+  @Override
+  public String getRemoteAddressString() {
+    return socket.getRemoteSocketAddress().toString();
+  }
+
+  @Override
+  public String getLocalAddressString() {
+    return socket.getLocalSocketAddress().toString();
+  }
+
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return in;
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return out;
+  }
+
+  @Override
+  public boolean isLocal() {
+    return isLocal;
+  }
+
+  @Override
+  public String toString() {
+    return "NioInetPeer(" + socket.toString() + ")";
+  }
+
+  @Override
+  public DomainSocket getDomainSocket() {
+    return null;
+  }
+}

+ 115 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/Peer.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.net;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.channels.ReadableByteChannel;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.net.unix.DomainSocket;
+
+/**
+ * Represents a connection to a peer.
+ */
+@InterfaceAudience.Private
+public interface Peer extends Closeable {
+  /**
+   * @return                The input stream channel associated with this
+   *                        peer, or null if it has none.
+   */
+  public ReadableByteChannel getInputStreamChannel();
+
+  /**
+   * Set the read timeout on this peer.
+   *
+   * @param timeoutMs       The timeout in milliseconds.
+   */
+  public void setReadTimeout(int timeoutMs) throws IOException;
+
+  /**
+   * @return                The receive buffer size.
+   */
+  public int getReceiveBufferSize() throws IOException;
+
+  /**
+   * @return                True if TCP_NODELAY is turned on.
+   */
+  public boolean getTcpNoDelay() throws IOException;
+
+  /**
+   * Set the write timeout on this peer.
+   *
+   * Note: this is not honored for BasicInetPeer.
+   * See {@link BasicSocketPeer#setWriteTimeout} for details.
+   * 
+   * @param timeoutMs       The timeout in milliseconds.
+   */
+  public void setWriteTimeout(int timeoutMs) throws IOException;
+
+  /**
+   * @return                true only if the peer is closed.
+   */
+  public boolean isClosed();
+  
+  /**
+   * Close the peer.
+   *
+   * It's safe to re-close a Peer that is already closed.
+   */
+  public void close() throws IOException;
+
+  /**
+   * @return               A string representing the remote end of our 
+   *                       connection to the peer.
+   */
+  public String getRemoteAddressString();
+
+  /**
+   * @return               A string representing the local end of our 
+   *                       connection to the peer.
+   */
+  public String getLocalAddressString();
+  
+  /**
+   * @return               An InputStream associated with the Peer.
+   *                       This InputStream will be valid until you close
+   *                       this peer with Peer#close.
+   */
+  public InputStream getInputStream() throws IOException;
+  
+  /**
+   * @return               An OutputStream associated with the Peer.
+   *                       This OutputStream will be valid until you close
+   *                       this peer with Peer#close.
+   */
+  public OutputStream getOutputStream() throws IOException;
+
+  /**
+   * @return               True if the peer resides on the same
+   *                       computer as we.
+   */
+  public boolean isLocal();
+
+  /**
+   * @return               The DomainSocket associated with the current
+   *                       peer, or null if there is none.
+   */
+  public DomainSocket getDomainSocket();
+}

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/PeerServer.java

@@ -0,0 +1,60 @@
+/**
+ * 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.net;
+
+import java.io.Closeable;
+import org.apache.hadoop.classification.InterfaceAudience;
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+
+@InterfaceAudience.Private
+public interface PeerServer extends Closeable {
+  /**
+   * Set the receive buffer size of the PeerServer.
+   * 
+   * @param size     The receive buffer size.
+   */
+  public void setReceiveBufferSize(int size) throws IOException;
+
+  /**
+   * Listens for a connection to be made to this server and accepts 
+   * it. The method blocks until a connection is made.
+   *
+   * @exception IOException  if an I/O error occurs when waiting for a
+   *               connection.
+   * @exception SecurityException  if a security manager exists and its  
+   *             <code>checkAccept</code> method doesn't allow the operation.
+   * @exception SocketTimeoutException if a timeout was previously set and
+   *             the timeout has been reached.
+   */
+  public Peer accept() throws IOException, SocketTimeoutException;
+
+  /**
+   * @return                 A string representation of the address we're
+   *                         listening on.
+   */
+  public String getListeningString();
+
+  /**
+   * Free the resources associated with this peer server.
+   * This normally includes sockets, etc.
+   *
+   * @throws IOException     If there is an error closing the PeerServer
+   */
+  public void close() throws IOException;
+}

+ 156 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/TcpPeerServer.java

@@ -0,0 +1,156 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.Server;
+
+@InterfaceAudience.Private
+public class TcpPeerServer implements PeerServer {
+  static Log LOG = LogFactory.getLog(TcpPeerServer.class);
+
+  private final ServerSocket serverSocket;
+
+  public static Peer peerFromSocket(Socket socket)
+      throws IOException {
+    Peer peer = null;
+    boolean success = false;
+    try {
+      // TCP_NODELAY is crucial here because of bad interactions between
+      // Nagle's Algorithm and Delayed ACKs. With connection keepalive
+      // between the client and DN, the conversation looks like:
+      //   1. Client -> DN: Read block X
+      //   2. DN -> Client: data for block X
+      //   3. Client -> DN: Status OK (successful read)
+      //   4. Client -> DN: Read block Y
+      // The fact that step #3 and #4 are both in the client->DN direction
+      // triggers Nagling. If the DN is using delayed ACKs, this results
+      // in a delay of 40ms or more.
+      //
+      // TCP_NODELAY disables nagling and thus avoids this performance
+      // disaster.
+      socket.setTcpNoDelay(true);
+      SocketChannel channel = socket.getChannel();
+      if (channel == null) {
+        peer = new BasicInetPeer(socket);
+      } else {
+        peer = new NioInetPeer(socket);
+      }
+      success = true;
+      return peer;
+    } finally {
+      if (!success) {
+        if (peer != null) peer.close();
+        socket.close();
+      }
+    }
+  }
+
+  public static Peer peerFromSocketAndKey(Socket s,
+        DataEncryptionKey key) throws IOException {
+    Peer peer = null;
+    boolean success = false;
+    try {
+      peer = peerFromSocket(s); 
+      if (key != null) {
+        peer = new EncryptedPeer(peer, key);
+      }
+      success = true;
+      return peer;
+    } finally {
+      if (!success) {
+        IOUtils.cleanup(null, peer);
+      }
+    }
+  }
+
+  /**
+   * Create a non-secure TcpPeerServer.
+   *
+   * @param socketWriteTimeout    The Socket write timeout in ms.
+   * @param bindAddr              The address to bind to.
+   * @throws IOException
+   */
+  public TcpPeerServer(int socketWriteTimeout,
+        InetSocketAddress bindAddr) throws IOException {
+    this.serverSocket = (socketWriteTimeout > 0) ?
+          ServerSocketChannel.open().socket() : new ServerSocket();
+    Server.bind(serverSocket, bindAddr, 0);
+  }
+
+  /**
+   * Create a secure TcpPeerServer.
+   *
+   * @param secureResources   Security resources.
+   */
+  public TcpPeerServer(SecureResources secureResources) {
+    this.serverSocket = secureResources.getStreamingSocket();
+  }
+  
+  /**
+   * @return     the IP address which this TcpPeerServer is listening on.
+   */
+  public InetSocketAddress getStreamingAddr() {
+    return new InetSocketAddress(
+        serverSocket.getInetAddress().getHostAddress(),
+        serverSocket.getLocalPort());
+  }
+
+  @Override
+  public void setReceiveBufferSize(int size) throws IOException {
+    this.serverSocket.setReceiveBufferSize(size);
+  }
+
+  @Override
+  public Peer accept() throws IOException, SocketTimeoutException {
+    Peer peer = peerFromSocket(serverSocket.accept());
+    return peer;
+  }
+
+  @Override
+  public String getListeningString() {
+    return serverSocket.getLocalSocketAddress().toString();
+  }
+  
+  @Override
+  public void close() throws IOException {
+    try {
+      serverSocket.close();
+    } catch(IOException e) {
+      LOG.error("error closing TcpPeerServer: ", e);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "TcpPeerServer(" + getListeningString() + ")";
+  }
+}

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java

@@ -107,6 +107,18 @@ public interface DataTransferProtocol {
       final String clientName,
       final DatanodeInfo[] targets) throws IOException;
 
+  /**
+   * Request short circuit access file descriptors from a DataNode.
+   *
+   * @param blk             The block to get file descriptors for.
+   * @param blockToken      Security token for accessing the block.
+   * @param maxVersion      Maximum version of the block data the client 
+   *                        can understand.
+   */
+  public void requestShortCircuitFds(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      int maxVersion) throws IOException;
+
   /**
    * Receive a block from a source datanode
    * and then notifies the namenode

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

@@ -34,7 +34,8 @@ public enum Op {
   REPLACE_BLOCK((byte)83),
   COPY_BLOCK((byte)84),
   BLOCK_CHECKSUM((byte)85),
-  TRANSFER_BLOCK((byte)86);
+  TRANSFER_BLOCK((byte)86),
+  REQUEST_SHORT_CIRCUIT_FDS((byte)87);
 
   /** The code for this operation. */
   public final byte code;

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 
@@ -76,6 +77,9 @@ public abstract class Receiver implements DataTransferProtocol {
     case TRANSFER_BLOCK:
       opTransferBlock(in);
       break;
+    case REQUEST_SHORT_CIRCUIT_FDS:
+      opRequestShortCircuitFds(in);
+      break;
     default:
       throw new IOException("Unknown op " + op + " in data stream");
     }
@@ -117,6 +121,15 @@ public abstract class Receiver implements DataTransferProtocol {
         PBHelper.convert(proto.getTargetsList()));
   }
 
+  /** Receive {@link Op#REQUEST_SHORT_CIRCUIT_FDS} */
+  private void opRequestShortCircuitFds(DataInputStream in) throws IOException {
+    final OpRequestShortCircuitAccessProto proto =
+      OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in));
+    requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
+        PBHelper.convert(proto.getHeader().getToken()),
+        proto.getMaxVersion());
+  }
+
   /** Receive OP_REPLACE_BLOCK */
   private void opReplaceBlock(DataInputStream in) throws IOException {
     OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -140,6 +141,17 @@ public class Sender implements DataTransferProtocol {
     send(out, Op.TRANSFER_BLOCK, proto);
   }
 
+  @Override
+  public void requestShortCircuitFds(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      int maxVersion) throws IOException {
+    OpRequestShortCircuitAccessProto proto =
+        OpRequestShortCircuitAccessProto.newBuilder()
+          .setHeader(DataTransferProtoUtil.buildBaseHeader(
+            blk, blockToken)).setMaxVersion(maxVersion).build();
+    send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
+  }
+  
   @Override
   public void replaceBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java

@@ -310,6 +310,23 @@ public class DelegationTokenSecretManager
       namesystem.logUpdateMasterKey(key);
     }
   }
+  
+  @Override //AbstractDelegationTokenManager
+  protected void logExpireToken(final DelegationTokenIdentifier dtId)
+      throws IOException {
+    synchronized (noInterruptsLock) {
+      // The edit logging code will fail catastrophically if it
+      // is interrupted during a logSync, since the interrupt
+      // closes the edit log files. Doing this inside the
+      // above lock and then checking interruption status
+      // prevents this bug.
+      if (Thread.interrupted()) {
+        throw new InterruptedIOException(
+            "Interrupted before expiring delegation token");
+      }
+      namesystem.logExpireDelegationToken(dtId);
+    }
+  }
 
   /** A utility method for creating credentials. */
   public static Credentials createCredentials(final NameNode namenode,

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -45,6 +45,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.net.TcpPeerServer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -208,9 +210,12 @@ public class JspHelper {
       // Use the block name for file name. 
     String file = BlockReaderFactory.getFileName(addr, poolId, blockId);
     BlockReader blockReader = BlockReaderFactory.newBlockReader(
-        conf, s, file,
+        conf, file,
         new ExtendedBlock(poolId, blockId, 0, genStamp), blockToken,
-        offsetIntoBlock, amtToRead, encryptionKey);
+        offsetIntoBlock, amtToRead,  true,
+        "JspHelper", TcpPeerServer.peerFromSocketAndKey(s, encryptionKey),
+        new DatanodeID(addr.getAddress().toString(),              
+            addr.getHostName(), poolId, addr.getPort(), 0, 0), null, false);
         
     byte[] buf = new byte[(int)amtToRead];
     int readOffset = 0;
@@ -229,8 +234,7 @@ public class JspHelper {
       amtToRead -= numRead;
       readOffset += numRead;
     }
-    blockReader = null;
-    s.close();
+    blockReader.close(null, null);
     out.print(HtmlQuoting.quoteHtmlChars(new String(buf, Charsets.UTF_8)));
   }
 

+ 147 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -53,19 +53,18 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
-import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.net.UnknownHostException;
 import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.ServerSocketChannel;
 import java.nio.channels.SocketChannel;
 import java.security.PrivilegedExceptionAction;
 import java.util.AbstractList;
@@ -93,6 +92,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.net.DomainPeerServer;
+import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -151,11 +152,11 @@ import org.apache.hadoop.io.ReadaheadPool;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -235,6 +236,7 @@ public class DataNode extends Configured
     LogFactory.getLog(DataNode.class.getName() + ".clienttrace");
   
   private static final String USAGE = "Usage: java DataNode [-rollback | -regular]";
+  static final int CURRENT_BLOCK_FORMAT_VERSION = 1;
 
   /**
    * Use {@link NetUtils#createSocketAddr(String)} instead.
@@ -252,6 +254,7 @@ public class DataNode extends Configured
   public final static String EMPTY_DEL_HINT = "";
   AtomicInteger xmitsInProgress = new AtomicInteger();
   Daemon dataXceiverServer = null;
+  Daemon localDataXceiverServer = null;
   ThreadGroup threadGroup = null;
   private DNConf dnConf;
   private volatile boolean heartbeatsDisabledForTests = false;
@@ -263,6 +266,7 @@ public class DataNode extends Configured
   private String hostName;
   private DatanodeID id;
   
+  final private String fileDescriptorPassingDisabledReason;
   boolean isBlockTokenEnabled;
   BlockPoolTokenSecretManager blockPoolTokenSecretManager;
   private boolean hasAnyBlockPoolRegistered = false;
@@ -311,6 +315,24 @@ public class DataNode extends Configured
     this.getHdfsBlockLocationsEnabled = conf.getBoolean(
         DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, 
         DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
+
+    // Determine whether we should try to pass file descriptors to clients.
+    if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
+              DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT)) {
+      String reason = DomainSocket.getLoadingFailureReason();
+      if (reason != null) {
+        LOG.warn("File descriptor passing is disabled because " + reason);
+        this.fileDescriptorPassingDisabledReason = reason;
+      } else {
+        LOG.info("File descriptor passing is enabled.");
+        this.fileDescriptorPassingDisabledReason = null;
+      }
+    } else {
+      this.fileDescriptorPassingDisabledReason =
+          "File descriptor passing was not configured.";
+      LOG.debug(this.fileDescriptorPassingDisabledReason);
+    }
+
     try {
       hostName = getHostName(conf);
       LOG.info("Configured hostname is " + hostName);
@@ -525,25 +547,63 @@ public class DataNode extends Configured
   
   private void initDataXceiver(Configuration conf) throws IOException {
     // find free port or use privileged port provided
-    ServerSocket ss;
-    if (secureResources == null) {
-      InetSocketAddress addr = DataNode.getStreamingAddr(conf);
-      ss = (dnConf.socketWriteTimeout > 0) ? 
-          ServerSocketChannel.open().socket() : new ServerSocket();
-          Server.bind(ss, addr, 0);
+    TcpPeerServer tcpPeerServer;
+    if (secureResources != null) {
+      tcpPeerServer = new TcpPeerServer(secureResources);
     } else {
-      ss = secureResources.getStreamingSocket();
+      tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
+          DataNode.getStreamingAddr(conf));
     }
-    ss.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE); 
-
-    streamingAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
-                                     ss.getLocalPort());
-
+    tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
+    streamingAddr = tcpPeerServer.getStreamingAddr();
     LOG.info("Opened streaming server at " + streamingAddr);
     this.threadGroup = new ThreadGroup("dataXceiverServer");
     this.dataXceiverServer = new Daemon(threadGroup, 
-        new DataXceiverServer(ss, conf, this));
+        new DataXceiverServer(tcpPeerServer, conf, this));
     this.threadGroup.setDaemon(true); // auto destroy when empty
+
+    if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
+              DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT) ||
+        conf.getBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+              DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
+      DomainPeerServer domainPeerServer =
+                getDomainPeerServer(conf, streamingAddr.getPort());
+      if (domainPeerServer != null) {
+        this.localDataXceiverServer = new Daemon(threadGroup,
+            new DataXceiverServer(domainPeerServer, conf, this));
+        LOG.info("Listening on UNIX domain socket: " +
+            domainPeerServer.getBindPath());
+      }
+    }
+  }
+
+  static DomainPeerServer getDomainPeerServer(Configuration conf,
+      int port) throws IOException {
+    String domainSocketPath =
+        conf.getTrimmed(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+            DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT);
+    if (domainSocketPath.isEmpty()) {
+      if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
+            DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT) &&
+         (!conf.getBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
+          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT))) {
+        LOG.warn("Although short-circuit local reads are configured, " +
+            "they are disabled because you didn't configure " +
+            DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY);
+      }
+      return null;
+    }
+    if (DomainSocket.getLoadingFailureReason() != null) {
+      throw new RuntimeException("Although a UNIX domain socket " +
+          "path is configured as " + domainSocketPath + ", we cannot " +
+          "start a localDataXceiverServer because " +
+          DomainSocket.getLoadingFailureReason());
+    }
+    DomainPeerServer domainPeerServer =
+      new DomainPeerServer(domainSocketPath, port);
+    domainPeerServer.setReceiveBufferSize(
+        HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
+    return domainPeerServer;
   }
   
   // calls specific to BP
@@ -1044,6 +1104,42 @@ public class DataNode extends Configured
     return info;
   }
 
+  @InterfaceAudience.LimitedPrivate("HDFS")
+  static public class ShortCircuitFdsUnsupportedException extends IOException {
+    private static final long serialVersionUID = 1L;
+    public ShortCircuitFdsUnsupportedException(String msg) {
+      super(msg);
+    }
+  }
+
+  @InterfaceAudience.LimitedPrivate("HDFS")
+  static public class ShortCircuitFdsVersionException extends IOException {
+    private static final long serialVersionUID = 1L;
+    public ShortCircuitFdsVersionException(String msg) {
+      super(msg);
+    }
+  }
+
+  FileInputStream[] requestShortCircuitFdsForRead(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> token, int maxVersion) 
+          throws ShortCircuitFdsUnsupportedException,
+            ShortCircuitFdsVersionException, IOException {
+    if (fileDescriptorPassingDisabledReason != null) {
+      throw new ShortCircuitFdsUnsupportedException(
+          fileDescriptorPassingDisabledReason);
+    }
+    checkBlockToken(blk, token, BlockTokenSecretManager.AccessMode.READ);
+    int blkVersion = CURRENT_BLOCK_FORMAT_VERSION;
+    if (maxVersion < blkVersion) {
+      throw new ShortCircuitFdsVersionException("Your client is too old " +
+        "to read this block!  Its format version is " + 
+        blkVersion + ", but the highest format version you can read is " +
+        maxVersion);
+    }
+    metrics.incrBlocksGetLocalPathInfo();
+    return data.getShortCircuitFdsForRead(blk);
+  }
+
   @Override
   public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks,
       List<Token<BlockTokenIdentifier>> tokens) throws IOException, 
@@ -1118,32 +1214,45 @@ public class DataNode extends Configured
     if (dataXceiverServer != null) {
       ((DataXceiverServer) this.dataXceiverServer.getRunnable()).kill();
       this.dataXceiverServer.interrupt();
-
-      // wait for all data receiver threads to exit
-      if (this.threadGroup != null) {
-        int sleepMs = 2;
-        while (true) {
-          this.threadGroup.interrupt();
-          LOG.info("Waiting for threadgroup to exit, active threads is " +
-                   this.threadGroup.activeCount());
-          if (this.threadGroup.activeCount() == 0) {
-            break;
-          }
-          try {
-            Thread.sleep(sleepMs);
-          } catch (InterruptedException e) {}
-          sleepMs = sleepMs * 3 / 2; // exponential backoff
-          if (sleepMs > 1000) {
-            sleepMs = 1000;
-          }
+    }
+    if (localDataXceiverServer != null) {
+      ((DataXceiverServer) this.localDataXceiverServer.getRunnable()).kill();
+      this.localDataXceiverServer.interrupt();
+    }
+    // wait for all data receiver threads to exit
+    if (this.threadGroup != null) {
+      int sleepMs = 2;
+      while (true) {
+        this.threadGroup.interrupt();
+        LOG.info("Waiting for threadgroup to exit, active threads is " +
+                 this.threadGroup.activeCount());
+        if (this.threadGroup.activeCount() == 0) {
+          break;
+        }
+        try {
+          Thread.sleep(sleepMs);
+        } catch (InterruptedException e) {}
+        sleepMs = sleepMs * 3 / 2; // exponential backoff
+        if (sleepMs > 1000) {
+          sleepMs = 1000;
         }
       }
-      // wait for dataXceiveServer to terminate
+      this.threadGroup = null;
+    }
+    if (this.dataXceiverServer != null) {
+      // wait for dataXceiverServer to terminate
       try {
         this.dataXceiverServer.join();
       } catch (InterruptedException ie) {
       }
     }
+    if (this.localDataXceiverServer != null) {
+      // wait for localDataXceiverServer to terminate
+      try {
+        this.localDataXceiverServer.join();
+      } catch (InterruptedException ie) {
+      }
+    }
     
     if(blockPoolManager != null) {
       try {
@@ -1538,6 +1647,9 @@ public class DataNode extends Configured
 
     // start dataXceiveServer
     dataXceiverServer.start();
+    if (localDataXceiverServer != null) {
+      localDataXceiverServer.start();
+    }
     ipcServer.start();
     startPlugins(conf);
   }

+ 110 - 49
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_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;
 import static org.apache.hadoop.util.Time.now;
@@ -28,6 +29,8 @@ import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InterruptedIOException;
@@ -39,6 +42,7 @@ import java.nio.channels.ClosedChannelException;
 import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
+import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -59,12 +63,13 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 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.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.SocketInputWrapper;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -79,8 +84,7 @@ class DataXceiver extends Receiver implements Runnable {
   public static final Log LOG = DataNode.LOG;
   static final Log ClientTraceLog = DataNode.ClientTraceLog;
   
-  private final Socket s;
-  private final boolean isLocal; //is a local connection?
+  private final Peer peer;
   private final String remoteAddress; // address of remote side
   private final String localAddress;  // local address of this daemon
   private final DataNode datanode;
@@ -88,7 +92,7 @@ class DataXceiver extends Receiver implements Runnable {
   private final DataXceiverServer dataXceiverServer;
   private final boolean connectToDnViaHostname;
   private long opStartTime; //the start time of receiving an Op
-  private final SocketInputWrapper socketIn;
+  private final InputStream socketIn;
   private OutputStream socketOut;
 
   /**
@@ -97,25 +101,23 @@ class DataXceiver extends Receiver implements Runnable {
    */
   private String previousOpClientName;
   
-  public static DataXceiver create(Socket s, DataNode dn,
+  public static DataXceiver create(Peer peer, DataNode dn,
       DataXceiverServer dataXceiverServer) throws IOException {
-    return new DataXceiver(s, dn, dataXceiverServer);
+    return new DataXceiver(peer, dn, dataXceiverServer);
   }
   
-  private DataXceiver(Socket s, 
-      DataNode datanode, 
+  private DataXceiver(Peer peer, DataNode datanode,
       DataXceiverServer dataXceiverServer) throws IOException {
 
-    this.s = s;
+    this.peer = peer;
     this.dnConf = datanode.getDnConf();
-    this.socketIn = NetUtils.getInputStream(s);
-    this.socketOut = NetUtils.getOutputStream(s, dnConf.socketWriteTimeout);
-    this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
+    this.socketIn = peer.getInputStream();
+    this.socketOut = peer.getOutputStream();
     this.datanode = datanode;
     this.dataXceiverServer = dataXceiverServer;
     this.connectToDnViaHostname = datanode.getDnConf().connectToDnViaHostname;
-    remoteAddress = s.getRemoteSocketAddress().toString();
-    localAddress = s.getLocalSocketAddress().toString();
+    remoteAddress = peer.getRemoteAddressString();
+    localAddress = peer.getLocalAddressString();
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Number of active connections is: "
@@ -155,11 +157,10 @@ class DataXceiver extends Receiver implements Runnable {
   public void run() {
     int opsProcessed = 0;
     Op op = null;
-    
-    dataXceiverServer.childSockets.add(s);
-    
+
+    dataXceiverServer.addPeer(peer);
     try {
-      
+      peer.setWriteTimeout(datanode.getDnConf().socketWriteTimeout);
       InputStream input = socketIn;
       if (dnConf.encryptDataTransfer) {
         IOStreamPair encryptedStreams = null;
@@ -169,8 +170,9 @@ class DataXceiver extends Receiver implements Runnable {
               dnConf.encryptionAlgorithm);
         } catch (InvalidMagicNumberException imne) {
           LOG.info("Failed to read expected encryption handshake from client " +
-              "at " + s.getInetAddress() + ". Perhaps the client is running an " +
-              "older version of Hadoop which does not support encryption");
+              "at " + peer.getRemoteAddressString() + ". Perhaps the client " +
+              "is running an older version of Hadoop which does not support " +
+              "encryption");
           return;
         }
         input = encryptedStreams.in;
@@ -189,9 +191,9 @@ class DataXceiver extends Receiver implements Runnable {
         try {
           if (opsProcessed != 0) {
             assert dnConf.socketKeepaliveTimeout > 0;
-            socketIn.setTimeout(dnConf.socketKeepaliveTimeout);
+            peer.setReadTimeout(dnConf.socketKeepaliveTimeout);
           } else {
-            socketIn.setTimeout(dnConf.socketTimeout);
+            peer.setReadTimeout(dnConf.socketTimeout);
           }
           op = readOp();
         } catch (InterruptedIOException ignored) {
@@ -202,7 +204,7 @@ class DataXceiver extends Receiver implements Runnable {
           if (opsProcessed > 0 &&
               (err instanceof EOFException || err instanceof ClosedChannelException)) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug("Cached " + s.toString() + " closing after " + opsProcessed + " ops");
+              LOG.debug("Cached " + peer + " closing after " + opsProcessed + " ops");
             }
           } else {
             throw err;
@@ -212,13 +214,13 @@ class DataXceiver extends Receiver implements Runnable {
 
         // restore normal timeout
         if (opsProcessed != 0) {
-          s.setSoTimeout(dnConf.socketTimeout);
+          peer.setReadTimeout(dnConf.socketTimeout);
         }
 
         opStartTime = now();
         processOp(op);
         ++opsProcessed;
-      } while (!s.isClosed() && dnConf.socketKeepaliveTimeout > 0);
+      } while (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0);
     } catch (Throwable t) {
       LOG.error(datanode.getDisplayName() + ":DataXceiver error processing " +
                 ((op == null) ? "unknown" : op.name()) + " operation " +
@@ -230,9 +232,70 @@ class DataXceiver extends Receiver implements Runnable {
             + datanode.getXceiverCount());
       }
       updateCurrentThreadName("Cleaning up");
+      dataXceiverServer.closePeer(peer);
       IOUtils.closeStream(in);
-      IOUtils.closeSocket(s);
-      dataXceiverServer.childSockets.remove(s);
+    }
+  }
+
+  @Override
+  public void requestShortCircuitFds(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> token,
+      int maxVersion) throws IOException {
+    updateCurrentThreadName("Passing file descriptors for block " + blk);
+    BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
+    FileInputStream fis[] = null;
+    try {
+      if (peer.getDomainSocket() == null) {
+        throw new IOException("You cannot pass file descriptors over " +
+            "anything but a UNIX domain socket.");
+      }
+      fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
+      bld.setStatus(SUCCESS);
+      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+    } catch (ShortCircuitFdsVersionException e) {
+      bld.setStatus(ERROR_UNSUPPORTED);
+      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+      bld.setMessage(e.getMessage());
+    } catch (ShortCircuitFdsUnsupportedException e) {
+      bld.setStatus(ERROR_UNSUPPORTED);
+      bld.setMessage(e.getMessage());
+    } catch (InvalidToken e) {
+      bld.setStatus(ERROR_ACCESS_TOKEN);
+      bld.setMessage(e.getMessage());
+    } catch (IOException e) {
+      bld.setStatus(ERROR);
+      bld.setMessage(e.getMessage());
+    }
+    try {
+      bld.build().writeDelimitedTo(socketOut);
+      if (fis != null) {
+        FileDescriptor fds[] = new FileDescriptor[fis.length];
+        for (int i = 0; i < fds.length; i++) {
+          fds[i] = fis[i].getFD();
+        }
+        byte buf[] = new byte[] { (byte)0 };
+        peer.getDomainSocket().
+          sendFileDescriptors(fds, buf, 0, buf.length);
+      }
+    } finally {
+      if (ClientTraceLog.isInfoEnabled()) {
+        DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
+            .getBlockPoolId());
+        BlockSender.ClientTraceLog.info(String.format(
+          String.format(
+            "src: %s, dest: %s, op: %s, blockid: %s, srvID: %s, " +
+              "success: %b",
+            "127.0.0.1",                   // src IP
+            "127.0.0.1",                   // dst IP
+            "REQUEST_SHORT_CIRCUIT_FDS",   // operation
+            blk.getBlockId(),             // block id
+            dnR.getStorageID(),
+            (fis != null)
+          )));
+      }
+      if (fis != null) {
+        IOUtils.cleanup(LOG, fis);
+      }
     }
   }
 
@@ -287,8 +350,9 @@ class DataXceiver extends Receiver implements Runnable {
           ClientReadStatusProto stat = ClientReadStatusProto.parseFrom(
               PBHelper.vintPrefixed(in));
           if (!stat.hasStatus()) {
-            LOG.warn("Client " + s.getInetAddress() + " did not send a valid status " +
-                     "code after reading. Will close connection.");
+            LOG.warn("Client " + peer.getRemoteAddressString() +
+                " did not send a valid status code after reading. " +
+                "Will close connection.");
             IOUtils.closeStream(out);
           }
         } catch (IOException ioe) {
@@ -321,7 +385,7 @@ class DataXceiver extends Receiver implements Runnable {
 
     //update metrics
     datanode.metrics.addReadBlockOp(elapsed());
-    datanode.metrics.incrReadsFromClient(isLocal);
+    datanode.metrics.incrReadsFromClient(peer.isLocal());
   }
 
   @Override
@@ -359,8 +423,8 @@ class DataXceiver extends Receiver implements Runnable {
       LOG.debug("isDatanode=" + isDatanode
           + ", isClient=" + isClient
           + ", isTransfer=" + isTransfer);
-      LOG.debug("writeBlock receive buf size " + s.getReceiveBufferSize() +
-                " tcp no delay " + s.getTcpNoDelay());
+      LOG.debug("writeBlock receive buf size " + peer.getReceiveBufferSize() +
+                " tcp no delay " + peer.getTcpNoDelay());
     }
 
     // We later mutate block's generation stamp and length, but we need to
@@ -391,8 +455,8 @@ class DataXceiver extends Receiver implements Runnable {
           stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
         // open a block receiver
         blockReceiver = new BlockReceiver(block, in, 
-            s.getRemoteSocketAddress().toString(),
-            s.getLocalSocketAddress().toString(),
+            peer.getRemoteAddressString(),
+            peer.getLocalAddressString(),
             stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
             clientname, srcDataNode, datanode, requestedChecksum);
       } else {
@@ -547,7 +611,7 @@ class DataXceiver extends Receiver implements Runnable {
 
     //update metrics
     datanode.metrics.addWriteBlockOp(elapsed());
-    datanode.metrics.incrWritesFromClient(isLocal);
+    datanode.metrics.incrWritesFromClient(peer.isLocal());
   }
 
   @Override
@@ -555,7 +619,7 @@ class DataXceiver extends Receiver implements Runnable {
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final DatanodeInfo[] targets) throws IOException {
-    checkAccess(null, true, blk, blockToken,
+    checkAccess(socketOut, true, blk, blockToken,
         Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
     previousOpClientName = clientName;
     updateCurrentThreadName(Op.TRANSFER_BLOCK + " " + blk);
@@ -642,8 +706,9 @@ class DataXceiver extends Receiver implements Runnable {
     }
 
     if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
-      String msg = "Not able to copy block " + block.getBlockId() + " to " 
-      + s.getRemoteSocketAddress() + " because threads quota is exceeded."; 
+      String msg = "Not able to copy block " + block.getBlockId() + " " +
+          "to " + peer.getRemoteAddressString() + " because threads " +
+          "quota is exceeded.";
       LOG.info(msg);
       sendResponse(ERROR, msg);
       return;
@@ -672,7 +737,7 @@ class DataXceiver extends Receiver implements Runnable {
       datanode.metrics.incrBytesRead((int) read);
       datanode.metrics.incrBlocksRead();
       
-      LOG.info("Copied " + block + " to " + s.getRemoteSocketAddress());
+      LOG.info("Copied " + block + " to " + peer.getRemoteAddressString());
     } catch (IOException ioe) {
       isOpSuccess = false;
       LOG.info("opCopyBlock " + block + " received exception " + ioe);
@@ -717,8 +782,9 @@ class DataXceiver extends Receiver implements Runnable {
     }
 
     if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
-      String msg = "Not able to receive block " + block.getBlockId() + " from " 
-          + s.getRemoteSocketAddress() + " because threads quota is exceeded."; 
+      String msg = "Not able to receive block " + block.getBlockId() +
+          " from " + peer.getRemoteAddressString() + " because threads " +
+          "quota is exceeded.";
       LOG.warn(msg);
       sendResponse(ERROR, msg);
       return;
@@ -795,7 +861,7 @@ class DataXceiver extends Receiver implements Runnable {
       // notify name node
       datanode.notifyNamenodeReceivedBlock(block, delHint);
 
-      LOG.info("Moved " + block + " from " + s.getRemoteSocketAddress());
+      LOG.info("Moved " + block + " from " + peer.getRemoteAddressString());
       
     } catch (IOException ioe) {
       opStatus = ERROR;
@@ -818,7 +884,7 @@ class DataXceiver extends Receiver implements Runnable {
       try {
         sendResponse(opStatus, errMsg);
       } catch (IOException ioe) {
-        LOG.warn("Error writing reply back to " + s.getRemoteSocketAddress());
+        LOG.warn("Error writing reply back to " + peer.getRemoteAddressString());
       }
       IOUtils.closeStream(proxyOut);
       IOUtils.closeStream(blockReceiver);
@@ -872,7 +938,7 @@ class DataXceiver extends Receiver implements Runnable {
   }
   
 
-  private void checkAccess(DataOutputStream out, final boolean reply, 
+  private void checkAccess(OutputStream out, final boolean reply, 
       final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> t,
       final Op op,
@@ -887,11 +953,6 @@ class DataXceiver extends Receiver implements Runnable {
       } catch(InvalidToken e) {
         try {
           if (reply) {
-            if (out == null) {
-              out = new DataOutputStream(
-                  NetUtils.getOutputStream(s, dnConf.socketWriteTimeout));
-            }
-            
             BlockOpResponseProto.Builder resp = BlockOpResponseProto.newBuilder()
               .setStatus(ERROR_ACCESS_TOKEN);
             if (mode == BlockTokenSecretManager.AccessMode.WRITE) {

+ 28 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -18,18 +18,16 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.IOException;
-import java.net.ServerSocket;
-import java.net.Socket;
 import java.net.SocketTimeoutException;
 import java.nio.channels.AsynchronousCloseException;
-import java.util.Collections;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.net.PeerServer;
 import org.apache.hadoop.hdfs.server.balancer.Balancer;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
@@ -45,11 +43,9 @@ import org.apache.hadoop.util.Daemon;
 class DataXceiverServer implements Runnable {
   public static final Log LOG = DataNode.LOG;
   
-  ServerSocket ss;
-  DataNode datanode;
-  // Record all sockets opened for data transfer
-  Set<Socket> childSockets = Collections.synchronizedSet(
-                                       new HashSet<Socket>());
+  private final PeerServer peerServer;
+  private final DataNode datanode;
+  private final Set<Peer> peers = new HashSet<Peer>();
   
   /**
    * Maximal number of concurrent xceivers per node.
@@ -109,10 +105,10 @@ class DataXceiverServer implements Runnable {
   long estimateBlockSize;
   
   
-  DataXceiverServer(ServerSocket ss, Configuration conf, 
+  DataXceiverServer(PeerServer peerServer, Configuration conf,
       DataNode datanode) {
     
-    this.ss = ss;
+    this.peerServer = peerServer;
     this.datanode = datanode;
     
     this.maxXceiverCount = 
@@ -130,12 +126,10 @@ class DataXceiverServer implements Runnable {
 
   @Override
   public void run() {
+    Peer peer = null;
     while (datanode.shouldRun) {
-      Socket s = null;
       try {
-        s = ss.accept();
-        s.setTcpNoDelay(true);
-        // Timeouts are set within DataXceiver.run()
+        peer = peerServer.accept();
 
         // Make sure the xceiver count is not exceeded
         int curXceiverCount = datanode.getXceiverCount();
@@ -146,7 +140,7 @@ class DataXceiverServer implements Runnable {
         }
 
         new Daemon(datanode.threadGroup,
-            DataXceiver.create(s, datanode, this))
+            DataXceiver.create(peer, datanode, this))
             .start();
       } catch (SocketTimeoutException ignored) {
         // wake up to see if should continue to run
@@ -157,10 +151,10 @@ class DataXceiverServer implements Runnable {
           LOG.warn(datanode.getDisplayName() + ":DataXceiverServer: ", ace);
         }
       } catch (IOException ie) {
-        IOUtils.closeSocket(s);
+        IOUtils.cleanup(null, peer);
         LOG.warn(datanode.getDisplayName() + ":DataXceiverServer: ", ie);
       } catch (OutOfMemoryError ie) {
-        IOUtils.closeSocket(s);
+        IOUtils.cleanup(null, peer);
         // DataNode can run out of memory if there is too many transfers.
         // Log the event, Sleep for 30 seconds, other transfers may complete by
         // then.
@@ -176,33 +170,35 @@ class DataXceiverServer implements Runnable {
         datanode.shouldRun = false;
       }
     }
+    synchronized (this) {
+      for (Peer p : peers) {
+        IOUtils.cleanup(LOG, p);
+      }
+    }
     try {
-      ss.close();
+      peerServer.close();
     } catch (IOException ie) {
       LOG.warn(datanode.getDisplayName()
           + " :DataXceiverServer: close exception", ie);
     }
   }
-  
+
   void kill() {
     assert datanode.shouldRun == false :
       "shoudRun should be set to false before killing";
     try {
-      this.ss.close();
+      this.peerServer.close();
     } catch (IOException ie) {
       LOG.warn(datanode.getDisplayName() + ":DataXceiverServer.kill(): ", ie);
     }
+  }
+  
+  synchronized void addPeer(Peer peer) {
+    peers.add(peer);
+  }
 
-    // close all the sockets that were accepted earlier
-    synchronized (childSockets) {
-      for (Iterator<Socket> it = childSockets.iterator();
-           it.hasNext();) {
-        Socket thissock = it.next();
-        try {
-          thissock.close();
-        } catch (IOException e) {
-        }
-      }
-    }
+  synchronized void closePeer(Peer peer) {
+    peers.remove(peer);
+    IOUtils.cleanup(null, peer);
   }
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
@@ -386,4 +387,6 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
       throws IOException;
 
+  FileInputStream[] getShortCircuitFdsForRead(ExtendedBlock block)
+      throws IOException;
 }

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

@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.DataChecksum;
@@ -1700,6 +1701,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     return info;
   }
   
+  @Override // FsDatasetSpi
+  public FileInputStream[] getShortCircuitFdsForRead(ExtendedBlock block) 
+      throws IOException {
+    File datafile = getBlockFile(block);
+    File metafile = FsDatasetUtil.getMetaFile(datafile,
+        block.getGenerationStamp());
+    FileInputStream fis[] = new FileInputStream[2];
+    boolean success = false;
+    try {
+      fis[0] = new FileInputStream(datafile);
+      fis[1] = new FileInputStream(metafile);
+      success = true;
+      return fis;
+    } finally {
+      if (!success) {
+        IOUtils.cleanup(null, fis);
+      }
+    }
+  }
+    
   @Override // FsDatasetSpi
   public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
       throws IOException {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -246,8 +246,7 @@ public class FSDirectory implements Closeable {
                 long preferredBlockSize,
                 String clientName,
                 String clientMachine,
-                DatanodeDescriptor clientNode,
-                long generationStamp) 
+                DatanodeDescriptor clientNode)
     throws FileAlreadyExistsException, QuotaExceededException,
       UnresolvedLinkException, SnapshotAccessControlException {
     waitForReady();

+ 138 - 140
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1221,7 +1221,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set permission for " + src, safeMode);
       }
@@ -1259,7 +1258,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set owner for " + src, safeMode);
       }
@@ -1313,9 +1311,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   LocatedBlocks getBlockLocations(String src, long offset, long length,
       boolean doAccessTime, boolean needBlockToken, boolean checkSafeMode)
       throws FileNotFoundException, UnresolvedLinkException, IOException {
-    FSPermissionChecker pc = getPermissionChecker();
     try {
-      return getBlockLocationsInt(pc, src, offset, length, doAccessTime,
+      return getBlockLocationsInt(src, offset, length, doAccessTime,
                                   needBlockToken, checkSafeMode);
     } catch (AccessControlException e) {
       logAuditEvent(false, "open", src);
@@ -1323,14 +1320,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  private LocatedBlocks getBlockLocationsInt(FSPermissionChecker pc,
-      String src, long offset, long length, boolean doAccessTime,
-      boolean needBlockToken, boolean checkSafeMode)
+  private LocatedBlocks getBlockLocationsInt(String src, long offset,
+      long length, boolean doAccessTime, boolean needBlockToken,
+      boolean checkSafeMode)
       throws FileNotFoundException, UnresolvedLinkException, IOException {
-    if (isPermissionEnabled) {
-      checkPathAccess(pc, src, FsAction.READ);
-    }
-
     if (offset < 0) {
       throw new HadoopIllegalArgumentException(
           "Negative offset is not supported. File: " + src);
@@ -1358,13 +1351,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Get block locations within the specified range, updating the
    * access times if necessary. 
    */
-  private LocatedBlocks getBlockLocationsUpdateTimes(String src,
-                                                       long offset, 
-                                                       long length,
-                                                       boolean doAccessTime, 
-                                                       boolean needBlockToken)
-      throws FileNotFoundException, UnresolvedLinkException, IOException {
-
+  private LocatedBlocks getBlockLocationsUpdateTimes(String src, long offset,
+      long length, boolean doAccessTime, boolean needBlockToken)
+      throws FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    FSPermissionChecker pc = getPermissionChecker();
     for (int attempt = 0; attempt < 2; attempt++) {
       boolean isReadOp = (attempt == 0);
       if (isReadOp) { // first attempt is with readlock
@@ -1380,6 +1371,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         } else {
           checkOperation(OperationCategory.WRITE);
         }
+        if (isPermissionEnabled) {
+          checkPathAccess(pc, src, FsAction.READ);
+        }
 
         // if the namenode is in safemode, then do not update access time
         if (isInSafeMode()) {
@@ -1426,6 +1420,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   void concat(String target, String [] srcs) 
       throws IOException, UnresolvedLinkException {
+    if(FSNamesystem.LOG.isDebugEnabled()) {
+      FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) +
+          " to " + target);
+    }
     try {
       concatInt(target, srcs);
     } catch (AccessControlException e) {
@@ -1436,11 +1434,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   private void concatInt(String target, String [] srcs) 
       throws IOException, UnresolvedLinkException {
-    if(FSNamesystem.LOG.isDebugEnabled()) {
-      FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) +
-          " to " + target);
-    }
-    
     // verify args
     if(target.isEmpty()) {
       throw new IllegalArgumentException("Target file name is empty");
@@ -1594,6 +1587,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   void setTimes(String src, long mtime, long atime) 
       throws IOException, UnresolvedLinkException {
+    if (!isAccessTimeSupported() && atime != -1) {
+      throw new IOException("Access time for hdfs is not configured. " +
+                            " Please set " + DFS_NAMENODE_ACCESSTIME_PRECISION_KEY + " configuration parameter.");
+    }
     try {
       setTimesInt(src, mtime, atime);
     } catch (AccessControlException e) {
@@ -1604,16 +1601,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   private void setTimesInt(String src, long mtime, long atime) 
     throws IOException, UnresolvedLinkException {
-    if (!isAccessTimeSupported() && atime != -1) {
-      throw new IOException("Access time for hdfs is not configured. " +
-                            " Please set " + DFS_NAMENODE_ACCESSTIME_PRECISION_KEY + " configuration parameter.");
-    }
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException("Cannot set times " + src, safeMode);
+      }
 
       // Write access is required to set access and modification times
       if (isPermissionEnabled) {
@@ -1639,6 +1635,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   void createSymlink(String target, String link,
       PermissionStatus dirPerms, boolean createParent) 
       throws IOException, UnresolvedLinkException {
+    if (!DFSUtil.isValidName(link)) {
+      throw new InvalidPathException("Invalid file name: " + link);
+    }
     try {
       createSymlinkInt(target, link, dirPerms, createParent);
     } catch (AccessControlException e) {
@@ -1650,17 +1649,34 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private void createSymlinkInt(String target, String link,
       PermissionStatus dirPerms, boolean createParent) 
       throws IOException, UnresolvedLinkException {
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* NameSystem.createSymlink: target="
+          + target + " link=" + link);
+    }
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
+      if (isInSafeMode()) {
+        throw new SafeModeException("Cannot create symlink " + link, safeMode);
+      }
       if (!createParent) {
         verifyParentDir(link);
       }
-      createSymlinkInternal(pc, target, link, dirPerms, createParent);
+      if (!dir.isValidToCreate(link)) {
+        throw new IOException("failed to create link " + link 
+            +" either because the filename is invalid or the file exists");
+      }
+      if (isPermissionEnabled) {
+        checkAncestorAccess(pc, link, FsAction.WRITE);
+      }
+      // validate that we have enough inodes.
+      checkFsObjectLimit();
+
+      // add symbolic link to namespace
+      dir.addSymlink(link, target, dirPerms, createParent);
       resultingStat = getAuditFileInfo(link, false);
     } finally {
       writeUnlock();
@@ -1669,37 +1685,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     logAuditEvent(true, "createSymlink", link, target, resultingStat);
   }
 
-  /**
-   * Create a symbolic link.
-   */
-  private void createSymlinkInternal(FSPermissionChecker pc, String target,
-      String link, PermissionStatus dirPerms, boolean createParent)
-      throws IOException, UnresolvedLinkException {
-    assert hasWriteLock();
-    if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* NameSystem.createSymlink: target=" + 
-        target + " link=" + link);
-    }
-    if (isInSafeMode()) {
-      throw new SafeModeException("Cannot create symlink " + link, safeMode);
-    }
-    if (!DFSUtil.isValidName(link)) {
-      throw new InvalidPathException("Invalid file name: " + link);
-    }
-    if (!dir.isValidToCreate(link)) {
-      throw new IOException("failed to create link " + link 
-          +" either because the filename is invalid or the file exists");
-    }
-    if (isPermissionEnabled) {
-      checkAncestorAccess(pc, link, FsAction.WRITE);
-    }
-    // validate that we have enough inodes.
-    checkFsObjectLimit();
-
-    // add symbolic link to namespace
-    dir.addSymlink(link, target, dirPerms, createParent);
-  }
-
   /**
    * Set replication for an existing file.
    * 
@@ -1819,13 +1804,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws AccessControlException, SafeModeException,
       FileAlreadyExistsException, UnresolvedLinkException,
       FileNotFoundException, ParentNotDirectoryException, IOException {
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: src=" + src
+          + ", holder=" + holder
+          + ", clientMachine=" + clientMachine
+          + ", createParent=" + createParent
+          + ", replication=" + replication
+          + ", createFlag=" + flag.toString());
+    }
+    if (!DFSUtil.isValidName(src)) {
+      throw new InvalidPathException(src);
+    }
+
     boolean skipSync = false;
     final HdfsFileStatus stat;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
       startFileInternal(pc, src, permissions, holder, clientMachine, flag,
           createParent, replication, blockSize);
       stat = dir.getFileInfo(src, false);
@@ -1868,21 +1864,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       AccessControlException, UnresolvedLinkException, FileNotFoundException,
       ParentNotDirectoryException, IOException {
     assert hasWriteLock();
-    if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: src=" + src
-          + ", holder=" + holder
-          + ", clientMachine=" + clientMachine
-          + ", createParent=" + createParent
-          + ", replication=" + replication
-          + ", createFlag=" + flag.toString());
-    }
+    checkOperation(OperationCategory.WRITE);
     if (isInSafeMode()) {
       throw new SafeModeException("Cannot create file" + src, safeMode);
     }
-    if (!DFSUtil.isValidName(src)) {
-      throw new InvalidPathException(src);
-    }
-
     // Verify that the destination does not exist as a directory already.
     final INodesInPath iip = dir.getINodesInPath4Write(src);
     final INode inode = iip.getLastINode();
@@ -1945,9 +1930,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
        checkFsObjectLimit();
 
         // increment global generation stamp
-        long genstamp = nextGenerationStamp();
         INodeFileUnderConstruction newNode = dir.addFile(src, permissions,
-            replication, blockSize, holder, clientMachine, clientNode, genstamp);
+            replication, blockSize, holder, clientMachine, clientNode);
         if (newNode == null) {
           throw new IOException("DIR* NameSystem.startFile: " +
                                 "Unable to add file to namespace.");
@@ -2013,21 +1997,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   boolean recoverLease(String src, String holder, String clientMachine)
       throws IOException {
+    if (!DFSUtil.isValidName(src)) {
+      throw new IOException("Invalid file name: " + src);
+    }
+  
     boolean skipSync = false;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
       if (isInSafeMode()) {
         throw new SafeModeException(
             "Cannot recover the lease of " + src, safeMode);
       }
-      if (!DFSUtil.isValidName(src)) {
-        throw new IOException("Invalid file name: " + src);
-      }
-  
       final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
       if (!inode.isUnderConstruction()) {
         return true;
@@ -2151,13 +2134,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           "Append is not enabled on this NameNode. Use the " +
           DFS_SUPPORT_APPEND_KEY + " configuration option to enable it.");
     }
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src=" + src
+          + ", holder=" + holder
+          + ", clientMachine=" + clientMachine);
+    }
+    if (!DFSUtil.isValidName(src)) {
+      throw new InvalidPathException(src);
+    }
+
     LocatedBlock lb = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-
       lb = startFileInternal(pc, src, null, holder, clientMachine, 
                         EnumSet.of(CreateFlag.APPEND), 
                         false, blockManager.maxReplication, 0);
@@ -2448,21 +2438,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   boolean abandonBlock(ExtendedBlock b, String src, String holder)
       throws LeaseExpiredException, FileNotFoundException,
       UnresolvedLinkException, IOException {
+    if(NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: " + b
+          + "of file " + src);
+    }
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-      //
-      // Remove the block from the pending creates list
-      //
-      if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
-                                      +b+"of file "+src);
-      }
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot abandon block " + b +
                                     " for fle" + src, safeMode);
       }
+      //
+      // Remove the block from the pending creates list
+      //
       INodeFileUnderConstruction file = checkLease(src, holder);
       dir.removeBlock(src, file, ExtendedBlock.getLocalBlock(b));
       if(NameNode.stateChangeLog.isDebugEnabled()) {
@@ -2524,19 +2514,23 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   boolean completeFile(String src, String holder, ExtendedBlock last) 
     throws SafeModeException, UnresolvedLinkException, IOException {
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " +
+          src + " for " + holder);
+    }
     checkBlock(last);
     boolean success = false;
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-
-      success = completeFileInternal(src, holder, 
-        ExtendedBlock.getLocalBlock(last));
+      success = completeFileInternal(src, holder,
+          ExtendedBlock.getLocalBlock(last));
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
+    NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
+        + holder);
     return success;
   }
 
@@ -2544,10 +2538,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       String holder, Block last) throws SafeModeException,
       UnresolvedLinkException, IOException {
     assert hasWriteLock();
-    if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " +
-          src + " for " + holder);
-    }
+    checkOperation(OperationCategory.WRITE);
     if (isInSafeMode()) {
       throw new SafeModeException("Cannot complete file " + src, safeMode);
     }
@@ -2588,9 +2579,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
     finalizeINodeFileUnderConstruction(src, pendingFile,
         iip.getLatestSnapshot());
-
-    NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
-        + holder);
     return true;
   }
 
@@ -2691,18 +2679,19 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   private boolean renameToInt(String src, String dst) 
     throws IOException, UnresolvedLinkException {
-    boolean status = false;
-    HdfsFileStatus resultingStat = null;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src +
           " to " + dst);
     }
+    if (!DFSUtil.isValidName(dst)) {
+      throw new IOException("Invalid name: " + dst);
+    }
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
+    boolean status = false;
+    HdfsFileStatus resultingStat = null;
     writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-
       status = renameToInternal(pc, src, dst);
       if (status) {
         resultingStat = getAuditFileInfo(dst, false);
@@ -2722,12 +2711,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private boolean renameToInternal(FSPermissionChecker pc, String src, String dst)
     throws IOException, UnresolvedLinkException {
     assert hasWriteLock();
+      checkOperation(OperationCategory.WRITE);
     if (isInSafeMode()) {
       throw new SafeModeException("Cannot rename " + src, safeMode);
     }
-    if (!DFSUtil.isValidName(dst)) {
-      throw new IOException("Invalid name: " + dst);
-    }
     if (isPermissionEnabled) {
       //We should not be doing this.  This is move() not renameTo().
       //but for now,
@@ -2749,16 +2736,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /** Rename src to dst */
   void renameTo(String src, String dst, Options.Rename... options)
       throws IOException, UnresolvedLinkException {
-    HdfsFileStatus resultingStat = null;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
           + src + " to " + dst);
     }
+    if (!DFSUtil.isValidName(dst)) {
+      throw new InvalidPathException("Invalid name: " + dst);
+    }
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
+    HdfsFileStatus resultingStat = null;
     writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
       renameToInternal(pc, src, dst, options);
       resultingStat = getAuditFileInfo(dst, false);
     } finally {
@@ -2777,12 +2766,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private void renameToInternal(FSPermissionChecker pc, String src, String dst,
       Options.Rename... options) throws IOException {
     assert hasWriteLock();
+    checkOperation(OperationCategory.WRITE);
     if (isInSafeMode()) {
       throw new SafeModeException("Cannot rename " + src, safeMode);
     }
-    if (!DFSUtil.isValidName(dst)) {
-      throw new InvalidPathException("Invalid name: " + dst);
-    }
     if (isPermissionEnabled) {
       checkParentAccess(pc, src, FsAction.WRITE);
       checkAncestorAccess(pc, dst, FsAction.WRITE);
@@ -2969,16 +2956,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   HdfsFileStatus getFileInfo(String src, boolean resolveLink) 
     throws AccessControlException, UnresolvedLinkException,
            StandbyException, IOException {
+    if (!DFSUtil.isValidName(src)) {
+      throw new InvalidPathException("Invalid file name: " + src);
+    }
     HdfsFileStatus stat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-
-      if (!DFSUtil.isValidName(src)) {
-        throw new InvalidPathException("Invalid file name: " + src);
-      }
       if (isPermissionEnabled) {
         checkTraverse(pc, src);
       }
@@ -3035,16 +3021,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   private boolean mkdirsInt(String src, PermissionStatus permissions,
       boolean createParent) throws IOException, UnresolvedLinkException {
-    HdfsFileStatus resultingStat = null;
-    boolean status = false;
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
     }
+    if (!DFSUtil.isValidName(src)) {
+      throw new InvalidPathException(src);
+    }
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
+    HdfsFileStatus resultingStat = null;
+    boolean status = false;
     writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
       status = mkdirsInternal(pc, src, permissions, createParent);
       if (status) {
         resultingStat = dir.getFileInfo(src, false);
@@ -3066,6 +3054,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       PermissionStatus permissions, boolean createParent) 
       throws IOException, UnresolvedLinkException {
     assert hasWriteLock();
+    checkOperation(OperationCategory.WRITE);   
     if (isInSafeMode()) {
       throw new SafeModeException("Cannot create directory " + src, safeMode);
     }
@@ -3077,9 +3066,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       // a new directory is not created.
       return true;
     }
-    if (!DFSUtil.isValidName(src)) {
-      throw new InvalidPathException(src);
-    }
     if (isPermissionEnabled) {
       checkAncestorAccess(pc, src, FsAction.WRITE);
     }
@@ -3355,8 +3341,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       boolean closeFile, boolean deleteblock, DatanodeID[] newtargets,
       String[] newtargetstorages)
       throws IOException, UnresolvedLinkException {
-    String src = "";
+    LOG.info("commitBlockSynchronization(lastblock=" + lastblock
+             + ", newgenerationstamp=" + newgenerationstamp
+             + ", newlength=" + newlength
+             + ", newtargets=" + Arrays.asList(newtargets)
+             + ", closeFile=" + closeFile
+             + ", deleteBlock=" + deleteblock
+             + ")");
     checkOperation(OperationCategory.WRITE);
+    String src = "";
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -3368,13 +3361,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           "Cannot commitBlockSynchronization while in safe mode",
           safeMode);
       }
-      LOG.info("commitBlockSynchronization(lastblock=" + lastblock
-               + ", newgenerationstamp=" + newgenerationstamp
-               + ", newlength=" + newlength
-               + ", newtargets=" + Arrays.asList(newtargets)
-               + ", closeFile=" + closeFile
-               + ", deleteBlock=" + deleteblock
-               + ")");
       final BlockInfo storedBlock = blockManager.getStoredBlock(ExtendedBlock
         .getLocalBlock(lastblock));
       if (storedBlock == null) {
@@ -3465,7 +3451,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot renew lease for " + holder, safeMode);
       }
@@ -4951,11 +4936,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     checkOperation(OperationCategory.WRITE);
+    NameNode.stateChangeLog.info("*DIR* reportBadBlocks");
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-      
-      NameNode.stateChangeLog.info("*DIR* reportBadBlocks");
       for (int i = 0; i < blocks.length; i++) {
         ExtendedBlock blk = blocks[i].getBlock();
         DatanodeInfo[] nodes = blocks[i].getLocations();
@@ -5018,6 +5002,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       ExtendedBlock newBlock, DatanodeID[] newNodes)
       throws IOException {
     checkOperation(OperationCategory.WRITE);
+    LOG.info("updatePipeline(block=" + oldBlock
+             + ", newGenerationStamp=" + newBlock.getGenerationStamp()
+             + ", newLength=" + newBlock.getNumBytes()
+             + ", newNodes=" + Arrays.asList(newNodes)
+             + ", clientName=" + clientName
+             + ")");
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -5027,12 +5017,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
         + oldBlock + " has different block identifier";
-      LOG.info("updatePipeline(block=" + oldBlock
-               + ", newGenerationStamp=" + newBlock.getGenerationStamp()
-               + ", newLength=" + newBlock.getNumBytes()
-               + ", newNodes=" + Arrays.asList(newNodes)
-               + ", clientName=" + clientName
-               + ")");
       updatePipelineInternal(clientName, oldBlock, newBlock, newNodes);
     } finally {
       writeUnlock();
@@ -5284,7 +5268,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot issue delegation token", safeMode);
       }
@@ -5409,6 +5392,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
   }
   
+  /**
+   * Log the cancellation of expired tokens to edit logs
+   * 
+   * @param id token identifier to cancel
+   */
+  public void logExpireDelegationToken(DelegationTokenIdentifier id) {
+    assert !isInSafeMode() :
+      "this should never be called while in safemode, since we stop " +
+      "the DT manager before entering safemode!";
+    // No need to hold FSN lock since we don't access any internal
+    // structures, and this is stopped before the FSN shuts itself
+    // down, etc.
+    getEditLog().logCancelDelegationToken(id);
+  }  
+  
   private void logReassignLease(String leaseHolder, String src,
       String newHolder) {
     assert hasWriteLock();

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -559,9 +560,10 @@ public class NamenodeFsck {
         String file = BlockReaderFactory.getFileName(targetAddr, block.getBlockPoolId(),
             block.getBlockId());
         blockReader = BlockReaderFactory.newBlockReader(
-            conf, s, file, block, lblock
-            .getBlockToken(), 0, -1,
-            namenode.getRpcServer().getDataEncryptionKey());
+            conf, file, block, lblock.getBlockToken(), 0, -1, true, "fsck",
+            TcpPeerServer.peerFromSocketAndKey(s, namenode.getRpcServer().
+                getDataEncryptionKey()),
+            chosenNode, null, false);
         
       }  catch (IOException ex) {
         // Put chosen node into dead list, continue

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto

@@ -74,6 +74,8 @@ message DeleteBlockPoolResponseProto {
  * Gets the file information where block and its metadata is stored
  * block - block for which path information is being requested
  * token - block token
+ *
+ * This message is deprecated in favor of file descriptor passing.
  */
 message GetBlockLocalPathInfoRequestProto {
   required ExtendedBlockProto block = 1;
@@ -84,6 +86,8 @@ message GetBlockLocalPathInfoRequestProto {
  * block - block for which file path information is being returned
  * localPath - file path where the block data is stored
  * localMetaPath - file path where the block meta data is stored
+ *
+ * This message is deprecated in favor of file descriptor passing.
  */
 message GetBlockLocalPathInfoResponseProto {
   required ExtendedBlockProto block = 1;

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

@@ -115,6 +115,16 @@ message OpBlockChecksumProto {
   required BaseHeaderProto header = 1;
 }
 
+message OpRequestShortCircuitAccessProto { 
+  required BaseHeaderProto header = 1;
+
+  /** In order to get short-circuit access to block data, clients must set this
+   * to the highest version of the block data that they can understand.
+   * Currently 1 is the only version, but more versions may exist in the future
+   * if the on-disk format changes.
+   */
+  required uint32 maxVersion = 2;
+}
 
 message PacketHeaderProto {
   // All fields must be fixed-length!
@@ -133,6 +143,7 @@ enum Status {
   ERROR_EXISTS = 4;
   ERROR_ACCESS_TOKEN = 5;
   CHECKSUM_OK = 6;
+  ERROR_UNSUPPORTED = 7;
 }
 
 message PipelineAckProto {
@@ -165,6 +176,16 @@ message BlockOpResponseProto {
 
   /** explanatory text which may be useful to log on the client side */
   optional string message = 5;
+
+  /** If the server chooses to agree to the request of a client for
+   * short-circuit access, it will send a response message with the relevant
+   * file descriptors attached.
+   *
+   * In the body of the message, this version number will be set to the
+   * specific version number of the block data that the client is about to
+   * read.
+   */
+  optional uint32 shortCircuitAccessVersion = 6;
 }
 
 /**

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

@@ -1231,6 +1231,17 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.domain.socket.path</name>
+  <value></value>
+  <description>
+    Optional.  This is a path to a UNIX domain socket that will be used for
+    communication between the DataNode and local HDFS clients.
+    If the string "_PORT" is present in this path, it will be replaced by the
+    TCP port of the DataNode.
+  </description>
+</property>
+
 <property>
   <name>dfs.datanode.fsdataset.volume.choosing.balanced-space-threshold</name>
   <value>10737418240</value> <!-- 10 GB -->

+ 68 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ShortCircuitLocalReads.apt.vm

@@ -0,0 +1,68 @@
+
+~~ Licensed 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. See accompanying LICENSE file.
+
+  ---
+  Hadoop Distributed File System-${project.version} - Short-Circuit Local Reads
+  ---
+  ---
+  ${maven.build.timestamp}
+
+HDFS Short-Circuit Local Reads
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=0}
+
+* {Background}
+
+  In <<<HDFS>>>, reads normally go through the <<<DataNode>>>.  Thus, when the
+  client asks the <<<DataNode>>> to read a file, the <<<DataNode>>> reads that
+  file off of the disk and sends the data to the client over a TCP socket.
+  So-called "short-circuit" reads bypass the <<<DataNode>>>, allowing the client
+  to read the file directly.  Obviously, this is only possible in cases where
+  the client is co-located with the data.  Short-circuit reads provide a
+  substantial performance boost to many applications.
+
+* {Configuration}
+
+  To configure short-circuit local reads, you will need to enable
+  <<<libhadoop.so>>>.  See
+  {{{../hadoop-common/NativeLibraries.html}Native
+  Libraries}} for details on enabling this library.
+
+  Short-circuit reads make use of a UNIX domain socket.  This is a special path
+  in the filesystem that allows the client and the DataNodes to communicate.
+  You will need to set a path to this socket.  The DataNode needs to be able to
+  create this path.  On the other hand, it should not be possible for any user
+  except the hdfs user or root to create this path.  For this reason, paths
+  under <<</var/run>>> or <<</var/lib>>> are often used.
+
+  Short-circuit local reads need to be configured on both the <<<DataNode>>>
+  and the client.
+
+* {Example Configuration}
+
+  Here is an example configuration.
+
+----
+<configuration>
+  <property>
+    <name>dfs.client.read.shortcircuit</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>dfs.domain.socket.path</name>
+    <value>/var/lib/hadoop-hdfs/dn_socket</value>
+  </property>
+</configuration>
+----

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java

@@ -28,9 +28,9 @@ import java.net.Socket;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+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;
@@ -150,12 +150,12 @@ public class BlockReaderTestUtil {
     sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
 
     return BlockReaderFactory.newBlockReader(
-      new DFSClient.Conf(conf),
-      sock, targetAddr.toString()+ ":" + block.getBlockId(), block,
+      conf,
+      targetAddr.toString()+ ":" + block.getBlockId(), block,
       testBlock.getBlockToken(), 
       offset, lenToRead,
-      conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-      true, "", null, null);
+      true, "BlockReaderTestUtil", TcpPeerServer.peerFromSocket(sock),
+      nodes[0], null, false);
   }
 
   /**
@@ -166,5 +166,4 @@ public class BlockReaderTestUtil {
     int ipcport = nodes[0].getIpcPort();
     return cluster.getDataNode(ipcport);
   }
-
 }

+ 34 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -2188,14 +2188,27 @@ public class MiniDFSCluster {
   /**
    * Get file correpsonding to a block
    * @param storageDir storage directory
-   * @param blk block to be corrupted
-   * @return file corresponding to the block
+   * @param blk the block
+   * @return data file corresponding to the block
    */
   public static File getBlockFile(File storageDir, ExtendedBlock blk) {
     return new File(getFinalizedDir(storageDir, blk.getBlockPoolId()), 
         blk.getBlockName());
   }
 
+  /**
+   * Get the latest metadata file correpsonding to a block
+   * @param storageDir storage directory
+   * @param blk the block
+   * @return metadata file corresponding to the block
+   */
+  public static File getBlockMetadataFile(File storageDir, ExtendedBlock blk) {
+    return new File(getFinalizedDir(storageDir, blk.getBlockPoolId()), 
+        blk.getBlockName() + "_" + blk.getGenerationStamp() +
+        Block.METADATA_EXTENSION);
+    
+  }
+
   /**
    * Shut down a cluster if it is not null
    * @param cluster cluster reference or null
@@ -2223,7 +2236,7 @@ public class MiniDFSCluster {
   }
   
   /**
-   * Get files related to a block for a given datanode
+   * Get the block data file for a block from a given datanode
    * @param dnIndex Index of the datanode to get block files for
    * @param block block for which corresponding files are needed
    */
@@ -2238,6 +2251,24 @@ public class MiniDFSCluster {
     }
     return null;
   }
+
+  /**
+   * Get the block metadata file for a block from a given datanode
+   * 
+   * @param dnIndex Index of the datanode to get block files for
+   * @param block block for which corresponding files are needed
+   */
+  public static File getBlockMetadataFile(int dnIndex, ExtendedBlock block) {
+    // Check for block file in the two storage directories of the datanode
+    for (int i = 0; i <=1 ; i++) {
+      File storageDir = MiniDFSCluster.getStorageDir(dnIndex, i);
+      File blockMetaFile = getBlockMetadataFile(storageDir, block);
+      if (blockMetaFile.exists()) {
+        return blockMetaFile;
+      }
+    }
+    return null;
+  }
   
   /**
    * Throw an exception if the MiniDFSCluster is not started with a single

+ 302 - 59
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java

@@ -17,90 +17,333 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+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.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class TestBlockReaderLocal {
-  static MiniDFSCluster cluster;
-  static HdfsConfiguration conf;
-
-  @BeforeClass
-  public static void setupCluster() throws IOException {
-    conf = new HdfsConfiguration();
-
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
-        false);
-    conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
-        UserGroupInformation.getCurrentUser().getShortUserName());
+  public static void assertArrayRegionsEqual(byte []buf1, int off1, byte []buf2,
+      int off2, int len) {
+    for (int i = 0; i < len; i++) {
+      if (buf1[off1 + i] != buf2[off2 + i]) {
+        Assert.fail("arrays differ at byte " +  i + ". " + 
+          "The first array has " + (int)buf1[off1 + i] + 
+          ", but the second array has " + (int)buf2[off2 + i]);
+      }
+    }
+  }
 
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+  /**
+   * Similar to IOUtils#readFully(). Reads bytes in a loop.
+   *
+   * @param reader           The BlockReaderLocal to read bytes from
+   * @param buf              The ByteBuffer to read into
+   * @param off              The offset in the buffer to read into
+   * @param len              The number of bytes to read.
+   * 
+   * @throws IOException     If it could not read the requested number of bytes
+   */
+  private static void readFully(BlockReaderLocal reader,
+      ByteBuffer buf, int off, int len) throws IOException {
+    int amt = len;
+    while (amt > 0) {
+      buf.limit(off + len);
+      buf.position(off);
+      long ret = reader.read(buf);
+      if (ret < 0) {
+        throw new EOFException( "Premature EOF from BlockReaderLocal " +
+            "after reading " + (len - amt) + " byte(s).");
+      }
+      amt -= ret;
+      off += ret;
+    }
   }
 
-  @AfterClass
-  public static void teardownCluster() {
-    cluster.shutdown();
+  private static interface BlockReaderLocalTest {
+    final int TEST_LENGTH = 12345;
+    public void setup(File blockFile, boolean usingChecksums)
+        throws IOException;
+    public void doTest(BlockReaderLocal reader, byte original[])
+        throws IOException;
   }
+  
+  public void runBlockReaderLocalTest(BlockReaderLocalTest test,
+      boolean checksum) throws IOException {
+    MiniDFSCluster cluster = null;
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, !checksum);
+    conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
+    FileInputStream dataIn = null, checkIn = null;
+    final Path TEST_PATH = new Path("/a");
+    final long RANDOM_SEED = 4567L;
+    BlockReaderLocal blockReaderLocal = null;
+    FSDataInputStream fsIn = null;
+    byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
+    
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      FileSystem fs = cluster.getFileSystem();
+      DFSTestUtil.createFile(fs, TEST_PATH,
+          BlockReaderLocalTest.TEST_LENGTH, (short)1, RANDOM_SEED);
+      try {
+        DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
+      } catch (InterruptedException e) {
+        Assert.fail("unexpected InterruptedException during " +
+            "waitReplication: " + e);
+      } catch (TimeoutException e) {
+        Assert.fail("unexpected TimeoutException during " +
+            "waitReplication: " + e);
+      }
+      fsIn = fs.open(TEST_PATH);
+      IOUtils.readFully(fsIn, original, 0,
+          BlockReaderLocalTest.TEST_LENGTH);
+      fsIn.close();
+      fsIn = null;
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
+      File dataFile = MiniDFSCluster.getBlockFile(0, block);
+      File metaFile = MiniDFSCluster.getBlockMetadataFile(0, block);
 
-  /**
-   * Test that, in the case of an error, the position and limit of a ByteBuffer
-   * are left unchanged. This is not mandated by ByteBufferReadable, but clients
-   * of this class might immediately issue a retry on failure, so it's polite.
-   */
+      DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
+      cluster.shutdown();
+      cluster = null;
+      test.setup(dataFile, checksum);
+      dataIn = new FileInputStream(dataFile);
+      checkIn = new FileInputStream(metaFile);
+      blockReaderLocal = new BlockReaderLocal(conf,
+          TEST_PATH.getName(), block, 0, -1,
+          dataIn, checkIn, datanodeID, checksum);
+      dataIn = null;
+      checkIn = null;
+      test.doTest(blockReaderLocal, original);
+    } finally {
+      if (fsIn != null) fsIn.close();
+      if (cluster != null) cluster.shutdown();
+      if (dataIn != null) dataIn.close();
+      if (checkIn != null) checkIn.close();
+      if (blockReaderLocal != null) blockReaderLocal.close(null, null);
+    }
+  }
+  
+  private static class TestBlockReaderLocalImmediateClose 
+      implements BlockReaderLocalTest {
+    @Override
+    public void setup(File blockFile, boolean usingChecksums)
+        throws IOException { }
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[]) 
+        throws IOException { }
+  }
+  
   @Test
-  public void testStablePositionAfterCorruptRead() throws Exception {
-    final short REPL_FACTOR = 1;
-    final long FILE_LENGTH = 512L;
-    cluster.waitActive();
-    FileSystem fs = cluster.getFileSystem();
+  public void testBlockReaderLocalImmediateClose() throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true);
+    runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false);
+  }
+  
+  private static class TestBlockReaderSimpleReads 
+      implements BlockReaderLocalTest {
+    @Override
+    public void setup(File blockFile, boolean usingChecksums)
+        throws IOException { }
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[]) 
+        throws IOException {
+      byte buf[] = new byte[TEST_LENGTH];
+      reader.readFully(buf, 0, 512);
+      assertArrayRegionsEqual(original, 0, buf, 0, 512);
+      reader.readFully(buf, 512, 512);
+      assertArrayRegionsEqual(original, 512, buf, 512, 512);
+      reader.readFully(buf, 1024, 513);
+      assertArrayRegionsEqual(original, 1024, buf, 1024, 513);
+      reader.readFully(buf, 1537, 514);
+      assertArrayRegionsEqual(original, 1537, buf, 1537, 514);
+    }
+  }
+  
+  @Test
+  public void testBlockReaderSimpleReads() throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true);
+  }
 
-    Path path = new Path("/corrupted");
+  @Test
+  public void testBlockReaderSimpleReadsNoChecksum() throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false);
+  }
+  
+  private static class TestBlockReaderLocalArrayReads2 
+      implements BlockReaderLocalTest {
+    @Override
+    public void setup(File blockFile, boolean usingChecksums)
+        throws IOException { }
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[]) 
+        throws IOException {
+      byte buf[] = new byte[TEST_LENGTH];
+      reader.readFully(buf, 0, 10);
+      assertArrayRegionsEqual(original, 0, buf, 0, 10);
+      reader.readFully(buf, 10, 100);
+      assertArrayRegionsEqual(original, 10, buf, 10, 100);
+      reader.readFully(buf, 110, 700);
+      assertArrayRegionsEqual(original, 110, buf, 110, 700);
+      reader.readFully(buf, 810, 1); // from offset 810 to offset 811
+      reader.readFully(buf, 811, 5);
+      assertArrayRegionsEqual(original, 811, buf, 811, 5);
+      reader.readFully(buf, 816, 900); // skip from offset 816 to offset 1716
+      reader.readFully(buf, 1716, 5);
+      assertArrayRegionsEqual(original, 1716, buf, 1716, 5);
+    }
+  }
+  
+  @Test
+  public void testBlockReaderLocalArrayReads2() throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
+        true);
+  }
 
-    DFSTestUtil.createFile(fs, path, FILE_LENGTH, REPL_FACTOR, 12345L);
-    DFSTestUtil.waitReplication(fs, path, REPL_FACTOR);
+  @Test
+  public void testBlockReaderLocalArrayReads2NoChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
+        false);
+  }
 
-    ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, path);
-    int blockFilesCorrupted = cluster.corruptBlockOnDataNodes(block);
-    assertEquals("All replicas not corrupted", REPL_FACTOR, blockFilesCorrupted);
+  private static class TestBlockReaderLocalByteBufferReads 
+      implements BlockReaderLocalTest {
+    @Override
+    public void setup(File blockFile, boolean usingChecksums)
+        throws IOException { }
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[]) 
+        throws IOException {
+      ByteBuffer buf = ByteBuffer.wrap(new byte[TEST_LENGTH]);
+      readFully(reader, buf, 0, 10);
+      assertArrayRegionsEqual(original, 0, buf.array(), 0, 10);
+      readFully(reader, buf, 10, 100);
+      assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
+      readFully(reader, buf, 110, 700);
+      assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
+      reader.skip(1); // skip from offset 810 to offset 811
+      readFully(reader, buf, 811, 5);
+      assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
+    }
+  }
+  
+  @Test
+  public void testBlockReaderLocalByteBufferReads()
+      throws IOException {
+    runBlockReaderLocalTest(
+        new TestBlockReaderLocalByteBufferReads(), true);
+  }
 
-    FSDataInputStream dis = cluster.getFileSystem().open(path);
-    ByteBuffer buf = ByteBuffer.allocateDirect((int)FILE_LENGTH);
-    boolean sawException = false;
-    try {
-      dis.read(buf);
-    } catch (ChecksumException ex) {
-      sawException = true;
+  @Test
+  public void testBlockReaderLocalByteBufferReadsNoChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(
+        new TestBlockReaderLocalByteBufferReads(), false);
+  }
+  
+  private static class TestBlockReaderLocalReadCorruptStart
+      implements BlockReaderLocalTest {
+    boolean usingChecksums = false;
+    @Override
+    public void setup(File blockFile, boolean usingChecksums)
+        throws IOException {
+      RandomAccessFile bf = null;
+      this.usingChecksums = usingChecksums;
+      try {
+        bf = new RandomAccessFile(blockFile, "rw");
+        bf.write(new byte[] {0,0,0,0,0,0,0,0,0,0,0,0,0,0});
+      } finally {
+        if (bf != null) bf.close();
+      }
     }
 
-    assertTrue(sawException);
-    assertEquals(0, buf.position());
-    assertEquals(buf.capacity(), buf.limit());
+    public void doTest(BlockReaderLocal reader, byte original[]) 
+        throws IOException {
+      byte buf[] = new byte[TEST_LENGTH];
+      if (usingChecksums) {
+        try {
+          reader.readFully(buf, 0, 10);
+          Assert.fail("did not detect corruption");
+        } catch (IOException e) {
+          // expected
+        }
+      } else {
+        reader.readFully(buf, 0, 10);
+      }
+    }
+  }
+  
+  @Test
+  public void testBlockReaderLocalReadCorruptStart()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true);
+  }
+  
+  private static class TestBlockReaderLocalReadCorrupt
+      implements BlockReaderLocalTest {
+    boolean usingChecksums = false;
+    @Override
+    public void setup(File blockFile, boolean usingChecksums) 
+        throws IOException {
+      RandomAccessFile bf = null;
+      this.usingChecksums = usingChecksums;
+      try {
+        bf = new RandomAccessFile(blockFile, "rw");
+        bf.seek(1539);
+        bf.write(new byte[] {0,0,0,0,0,0,0,0,0,0,0,0,0,0});
+      } finally {
+        if (bf != null) bf.close();
+      }
+    }
 
-    dis = cluster.getFileSystem().open(path);
-    buf.position(3);
-    buf.limit(25);
-    sawException = false;
-    try {
-      dis.read(buf);
-    } catch (ChecksumException ex) {
-      sawException = true;
+    public void doTest(BlockReaderLocal reader, byte original[]) 
+        throws IOException {
+      byte buf[] = new byte[TEST_LENGTH];
+      try {
+        reader.readFully(buf, 0, 10);
+        assertArrayRegionsEqual(original, 0, buf, 0, 10);
+        reader.readFully(buf, 10, 100);
+        assertArrayRegionsEqual(original, 10, buf, 10, 100);
+        reader.readFully(buf, 110, 700);
+        assertArrayRegionsEqual(original, 110, buf, 110, 700);
+        reader.skip(1); // skip from offset 810 to offset 811
+        reader.readFully(buf, 811, 5);
+        assertArrayRegionsEqual(original, 811, buf, 811, 5);
+        reader.readFully(buf, 816, 900);
+        if (usingChecksums) {
+          // We should detect the corruption when using a checksum file.
+          Assert.fail("did not detect corruption");
+        }
+      } catch (ChecksumException e) {
+        if (!usingChecksums) {
+          Assert.fail("didn't expect to get ChecksumException: not " +
+              "using checksums.");
+        }
+      }
     }
+  }
 
-    assertTrue(sawException);
-    assertEquals(3, buf.position());
-    assertEquals(25, buf.limit());
+  @Test
+  public void testBlockReaderLocalReadCorrupt()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true);
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false);
   }
 }

+ 154 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java

@@ -0,0 +1,154 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+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.UserGroupInformation;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestBlockReaderLocalLegacy {
+  @BeforeClass
+  public static void setupCluster() throws IOException {
+    DFSInputStream.tcpReadsDisabledForTesting = true;
+    DomainSocket.disableBindPathValidation();
+  }
+  
+  private static HdfsConfiguration getConfiguration(
+      TemporarySocketDirectory socketDir) throws IOException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    if (socketDir == null) {
+      conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, "");
+    } else {
+      conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        new File(socketDir.getDir(), "TestBlockReaderLocalLegacy.%d.sock").
+          getAbsolutePath());
+    }
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+        false);
+    conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
+        UserGroupInformation.getCurrentUser().getShortUserName());
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
+    return conf;
+  }
+
+  /**
+   * Test that, in the case of an error, the position and limit of a ByteBuffer
+   * are left unchanged. This is not mandated by ByteBufferReadable, but clients
+   * of this class might immediately issue a retry on failure, so it's polite.
+   */
+  @Test
+  public void testStablePositionAfterCorruptRead() throws Exception {
+    final short REPL_FACTOR = 1;
+    final long FILE_LENGTH = 512L;
+    
+    HdfsConfiguration conf = getConfiguration(null);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    FileSystem fs = cluster.getFileSystem();
+
+    Path path = new Path("/corrupted");
+
+    DFSTestUtil.createFile(fs, path, FILE_LENGTH, REPL_FACTOR, 12345L);
+    DFSTestUtil.waitReplication(fs, path, REPL_FACTOR);
+
+    ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, path);
+    int blockFilesCorrupted = cluster.corruptBlockOnDataNodes(block);
+    assertEquals("All replicas not corrupted", REPL_FACTOR, blockFilesCorrupted);
+
+    FSDataInputStream dis = cluster.getFileSystem().open(path);
+    ByteBuffer buf = ByteBuffer.allocateDirect((int)FILE_LENGTH);
+    boolean sawException = false;
+    try {
+      dis.read(buf);
+    } catch (ChecksumException ex) {
+      sawException = true;
+    }
+
+    assertTrue(sawException);
+    assertEquals(0, buf.position());
+    assertEquals(buf.capacity(), buf.limit());
+
+    dis = cluster.getFileSystem().open(path);
+    buf.position(3);
+    buf.limit(25);
+    sawException = false;
+    try {
+      dis.read(buf);
+    } catch (ChecksumException ex) {
+      sawException = true;
+    }
+
+    assertTrue(sawException);
+    assertEquals(3, buf.position());
+    assertEquals(25, buf.limit());
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testBothOldAndNewShortCircuitConfigured() throws Exception {
+    final short REPL_FACTOR = 1;
+    final int FILE_LENGTH = 512;
+    Assume.assumeTrue(null == DomainSocket.getLoadingFailureReason());
+    TemporarySocketDirectory socketDir = new TemporarySocketDirectory();
+    HdfsConfiguration conf = getConfiguration(socketDir);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    socketDir.close();
+    FileSystem fs = cluster.getFileSystem();
+
+    Path path = new Path("/foo");
+    byte orig[] = new byte[FILE_LENGTH];
+    for (int i = 0; i < orig.length; i++) {
+      orig[i] = (byte)(i%10);
+    }
+    FSDataOutputStream fos = fs.create(path, (short)1);
+    fos.write(orig);
+    fos.close();
+    DFSTestUtil.waitReplication(fs, path, REPL_FACTOR);
+    FSDataInputStream fis = cluster.getFileSystem().open(path);
+    byte buf[] = new byte[FILE_LENGTH];
+    IOUtils.readFully(fis, buf, 0, FILE_LENGTH);
+    fis.close();
+    Assert.assertArrayEquals(orig, buf);
+    Arrays.equals(orig, buf);
+    cluster.shutdown();
+  }
+}

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java

@@ -61,7 +61,7 @@ public class TestClientBlockVerification {
         util.getBlockReader(testBlock, 0, FILE_SIZE_K * 1024));
     util.readAndCheckEOS(reader, FILE_SIZE_K * 1024, true);
     verify(reader).sendReadResult(Status.CHECKSUM_OK);
-    reader.close();
+    reader.close(null, null);
   }
 
   /**
@@ -76,7 +76,7 @@ public class TestClientBlockVerification {
     // We asked the blockreader for the whole file, and only read
     // half of it, so no CHECKSUM_OK
     verify(reader, never()).sendReadResult(Status.CHECKSUM_OK);
-    reader.close();
+    reader.close(null, null);
   }
 
   /**
@@ -92,7 +92,7 @@ public class TestClientBlockVerification {
     // And read half the file
     util.readAndCheckEOS(reader, FILE_SIZE_K * 1024 / 2, true);
     verify(reader).sendReadResult(Status.CHECKSUM_OK);
-    reader.close();
+    reader.close(null, null);
   }
 
   /**
@@ -111,7 +111,7 @@ public class TestClientBlockVerification {
             util.getBlockReader(testBlock, startOffset, length));
         util.readAndCheckEOS(reader, length, true);
         verify(reader).sendReadResult(Status.CHECKSUM_OK);
-        reader.close();
+        reader.close(null, null);
       }
     }
   }

+ 29 - 160
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java

@@ -18,28 +18,20 @@
 package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.security.PrivilegedExceptionAction;
+
+import junit.framework.Assert;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.security.token.Token;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Matchers;
 import org.mockito.Mockito;
@@ -55,59 +47,31 @@ public class TestConnCache {
 
   static final int BLOCK_SIZE = 4096;
   static final int FILE_SIZE = 3 * BLOCK_SIZE;
-  final static int CACHE_SIZE = 4;
-  final static long CACHE_EXPIRY_MS = 200;
-  static Configuration conf = null;
-  static MiniDFSCluster cluster = null;
-  static FileSystem fs = null;
-  static SocketCache cache;
-
-  static final Path testFile = new Path("/testConnCache.dat");
-  static byte authenticData[] = null;
-
-  static BlockReaderTestUtil util = null;
-
 
   /**
    * A mock Answer to remember the BlockReader used.
    *
    * It verifies that all invocation to DFSInputStream.getBlockReader()
-   * use the same socket.
+   * use the same peer.
    */
   private class MockGetBlockReader implements Answer<RemoteBlockReader2> {
     public RemoteBlockReader2 reader = null;
-    private Socket sock = null;
+    private Peer peer = null;
 
     @Override
     public RemoteBlockReader2 answer(InvocationOnMock invocation) throws Throwable {
       RemoteBlockReader2 prevReader = reader;
       reader = (RemoteBlockReader2) invocation.callRealMethod();
-      if (sock == null) {
-        sock = reader.dnSock;
+      if (peer == null) {
+        peer = reader.getPeer();
       } else if (prevReader != null) {
-        assertSame("DFSInputStream should use the same socket",
-                   sock, reader.dnSock);
+        Assert.assertSame("DFSInputStream should use the same peer",
+                   peer, reader.getPeer());
       }
       return reader;
     }
   }
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    final int REPLICATION_FACTOR = 1;
-
-    /* create a socket cache. There is only one socket cache per jvm */
-    cache = SocketCache.getInstance(CACHE_SIZE, CACHE_EXPIRY_MS);
-
-    util = new BlockReaderTestUtil(REPLICATION_FACTOR);
-    cluster = util.getCluster();
-    conf = util.getConf();
-    fs = cluster.getFileSystem();
-
-    authenticData = util.writeFile(testFile, FILE_SIZE / 1024);
-  }
-
-
   /**
    * (Optionally) seek to position, read and verify data.
    *
@@ -117,9 +81,10 @@ public class TestConnCache {
                      long pos,
                      byte[] buffer,
                      int offset,
-                     int length)
+                     int length,
+                     byte[] authenticData)
       throws IOException {
-    assertTrue("Test buffer too small", buffer.length >= offset + length);
+    Assert.assertTrue("Test buffer too small", buffer.length >= offset + length);
 
     if (pos >= 0)
       in.seek(pos);
@@ -129,7 +94,7 @@ public class TestConnCache {
 
     while (length > 0) {
       int cnt = in.read(buffer, offset, length);
-      assertTrue("Error in read", cnt > 0);
+      Assert.assertTrue("Error in read", cnt > 0);
       offset += cnt;
       length -= cnt;
     }
@@ -144,116 +109,23 @@ public class TestConnCache {
     }
   }
 
-  /**
-   * Test the SocketCache itself.
-   */
-  @Test
-  public void testSocketCache() throws Exception {
-    // Make a client
-    InetSocketAddress nnAddr =
-        new InetSocketAddress("localhost", cluster.getNameNodePort());
-    DFSClient client = new DFSClient(nnAddr, conf);
-
-    // Find out the DN addr
-    LocatedBlock block =
-        client.getNamenode().getBlockLocations(
-            testFile.toString(), 0, FILE_SIZE)
-        .getLocatedBlocks().get(0);
-    DataNode dn = util.getDataNode(block);
-    InetSocketAddress dnAddr = dn.getXferAddress();
-
-
-    // Make some sockets to the DN
-    Socket[] dnSockets = new Socket[CACHE_SIZE];
-    for (int i = 0; i < dnSockets.length; ++i) {
-      dnSockets[i] = client.socketFactory.createSocket(
-          dnAddr.getAddress(), dnAddr.getPort());
-    }
-
-
-    // Insert a socket to the NN
-    Socket nnSock = new Socket(nnAddr.getAddress(), nnAddr.getPort());
-    cache.put(nnSock, null);
-    assertSame("Read the write", nnSock, cache.get(nnAddr).sock);
-    cache.put(nnSock, null);
-
-    // Insert DN socks
-    for (Socket dnSock : dnSockets) {
-      cache.put(dnSock, null);
-    }
-
-    assertEquals("NN socket evicted", null, cache.get(nnAddr));
-    assertTrue("Evicted socket closed", nnSock.isClosed());
- 
-    // Lookup the DN socks
-    for (Socket dnSock : dnSockets) {
-      assertEquals("Retrieve cached sockets", dnSock, cache.get(dnAddr).sock);
-      dnSock.close();
-    }
-
-    assertEquals("Cache is empty", 0, cache.size());
-  }
-
-
-  /**
-   * Test the SocketCache expiry.
-   * Verify that socket cache entries expire after the set
-   * expiry time.
-   */
-  @Test
-  public void testSocketCacheExpiry() throws Exception {
-    // Make a client
-    InetSocketAddress nnAddr =
-        new InetSocketAddress("localhost", cluster.getNameNodePort());
-    DFSClient client = new DFSClient(nnAddr, conf);
-
-    // Find out the DN addr
-    LocatedBlock block =
-        client.getNamenode().getBlockLocations(
-            testFile.toString(), 0, FILE_SIZE)
-        .getLocatedBlocks().get(0);
-    DataNode dn = util.getDataNode(block);
-    InetSocketAddress dnAddr = dn.getXferAddress();
-
-
-    // Make some sockets to the DN and put in cache
-    Socket[] dnSockets = new Socket[CACHE_SIZE];
-    for (int i = 0; i < dnSockets.length; ++i) {
-      dnSockets[i] = client.socketFactory.createSocket(
-          dnAddr.getAddress(), dnAddr.getPort());
-      cache.put(dnSockets[i], null);
-    }
-
-    // Client side still has the sockets cached
-    assertEquals(CACHE_SIZE, client.socketCache.size());
-
-    //sleep for a second and see if it expired
-    Thread.sleep(CACHE_EXPIRY_MS + 1000);
-    
-    // Client side has no sockets cached
-    assertEquals(0, client.socketCache.size());
-
-    //sleep for another second and see if 
-    //the daemon thread runs fine on empty cache
-    Thread.sleep(CACHE_EXPIRY_MS + 1000);
-  }
-
-
   /**
    * Read a file served entirely from one DN. Seek around and read from
    * different offsets. And verify that they all use the same socket.
-   *
-   * @throws java.io.IOException
+   * @throws Exception 
    */
   @Test
   @SuppressWarnings("unchecked")
-  public void testReadFromOneDN() throws IOException {
-    LOG.info("Starting testReadFromOneDN()");
+  public void testReadFromOneDN() throws Exception {
+    BlockReaderTestUtil util = new BlockReaderTestUtil(1,
+        new HdfsConfiguration());
+    final Path testFile = new Path("/testConnCache.dat");
+    byte authenticData[] = util.writeFile(testFile, FILE_SIZE / 1024);
     DFSClient client = new DFSClient(
-        new InetSocketAddress("localhost", cluster.getNameNodePort()), conf);
-    DFSInputStream in = spy(client.open(testFile.toString()));
+        new InetSocketAddress("localhost",
+            util.getCluster().getNameNodePort()), util.getConf());
+    DFSInputStream in = Mockito.spy(client.open(testFile.toString()));
     LOG.info("opened " + testFile.toString());
-
     byte[] dataBuf = new byte[BLOCK_SIZE];
 
     MockGetBlockReader answer = new MockGetBlockReader();
@@ -270,18 +142,15 @@ public class TestConnCache {
                            Matchers.anyString());
 
     // Initial read
-    pread(in, 0, dataBuf, 0, dataBuf.length);
+    pread(in, 0, dataBuf, 0, dataBuf.length, authenticData);
     // Read again and verify that the socket is the same
-    pread(in, FILE_SIZE - dataBuf.length, dataBuf, 0, dataBuf.length);
-    pread(in, 1024, dataBuf, 0, dataBuf.length);
-    pread(in, -1, dataBuf, 0, dataBuf.length);            // No seek; just read
-    pread(in, 64, dataBuf, 0, dataBuf.length / 2);
+    pread(in, FILE_SIZE - dataBuf.length, dataBuf, 0, dataBuf.length,
+        authenticData);
+    pread(in, 1024, dataBuf, 0, dataBuf.length, authenticData);
+    // No seek; just read
+    pread(in, -1, dataBuf, 0, dataBuf.length, authenticData);
+    pread(in, 64, dataBuf, 0, dataBuf.length / 2, authenticData);
 
     in.close();
   }
-
-  @AfterClass
-  public static void teardownCluster() throws Exception {
-    util.shutdown();
-  }
 }

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

@@ -53,7 +53,7 @@ import com.google.common.base.Joiner;
 */
 public class TestDFSUpgrade {
  
-  private static final int EXPECTED_TXID = 49;
+  private static final int EXPECTED_TXID = 45;
   private static final Log LOG = LogFactory.getLog(TestDFSUpgrade.class.getName());
   private Configuration conf;
   private int testCounter = 0;

+ 10 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -70,7 +71,7 @@ public class TestDataTransferKeepalive {
       .numDataNodes(1).build();
     fs = cluster.getFileSystem();
     dfsClient = ((DistributedFileSystem)fs).dfs;
-    dfsClient.socketCache.clear();
+    dfsClient.peerCache.clear();
 
     String poolId = cluster.getNamesystem().getBlockPoolId();
     dn = cluster.getDataNodes().get(0);
@@ -93,13 +94,13 @@ public class TestDataTransferKeepalive {
     DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
 
     // Clients that write aren't currently re-used.
-    assertEquals(0, dfsClient.socketCache.size());
+    assertEquals(0, dfsClient.peerCache.size());
     assertXceiverCount(0);
 
     // Reads the file, so we should get a
     // cached socket, and should have an xceiver on the other side.
     DFSTestUtil.readFile(fs, TEST_FILE);
-    assertEquals(1, dfsClient.socketCache.size());
+    assertEquals(1, dfsClient.peerCache.size());
     assertXceiverCount(1);
 
     // Sleep for a bit longer than the keepalive timeout
@@ -110,13 +111,13 @@ public class TestDataTransferKeepalive {
     // The socket is still in the cache, because we don't
     // notice that it's closed until we try to read
     // from it again.
-    assertEquals(1, dfsClient.socketCache.size());
+    assertEquals(1, dfsClient.peerCache.size());
     
     // Take it out of the cache - reading should
     // give an EOF.
-    Socket s = dfsClient.socketCache.get(dnAddr).sock;
-    assertNotNull(s);
-    assertEquals(-1, NetUtils.getInputStream(s).read());
+    Peer peer = dfsClient.peerCache.get(dn.getDatanodeId(), false);
+    assertNotNull(peer);
+    assertEquals(-1, peer.getInputStream().read());
   }
 
   /**
@@ -175,14 +176,14 @@ public class TestDataTransferKeepalive {
     }
     
     DFSClient client = ((DistributedFileSystem)fs).dfs;
-    assertEquals(5, client.socketCache.size());
+    assertEquals(5, client.peerCache.size());
     
     // Let all the xceivers timeout
     Thread.sleep(1500);
     assertXceiverCount(0);
 
     // Client side still has the sockets cached
-    assertEquals(5, client.socketCache.size());
+    assertEquals(5, client.peerCache.size());
 
     // Reading should not throw an exception.
     DFSTestUtil.readFile(fs, TEST_FILE);

+ 62 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java

@@ -0,0 +1,62 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+/**
+ * This class tests disabling client connection caching in a single node
+ * mini-cluster.
+ */
+public class TestDisableConnCache {
+  static final Log LOG = LogFactory.getLog(TestDisableConnCache.class);
+
+  static final int BLOCK_SIZE = 4096;
+  static final int FILE_SIZE = 3 * BLOCK_SIZE;
+  
+  /**
+   * Test that the socket cache can be disabled by setting the capacity to
+   * 0. Regression test for HDFS-3365.
+   * @throws Exception 
+   */
+  @Test
+  public void testDisableCache() throws Exception {
+    HdfsConfiguration confWithoutCache = new HdfsConfiguration();
+    // Configure a new instance with no peer caching, ensure that it doesn't
+    // cache anything
+    confWithoutCache.setInt(
+        DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 0);
+    BlockReaderTestUtil util = new BlockReaderTestUtil(1, confWithoutCache);
+    final Path testFile = new Path("/testConnCache.dat");
+    util.writeFile(testFile, FILE_SIZE / 1024);
+    FileSystem fsWithoutCache = FileSystem.newInstance(util.getConf());
+    try {
+      DFSTestUtil.readFile(fsWithoutCache, testFile);
+      assertEquals(0, ((DistributedFileSystem)fsWithoutCache).dfs.peerCache.size());
+    } finally {
+      fsWithoutCache.close();
+      util.shutdown();
+    }
+  }
+}

+ 127 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileInputStreamCache.java

@@ -0,0 +1,127 @@
+/**
+ * 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.FileOutputStream;
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.junit.Test;
+
+public class TestFileInputStreamCache {
+  static final Log LOG = LogFactory.getLog(TestFileInputStreamCache.class);
+
+  @Test
+  public void testCreateAndDestroy() throws Exception {
+    FileInputStreamCache cache = new FileInputStreamCache(10, 1000);
+    cache.close();
+  }
+  
+  private static class TestFileDescriptorPair {
+    TemporarySocketDirectory dir = new TemporarySocketDirectory();
+    FileInputStream fis[];
+
+    public TestFileDescriptorPair() throws IOException {
+      fis = new FileInputStream[2];
+      for (int i = 0; i < 2; i++) {
+        String name = dir.getDir() + "/file" + i;
+        FileOutputStream fos = new FileOutputStream(name);
+        fos.write(1);
+        fos.close();
+        fis[i] = new FileInputStream(name);
+      }
+    }
+
+    public FileInputStream[] getFileInputStreams() {
+      return fis;
+    }
+
+    public void close() throws IOException {
+      IOUtils.cleanup(LOG, fis);
+      dir.close();
+    }
+
+    public boolean compareWith(FileInputStream other[]) {
+      if ((other == null) || (fis == null)) {
+        return other == fis;
+      }
+      if (fis.length != other.length) return false;
+      for (int i = 0; i < fis.length; i++) {
+        if (fis[i] != other[i]) return false;
+      }
+      return true;
+    }
+  }
+
+  @Test
+  public void testAddAndRetrieve() throws Exception {
+    FileInputStreamCache cache = new FileInputStreamCache(1, 1000000);
+    DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
+        "xyzzy", 8080, 9090, 7070);
+    ExtendedBlock block = new ExtendedBlock("poolid", 123);
+    TestFileDescriptorPair pair = new TestFileDescriptorPair();
+    cache.put(dnId, block, pair.getFileInputStreams());
+    FileInputStream fis[] = cache.get(dnId, block);
+    Assert.assertTrue(pair.compareWith(fis));
+    pair.close();
+    cache.close();
+  }
+
+  @Test
+  public void testExpiry() throws Exception {
+    FileInputStreamCache cache = new FileInputStreamCache(1, 10);
+    DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
+        "xyzzy", 8080, 9090, 7070);
+    ExtendedBlock block = new ExtendedBlock("poolid", 123);
+    TestFileDescriptorPair pair = new TestFileDescriptorPair();
+    cache.put(dnId, block, pair.getFileInputStreams());
+    Thread.sleep(cache.getExpiryTimeMs() * 100);
+    FileInputStream fis[] = cache.get(dnId, block);
+    Assert.assertNull(fis);
+    pair.close();
+    cache.close();
+  }
+
+  @Test
+  public void testEviction() throws Exception {
+    FileInputStreamCache cache = new FileInputStreamCache(1, 10000000);
+    DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
+        "xyzzy", 8080, 9090, 7070);
+    ExtendedBlock block = new ExtendedBlock("poolid", 123);
+    TestFileDescriptorPair pair = new TestFileDescriptorPair();
+    cache.put(dnId, block, pair.getFileInputStreams());
+    DatanodeID dnId2 = new DatanodeID("127.0.0.1", "localhost", 
+        "xyzzy", 8081, 9091, 7071);
+    TestFileDescriptorPair pair2 = new TestFileDescriptorPair();
+    cache.put(dnId2, block, pair2.getFileInputStreams());
+    FileInputStream fis[] = cache.get(dnId, block);
+    Assert.assertNull(fis);
+    FileInputStream fis2[] = cache.get(dnId2, block);
+    Assert.assertTrue(pair2.compareWith(fis2));
+    pair.close();
+    cache.close();
+  }
+}

+ 11 - 34
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java

@@ -17,53 +17,30 @@
  */
 package org.apache.hadoop.hdfs;
 
-import java.io.IOException;
-
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Test;
 
 public class TestParallelRead extends TestParallelReadUtil {
-
   @BeforeClass
   static public void setupCluster() throws Exception {
-    setupCluster(DEFAULT_REPLICATION_FACTOR, new HdfsConfiguration());
+    // This is a test of the normal (TCP) read path.  For this reason, we turn
+    // off both short-circuit local reads and UNIX domain socket data traffic.
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+                    false);
+    // dfs.domain.socket.path should be ignored because the previous two keys
+    // were set to false.  This is a regression test for HDFS-4473.
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, "/will/not/be/created");
+
+    setupCluster(DEFAULT_REPLICATION_FACTOR, conf);
   }
 
   @AfterClass
   static public void teardownCluster() throws Exception {
     TestParallelReadUtil.teardownCluster();
   }
-
-  /**
-   * Do parallel read several times with different number of files and threads.
-   *
-   * Note that while this is the only "test" in a junit sense, we're actually
-   * dispatching a lot more. Failures in the other methods (and other threads)
-   * need to be manually collected, which is inconvenient.
-   */
-  @Test
-  public void testParallelReadCopying() throws IOException {
-    runTestWorkload(new CopyingReadWorkerHelper());
-  }
-
-  @Test
-  public void testParallelReadByteBuffer() throws IOException {
-    runTestWorkload(new DirectReadWorkerHelper());
-  }
-
-  @Test
-  public void testParallelReadMixed() throws IOException {
-    runTestWorkload(new MixedWorkloadHelper());
-  }
-  
-  @Test
-  public void testParallelNoChecksums() throws IOException {
-    verifyChecksums = false;
-    runTestWorkload(new MixedWorkloadHelper());
-  }
-
 }

+ 35 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java

@@ -32,12 +32,18 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
 
 /**
  * Driver class for testing the use of DFSInputStream by multiple concurrent
- * readers, using the different read APIs. See subclasses for the actual test
- * cases.
+ * readers, using the different read APIs.
+ *
+ * This class is marked as @Ignore so that junit doesn't try to execute the
+ * tests in here directly.  They are executed from subclasses.
  */
+@Ignore
 public class TestParallelReadUtil {
 
   static final Log LOG = LogFactory.getLog(TestParallelReadUtil.class);
@@ -388,4 +394,31 @@ public class TestParallelReadUtil {
     util.shutdown();
   }
 
+  /**
+   * Do parallel read several times with different number of files and threads.
+   *
+   * Note that while this is the only "test" in a junit sense, we're actually
+   * dispatching a lot more. Failures in the other methods (and other threads)
+   * need to be manually collected, which is inconvenient.
+   */
+  @Test
+  public void testParallelReadCopying() throws IOException {
+    runTestWorkload(new CopyingReadWorkerHelper());
+  }
+
+  @Test
+  public void testParallelReadByteBuffer() throws IOException {
+    runTestWorkload(new DirectReadWorkerHelper());
+  }
+
+  @Test
+  public void testParallelReadMixed() throws IOException {
+    runTestWorkload(new MixedWorkloadHelper());
+  }
+  
+  @Test
+  public void testParallelNoChecksums() throws IOException {
+    verifyChecksums = false;
+    runTestWorkload(new MixedWorkloadHelper());
+  }
 }

+ 9 - 31
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelLocalRead.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitLegacyRead.java

@@ -17,25 +17,25 @@
  */
 package org.apache.hadoop.hdfs;
 
-import java.io.IOException;
-
+import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestParallelLocalRead extends TestParallelReadUtil {
 
+public class TestParallelShortCircuitLegacyRead extends TestParallelReadUtil {
   @BeforeClass
   static public void setupCluster() throws Exception {
+    DFSInputStream.tcpReadsDisabledForTesting = true;
     HdfsConfiguration conf = new HdfsConfiguration();
-
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, "");
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
-        false);
+    conf.setBoolean(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
     conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
         UserGroupInformation.getCurrentUser().getShortUserName());
-
+    DomainSocket.disableBindPathValidation();
     setupCluster(1, conf);
   }
 
@@ -43,26 +43,4 @@ public class TestParallelLocalRead extends TestParallelReadUtil {
   static public void teardownCluster() throws Exception {
     TestParallelReadUtil.teardownCluster();
   }
-
-  /**
-   * Do parallel read several times with different number of files and threads.
-   *
-   * Note that while this is the only "test" in a junit sense, we're actually
-   * dispatching a lot more. Failures in the other methods (and other threads)
-   * need to be manually collected, which is inconvenient.
-   */
-  @Test
-  public void testParallelReadCopying() throws IOException {
-    runTestWorkload(new CopyingReadWorkerHelper());
-  }
-
-  @Test
-  public void testParallelReadByteBuffer() throws IOException {
-    runTestWorkload(new DirectReadWorkerHelper());
-  }
-
-  @Test
-  public void testParallelReadMixed() throws IOException {
-    runTestWorkload(new MixedWorkloadHelper());
-  }
 }

+ 59 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitRead.java

@@ -0,0 +1,59 @@
+/**
+ * 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.File;
+
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import static org.hamcrest.CoreMatchers.*;
+
+public class TestParallelShortCircuitRead extends TestParallelReadUtil {
+  private static TemporarySocketDirectory sockDir;
+
+  @BeforeClass
+  static public void setupCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
+    DFSInputStream.tcpReadsDisabledForTesting = true;
+    sockDir = new TemporarySocketDirectory();
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+      new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    DomainSocket.disableBindPathValidation();
+    setupCluster(1, conf);
+  }
+
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+
+  @AfterClass
+  static public void teardownCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
+    sockDir.close();
+    TestParallelReadUtil.teardownCluster();
+  }
+}

+ 59 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadNoChecksum.java

@@ -0,0 +1,59 @@
+/**
+ * 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.File;
+
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import static org.hamcrest.CoreMatchers.*;
+
+public class TestParallelShortCircuitReadNoChecksum extends TestParallelReadUtil {
+  private static TemporarySocketDirectory sockDir;
+
+  @BeforeClass
+  static public void setupCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
+    DFSInputStream.tcpReadsDisabledForTesting = true;
+    sockDir = new TemporarySocketDirectory();
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+      new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
+    DomainSocket.disableBindPathValidation();
+    setupCluster(1, conf);
+  }
+
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+
+  @AfterClass
+  static public void teardownCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
+    sockDir.close();
+    TestParallelReadUtil.teardownCluster();
+  }
+}

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java

@@ -0,0 +1,73 @@
+/**
+ * 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.File;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import static org.hamcrest.CoreMatchers.*;
+
+/**
+ * This class tests short-circuit local reads without any FileInputStream or
+ * Socket caching.  This is a regression test for HDFS-4417.
+ */
+public class TestParallelShortCircuitReadUnCached extends TestParallelReadUtil {
+  private static TemporarySocketDirectory sockDir;
+
+  @BeforeClass
+  static public void setupCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
+    sockDir = new TemporarySocketDirectory();
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+      new File(sockDir.getDir(), 
+        "TestParallelShortCircuitReadUnCached._PORT.sock").getAbsolutePath());
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.setBoolean(DFSConfigKeys.
+        DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
+    // We want to test reading from stale sockets.
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY, 1);
+    conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY,
+        5 * 60 * 1000);
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 32);
+    // Avoid using the FileInputStreamCache.
+    conf.setInt(DFSConfigKeys.
+        DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY, 0);
+    DomainSocket.disableBindPathValidation();
+    DFSInputStream.tcpReadsDisabledForTesting = true;
+    setupCluster(1, conf);
+  }
+
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+
+  @AfterClass
+  static public void teardownCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
+    sockDir.close();
+    TestParallelReadUtil.teardownCluster();
+  }
+}

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelUnixDomainRead.java

@@ -0,0 +1,58 @@
+/**
+ * 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.File;
+
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import static org.hamcrest.CoreMatchers.*;
+
+public class TestParallelUnixDomainRead extends TestParallelReadUtil {
+  private static TemporarySocketDirectory sockDir;
+
+  @BeforeClass
+  static public void setupCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
+    DFSInputStream.tcpReadsDisabledForTesting = true;
+    sockDir = new TemporarySocketDirectory();
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+      new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
+    DomainSocket.disableBindPathValidation();
+    setupCluster(1, conf);
+  }
+
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+
+  @AfterClass
+  static public void teardownCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
+    sockDir.close();
+    TestParallelReadUtil.teardownCluster();
+  }
+}

+ 288 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java

@@ -0,0 +1,288 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.channels.ReadableByteChannel;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.collect.HashMultiset;
+
+public class TestPeerCache {
+  static final Log LOG = LogFactory.getLog(TestPeerCache.class);
+
+  private static class FakePeer implements Peer {
+    private boolean closed = false;
+    private final boolean hasDomain;
+
+    private DatanodeID dnId;
+
+    public FakePeer(DatanodeID dnId, boolean hasDomain) {
+      this.dnId = dnId;
+      this.hasDomain = hasDomain;
+    }
+
+    @Override
+    public ReadableByteChannel getInputStreamChannel() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setReadTimeout(int timeoutMs) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getReceiveBufferSize() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean getTcpNoDelay() throws IOException {
+      return false;
+    }
+
+    @Override
+    public void setWriteTimeout(int timeoutMs) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean isClosed() {
+      return closed;
+    }
+  
+    @Override
+    public void close() throws IOException {
+      closed = true;
+    }
+
+    @Override
+    public String getRemoteAddressString() {
+      return dnId.getInfoAddr();
+    }
+
+    @Override
+    public String getLocalAddressString() {
+      return "127.0.0.1:123";
+    }
+
+    @Override
+    public InputStream getInputStream() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+  
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+  
+    @Override
+    public boolean isLocal() {
+      return true;
+    }
+  
+    @Override
+    public String toString() {
+      return "FakePeer(dnId=" + dnId + ")";
+    }
+
+    @Override
+    public DomainSocket getDomainSocket() {
+      if (!hasDomain) return null;
+      // Return a mock which throws an exception whenever any function is
+      // called.
+      return Mockito.mock(DomainSocket.class,
+          new Answer<Object>() {
+            @Override
+            public Object answer(InvocationOnMock invocation)
+                throws Throwable {
+              throw new RuntimeException("injected fault.");
+          } });
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof FakePeer)) return false;
+      FakePeer other = (FakePeer)o;
+      return hasDomain == other.hasDomain &&
+          dnId.equals(other.dnId);
+    }
+
+    @Override
+    public int hashCode() {
+      return dnId.hashCode() ^ (hasDomain ? 1 : 0);
+    }
+  }
+
+  @Test
+  public void testAddAndRetrieve() throws Exception {
+    PeerCache cache = new PeerCache(3, 100000);
+    DatanodeID dnId = new DatanodeID("192.168.0.1",
+          "fakehostname", "fake_storage_id",
+          100, 101, 102);
+    FakePeer peer = new FakePeer(dnId, false);
+    cache.put(dnId, peer);
+    assertTrue(!peer.isClosed());
+    assertEquals(1, cache.size());
+    assertEquals(peer, cache.get(dnId, false));
+    assertEquals(0, cache.size());
+    cache.close();
+  }
+
+  @Test
+  public void testExpiry() throws Exception {
+    final int CAPACITY = 3;
+    final int EXPIRY_PERIOD = 10;
+    PeerCache cache = new PeerCache(CAPACITY, EXPIRY_PERIOD);
+    DatanodeID dnIds[] = new DatanodeID[CAPACITY];
+    FakePeer peers[] = new FakePeer[CAPACITY];
+    for (int i = 0; i < CAPACITY; ++i) {
+      dnIds[i] = new DatanodeID("192.168.0.1",
+          "fakehostname_" + i, "fake_storage_id",
+          100, 101, 102);
+      peers[i] = new FakePeer(dnIds[i], false);
+    }
+    for (int i = 0; i < CAPACITY; ++i) {
+      cache.put(dnIds[i], peers[i]);
+    }
+
+    // Wait for the peers to expire
+    Thread.sleep(EXPIRY_PERIOD * 50);
+    assertEquals(0, cache.size());
+
+    // make sure that the peers were closed when they were expired
+    for (int i = 0; i < CAPACITY; ++i) {
+      assertTrue(peers[i].isClosed());
+    }
+
+    // sleep for another second and see if 
+    // the daemon thread runs fine on empty cache
+    Thread.sleep(EXPIRY_PERIOD * 50);
+    cache.close();
+  }
+
+  @Test
+  public void testEviction() throws Exception {
+    final int CAPACITY = 3;
+    PeerCache cache = new PeerCache(CAPACITY, 100000);
+    DatanodeID dnIds[] = new DatanodeID[CAPACITY + 1];
+    FakePeer peers[] = new FakePeer[CAPACITY + 1];
+    for (int i = 0; i < dnIds.length; ++i) {
+      dnIds[i] = new DatanodeID("192.168.0.1",
+          "fakehostname_" + i, "fake_storage_id_" + i,
+          100, 101, 102);
+      peers[i] = new FakePeer(dnIds[i], false);
+    }
+    for (int i = 0; i < CAPACITY; ++i) {
+      cache.put(dnIds[i], peers[i]);
+    }
+    // Check that the peers are cached
+    assertEquals(CAPACITY, cache.size());
+
+    // Add another entry and check that the first entry was evicted
+    cache.put(dnIds[CAPACITY], peers[CAPACITY]);
+    assertEquals(CAPACITY, cache.size());
+    assertSame(null, cache.get(dnIds[0], false));
+
+    // Make sure that the other entries are still there
+    for (int i = 1; i < CAPACITY; ++i) {
+      Peer peer = cache.get(dnIds[i], false);
+      assertSame(peers[i], peer);
+      assertTrue(!peer.isClosed());
+      peer.close();
+    }
+    assertEquals(1, cache.size());
+    cache.close();
+  }
+
+  @Test
+  public void testMultiplePeersWithSameKey() throws Exception {
+    final int CAPACITY = 3;
+    PeerCache cache = new PeerCache(CAPACITY, 100000);
+    DatanodeID dnId = new DatanodeID("192.168.0.1",
+          "fakehostname", "fake_storage_id",
+          100, 101, 102);
+    HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
+    for (int i = 0; i < CAPACITY; ++i) {
+      FakePeer peer = new FakePeer(dnId, false);
+      peers.add(peer);
+      cache.put(dnId, peer);
+    }
+    // Check that all of the peers ended up in the cache
+    assertEquals(CAPACITY, cache.size());
+    while (!peers.isEmpty()) {
+      Peer peer = cache.get(dnId, false);
+      assertTrue(peer != null);
+      assertTrue(!peer.isClosed());
+      peers.remove(peer);
+    }
+    assertEquals(0, cache.size());
+    cache.close();
+  }
+
+  @Test
+  public void testDomainSocketPeers() throws Exception {
+    final int CAPACITY = 3;
+    PeerCache cache = new PeerCache(CAPACITY, 100000);
+    DatanodeID dnId = new DatanodeID("192.168.0.1",
+          "fakehostname", "fake_storage_id",
+          100, 101, 102);
+    HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
+    for (int i = 0; i < CAPACITY; ++i) {
+      FakePeer peer = new FakePeer(dnId, i == CAPACITY - 1);
+      peers.add(peer);
+      cache.put(dnId, peer);
+    }
+    // Check that all of the peers ended up in the cache
+    assertEquals(CAPACITY, cache.size());
+    // Test that get(requireDomainPeer=true) finds the peer with the 
+    // domain socket.
+    Peer peer = cache.get(dnId, true);
+    assertTrue(peer.getDomainSocket() != null);
+    peers.remove(peer);
+    // Test that get(requireDomainPeer=true) returns null when there are
+    // no more peers with domain sockets.
+    peer = cache.get(dnId, true);
+    assertTrue(peer == null);
+    // Check that all of the other peers ended up in the cache.
+    while (!peers.isEmpty()) {
+      peer = cache.get(dnId, false);
+      assertTrue(peer != null);
+      assertTrue(!peer.isClosed());
+      peers.remove(peer);
+    }
+    assertEquals(0, cache.size());
+    cache.close();
+  }
+}

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java

@@ -99,7 +99,7 @@ public class TestSafeMode {
    */
   @Test
   public void testManualSafeMode() throws IOException {      
-    fs = (DistributedFileSystem)cluster.getFileSystem();
+    fs = cluster.getFileSystem();
     Path file1 = new Path("/tmp/testManualSafeMode/file1");
     Path file2 = new Path("/tmp/testManualSafeMode/file2");
     
@@ -112,7 +112,7 @@ public class TestSafeMode {
     // now bring up just the NameNode.
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
     cluster.waitActive();
-    dfs = (DistributedFileSystem)cluster.getFileSystem();
+    dfs = cluster.getFileSystem();
     
     assertTrue("No datanode is started. Should be in SafeMode", 
                dfs.setSafeMode(SafeModeAction.SAFEMODE_GET));
@@ -322,11 +322,11 @@ public class TestSafeMode {
         fs.rename(file1, new Path("file2"));
       }});
 
-    try {
-      fs.setTimes(file1, 0, 0);
-    } catch (IOException ioe) {
-      fail("Set times failed while in SM");
-    }
+    runFsFun("Set time while in SM", new FSRun() {
+      @Override
+      public void run(FileSystem fs) throws IOException {
+        fs.setTimes(file1, 0, 0);
+      }});
 
     try {
       DFSTestUtil.readFile(fs, file1);
@@ -350,7 +350,7 @@ public class TestSafeMode {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
 
     cluster.restartNameNode();
-    fs = (DistributedFileSystem)cluster.getFileSystem();
+    fs = cluster.getFileSystem();
 
     String tipMsg = cluster.getNamesystem().getSafemode();
     assertTrue("Safemode tip message looks right: " + tipMsg,
@@ -375,7 +375,7 @@ public class TestSafeMode {
    * @throws IOException when there's an issue connecting to the test DFS.
    */
   public void testSafeModeUtils() throws IOException {
-    dfs = (DistributedFileSystem)cluster.getFileSystem();
+    dfs = cluster.getFileSystem();
 
     // Enter safemode.
     dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);

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

@@ -21,10 +21,13 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 
 import java.io.EOFException;
+import java.io.File;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.net.URI;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -32,7 +35,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -40,15 +42,21 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 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.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import static org.hamcrest.CoreMatchers.*;
 
 /**
  * Test for short circuit read functionality using {@link BlockReaderLocal}.
@@ -58,9 +66,24 @@ import org.junit.Test;
  * system.
  */
 public class TestShortCircuitLocalRead {
+  private static TemporarySocketDirectory sockDir;
 
-  static final String DIR = "/" + TestShortCircuitLocalRead.class.getSimpleName() + "/";
+  @BeforeClass
+  public static void init() {
+    sockDir = new TemporarySocketDirectory();
+    DomainSocket.disableBindPathValidation();
+  }
+
+  @AfterClass
+  public static void shutdown() throws IOException {
+    sockDir.close();
+  }
 
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+  
   static final long seed = 0xDEADBEEFL;
   static final int blockSize = 5120;
   boolean simulatedStorage = false;
@@ -84,7 +107,9 @@ public class TestShortCircuitLocalRead {
     for (int idx = 0; idx < len; idx++) {
       if (expected[from + idx] != actual[idx]) {
         Assert.fail(message + " byte " + (from + idx) + " differs. expected "
-            + expected[from + idx] + " actual " + actual[idx]);
+            + expected[from + idx] + " actual " + actual[idx] +
+            "\nexpected: " + StringUtils.byteToHexString(expected, from, from + len) +
+            "\nactual:   " + StringUtils.byteToHexString(actual, 0, len));
       }
     }
   }
@@ -96,11 +121,13 @@ public class TestShortCircuitLocalRead {
   /** Check file content, reading as user {@code readingUser} */
   static void checkFileContent(URI uri, Path name, byte[] expected,
       int readOffset, String readingUser, Configuration conf,
-      boolean shortCircuitFails)
+      boolean legacyShortCircuitFails)
       throws IOException, InterruptedException {
     // Ensure short circuit is enabled
     DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
-    assertTrue(fs.getClient().getShortCircuitLocalReads());
+    if (legacyShortCircuitFails) {
+      assertTrue(fs.getClient().useLegacyBlockReaderLocal());
+    }
     
     FSDataInputStream stm = fs.open(name);
     byte[] actual = new byte[expected.length-readOffset];
@@ -127,9 +154,8 @@ public class TestShortCircuitLocalRead {
     }
     checkData(actual, readOffset, expected, "Read 3");
     
-    if (shortCircuitFails) {
-      // short circuit should be disabled due to failure
-      assertFalse(fs.getClient().getShortCircuitLocalReads());
+    if (legacyShortCircuitFails) {
+      assertFalse(fs.getClient().useLegacyBlockReaderLocal());
     }
     stm.close();
   }
@@ -145,11 +171,13 @@ public class TestShortCircuitLocalRead {
   /** Check the file content, reading as user {@code readingUser} */
   static void checkFileContentDirect(URI uri, Path name, byte[] expected,
       int readOffset, String readingUser, Configuration conf,
-      boolean shortCircuitFails)
+      boolean legacyShortCircuitFails)
       throws IOException, InterruptedException {
     // Ensure short circuit is enabled
     DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
-    assertTrue(fs.getClient().getShortCircuitLocalReads());
+    if (legacyShortCircuitFails) {
+      assertTrue(fs.getClient().useLegacyBlockReaderLocal());
+    }
     
     HdfsDataInputStream stm = (HdfsDataInputStream)fs.open(name);
 
@@ -180,33 +208,45 @@ public class TestShortCircuitLocalRead {
       nread += nbytes;
     }
     checkData(arrayFromByteBuffer(actual), readOffset, expected, "Read 3");
-    if (shortCircuitFails) {
-      // short circuit should be disabled due to failure
-      assertFalse(fs.getClient().getShortCircuitLocalReads());
+    if (legacyShortCircuitFails) {
+      assertFalse(fs.getClient().useLegacyBlockReaderLocal());
     }
     stm.close();
   }
 
+  public void doTestShortCircuitReadLegacy(boolean ignoreChecksum, int size,
+      int readOffset, String shortCircuitUser, String readingUser,
+      boolean legacyShortCircuitFails) throws IOException, InterruptedException {
+    doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
+        shortCircuitUser, readingUser, legacyShortCircuitFails);
+  }
+
   public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
       int readOffset) throws IOException, InterruptedException {
     String shortCircuitUser = getCurrentUser();
-    doTestShortCircuitRead(ignoreChecksum, size, readOffset, shortCircuitUser,
-        shortCircuitUser, false);
+    doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
+        null, getCurrentUser(), false);
   }
   
   /**
    * Test that file data can be read by reading the block file
    * directly from the local store.
    */
-  public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
+  public void doTestShortCircuitReadImpl(boolean ignoreChecksum, int size,
       int readOffset, String shortCircuitUser, String readingUser,
-      boolean shortCircuitFails) throws IOException, InterruptedException {
+      boolean legacyShortCircuitFails) throws IOException, InterruptedException {
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
         ignoreChecksum);
-    conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
-        shortCircuitUser);
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        new File(sockDir.getDir(),
+          "TestShortCircuitLocalRead._PORT.sock").getAbsolutePath());
+    if (shortCircuitUser != null) {
+      conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
+          shortCircuitUser);
+      conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
+    }
     if (simulatedStorage) {
       SimulatedFSDataset.setFactory(conf);
     }
@@ -228,9 +268,9 @@ public class TestShortCircuitLocalRead {
       
       URI uri = cluster.getURI();
       checkFileContent(uri, file1, fileData, readOffset, readingUser, conf,
-          shortCircuitFails);
+          legacyShortCircuitFails);
       checkFileContentDirect(uri, file1, fileData, readOffset, readingUser,
-          conf, shortCircuitFails);
+          conf, legacyShortCircuitFails);
     } finally {
       fs.close();
       cluster.shutdown();
@@ -255,6 +295,12 @@ public class TestShortCircuitLocalRead {
     doTestShortCircuitRead(true, 13, 5);
   }
   
+  @Test(timeout=10000)
+  public void testLocalReadLegacy() throws Exception {
+    doTestShortCircuitReadLegacy(true, 13, 0, getCurrentUser(),
+        getCurrentUser(), false);
+  }
+
   /**
    * Try a short circuit from a reader that is not allowed to
    * to use short circuit. The test ensures reader falls back to non
@@ -262,7 +308,7 @@ public class TestShortCircuitLocalRead {
    */
   @Test(timeout=10000)
   public void testLocalReadFallback() throws Exception {
-    doTestShortCircuitRead(true, 13, 0, getCurrentUser(), "notallowed", true);
+    doTestShortCircuitReadLegacy(true, 13, 0, getCurrentUser(), "notallowed", true);
   }
   
   @Test(timeout=10000)
@@ -276,7 +322,7 @@ public class TestShortCircuitLocalRead {
     doTestShortCircuitRead(false, 10*blockSize+100, 777);
     doTestShortCircuitRead(true, 10*blockSize+100, 777);
   }
-   
+
   private ClientDatanodeProtocol getProxy(UserGroupInformation ugi,
       final DatanodeID dnInfo, final Configuration conf) throws IOException,
       InterruptedException {
@@ -301,21 +347,15 @@ public class TestShortCircuitLocalRead {
   }
   
   @Test(timeout=10000)
-  public void testGetBlockLocalPathInfo() throws IOException, InterruptedException {
+  public void testDeprecatedGetBlockLocalPathInfoRpc()
+      throws IOException, InterruptedException {
     final Configuration conf = new Configuration();
-    conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
-        "alloweduser1,alloweduser2");
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .format(true).build();
     cluster.waitActive();
-    final DataNode dn = cluster.getDataNodes().get(0);
     FileSystem fs = cluster.getFileSystem();
     try {
       DFSTestUtil.createFile(fs, new Path("/tmp/x"), 16, (short) 1, 23);
-      UserGroupInformation aUgi1 =
-          UserGroupInformation.createRemoteUser("alloweduser1");
-      UserGroupInformation aUgi2 =
-          UserGroupInformation.createRemoteUser("alloweduser2");
       LocatedBlocks lb = cluster.getNameNode().getRpcServer()
           .getBlockLocations("/tmp/x", 0, 16);
       // Create a new block object, because the block inside LocatedBlock at
@@ -323,29 +363,11 @@ public class TestShortCircuitLocalRead {
       ExtendedBlock blk = new ExtendedBlock(lb.get(0).getBlock());
       Token<BlockTokenIdentifier> token = lb.get(0).getBlockToken();
       final DatanodeInfo dnInfo = lb.get(0).getLocations()[0];
-      ClientDatanodeProtocol proxy = getProxy(aUgi1, dnInfo, conf);
-      // This should succeed
-      BlockLocalPathInfo blpi = proxy.getBlockLocalPathInfo(blk, token);
-      Assert.assertEquals(
-          DataNodeTestUtils.getFSDataset(dn).getBlockLocalPathInfo(blk).getBlockPath(),
-          blpi.getBlockPath());
-
-      // Try with the other allowed user
-      proxy = getProxy(aUgi2, dnInfo, conf);
-
-      // This should succeed as well
-      blpi = proxy.getBlockLocalPathInfo(blk, token);
-      Assert.assertEquals(
-          DataNodeTestUtils.getFSDataset(dn).getBlockLocalPathInfo(blk).getBlockPath(),
-          blpi.getBlockPath());
-
-      // Now try with a disallowed user
-      UserGroupInformation bUgi = UserGroupInformation
-          .createRemoteUser("notalloweduser");
-      proxy = getProxy(bUgi, dnInfo, conf);
+      ClientDatanodeProtocol proxy = 
+          DFSUtil.createClientDatanodeProtocolProxy(dnInfo, conf, 60000, false);
       try {
         proxy.getBlockLocalPathInfo(blk, token);
-        Assert.fail("The call should have failed as " + bUgi.getShortUserName()
+        Assert.fail("The call should have failed as this user "
             + " is not allowed to call getBlockLocalPathInfo");
       } catch (IOException ex) {
         Assert.assertTrue(ex.getMessage().contains(
@@ -363,8 +385,9 @@ public class TestShortCircuitLocalRead {
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
-    conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
-        getCurrentUser());
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        "/tmp/testSkipWithVerifyChecksum._PORT");
+    DomainSocket.disableBindPathValidation();
     if (simulatedStorage) {
       SimulatedFSDataset.setFactory(conf);
     }
@@ -402,6 +425,88 @@ public class TestShortCircuitLocalRead {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testHandleTruncatedBlockFile() throws IOException {
+    MiniDFSCluster cluster = null;
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        "/tmp/testHandleTruncatedBlockFile._PORT");
+    conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
+    final Path TEST_PATH = new Path("/a");
+    final Path TEST_PATH2 = new Path("/b");
+    final long RANDOM_SEED = 4567L;
+    final long RANDOM_SEED2 = 4568L;
+    FSDataInputStream fsIn = null;
+    final int TEST_LENGTH = 3456;
+    
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      FileSystem fs = cluster.getFileSystem();
+      DFSTestUtil.createFile(fs, TEST_PATH,
+          TEST_LENGTH, (short)1, RANDOM_SEED);
+      DFSTestUtil.createFile(fs, TEST_PATH2,
+          TEST_LENGTH, (short)1, RANDOM_SEED2);
+      fsIn = cluster.getFileSystem().open(TEST_PATH2);
+      byte original[] = new byte[TEST_LENGTH];
+      IOUtils.readFully(fsIn, original, 0, TEST_LENGTH);
+      fsIn.close();
+      fsIn = null;
+      try {
+        DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
+      } catch (InterruptedException e) {
+        Assert.fail("unexpected InterruptedException during " +
+            "waitReplication: " + e);
+      } catch (TimeoutException e) {
+        Assert.fail("unexpected TimeoutException during " +
+            "waitReplication: " + e);
+      }
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
+      File dataFile = MiniDFSCluster.getBlockFile(0, block);
+      cluster.shutdown();
+      cluster = null;
+      RandomAccessFile raf = null;
+      try {
+        raf = new RandomAccessFile(dataFile, "rw");
+        raf.setLength(0);
+      } finally {
+        if (raf != null) raf.close();
+      }
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).format(false).build();
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      fsIn = fs.open(TEST_PATH);
+      try {
+        byte buf[] = new byte[100];
+        fsIn.seek(2000);
+        fsIn.readFully(buf, 0, buf.length);
+        Assert.fail("shouldn't be able to read from corrupt 0-length " +
+            "block file.");
+      } catch (IOException e) {
+        DFSClient.LOG.error("caught exception ", e);
+      }
+      fsIn.close();
+      fsIn = null;
+
+      // We should still be able to read the other file.
+      // This is important because it indicates that we detected that the 
+      // previous block was corrupt, rather than blaming the problem on
+      // communication.
+      fsIn = fs.open(TEST_PATH2);
+      byte buf[] = new byte[original.length];
+      fsIn.readFully(buf, 0, buf.length);
+      TestBlockReaderLocal.assertArrayRegionsEqual(original, 0, buf, 0,
+          original.length);
+      fsIn.close();
+      fsIn = null;
+    } finally {
+      if (fsIn != null) fsIn.close();
+      if (cluster != null) cluster.shutdown();
+    }
+  }
      
   /**
    * Test to run benchmarks between short circuit read vs regular read with
@@ -424,6 +529,8 @@ public class TestShortCircuitLocalRead {
     // Setup create a file
     final Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, shortcircuit);
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        "/tmp/TestShortCircuitLocalRead._PORT");
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
         checksum);
     

+ 0 - 171
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSocketCache.java

@@ -1,171 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.spy;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.security.PrivilegedExceptionAction;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-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.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.security.token.Token;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * This class tests the client connection caching in a single node
- * mini-cluster.
- */
-public class TestSocketCache {
-  static final Log LOG = LogFactory.getLog(TestSocketCache.class);
-
-  static final int BLOCK_SIZE = 4096;
-  static final int FILE_SIZE = 3 * BLOCK_SIZE;
-  final static int CACHE_SIZE = 4;
-  final static long CACHE_EXPIRY_MS = 200;
-  static Configuration conf = null;
-  static MiniDFSCluster cluster = null;
-  static FileSystem fs = null;
-  static SocketCache cache;
-
-  static final Path testFile = new Path("/testConnCache.dat");
-  static byte authenticData[] = null;
-
-  static BlockReaderTestUtil util = null;
-
-
-  /**
-   * A mock Answer to remember the BlockReader used.
-   *
-   * It verifies that all invocation to DFSInputStream.getBlockReader()
-   * use the same socket.
-   */
-  private class MockGetBlockReader implements Answer<RemoteBlockReader2> {
-    public RemoteBlockReader2 reader = null;
-    private Socket sock = null;
-
-    @Override
-    public RemoteBlockReader2 answer(InvocationOnMock invocation) throws Throwable {
-      RemoteBlockReader2 prevReader = reader;
-      reader = (RemoteBlockReader2) invocation.callRealMethod();
-      if (sock == null) {
-        sock = reader.dnSock;
-      } else if (prevReader != null) {
-        assertSame("DFSInputStream should use the same socket",
-                   sock, reader.dnSock);
-      }
-      return reader;
-    }
-  }
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    final int REPLICATION_FACTOR = 1;
-
-    HdfsConfiguration confWithoutCache = new HdfsConfiguration();
-    confWithoutCache.setInt(
-        DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 0);
-    util = new BlockReaderTestUtil(REPLICATION_FACTOR, confWithoutCache);
-    cluster = util.getCluster();
-    conf = util.getConf();
-
-    authenticData = util.writeFile(testFile, FILE_SIZE / 1024);
-  }
-
-
-  /**
-   * (Optionally) seek to position, read and verify data.
-   *
-   * Seek to specified position if pos is non-negative.
-   */
-  private void pread(DFSInputStream in,
-                     long pos,
-                     byte[] buffer,
-                     int offset,
-                     int length)
-      throws IOException {
-    assertTrue("Test buffer too small", buffer.length >= offset + length);
-
-    if (pos >= 0)
-      in.seek(pos);
-
-    LOG.info("Reading from file of size " + in.getFileLength() +
-             " at offset " + in.getPos());
-
-    while (length > 0) {
-      int cnt = in.read(buffer, offset, length);
-      assertTrue("Error in read", cnt > 0);
-      offset += cnt;
-      length -= cnt;
-    }
-
-    // Verify
-    for (int i = 0; i < length; ++i) {
-      byte actual = buffer[i];
-      byte expect = authenticData[(int)pos + i];
-      assertEquals("Read data mismatch at file offset " + (pos + i) +
-                   ". Expects " + expect + "; got " + actual,
-                   actual, expect);
-    }
-  }
-
-  
-  /**
-   * Test that the socket cache can be disabled by setting the capacity to
-   * 0. Regression test for HDFS-3365.
-   */
-  @Test
-  public void testDisableCache() throws IOException {
-    LOG.info("Starting testDisableCache()");
-
-    // Configure a new instance with no caching, ensure that it doesn't
-    // cache anything
-
-    FileSystem fsWithoutCache = FileSystem.newInstance(conf);
-    try {
-      DFSTestUtil.readFile(fsWithoutCache, testFile);
-      assertEquals(0, ((DistributedFileSystem)fsWithoutCache).dfs.socketCache.size());
-    } finally {
-      fsWithoutCache.close();
-    }
-  }
-
-  @AfterClass
-  public static void teardownCluster() throws Exception {
-    util.shutdown();
-  }
-}

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+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;
@@ -145,8 +146,9 @@ public class TestBlockTokenWithDFS {
       String file = BlockReaderFactory.getFileName(targetAddr, 
           "test-blockpoolid", block.getBlockId());
       blockReader = BlockReaderFactory.newBlockReader(
-          conf, s, file, block, 
-          lblock.getBlockToken(), 0, -1, null);
+          conf, file, block, lblock.getBlockToken(), 0, -1,
+          true, "TestBlockTokenWithDFS", TcpPeerServer.peerFromSocket(s),
+          nodes[0], null, false);
 
     } catch (IOException ex) {
       if (ex instanceof InvalidBlockTokenException) {

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -962,6 +963,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public FileInputStream[] getShortCircuitFdsForRead(ExtendedBlock block)
+      throws IOException {
+    throw new UnsupportedOperationException();
+  }
   
   @Override
   public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java

@@ -32,11 +32,13 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -280,10 +282,11 @@ public class TestDataNodeVolumeFailure {
     String file = BlockReaderFactory.getFileName(targetAddr, 
         "test-blockpoolid",
         block.getBlockId());
-    BlockReaderFactory.newBlockReader(conf, s, file, block, lblock
-        .getBlockToken(), 0, -1, null);
-
-    // nothing - if it fails - it will throw and exception
+    BlockReader blockReader =
+      BlockReaderFactory.newBlockReader(conf, file, block,
+        lblock.getBlockToken(), 0, -1, true, "TestDataNodeVolumeFailure",
+        TcpPeerServer.peerFromSocket(s), datanode, null, false);
+    blockReader.close(null, null);
   }
   
   /**

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

@@ -1129,7 +1129,7 @@ public class TestCheckpoint {
         throw new IOException(e);
       }
       
-      final int EXPECTED_TXNS_FIRST_SEG = 12;
+      final int EXPECTED_TXNS_FIRST_SEG = 11;
       
       // the following steps should have happened:
       //   edits_inprogress_1 -> edits_1-12  (finalized)

+ 69 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import static org.junit.Assert.assertEquals;
 
 import java.io.File;
@@ -30,12 +31,14 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Test;
+import static org.mockito.Mockito.*;
 
 /**
  * This class tests the creation and validation of a checkpoint.
@@ -163,4 +166,70 @@ public class TestSecurityTokenEditLog {
       if(cluster != null) cluster.shutdown();
     }
   }
+  
+  @Test(timeout=10000)
+  public void testEditsForCancelOnTokenExpire() throws IOException,
+  InterruptedException {
+    long renewInterval = 2000;
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    conf.setLong(DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, renewInterval);
+    conf.setLong(DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, renewInterval*2);
+
+    Text renewer = new Text(UserGroupInformation.getCurrentUser().getUserName());
+    FSImage fsImage = mock(FSImage.class);
+    FSEditLog log = mock(FSEditLog.class);
+    doReturn(log).when(fsImage).getEditLog();   
+    FSNamesystem fsn = new FSNamesystem(conf, fsImage);
+    
+    DelegationTokenSecretManager dtsm = fsn.getDelegationTokenSecretManager();
+    try {
+      dtsm.startThreads();
+      
+      // get two tokens
+      Token<DelegationTokenIdentifier> token1 = fsn.getDelegationToken(renewer);
+      Token<DelegationTokenIdentifier> token2 = fsn.getDelegationToken(renewer);
+      DelegationTokenIdentifier ident1 =
+          (DelegationTokenIdentifier)token1.decodeIdentifier();
+      DelegationTokenIdentifier ident2 =
+          (DelegationTokenIdentifier)token2.decodeIdentifier();
+      
+      // verify we got the tokens
+      verify(log, times(1)).logGetDelegationToken(eq(ident1), anyLong());
+      verify(log, times(1)).logGetDelegationToken(eq(ident2), anyLong());
+      
+      // this is a little tricky because DTSM doesn't let us set scan interval
+      // so need to periodically sleep, then stop/start threads to force scan
+      
+      // renew first token 1/2 to expire
+      Thread.sleep(renewInterval/2);
+      fsn.renewDelegationToken(token2);
+      verify(log, times(1)).logRenewDelegationToken(eq(ident2), anyLong());
+      // force scan and give it a little time to complete
+      dtsm.stopThreads(); dtsm.startThreads();
+      Thread.sleep(250);
+      // no token has expired yet 
+      verify(log, times(0)).logCancelDelegationToken(eq(ident1));
+      verify(log, times(0)).logCancelDelegationToken(eq(ident2));
+      
+      // sleep past expiration of 1st non-renewed token
+      Thread.sleep(renewInterval/2);
+      dtsm.stopThreads(); dtsm.startThreads();
+      Thread.sleep(250);
+      // non-renewed token should have implicitly been cancelled
+      verify(log, times(1)).logCancelDelegationToken(eq(ident1));
+      verify(log, times(0)).logCancelDelegationToken(eq(ident2));
+      
+      // sleep past expiration of 2nd renewed token
+      Thread.sleep(renewInterval/2);
+      dtsm.stopThreads(); dtsm.startThreads();
+      Thread.sleep(250);
+      // both tokens should have been implicitly cancelled by now
+      verify(log, times(1)).logCancelDelegationToken(eq(ident1));
+      verify(log, times(1)).logCancelDelegationToken(eq(ident2));
+    } finally {
+      dtsm.stopThreads();
+    }
+  }
 }

+ 197 - 55
hadoop-mapreduce-project/CHANGES.txt

@@ -14,10 +14,11 @@ Trunk (Unreleased)
     MAPREDUCE-4887. Add RehashPartitioner, to smooth distributions
     with poor implementations of Object#hashCode().  (Radim Kolar via cutting)
 
-  IMPROVEMENTS
+    HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows
+    Azure environments. (See breakdown of tasks below for subtasks and
+    contributors)
 
-    MAPREDUCE-3787. [Gridmix] Optimize job monitoring and STRESS mode for
-                    faster job submission. (amarrk)
+  IMPROVEMENTS
 
     MAPREDUCE-3481. [Gridmix] Improve Gridmix STRESS mode. (amarrk)
 
@@ -30,9 +31,6 @@ Trunk (Unreleased)
     MAPREDUCE-2733. [Gridmix] Gridmix3 cpu emulation system tests.
                     (Vinay Thota via amarrk)
 
-    MAPREDUCE-3008. Improvements to cumulative CPU emulation for short running
-                    tasks in Gridmix. (amarrk)
-
     MAPREDUCE-2836. Provide option to fail jobs when submitted to non-existent
     fair scheduler pools. (Ahmed Radwan via todd)
 
@@ -71,39 +69,14 @@ Trunk (Unreleased)
     MAPREDUCE-4735. Make arguments in TestDFSIO case insensitive.
     (Brandon Li via suresh)
 
+    MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
+    (Srikanth Sundarrajan via amareshwari)
+
   BUG FIXES
 
     MAPREDUCE-4272. SortedRanges.Range#compareTo is not spec compliant.
     (Yu Gao via llu)
 
-    MAPREDUCE-4356. [Rumen] Provide access to the method
-                    ParsedTask.obtainTaskAttempts(). (ravigummadi)
-
-    MAPREDUCE-4100. [Gridmix] Bug fixed in compression emulation feature for 
-                    map only jobs. (amarrk)
-
-    MAPREDUCE-4149. [Rumen] Rumen fails to parse certain counter
-                    strings. (ravigummadi)
-
-    MAPREDUCE-4083. [Gridmix] NPE in cpu emulation. (amarrk)
-
-    MAPREDUCE-4087. [Gridmix] GenerateDistCacheData job of Gridmix can
-                    become slow in some cases (ravigummadi).
-
-    MAPREDUCE-3953. [Gridmix] Gridmix throws NPE and does not simulate a
-                    job if the trace contains null taskStatus for a task.
-                    (ravigummadi)
-
-    MAPREDUCE-3829. [Gridmix] Gridmix should give better error message when
-                    input data directory already exists and -generate opton is
-                    given.(ravigummadi)
-
-    MAPREDUCE-2722. [Gridmix] Gridmix simulated job's map's hdfsBytesRead
-                    counter is wrong when compressed input is used.(ravigummadi)
-
-    MAPREDUCE-3757. [Rumen] Fixed Rumen Folder to adjust shuffleFinished and
-                    sortFinished times when needed.(ravigummadi)
-
     MAPREDUCE-3194. "mapred mradmin" command is broken in mrv2
                      (Jason Lowe via bobby)
 
@@ -155,33 +128,34 @@ Trunk (Unreleased)
     MAPREDUCE-5012. Typo in javadoc for IdentityMapper class. (Adam Monsen
     via suresh)
 
-    MAPREDUCE-5006. Fix failing streaming tests due to MAPREDUCE-4994.
-    (Sandy Ryza via tomwhite)
+    MAPREDUCE-5078. TestMRAppMaster fails on Windows due to mismatched path
+    separators. (Chris Nauroth via sseth)
 
-Release 2.0.5-alpha - UNRELEASED
+    MAPREDUCE-4885. Streaming tests have multiple failures on Windows. (Chris
+    Nauroth via bikas)
 
-  INCOMPATIBLE CHANGES
+  BREAKDOWN OF HADOOP-8562 SUBTASKS
 
-  NEW FEATURES
+    MAPREDUCE-4739. Some MapReduce tests fail to find winutils.
+    (Chris Nauroth via suresh)
 
-  IMPROVEMENTS
-
-    MAPREDUCE-5129. Allow tags to JobHistory for deeper analytics. (billie via
-    acmurthy)
+    MAPREDUCE-4780. MapReduce distribution build fails on Windows.
+    (Chris Nauroth via suresh)
 
-  OPTIMIZATIONS
+    MAPREDUCE-4790. MapReduce build script would be more readable using abspath.
+    (Chris Nauroth via suresh)
 
-  BUG FIXES
+    MAPREDUCE-4869. Fix TestMapReduceChildJVM. (Chris Nauroth via acmurthy)
 
-    MAPREDUCE-5113. Streaming input/output types are ignored with java 
-    mapper/reducer. (sandyr via tucu)
+    MAPREDUCE-4870. Fix TestMRJobsWithHistoryService. (Chris Nauroth via acmurthy)
 
-    MAPREDUCE-5098. Fix findbugs warnings in gridmix. (kkambatl via tucu)
+    MAPREDUCE-4983. Fixed various platform specific assumptions in various tests,
+    so that they can pass on Windows too. (Chris Nauroth via vinodkv)
 
-    MAPREDUCE-5086. MR app master deletes staging dir when sent a reboot
-    command from the RM. (Jian He via jlowe)
+    HADOOP-9372. Fix bad timeout annotations on tests.
+    (Arpit Agarwal via suresh)
 
-Release 2.0.4-beta - UNRELEASED
+Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
@@ -189,9 +163,41 @@ Release 2.0.4-beta - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-3008. Improvements to cumulative CPU emulation for short running
+    tasks in Gridmix. (amarrk via tgraves)
+
     MAPREDUCE-5033. mapred shell script should respect usage flags
     (--help -help -h). (Andrew Wang via atm)
 
+    MAPREDUCE-4892. Modify CombineFileInputFormat to not skew input slits'
+    allocation on small clusters. (Bikas Saha via vinodkv)
+    
+    MAPREDUCE-4990. Construct debug strings conditionally in
+    ShuffleHandler.Shuffle#sendMapOutput(). (kkambatl via tucu)
+    
+    MAPREDUCE-4875. coverage fixing for org.apache.hadoop.mapred
+    (Aleksey Gorshkov via bobby)
+
+    MAPREDUCE-5129. Allow tags to JobHistory for deeper analytics. (billie via
+    acmurthy)
+
+    MAPREDUCE-3787. [Gridmix] Optimize job monitoring and STRESS mode for
+    faster job submission. (amarrk via tgraves)
+
+    MAPREDUCE-5079. Changes job recovery to restore state directly from job
+    history, instaed of simulating state machine events.
+    (Jason Lowe and Robert Parker via sseth)
+
+    MAPREDUCE-4981. Add WordMean, WordMedian, WordStandardDeviation
+    to ExamplesDriver. (Plamen Jeliazkov via shv)
+
+    MAPREUDUCE-5059. Change average merge time on Job overview page to be the
+    time delta between the end of the shuffle and the start of the reduce.
+    (Omkar Vinit Joshi via vinodkv)
+
+    MAPREDUCE-4985. Add compression option to TestDFSIO usage.
+    (Plamen Jeliazkov via shv)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -218,15 +224,108 @@ Release 2.0.4-beta - UNRELEASED
     MAPREDUCE-5008. Merger progress miscounts with respect to EOF_MARKER.
     (Sandy Ryza via tomwhite)
 
+    MAPREDUCE-4693. History server should include counters for failed tasks.
+    (Xuan Gong via sseth)
+
+    MAPREDUCE-4896. mapred queue -info spits out ugly exception when queue does
+    not exist. (sandyr via tucu)
+
+    MAPREDUCE-3685. Fix bugs in MergeManager to ensure compression codec is
+    appropriately used and that on-disk segments are correctly sorted on
+    file-size. (Anty Rao and Ravi Prakash via acmurthy)
+    
+    MAPREDUCE-4571. TestHsWebServicesJobs fails on jdk7. (tgraves via tucu)
+
+    MAPREDUCE-4716. TestHsWebServicesJobsQuery.testJobsQueryStateInvalid
+    fails with jdk7. (tgraves via tucu)
+
+    MAPREDUCE-5075. DistCp leaks input file handles since ThrottledInputStream
+    does not close the wrapped InputStream.  (Chris Nauroth via szetszwo)
+
+    MAPREDUCE-3872. Fix an event handling races in ContainerLauncherImpl.
+    (Robert Kanter via sseth)
+
+    MAPREDUCE-5062. Fix MR AM to read max-retries from the RM. (Zhijie Shen via
+    vinodkv)
+
+    MAPREDUCE-3829. [Gridmix] Gridmix should give better error message when
+    input data directory already exists and -generate opton is
+    given.(ravigummadi via tgraves)
+
+    MAPREDUCE-2722. [Gridmix] Gridmix simulated job's map's hdfsBytesRead
+    counter is wrong when compressed input is used.(ravigummadi via tgraves)
+
+    MAPREDUCE-3953. [Gridmix] Gridmix throws NPE and does not simulate a
+    job if the trace contains null taskStatus for a task.  (ravigummadi via 
+    tgraves)
+
+    MAPREDUCE-4087. [Gridmix] GenerateDistCacheData job of Gridmix can
+    become slow in some cases (ravigummadi via tgraves).
+
+    MAPREDUCE-5077. Remove mapreduce.util.ResourceCalculatorPlugin and related
+    code. (Karthik Kambatla via sseth)
+
+    MAPREDUCE-4083. [Gridmix] NPE in cpu emulation. (amarrk via tgraves)
+
+    MAPREDUCE-4100. [Gridmix] Bug fixed in compression emulation feature for 
+    map only jobs. (amarrk via tgraves)
+
+    MAPREDUCE-4356. [Rumen] Provide access to the method
+    ParsedTask.obtainTaskAttempts(). (ravigummadi via tgraves)
+
+    MAPREDUCE-4149. [Rumen] Rumen fails to parse certain counter
+    strings. (ravigummadi via tgraves) 
+
+    MAPREDUCE-3757. [Rumen] Fixed Rumen Folder to adjust shuffleFinished and
+    sortFinished times when needed. (Ravi Gummadi via tgraves)
+
+    MAPREDUCE-5138. Fix LocalDistributedCacheManager after YARN-112. (Omkar Vinit
+    Joshi via vinodkv)
+
+    MAPREDUCE-5113. Streaming input/output types are ignored with java 
+    mapper/reducer. (sandyr via tucu)
+
+    MAPREDUCE-5098. Fix findbugs warnings in gridmix. (kkambatl via tucu)
+
+    MAPREDUCE-5086. MR app master deletes staging dir when sent a reboot
+    command from the RM. (Jian He via jlowe)
+
+    MAPREDUCE-5137. AM web UI: clicking on Map Task results in 500 error
+    (Thomas Graves via jlowe)
+
+    MAPREDUCE-5136. TestJobImpl->testJobNoTasks fails with IBM JAVA (Amir
+    Sanjar via jlowe)
+
+    MAPREDUCE-5139. Update MR AM to use the modified startContainer API after
+    YARN-486. (Xuan Gong via vinodkv)
+
+Release 2.0.4-alpha - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    MAPREDUCE-5006. Fix failing streaming tests due to MAPREDUCE-4994.
+    (Sandy Ryza via tomwhite)
+
+    MAPREDUCE-5088. MR Client gets an renewer token exception while Oozie is
+    submitting a job (Daryn Sharp via cos)
+
     MAPREDUCE-5117. Changed MRClientProtocolPBClientImpl to be closeable and thus
     fix failures in renewal of HistoryServer's delegations tokens. (Siddharth
     Seth via vinodkv)
 
-    MAPREDUCE-5088. MR Client gets an renewer token exception while Oozie is
-    submitting a job (Daryn Sharp via cos)
+    MAPREDUCE-5083. MiniMRCluster should use a random component when creating an
+    actual cluster (Siddharth Seth via hitesh)
 
-    MAPREDUCE-5138. Fix LocalDistributedCacheManager after YARN-112. (Omkar Vinit
-    Joshi via vinodkv)
+    MAPREDUCE-5094. Disabled memory monitoring by default in MiniMRYarnCluster
+    to avoid some downstream tests failing. (Siddharth Seth via vinodkv)
 
 Release 2.0.3-alpha - 2013-02-06 
 
@@ -744,6 +843,18 @@ Release 2.0.0-alpha - 05-23-2012
     MAPREDUCE-4444. nodemanager fails to start when one of the local-dirs is
     bad (Jason Lowe via bobby)
 
+Release 0.23.8 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 0.23.7 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -758,6 +869,12 @@ Release 0.23.7 - UNRELEASED
     MAPREDUCE-4989. JSONify DataTables input data for Attempts page (Ravi
     Prakash via jlowe)
 
+    MAPREDUCE-5027. Shuffle does not limit number of outstanding connections
+    (Robert Parker via jeagles)
+    
+    MAPREDUCE-4972. Coverage fixing for org.apache.hadoop.mapreduce.jobhistory
+    (Aleksey Gorshkov via bobby)
+
   OPTIMIZATIONS
 
     MAPREDUCE-4946. Fix a performance problem for large jobs by reducing the
@@ -777,11 +894,36 @@ Release 0.23.7 - UNRELEASED
     MAPREDUCE-5009. Killing the Task Attempt slated for commit does not clear
     the value from the Task commitAttempt member (Robert Parker via jeagles)
 
+    MAPREDUCE-4871. AM uses mapreduce.jobtracker.split.metainfo.maxsize but
+    mapred-default has mapreduce.job.split.metainfo.maxsize (Jason Lowe via
+    jeagles)
+
+    MAPREDUCE-4794. DefaultSpeculator generates error messages on normal
+    shutdown (Jason Lowe via jeagles)
+
+    MAPREDUCE-5043. Fetch failure processing can cause AM event queue to
+    backup and eventually OOM (Jason Lowe via bobby)
+
+    MAPREDUCE-5023. History Server Web Services missing Job Counters (Ravi
+    Prakash via tgraves)
+
+    MAPREDUCE-5060. Fetch failures that time out only count against the first
+    map task (Robert Joseph Evans via jlowe)
+
+    MAPREDUCE-5042. Reducer unable to fetch for a map task that was recovered
+    (Jason Lowe via bobby)
+
+    MAPREDUCE-5053. java.lang.InternalError from decompression codec cause
+    reducer to fail (Robert Parker via jeagles)
+
     MAPREDUCE-4991. coverage for gridmix (Aleksey Gorshkov via tgraves)
 
     MAPREDUCE-5007. fix coverage org.apache.hadoop.mapreduce.v2.hs (Aleksey 
     Gorshkov via tgraves)
 
+    MAPREDUCE-5137. AM web UI: clicking on Map Task results in 500 error
+    (Thomas Graves via jlowe)
+
 Release 0.23.6 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 116 - 58
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -24,9 +24,12 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.io.IOUtils;
@@ -46,6 +49,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.jobhistory.AMStartedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.EventReader;
@@ -54,6 +58,9 @@ import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryCopyService;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
@@ -61,6 +68,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.client.MRClientService;
@@ -74,6 +82,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobFinishEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobStartEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
@@ -84,8 +93,6 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl;
 import org.apache.hadoop.mapreduce.v2.app.local.LocalContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
-import org.apache.hadoop.mapreduce.v2.app.recover.Recovery;
-import org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
@@ -94,6 +101,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -167,7 +175,6 @@ public class MRAppMaster extends CompositeService {
   private AppContext context;
   private Dispatcher dispatcher;
   private ClientService clientService;
-  private Recovery recoveryServ;
   private ContainerAllocator containerAllocator;
   private ContainerLauncher containerLauncher;
   private EventHandler<CommitterEvent> committerEventHandler;
@@ -180,7 +187,6 @@ public class MRAppMaster extends CompositeService {
   private OutputCommitter committer;
   private JobEventDispatcher jobEventDispatcher;
   private JobHistoryEventHandler jobHistoryEventHandler;
-  private boolean inRecovery = false;
   private SpeculatorEventDispatcher speculatorEventDispatcher;
 
   private Job job;
@@ -193,6 +199,8 @@ public class MRAppMaster extends CompositeService {
   private String shutDownMessage = null;
   JobStateInternal forcedState = null;
 
+  private long recoveredJobStartTime = 0;
+
   public MRAppMaster(ApplicationAttemptId applicationAttemptId,
       ContainerId containerId, String nmHost, int nmPort, int nmHttpPort,
       long appSubmitTime, int maxAppAttempts) {
@@ -340,34 +348,9 @@ public class MRAppMaster extends CompositeService {
       }
     } else {
       committer = createOutputCommitter(conf);
-      boolean recoveryEnabled = conf.getBoolean(
-          MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
-      boolean recoverySupportedByCommitter = committer.isRecoverySupported();
-
-      // If a shuffle secret was not provided by the job client then this app
-      // attempt will generate one.  However that disables recovery if there
-      // are reducers as the shuffle secret would be app attempt specific.
-      boolean shuffleKeyValidForRecovery = (numReduceTasks > 0 &&
-          TokenCache.getShuffleSecretKey(fsTokens) != null);
-
-      if (recoveryEnabled && recoverySupportedByCommitter
-          && shuffleKeyValidForRecovery && appAttemptID.getAttemptId() > 1) {
-        LOG.info("Recovery is enabled. "
-            + "Will try to recover from previous life on best effort basis.");
-        recoveryServ = createRecoveryService(context);
-        addIfService(recoveryServ);
-        dispatcher = recoveryServ.getDispatcher();
-        clock = recoveryServ.getClock();
-        inRecovery = true;
-      } else {
-        LOG.info("Not starting RecoveryService: recoveryEnabled: "
-            + recoveryEnabled + " recoverySupportedByCommitter: "
-            + recoverySupportedByCommitter + " shuffleKeyValidForRecovery: "
-            + shuffleKeyValidForRecovery + " ApplicationAttemptID: "
-            + appAttemptID.getAttemptId());
-        dispatcher = createDispatcher();
-        addIfService(dispatcher);
-      }
+
+      dispatcher = createDispatcher();
+      addIfService(dispatcher);
 
       //service to handle requests from JobClient
       clientService = createClientService(context);
@@ -595,15 +578,6 @@ public class MRAppMaster extends CompositeService {
     return new JobFinishEventHandler();
   }
 
-  /**
-   * Create the recovery service.
-   * @return an instance of the recovery service.
-   */
-  protected Recovery createRecoveryService(AppContext appContext) {
-    return new RecoveryService(appContext.getApplicationAttemptId(),
-        appContext.getClock(), getCommitter(), isNewApiCommitter());
-  }
-
   /** Create and initialize (but don't start) a single job. 
    * @param forcedState a state to force the job into or null for normal operation. 
    * @param diagnostic a diagnostic message to include with the job.
@@ -615,7 +589,8 @@ public class MRAppMaster extends CompositeService {
     Job newJob =
         new JobImpl(jobId, appAttemptID, conf, dispatcher.getEventHandler(),
             taskAttemptListener, jobTokenSecretManager, fsTokens, clock,
-            completedTasksFromPreviousRun, metrics, newApiCommitter,
+            completedTasksFromPreviousRun, metrics,
+            committer, newApiCommitter,
             currentUser.getUserName(), appSubmitTime, amInfos, context, 
             forcedState, diagnostic);
     ((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
@@ -978,18 +953,8 @@ public class MRAppMaster extends CompositeService {
   public void start() {
 
     amInfos = new LinkedList<AMInfo>();
-
-    // Pull completedTasks etc from recovery
-    if (inRecovery) {
-      completedTasksFromPreviousRun = recoveryServ.getCompletedTasks();
-      amInfos = recoveryServ.getAMInfos();
-    } else {
-      // Get the amInfos anyways irrespective of whether recovery is enabled or
-      // not IF this is not the first AM generation
-      if (appAttemptID.getAttemptId() != 1) {
-        amInfos.addAll(readJustAMInfos());
-      }
-    }
+    completedTasksFromPreviousRun = new HashMap<TaskId, TaskInfo>();
+    processRecovery();
 
     // Current an AMInfo for the current AM generation.
     AMInfo amInfo =
@@ -1051,13 +1016,105 @@ public class MRAppMaster extends CompositeService {
     startJobs();
   }
 
+  private void processRecovery() {
+    if (appAttemptID.getAttemptId() == 1) {
+      return;  // no need to recover on the first attempt
+    }
+
+    boolean recoveryEnabled = getConfig().getBoolean(
+        MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE,
+        MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE_DEFAULT);
+    boolean recoverySupportedByCommitter =
+        committer != null && committer.isRecoverySupported();
+
+    // If a shuffle secret was not provided by the job client then this app
+    // attempt will generate one.  However that disables recovery if there
+    // are reducers as the shuffle secret would be app attempt specific.
+    int numReduceTasks = getConfig().getInt(MRJobConfig.NUM_REDUCES, 0);
+    boolean shuffleKeyValidForRecovery = (numReduceTasks > 0 &&
+        TokenCache.getShuffleSecretKey(fsTokens) != null);
+
+    if (recoveryEnabled && recoverySupportedByCommitter
+          && shuffleKeyValidForRecovery) {
+      LOG.info("Recovery is enabled. "
+          + "Will try to recover from previous life on best effort basis.");
+      try {
+        parsePreviousJobHistory();
+      } catch (IOException e) {
+        LOG.warn("Unable to parse prior job history, aborting recovery", e);
+        // try to get just the AMInfos
+        amInfos.addAll(readJustAMInfos());
+      }
+    } else {
+      LOG.info("Will not try to recover. recoveryEnabled: "
+            + recoveryEnabled + " recoverySupportedByCommitter: "
+            + recoverySupportedByCommitter + " shuffleKeyValidForRecovery: "
+            + shuffleKeyValidForRecovery + " ApplicationAttemptID: "
+            + appAttemptID.getAttemptId());
+      // Get the amInfos anyways whether recovery is enabled or not
+      amInfos.addAll(readJustAMInfos());
+    }
+  }
+
+  private static FSDataInputStream getPreviousJobHistoryStream(
+      Configuration conf, ApplicationAttemptId appAttemptId)
+      throws IOException {
+    Path historyFile = JobHistoryUtils.getPreviousJobHistoryPath(conf,
+        appAttemptId);
+    LOG.info("Previous history file is at " + historyFile);
+    return historyFile.getFileSystem(conf).open(historyFile);
+  }
+
+  private void parsePreviousJobHistory() throws IOException {
+    FSDataInputStream in = getPreviousJobHistoryStream(getConfig(),
+        appAttemptID);
+    JobHistoryParser parser = new JobHistoryParser(in);
+    JobInfo jobInfo = parser.parse();
+    Exception parseException = parser.getParseException();
+    if (parseException != null) {
+      LOG.info("Got an error parsing job-history file" +
+          ", ignoring incomplete events.", parseException);
+    }
+    Map<org.apache.hadoop.mapreduce.TaskID, TaskInfo> taskInfos = jobInfo
+        .getAllTasks();
+    for (TaskInfo taskInfo : taskInfos.values()) {
+      if (TaskState.SUCCEEDED.toString().equals(taskInfo.getTaskStatus())) {
+        Iterator<Entry<TaskAttemptID, TaskAttemptInfo>> taskAttemptIterator =
+            taskInfo.getAllTaskAttempts().entrySet().iterator();
+        while (taskAttemptIterator.hasNext()) {
+          Map.Entry<TaskAttemptID, TaskAttemptInfo> currentEntry = taskAttemptIterator.next();
+          if (!jobInfo.getAllCompletedTaskAttempts().containsKey(currentEntry.getKey())) {
+            taskAttemptIterator.remove();
+          }
+        }
+        completedTasksFromPreviousRun
+            .put(TypeConverter.toYarn(taskInfo.getTaskId()), taskInfo);
+        LOG.info("Read from history task "
+            + TypeConverter.toYarn(taskInfo.getTaskId()));
+      }
+    }
+    LOG.info("Read completed tasks from history "
+        + completedTasksFromPreviousRun.size());
+    recoveredJobStartTime = jobInfo.getLaunchTime();
+
+    // recover AMInfos
+    List<JobHistoryParser.AMInfo> jhAmInfoList = jobInfo.getAMInfos();
+    if (jhAmInfoList != null) {
+      for (JobHistoryParser.AMInfo jhAmInfo : jhAmInfoList) {
+        AMInfo amInfo = MRBuilderUtils.newAMInfo(jhAmInfo.getAppAttemptId(),
+            jhAmInfo.getStartTime(), jhAmInfo.getContainerId(),
+            jhAmInfo.getNodeManagerHost(), jhAmInfo.getNodeManagerPort(),
+            jhAmInfo.getNodeManagerHttpPort());
+        amInfos.add(amInfo);
+      }
+    }
+  }
+
   private List<AMInfo> readJustAMInfos() {
     List<AMInfo> amInfos = new ArrayList<AMInfo>();
     FSDataInputStream inputStream = null;
     try {
-      inputStream =
-          RecoveryService.getPreviousJobHistoryFileStream(getConfig(),
-            appAttemptID);
+      inputStream = getPreviousJobHistoryStream(getConfig(), appAttemptID);
       EventReader jobHistoryEventReader = new EventReader(inputStream);
 
       // All AMInfos are contiguous. Track when the first AMStartedEvent
@@ -1108,7 +1165,8 @@ public class MRAppMaster extends CompositeService {
   @SuppressWarnings("unchecked")
   protected void startJobs() {
     /** create a job-start event to get this ball rolling */
-    JobEvent startJobEvent = new JobEvent(job.getID(), JobEventType.JOB_START);
+    JobEvent startJobEvent = new JobStartEvent(job.getID(),
+        recoveredJobStartTime);
     /** send the job-start event. this triggers the job execution. */
     dispatcher.getEventHandler().handle(startJobEvent);
   }

+ 23 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/package-info.java → hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobStartEvent.java

@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -15,6 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-@InterfaceAudience.Private
-package org.apache.hadoop.mapreduce.v2.app.recover;
-import org.apache.hadoop.classification.InterfaceAudience;
+
+package org.apache.hadoop.mapreduce.v2.app.job.event;
+
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+
+public class JobStartEvent extends JobEvent {
+
+  long recoveredJobStartTime;
+
+  public JobStartEvent(JobId jobID) {
+    this(jobID, 0);
+  }
+
+  public JobStartEvent(JobId jobID, long recoveredJobStartTime) {
+    super(jobID, JobEventType.JOB_START);
+    this.recoveredJobStartTime = recoveredJobStartTime;
+  }
+
+  public long getRecoveredJobStartTime() {
+    return recoveredJobStartTime;
+  }
+}

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptEventType.java

@@ -26,6 +26,7 @@ public enum TaskAttemptEventType {
   //Producer:Task
   TA_SCHEDULE,
   TA_RESCHEDULE,
+  TA_RECOVER,
 
   //Producer:Client, Task
   TA_KILL,

+ 50 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptRecoverEvent.java

@@ -0,0 +1,50 @@
+/**
+ * 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.mapreduce.v2.app.job.event;
+
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+
+public class TaskAttemptRecoverEvent extends TaskAttemptEvent {
+
+  private TaskAttemptInfo taInfo;
+  private OutputCommitter committer;
+  private boolean recoverAttemptOutput;
+
+  public TaskAttemptRecoverEvent(TaskAttemptId id, TaskAttemptInfo taInfo,
+      OutputCommitter committer, boolean recoverOutput) {
+    super(id, TaskAttemptEventType.TA_RECOVER);
+    this.taInfo = taInfo;
+    this.committer = committer;
+    this.recoverAttemptOutput = recoverOutput;
+  }
+
+  public TaskAttemptInfo getTaskAttemptInfo() {
+    return taInfo;
+  }
+
+  public OutputCommitter getCommitter() {
+    return committer;
+  }
+
+  public boolean getRecoverOutput() {
+    return recoverAttemptOutput;
+  }
+}

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskEventType.java

@@ -28,6 +28,7 @@ public enum TaskEventType {
 
   //Producer:Job
   T_SCHEDULE,
+  T_RECOVER,
 
   //Producer:Speculator
   T_ADD_SPEC_ATTEMPT,

+ 50 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskRecoverEvent.java

@@ -0,0 +1,50 @@
+/**
+ * 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.mapreduce.v2.app.job.event;
+
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+
+public class TaskRecoverEvent extends TaskEvent {
+
+  private TaskInfo taskInfo;
+  private OutputCommitter committer;
+  private boolean recoverTaskOutput;
+
+  public TaskRecoverEvent(TaskId taskID, TaskInfo taskInfo,
+      OutputCommitter committer, boolean recoverTaskOutput) {
+    super(taskID, TaskEventType.T_RECOVER);
+    this.taskInfo = taskInfo;
+    this.committer = committer;
+    this.recoverTaskOutput = recoverTaskOutput;
+  }
+
+  public TaskInfo getTaskInfo() {
+    return taskInfo;
+  }
+
+  public OutputCommitter getOutputCommitter() {
+    return committer;
+  }
+
+  public boolean getRecoverTaskOutput() {
+    return recoverTaskOutput;
+  }
+}

+ 24 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.jobhistory.JobFinishedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
@@ -92,6 +93,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobFinishEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobSetupFailedEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobStartEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptCompletedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptFetchFailureEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
@@ -101,6 +103,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskRecoverEvent;
 import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
@@ -159,6 +162,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
   private final Lock writeLock;
   private final JobId jobId;
   private final String jobName;
+  private final OutputCommitter committer;
   private final boolean newApiCommitter;
   private final org.apache.hadoop.mapreduce.JobID oldJobId;
   private final TaskAttemptListener taskAttemptListener;
@@ -602,7 +606,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       JobTokenSecretManager jobTokenSecretManager,
       Credentials fsTokenCredentials, Clock clock,
       Map<TaskId, TaskInfo> completedTasksFromPreviousRun, MRAppMetrics metrics,
-      boolean newApiCommitter, String userName,
+      OutputCommitter committer, boolean newApiCommitter, String userName,
       long appSubmitTime, List<AMInfo> amInfos, AppContext appContext,
       JobStateInternal forcedState, String forcedDiagnostic) {
     this.applicationAttemptId = applicationAttemptId;
@@ -618,6 +622,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     this.queueName = conf.get(MRJobConfig.QUEUE_NAME, "default");
     this.appSubmitTime = appSubmitTime;
     this.oldJobId = TypeConverter.fromYarn(jobId);
+    this.committer = committer;
     this.newApiCommitter = newApiCommitter;
 
     this.taskAttemptListener = taskAttemptListener;
@@ -888,10 +893,16 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     }
   }
 
-  protected void scheduleTasks(Set<TaskId> taskIDs) {
+  protected void scheduleTasks(Set<TaskId> taskIDs,
+      boolean recoverTaskOutput) {
     for (TaskId taskID : taskIDs) {
-      eventHandler.handle(new TaskEvent(taskID, 
-          TaskEventType.T_SCHEDULE));
+      TaskInfo taskInfo = completedTasksFromPreviousRun.remove(taskID);
+      if (taskInfo != null) {
+        eventHandler.handle(new TaskRecoverEvent(taskID, taskInfo,
+            committer, recoverTaskOutput));
+      } else {
+        eventHandler.handle(new TaskEvent(taskID, TaskEventType.T_SCHEDULE));
+      }
     }
   }
 
@@ -1421,7 +1432,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                 job.conf, splits[i], 
                 job.taskAttemptListener, 
                 job.jobToken, job.fsTokens,
-                job.clock, job.completedTasksFromPreviousRun, 
+                job.clock,
                 job.applicationAttemptId.getAttemptId(),
                 job.metrics, job.appContext);
         job.addTask(task);
@@ -1439,7 +1450,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                 job.conf, job.numMapTasks, 
                 job.taskAttemptListener, job.jobToken,
                 job.fsTokens, job.clock,
-                job.completedTasksFromPreviousRun, 
                 job.applicationAttemptId.getAttemptId(),
                 job.metrics, job.appContext);
         job.addTask(task);
@@ -1475,8 +1485,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     @Override
     public void transition(JobImpl job, JobEvent event) {
       job.setupProgress = 1.0f;
-      job.scheduleTasks(job.mapTasks);  // schedule (i.e., start) the maps
-      job.scheduleTasks(job.reduceTasks);
+      job.scheduleTasks(job.mapTasks, job.numReduceTasks == 0);
+      job.scheduleTasks(job.reduceTasks, true);
 
       // If we have no tasks, just transition to job completed
       if (job.numReduceTasks == 0 && job.numMapTasks == 0) {
@@ -1507,7 +1517,12 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
      */
     @Override
     public void transition(JobImpl job, JobEvent event) {
-      job.startTime = job.clock.getTime();
+      JobStartEvent jse = (JobStartEvent) event;
+      if (jse.getRecoveredJobStartTime() != 0) {
+        job.startTime = jse.getRecoveredJobStartTime();
+      } else {
+        job.startTime = job.clock.getTime();
+      }
       JobInitedEvent jie =
         new JobInitedEvent(job.oldJobId,
              job.startTime,

+ 2 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/MapTaskImpl.java

@@ -18,17 +18,13 @@
 
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
-import java.util.Map;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapTaskAttemptImpl;
 import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
@@ -49,11 +45,10 @@ public class MapTaskImpl extends TaskImpl {
       TaskAttemptListener taskAttemptListener,
       Token<JobTokenIdentifier> jobToken,
       Credentials credentials, Clock clock,
-      Map<TaskId, TaskInfo> completedTasksFromPreviousRun, int startCount,
-      MRAppMetrics metrics, AppContext appContext) {
+      int appAttemptId, MRAppMetrics metrics, AppContext appContext) {
     super(jobId, TaskType.MAP, partition, eventHandler, remoteJobConfFile,
         conf, taskAttemptListener, jobToken, credentials, clock,
-        completedTasksFromPreviousRun, startCount, metrics, appContext);
+        appAttemptId, metrics, appContext);
     this.taskSplitMetaInfo = taskSplitMetaInfo;
   }
 

+ 2 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/ReduceTaskImpl.java

@@ -18,16 +18,12 @@
 
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
-import java.util.Map;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.ReduceTaskAttemptImpl;
 import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
@@ -47,11 +43,10 @@ public class ReduceTaskImpl extends TaskImpl {
       int numMapTasks, TaskAttemptListener taskAttemptListener,
       Token<JobTokenIdentifier> jobToken,
       Credentials credentials, Clock clock,
-      Map<TaskId, TaskInfo> completedTasksFromPreviousRun, int startCount,
-      MRAppMetrics metrics, AppContext appContext) {
+      int appAttemptId, MRAppMetrics metrics, AppContext appContext) {
     super(jobId, TaskType.REDUCE, partition, eventHandler, jobFile, conf,
         taskAttemptListener, jobToken, credentials, clock,
-        completedTasksFromPreviousRun, startCount, metrics, appContext);
+        appAttemptId, metrics, appContext);
     this.numMapTasks = numMapTasks;
   }
 

+ 180 - 56
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -56,10 +56,12 @@ import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskCounter;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinishedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.ReduceAttemptFinishedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStartedEvent;
@@ -89,6 +91,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdate
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptRecoverEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
@@ -111,6 +114,7 @@ import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerToken;
@@ -204,6 +208,11 @@ public abstract class TaskAttemptImpl implements
          TaskAttemptEventType.TA_KILL, new KilledTransition())
      .addTransition(TaskAttemptStateInternal.NEW, TaskAttemptStateInternal.FAILED,
          TaskAttemptEventType.TA_FAILMSG, new FailedTransition())
+     .addTransition(TaskAttemptStateInternal.NEW,
+         EnumSet.of(TaskAttemptStateInternal.FAILED,
+             TaskAttemptStateInternal.KILLED,
+             TaskAttemptStateInternal.SUCCEEDED),
+         TaskAttemptEventType.TA_RECOVER, new RecoverTransition())
      .addTransition(TaskAttemptStateInternal.NEW,
           TaskAttemptStateInternal.NEW,
           TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
@@ -759,8 +768,8 @@ public abstract class TaskAttemptImpl implements
     // The null fields are per-container and will be constructed for each
     // container separately.
     ContainerLaunchContext container = BuilderUtils
-        .newContainerLaunchContext(null, conf
-            .get(MRJobConfig.USER_NAME), null, localResources,
+        .newContainerLaunchContext(conf
+            .get(MRJobConfig.USER_NAME), localResources,
             environment, null, serviceData, taskCredentialsBuffer,
             applicationACLs);
 
@@ -769,10 +778,9 @@ public abstract class TaskAttemptImpl implements
 
   static ContainerLaunchContext createContainerLaunchContext(
       Map<ApplicationAccessType, String> applicationACLs,
-      ContainerId containerID, Configuration conf,
-      Token<JobTokenIdentifier> jobToken, Task remoteTask,
+      Configuration conf, Token<JobTokenIdentifier> jobToken, Task remoteTask,
       final org.apache.hadoop.mapred.JobID oldJobId,
-      Resource assignedCapability, WrappedJvmID jvmID,
+      WrappedJvmID jvmID,
       TaskAttemptListener taskAttemptListener,
       Credentials credentials) {
 
@@ -805,7 +813,7 @@ public abstract class TaskAttemptImpl implements
 
     // Construct the actual Container
     ContainerLaunchContext container = BuilderUtils.newContainerLaunchContext(
-        containerID, commonContainerSpec.getUser(), assignedCapability,
+        commonContainerSpec.getUser(),
         commonContainerSpec.getLocalResources(), myEnv, commands,
         myServiceData, commonContainerSpec.getContainerTokens().duplicate(),
         applicationACLs);
@@ -1082,6 +1090,102 @@ public abstract class TaskAttemptImpl implements
     this.avataar = avataar;
   }
   
+  @SuppressWarnings("unchecked")
+  public TaskAttemptStateInternal recover(TaskAttemptInfo taInfo,
+      OutputCommitter committer, boolean recoverOutput) {
+    containerID = taInfo.getContainerId();
+    containerNodeId = ConverterUtils.toNodeId(taInfo.getHostname() + ":"
+        + taInfo.getPort());
+    containerMgrAddress = StringInterner.weakIntern(
+        containerNodeId.toString());
+    nodeHttpAddress = StringInterner.weakIntern(taInfo.getHostname() + ":"
+        + taInfo.getHttpPort());
+    computeRackAndLocality();
+    launchTime = taInfo.getStartTime();
+    finishTime = (taInfo.getFinishTime() != -1) ?
+        taInfo.getFinishTime() : clock.getTime();
+    shufflePort = taInfo.getShufflePort();
+    trackerName = taInfo.getHostname();
+    httpPort = taInfo.getHttpPort();
+    sendLaunchedEvents();
+
+    reportedStatus.id = attemptId;
+    reportedStatus.progress = 1.0f;
+    reportedStatus.counters = taInfo.getCounters();
+    reportedStatus.stateString = taInfo.getState();
+    reportedStatus.phase = Phase.CLEANUP;
+    reportedStatus.mapFinishTime = taInfo.getMapFinishTime();
+    reportedStatus.shuffleFinishTime = taInfo.getShuffleFinishTime();
+    reportedStatus.sortFinishTime = taInfo.getSortFinishTime();
+    addDiagnosticInfo(taInfo.getError());
+
+    boolean needToClean = false;
+    String recoveredState = taInfo.getTaskStatus();
+    if (recoverOutput
+        && TaskAttemptState.SUCCEEDED.toString().equals(recoveredState)) {
+      TaskAttemptContext tac = new TaskAttemptContextImpl(conf,
+          TypeConverter.fromYarn(attemptId));
+      try {
+        committer.recoverTask(tac);
+        LOG.info("Recovered output from task attempt " + attemptId);
+      } catch (Exception e) {
+        LOG.error("Unable to recover task attempt " + attemptId, e);
+        LOG.info("Task attempt " + attemptId + " will be recovered as KILLED");
+        recoveredState = TaskAttemptState.KILLED.toString();
+        needToClean = true;
+      }
+    }
+
+    TaskAttemptStateInternal attemptState;
+    if (TaskAttemptState.SUCCEEDED.toString().equals(recoveredState)) {
+      attemptState = TaskAttemptStateInternal.SUCCEEDED;
+      reportedStatus.taskState = TaskAttemptState.SUCCEEDED;
+      eventHandler.handle(createJobCounterUpdateEventTASucceeded(this));
+      logAttemptFinishedEvent(attemptState);
+    } else if (TaskAttemptState.FAILED.toString().equals(recoveredState)) {
+      attemptState = TaskAttemptStateInternal.FAILED;
+      reportedStatus.taskState = TaskAttemptState.FAILED;
+      eventHandler.handle(createJobCounterUpdateEventTAFailed(this, false));
+      TaskAttemptUnsuccessfulCompletionEvent tauce =
+          createTaskAttemptUnsuccessfulCompletionEvent(this,
+              TaskAttemptStateInternal.FAILED);
+      eventHandler.handle(
+          new JobHistoryEvent(attemptId.getTaskId().getJobId(), tauce));
+    } else {
+      if (!TaskAttemptState.KILLED.toString().equals(recoveredState)) {
+        if (String.valueOf(recoveredState).isEmpty()) {
+          LOG.info("TaskAttempt" + attemptId
+              + " had not completed, recovering as KILLED");
+        } else {
+          LOG.warn("TaskAttempt " + attemptId + " found in unexpected state "
+              + recoveredState + ", recovering as KILLED");
+        }
+        addDiagnosticInfo("Killed during application recovery");
+        needToClean = true;
+      }
+      attemptState = TaskAttemptStateInternal.KILLED;
+      reportedStatus.taskState = TaskAttemptState.KILLED;
+      eventHandler.handle(createJobCounterUpdateEventTAKilled(this, false));
+      TaskAttemptUnsuccessfulCompletionEvent tauce =
+          createTaskAttemptUnsuccessfulCompletionEvent(this,
+              TaskAttemptStateInternal.KILLED);
+      eventHandler.handle(
+          new JobHistoryEvent(attemptId.getTaskId().getJobId(), tauce));
+    }
+
+    if (needToClean) {
+      TaskAttemptContext tac = new TaskAttemptContextImpl(conf,
+          TypeConverter.fromYarn(attemptId));
+      try {
+        committer.abortTask(tac);
+      } catch (Exception e) {
+        LOG.warn("Task cleanup failed for attempt " + attemptId, e);
+      }
+    }
+
+    return attemptState;
+  }
+
   private static TaskAttemptState getExternalState(
       TaskAttemptStateInternal smState) {
     switch (smState) {
@@ -1122,6 +1226,24 @@ public abstract class TaskAttemptImpl implements
     }
   }
 
+  private void computeRackAndLocality() {
+    nodeRackName = RackResolver.resolve(
+        containerNodeId.getHost()).getNetworkLocation();
+
+    locality = Locality.OFF_SWITCH;
+    if (dataLocalHosts.size() > 0) {
+      String cHost = resolveHost(containerNodeId.getHost());
+      if (dataLocalHosts.contains(cHost)) {
+        locality = Locality.NODE_LOCAL;
+      }
+    }
+    if (locality == Locality.OFF_SWITCH) {
+      if (dataLocalRacks.contains(nodeRackName)) {
+        locality = Locality.RACK_LOCAL;
+      }
+    }
+  }
+
   private static long computeSlotMillis(TaskAttemptImpl taskAttempt) {
     TaskType taskType = taskAttempt.getID().getTaskId().getTaskType();
     int slotMemoryReq =
@@ -1141,6 +1263,18 @@ public abstract class TaskAttemptImpl implements
     return slotMillisIncrement;
   }
 
+  private static JobCounterUpdateEvent createJobCounterUpdateEventTASucceeded(
+      TaskAttemptImpl taskAttempt) {
+    long slotMillis = computeSlotMillis(taskAttempt);
+    TaskId taskId = taskAttempt.attemptId.getTaskId();
+    JobCounterUpdateEvent jce = new JobCounterUpdateEvent(taskId.getJobId());
+    jce.addCounterUpdate(
+      taskId.getTaskType() == TaskType.MAP ?
+        JobCounter.SLOTS_MILLIS_MAPS : JobCounter.SLOTS_MILLIS_REDUCES,
+        slotMillis);
+    return jce;
+  }
+
   private static JobCounterUpdateEvent createJobCounterUpdateEventTAFailed(
       TaskAttemptImpl taskAttempt, boolean taskAlreadyCompleted) {
     TaskType taskType = taskAttempt.getID().getTaskId().getTaskType();
@@ -1210,6 +1344,26 @@ public abstract class TaskAttemptImpl implements
     return tauce;
   }
 
+  @SuppressWarnings("unchecked")
+  private void sendLaunchedEvents() {
+    JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptId.getTaskId()
+        .getJobId());
+    jce.addCounterUpdate(attemptId.getTaskId().getTaskType() == TaskType.MAP ?
+        JobCounter.TOTAL_LAUNCHED_MAPS : JobCounter.TOTAL_LAUNCHED_REDUCES, 1);
+    eventHandler.handle(jce);
+
+    LOG.info("TaskAttempt: [" + attemptId
+        + "] using containerId: [" + containerID + " on NM: ["
+        + containerMgrAddress + "]");
+    TaskAttemptStartedEvent tase =
+      new TaskAttemptStartedEvent(TypeConverter.fromYarn(attemptId),
+          TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
+          launchTime, trackerName, httpPort, shufflePort, containerID,
+          locality.toString(), avataar.toString());
+    eventHandler.handle(
+        new JobHistoryEvent(attemptId.getTaskId().getJobId(), tase));
+  }
+
   private WrappedProgressSplitsBlock getProgressSplitBlock() {
     readLock.lock();
     try {
@@ -1342,8 +1496,6 @@ public abstract class TaskAttemptImpl implements
           taskAttempt.containerNodeId.toString());
       taskAttempt.nodeHttpAddress = StringInterner.weakIntern(
           cEvent.getContainer().getNodeHttpAddress());
-      taskAttempt.nodeRackName = RackResolver.resolve(
-          taskAttempt.containerNodeId.getHost()).getNetworkLocation();
       taskAttempt.containerToken = cEvent.getContainer().getContainerToken();
       taskAttempt.assignedCapability = cEvent.getContainer().getResource();
       // this is a _real_ Task (classic Hadoop mapred flavor):
@@ -1354,32 +1506,18 @@ public abstract class TaskAttemptImpl implements
       taskAttempt.taskAttemptListener.registerPendingTask(
           taskAttempt.remoteTask, taskAttempt.jvmID);
 
-      taskAttempt.locality = Locality.OFF_SWITCH;
-      if (taskAttempt.dataLocalHosts.size() > 0) {
-        String cHost = taskAttempt.resolveHost(
-            taskAttempt.containerNodeId.getHost());
-        if (taskAttempt.dataLocalHosts.contains(cHost)) {
-          taskAttempt.locality = Locality.NODE_LOCAL;
-        }
-      }
-      if (taskAttempt.locality == Locality.OFF_SWITCH) {
-        if (taskAttempt.dataLocalRacks.contains(taskAttempt.nodeRackName)) {
-          taskAttempt.locality = Locality.RACK_LOCAL;
-        }
-      }
+      taskAttempt.computeRackAndLocality();
       
       //launch the container
       //create the container object to be launched for a given Task attempt
       ContainerLaunchContext launchContext = createContainerLaunchContext(
-          cEvent.getApplicationACLs(), taskAttempt.containerID,
-          taskAttempt.conf, taskAttempt.jobToken, taskAttempt.remoteTask,
-          taskAttempt.oldJobId, taskAttempt.assignedCapability,
-          taskAttempt.jvmID, taskAttempt.taskAttemptListener,
-          taskAttempt.credentials);
+          cEvent.getApplicationACLs(), taskAttempt.conf, taskAttempt.jobToken,
+          taskAttempt.remoteTask, taskAttempt.oldJobId, taskAttempt.jvmID,
+          taskAttempt.taskAttemptListener, taskAttempt.credentials);
       taskAttempt.eventHandler.handle(new ContainerRemoteLaunchEvent(
           taskAttempt.attemptId, taskAttempt.containerID,
           taskAttempt.containerMgrAddress, taskAttempt.containerToken,
-          launchContext, taskAttempt.remoteTask));
+          launchContext, taskAttempt.assignedCapability, taskAttempt.remoteTask));
 
       // send event to speculator that our container needs are satisfied
       taskAttempt.eventHandler.handle
@@ -1471,27 +1609,7 @@ public abstract class TaskAttemptImpl implements
                                                                   // Costly?
       taskAttempt.trackerName = nodeHttpInetAddr.getHostName();
       taskAttempt.httpPort = nodeHttpInetAddr.getPort();
-      JobCounterUpdateEvent jce =
-          new JobCounterUpdateEvent(taskAttempt.attemptId.getTaskId()
-              .getJobId());
-      jce.addCounterUpdate(
-          taskAttempt.attemptId.getTaskId().getTaskType() == TaskType.MAP ? 
-              JobCounter.TOTAL_LAUNCHED_MAPS: JobCounter.TOTAL_LAUNCHED_REDUCES
-              , 1);
-      taskAttempt.eventHandler.handle(jce);
-      
-      LOG.info("TaskAttempt: [" + taskAttempt.attemptId
-          + "] using containerId: [" + taskAttempt.containerID + " on NM: ["
-          + taskAttempt.containerMgrAddress + "]");
-      TaskAttemptStartedEvent tase =
-        new TaskAttemptStartedEvent(TypeConverter.fromYarn(taskAttempt.attemptId),
-            TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId().getTaskType()),
-            taskAttempt.launchTime,
-            nodeHttpInetAddr.getHostName(), nodeHttpInetAddr.getPort(),
-            taskAttempt.shufflePort, taskAttempt.containerID,
-            taskAttempt.locality.toString(), taskAttempt.avataar.toString());
-      taskAttempt.eventHandler.handle
-          (new JobHistoryEvent(taskAttempt.attemptId.getTaskId().getJobId(), tase));
+      taskAttempt.sendLaunchedEvents();
       taskAttempt.eventHandler.handle
           (new SpeculatorEvent
               (taskAttempt.attemptId, true, taskAttempt.clock.getTime()));
@@ -1540,14 +1658,8 @@ public abstract class TaskAttemptImpl implements
         TaskAttemptEvent event) {
       //set the finish time
       taskAttempt.setFinishTime();
-      long slotMillis = computeSlotMillis(taskAttempt);
-      TaskId taskId = taskAttempt.attemptId.getTaskId();
-      JobCounterUpdateEvent jce = new JobCounterUpdateEvent(taskId.getJobId());
-      jce.addCounterUpdate(
-        taskId.getTaskType() == TaskType.MAP ? 
-          JobCounter.SLOTS_MILLIS_MAPS : JobCounter.SLOTS_MILLIS_REDUCES,
-          slotMillis);
-      taskAttempt.eventHandler.handle(jce);
+      taskAttempt.eventHandler.handle(
+          createJobCounterUpdateEventTASucceeded(taskAttempt));
       taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
           taskAttempt.attemptId,
@@ -1585,6 +1697,18 @@ public abstract class TaskAttemptImpl implements
     }
   }
 
+  private static class RecoverTransition implements
+      MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptStateInternal> {
+
+    @Override
+    public TaskAttemptStateInternal transition(TaskAttemptImpl taskAttempt,
+        TaskAttemptEvent event) {
+      TaskAttemptRecoverEvent tare = (TaskAttemptRecoverEvent) event;
+      return taskAttempt.recover(tare.getTaskAttemptInfo(),
+          tare.getCommitter(), tare.getRecoverOutput());
+    }
+  }
+
   @SuppressWarnings({ "unchecked" })
   private void logAttemptFinishedEvent(TaskAttemptStateInternal state) {
     //Log finished events only if an attempt started.

Niektoré súbory nie sú zobrazené, pretože je v týchto rozdielových dátach zmenené mnoho súborov