Parcourir la source

Merge trunk into auto-failover branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3042@1309164 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon il y a 13 ans
Parent
commit
39a5fd5c71
100 fichiers modifiés avec 2933 ajouts et 1563 suppressions
  1. 3 0
      hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
  2. 21 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 11 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java
  4. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
  5. 0 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java
  6. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
  7. 49 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java
  8. 98 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
  9. 8 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenIdentifier.java
  10. 28 1
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  12. 71 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  13. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  14. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  16. 6 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  17. 5 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
  18. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
  19. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  20. 10 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  21. 10 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  22. 16 41
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
  23. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaAlreadyExistsException.java
  25. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java
  26. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
  27. 9 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  28. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
  29. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
  30. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
  31. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  32. 286 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
  33. 29 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
  34. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetFactory.java
  35. 21 955
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  36. 107 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
  37. 288 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  38. 172 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
  39. 228 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LDir.java
  40. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
  41. 228 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java
  42. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  43. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  44. 9 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  45. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
  46. 6 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  47. 51 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java
  48. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  49. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java
  50. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
  51. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java
  52. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
  53. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
  54. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java
  55. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
  56. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
  57. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
  58. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  59. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java
  60. 128 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java
  61. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
  62. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java
  63. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
  64. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java
  65. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
  66. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
  67. 23 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
  68. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
  69. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
  70. 17 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
  71. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
  72. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
  73. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
  74. 15 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
  75. 36 32
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
  76. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java
  77. 23 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
  78. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
  79. 45 0
      hadoop-mapreduce-project/CHANGES.txt
  80. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  81. 28 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
  82. 62 26
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
  83. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  84. 0 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java
  85. 2 104
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
  86. 69 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
  87. 33 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  88. 106 49
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
  89. 7 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  90. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
  91. 70 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
  92. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
  93. 17 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java
  94. 170 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
  95. 2 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
  96. 3 3
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
  97. 2 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
  98. 16 7
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
  99. 6 8
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
  100. 3 5
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java

+ 3 - 0
hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml

@@ -155,6 +155,9 @@
       <excludes>
         <exclude>org.apache.hadoop:hadoop-common</exclude>
         <exclude>org.apache.hadoop:hadoop-hdfs</exclude>
+	<!-- use slf4j from common to avoid multiple binding warnings -->
+	<exclude>org.slf4j:slf4j-api</exclude>
+	<exclude>org.slf4j:slf4j-log4j12</exclude>
       </excludes>
     </dependencySet>
   </dependencySets>

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

@@ -140,6 +140,9 @@ Release 2.0.0 - UNRELEASED
 
     HADOOP-8206. Common portion of a ZK-based failover controller (todd)
 
+    HADOOP-8210. Common side of HDFS-3148: The client should be able
+    to use multiple local interfaces for data transfer. (eli)
+
   IMPROVEMENTS
 
     HADOOP-7524. Change RPC to allow multiple protocols including multuple
@@ -239,6 +242,9 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8236. haadmin should have configurable timeouts for failover
     commands. (todd)
 
+    HADOOP-8242. AbstractDelegationTokenIdentifier: add getter methods
+    for owner and realuser. (Colin Patrick McCabe via eli)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -308,6 +314,9 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8238. NetUtils#getHostNameOfIP blows up if given ip:port
     string w/o port. (eli)
 
+    HADOOP-8243. Security support broken in CLI (manual) failover controller
+    (todd)
+
   BREAKDOWN OF HADOOP-7454 SUBTASKS
 
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@@ -360,6 +369,18 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8116. RetriableCommand is using RetryPolicy incorrectly after
     HADOOP-7896. (atm)
 
+Release 0.23.3 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 0.23.2 - UNRELEASED 
 
   INCOMPATIBLE CHANGES

+ 11 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java

@@ -52,14 +52,22 @@ public class FailoverController {
   public FailoverController(Configuration conf) {
     this.conf = conf;
     
-    this.gracefulFenceTimeout = conf.getInt(
+    this.gracefulFenceTimeout = getGracefulFenceTimeout(conf);
+    this.rpcTimeoutToNewActive = getRpcTimeoutToNewActive(conf);
+  }
+
+  static int getGracefulFenceTimeout(Configuration conf) {
+    return conf.getInt(
         CommonConfigurationKeys.HA_FC_GRACEFUL_FENCE_TIMEOUT_KEY,
         CommonConfigurationKeys.HA_FC_GRACEFUL_FENCE_TIMEOUT_DEFAULT);
-    this.rpcTimeoutToNewActive = conf.getInt(
+  }
+  
+  static int getRpcTimeoutToNewActive(Configuration conf) {
+    return conf.getInt(
         CommonConfigurationKeys.HA_FC_NEW_ACTIVE_TIMEOUT_KEY,
         CommonConfigurationKeys.HA_FC_NEW_ACTIVE_TIMEOUT_DEFAULT);
   }
-
+  
   /**
    * Perform pre-failover checks on the given service we plan to
    * failover to, eg to prevent failing over to a service (eg due

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java

@@ -114,7 +114,8 @@ public abstract class HAAdmin extends Configured implements Tool {
       return -1;
     }
     
-    HAServiceProtocol proto = resolveTarget(argv[1]).getProxy();
+    HAServiceProtocol proto = resolveTarget(argv[1]).getProxy(
+        getConf(), 0);
     HAServiceProtocolHelper.transitionToActive(proto);
     return 0;
   }
@@ -127,7 +128,8 @@ public abstract class HAAdmin extends Configured implements Tool {
       return -1;
     }
     
-    HAServiceProtocol proto = resolveTarget(argv[1]).getProxy();
+    HAServiceProtocol proto = resolveTarget(argv[1]).getProxy(
+        getConf(), 0);
     HAServiceProtocolHelper.transitionToStandby(proto);
     return 0;
   }

+ 0 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java

@@ -68,11 +68,4 @@ public abstract class HAServiceTarget {
         getAddress(),
         confCopy, factory, timeoutMs);
   }
-
-  /**
-   * @return a proxy to connect to the target HA Service.
-   */
-  public final HAServiceProtocol getProxy() throws IOException {
-    return getProxy(new Configuration(), 0); // default conf, timeout
-  }
 }

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java

@@ -250,7 +250,8 @@ public abstract class ZKFailoverController implements Tool {
   private synchronized void becomeActive() throws ServiceFailedException {
     LOG.info("Trying to make " + localTarget + " active...");
     try {
-      HAServiceProtocolHelper.transitionToActive(localTarget.getProxy());
+      HAServiceProtocolHelper.transitionToActive(localTarget.getProxy(
+          conf, FailoverController.getRpcTimeoutToNewActive(conf)));
       LOG.info("Successfully transitioned " + localTarget +
           " to active state");
     } catch (Throwable t) {
@@ -277,7 +278,8 @@ public abstract class ZKFailoverController implements Tool {
     LOG.info("ZK Election indicated that " + localTarget +
         " should become standby");
     try {
-      localTarget.getProxy().transitionToStandby();
+      int timeout = FailoverController.getGracefulFenceTimeout(conf);
+      localTarget.getProxy(conf, timeout).transitionToStandby();
       LOG.info("Successfully transitioned " + localTarget +
           " to standby state");
     } catch (Exception e) {

+ 49 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java

@@ -27,7 +27,10 @@ import java.net.InetAddress;
 import java.net.NetworkInterface;
 import java.net.SocketException;
 import java.net.UnknownHostException;
+import java.util.Collections;
 import java.util.Enumeration;
+import java.util.LinkedHashSet;
+import java.util.Set;
 import java.util.Vector;
 
 import javax.naming.NamingException;
@@ -113,6 +116,31 @@ public class DNS {
     return null;
   }
 
+  /**
+   * @param nif network interface to get addresses for
+   * @return set containing addresses for each subinterface of nif,
+   *    see below for the rationale for using an ordered set
+   */
+  private static LinkedHashSet<InetAddress> getSubinterfaceInetAddrs(
+      NetworkInterface nif) {
+    LinkedHashSet<InetAddress> addrs = new LinkedHashSet<InetAddress>();
+    Enumeration<NetworkInterface> subNifs = nif.getSubInterfaces();
+    while (subNifs.hasMoreElements()) {
+      NetworkInterface subNif = subNifs.nextElement();
+      addrs.addAll(Collections.list(subNif.getInetAddresses()));
+    }
+    return addrs;
+  }
+
+  /**
+   * Like {@link DNS#getIPs(String, boolean), but returns all
+   * IPs associated with the given interface and its subinterfaces.
+   */
+  public static String[] getIPs(String strInterface)
+      throws UnknownHostException {
+    return getIPs(strInterface, true);
+  }
+
   /**
    * Returns all the IPs associated with the provided interface, if any, in
    * textual form.
@@ -120,6 +148,9 @@ public class DNS {
    * @param strInterface
    *            The name of the network interface or sub-interface to query
    *            (eg eth0 or eth0:0) or the string "default"
+   * @param returnSubinterfaces
+   *            Whether to return IPs associated with subinterfaces of
+   *            the given interface
    * @return A string vector of all the IPs associated with the provided
    *         interface. The local host IP is returned if the interface
    *         name "default" is specified or there is an I/O error looking
@@ -128,8 +159,8 @@ public class DNS {
    *             If the given interface is invalid
    * 
    */
-  public static String[] getIPs(String strInterface)
-    throws UnknownHostException {
+  public static String[] getIPs(String strInterface,
+      boolean returnSubinterfaces) throws UnknownHostException {
     if ("default".equals(strInterface)) {
       return new String[] { cachedHostAddress };
     }
@@ -147,12 +178,22 @@ public class DNS {
     if (netIf == null) {
       throw new UnknownHostException("No such interface " + strInterface);
     }
-    Vector<String> ips = new Vector<String>();
-    Enumeration<InetAddress> addrs = netIf.getInetAddresses();
-    while (addrs.hasMoreElements()) {
-      ips.add(addrs.nextElement().getHostAddress());
+
+    // NB: Using a LinkedHashSet to preserve the order for callers
+    // that depend on a particular element being 1st in the array.
+    // For example, getDefaultIP always returns the first element.
+    LinkedHashSet<InetAddress> allAddrs = new LinkedHashSet<InetAddress>();
+    allAddrs.addAll(Collections.list(netIf.getInetAddresses()));
+    if (!returnSubinterfaces) {
+      allAddrs.removeAll(getSubinterfaceInetAddrs(netIf));
+    }
+
+    String ips[] = new String[allAddrs.size()];
+    int i = 0;
+    for (InetAddress addr : allAddrs) {
+      ips[i++] = addr.getHostAddress();
     }
-    return ips.toArray(new String[] {});
+    return ips;
   }
 
 
@@ -256,7 +297,7 @@ public class DNS {
     return address;
   }
 
-    /**
+  /**
    * Returns all the host names associated by the default nameserver with the
    * address bound to the specified network interface
    * 

+ 98 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java

@@ -43,6 +43,8 @@ import javax.net.SocketFactory;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.net.util.SubnetUtils;
+import org.apache.commons.net.util.SubnetUtils.SubnetInfo;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -51,6 +53,8 @@ import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import com.google.common.base.Preconditions;
+
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Unstable
 public class NetUtils {
@@ -469,11 +473,27 @@ public class NetUtils {
    * @see java.net.Socket#connect(java.net.SocketAddress, int)
    * 
    * @param socket
-   * @param endpoint 
-   * @param timeout - timeout in milliseconds
+   * @param address the remote address
+   * @param timeout timeout in milliseconds
+   */
+  public static void connect(Socket socket,
+      SocketAddress address,
+      int timeout) throws IOException {
+    connect(socket, address, null, timeout);
+  }
+
+  /**
+   * Like {@link NetUtils#connect(Socket, SocketAddress, int)} but
+   * also takes a local address and port to bind the socket to. 
+   * 
+   * @param socket
+   * @param address the remote address
+   * @param localAddr the local address to bind the socket to
+   * @param timeout timeout in milliseconds
    */
   public static void connect(Socket socket, 
-                             SocketAddress endpoint, 
+                             SocketAddress endpoint,
+                             SocketAddress localAddr,
                              int timeout) throws IOException {
     if (socket == null || endpoint == null || timeout < 0) {
       throw new IllegalArgumentException("Illegal argument for connect()");
@@ -481,6 +501,15 @@ public class NetUtils {
     
     SocketChannel ch = socket.getChannel();
     
+    if (localAddr != null) {
+      Class localClass = localAddr.getClass();
+      Class remoteClass = endpoint.getClass();
+      Preconditions.checkArgument(localClass.equals(remoteClass),
+          "Local address %s must be of same family as remote address %s.",
+          localAddr, endpoint);
+      socket.bind(localAddr);
+    }
+
     if (ch == null) {
       // let the default implementation handle it.
       socket.connect(endpoint, timeout);
@@ -769,4 +798,70 @@ public class NetUtils {
         ("\"" + hostname + "\"")
         : UNKNOWN_HOST;
   }
+
+  /**
+   * @return true if the given string is a subnet specified
+   *     using CIDR notation, false otherwise
+   */
+  public static boolean isValidSubnet(String subnet) {
+    try {
+      new SubnetUtils(subnet);
+      return true;
+    } catch (IllegalArgumentException iae) {
+      return false;
+    }
+  }
+
+  /**
+   * Add all addresses associated with the given nif in the
+   * given subnet to the given list.
+   */
+  private static void addMatchingAddrs(NetworkInterface nif,
+      SubnetInfo subnetInfo, List<InetAddress> addrs) {
+    Enumeration<InetAddress> ifAddrs = nif.getInetAddresses();
+    while (ifAddrs.hasMoreElements()) {
+      InetAddress ifAddr = ifAddrs.nextElement();
+      if (subnetInfo.isInRange(ifAddr.getHostAddress())) {
+        addrs.add(ifAddr);
+      }
+    }
+  }
+
+  /**
+   * Return an InetAddress for each interface that matches the
+   * given subnet specified using CIDR notation.
+   *
+   * @param subnet subnet specified using CIDR notation
+   * @param returnSubinterfaces
+   *            whether to return IPs associated with subinterfaces
+   * @throws IllegalArgumentException if subnet is invalid
+   */
+  public static List<InetAddress> getIPs(String subnet,
+      boolean returnSubinterfaces) {
+    List<InetAddress> addrs = new ArrayList<InetAddress>();
+    SubnetInfo subnetInfo = new SubnetUtils(subnet).getInfo();
+    Enumeration<NetworkInterface> nifs;
+
+    try {
+      nifs = NetworkInterface.getNetworkInterfaces();
+    } catch (SocketException e) {
+      LOG.error("Unable to get host interfaces", e);
+      return addrs;
+    }
+
+    while (nifs.hasMoreElements()) {
+      NetworkInterface nif = nifs.nextElement();
+      // NB: adding addresses even if the nif is not up
+      addMatchingAddrs(nif, subnetInfo, addrs);
+
+      if (!returnSubinterfaces) {
+        continue;
+      }
+      Enumeration<NetworkInterface> subNifs = nif.getSubInterfaces();
+      while (subNifs.hasMoreElements()) {
+        addMatchingAddrs(subNifs.nextElement(), subnetInfo, addrs);
+      }
+    }
+    return addrs;
+  }
 }

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenIdentifier.java

@@ -96,10 +96,18 @@ extends TokenIdentifier {
     }
   }
 
+  public Text getOwner() {
+    return owner;
+  }
+
   public Text getRenewer() {
     return renewer;
   }
   
+  public Text getRealUser() {
+    return realUser;
+  }
+  
   public void setIssueDate(long issueDate) {
     this.issueDate = issueDate;
   }

+ 28 - 1
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -109,6 +109,9 @@ Trunk (unreleased changes)
 
     HDFS-3116. Typo in fetchdt error message. (AOE Takashi via atm)
 
+    HDFS-3126. Journal stream from Namenode to BackupNode needs to have 
+    timeout. (Hari Mankude via suresh)
+    
 Release 2.0.0 - UNRELEASED 
 
   INCOMPATIBLE CHANGES
@@ -179,6 +182,9 @@ Release 2.0.0 - UNRELEASED
 
     HDFS-3167. CLI-based driver for MiniDFSCluster. (Henry Robinson via atm)
 
+    HDFS-3148. The client should be able to use multiple local interfaces
+    for data transfer. (eli)
+
   IMPROVEMENTS
 
     HDFS-2018. Move all journal stream management code into one place.
@@ -296,6 +302,10 @@ Release 2.0.0 - UNRELEASED
 
     HDFS-3144. Refactor DatanodeID#getName by use. (eli)
 
+    HDFS-3130. Move fsdataset implementation to a package.  (szetszwo)
+
+    HDFS-3120. Enable hsync and hflush by default. (eli)
+
   OPTIMIZATIONS
 
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
@@ -708,7 +718,24 @@ Release 2.0.0 - UNRELEASED
     
     HDFS-3039. Address findbugs and javadoc warnings on branch. (todd via atm)
 
-Release 0.23.2 - UNRELEASED 
+Release 0.23.3 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HDFS-3166. Add timeout to Hftp connections.  (Daryn Sharp via szetszwo)
+
+    HDFS-3176. Use MD5MD5CRC32FileChecksum.readFields() in JsonUtil .  (Kihwal
+    Lee via szetszwo)
+
+Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -218,7 +218,7 @@
        the OBL warning.
      -->
      <Match>
-       <Class name="org.apache.hadoop.hdfs.server.datanode.FSDataset" />
+       <Class name="org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl" />
        <Method name="getTmpInputStreams" />
        <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
      </Match>

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

@@ -57,12 +57,16 @@ import java.io.OutputStream;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
+import java.net.SocketAddress;
 import java.net.URI;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 
 import javax.net.SocketFactory;
 
@@ -123,6 +127,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -132,7 +137,9 @@ import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+import com.google.common.net.InetAddresses;
 
 /********************************************************
  * DFSClient can connect to a Hadoop Filesystem and 
@@ -168,6 +175,8 @@ public class DFSClient implements java.io.Closeable {
   final LeaseRenewer leaserenewer;
   final SocketCache socketCache;
   final Conf dfsClientConf;
+  private Random r = new Random();
+  private SocketAddress[] localInterfaceAddrs;
 
   /**
    * DFSClient configuration 
@@ -361,6 +370,68 @@ public class DFSClient implements java.io.Closeable {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Short circuit read is " + shortCircuitLocalReads);
     }
+    String localInterfaces[] =
+      conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
+    localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
+    if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
+      LOG.debug("Using local interfaces [" +
+      Joiner.on(',').join(localInterfaces)+ "] with addresses [" +
+      Joiner.on(',').join(localInterfaceAddrs) + "]");
+    }
+  }
+
+  /**
+   * Return the socket addresses to use with each configured
+   * local interface. Local interfaces may be specified by IP
+   * address, IP address range using CIDR notation, interface
+   * name (e.g. eth0) or sub-interface name (e.g. eth0:0).
+   * The socket addresses consist of the IPs for the interfaces
+   * and the ephemeral port (port 0). If an IP, IP range, or
+   * interface name matches an interface with sub-interfaces
+   * only the IP of the interface is used. Sub-interfaces can
+   * be used by specifying them explicitly (by IP or name).
+   * 
+   * @return SocketAddresses for the configured local interfaces,
+   *    or an empty array if none are configured
+   * @throws UnknownHostException if a given interface name is invalid
+   */
+  private static SocketAddress[] getLocalInterfaceAddrs(
+      String interfaceNames[]) throws UnknownHostException {
+    List<SocketAddress> localAddrs = new ArrayList<SocketAddress>();
+    for (String interfaceName : interfaceNames) {
+      if (InetAddresses.isInetAddress(interfaceName)) {
+        localAddrs.add(new InetSocketAddress(interfaceName, 0));
+      } else if (NetUtils.isValidSubnet(interfaceName)) {
+        for (InetAddress addr : NetUtils.getIPs(interfaceName, false)) {
+          localAddrs.add(new InetSocketAddress(addr, 0));
+        }
+      } else {
+        for (String ip : DNS.getIPs(interfaceName, false)) {
+          localAddrs.add(new InetSocketAddress(ip, 0));
+        }
+      }
+    }
+    return localAddrs.toArray(new SocketAddress[localAddrs.size()]);
+  }
+
+  /**
+   * Select one of the configured local interfaces at random. We use a random
+   * interface because other policies like round-robin are less effective
+   * given that we cache connections to datanodes.
+   *
+   * @return one of the local interface addresses at random, or null if no
+   *    local interfaces are configured
+   */
+  SocketAddress getRandomLocalInterfaceAddr() {
+    if (localInterfaceAddrs.length == 0) {
+      return null;
+    }
+    final int idx = r.nextInt(localInterfaceAddrs.length);
+    final SocketAddress addr = localInterfaceAddrs[idx];
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using local interface " + addr);
+    }
+    return addr;
   }
 
   /**

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

@@ -197,6 +197,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY = "dfs.namenode.checkpoint.edits.dir";
   public static final String  DFS_HOSTS = "dfs.hosts";
   public static final String  DFS_HOSTS_EXCLUDE = "dfs.hosts.exclude";
+  public static final String  DFS_CLIENT_LOCAL_INTERFACES = "dfs.client.local.interfaces";
 
   // Much code in hdfs is not yet updated to use these keys.
   public static final String  DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY = "dfs.client.block.write.locateFollowingBlock.retries";

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

@@ -850,7 +850,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
         // disaster.
         sock.setTcpNoDelay(true);
 
-        NetUtils.connect(sock, dnAddr, dfsClient.getConf().socketTimeout);
+        NetUtils.connect(sock, dnAddr,
+            dfsClient.getRandomLocalInterfaceAddr(),
+            dfsClient.getConf().socketTimeout);
         sock.setSoTimeout(dfsClient.getConf().socketTimeout);
       }
 

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

@@ -1171,7 +1171,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       NetUtils.createSocketAddr(first.getXferAddr());
     final Socket sock = client.socketFactory.createSocket();
     final int timeout = client.getDatanodeReadTimeout(length);
-    NetUtils.connect(sock, isa, timeout);
+    NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), timeout);
     sock.setSoTimeout(timeout);
     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
     if(DFSClient.LOG.isDebugEnabled()) {

+ 6 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
+import org.apache.hadoop.hdfs.web.URLUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
@@ -293,15 +294,6 @@ public class HftpFileSystem extends FileSystem
     }
     return ugiParamenter.toString();
   }
-  
-  static Void throwIOExceptionFromConnection(
-      final HttpURLConnection connection, final IOException ioe
-      ) throws IOException {
-    final int code = connection.getResponseCode();
-    final String s = connection.getResponseMessage();
-    throw s == null? ioe:
-        new IOException(s + " (error code=" + code + ")", ioe);
-  }
 
   /**
    * Open an HTTP connection to the namenode to read file data and metadata.
@@ -312,13 +304,10 @@ public class HftpFileSystem extends FileSystem
       throws IOException {
     query = addDelegationTokenParam(query);
     final URL url = getNamenodeURL(path, query);
-    final HttpURLConnection connection = (HttpURLConnection)url.openConnection();
-    try {
-      connection.setRequestMethod("GET");
-      connection.connect();
-    } catch (IOException ioe) {
-      throwIOExceptionFromConnection(connection, ioe);
-    }
+    final HttpURLConnection connection =
+        (HttpURLConnection)URLUtils.openConnection(url);
+    connection.setRequestMethod("GET");
+    connection.connect();
     return connection;
   }
 
@@ -342,7 +331,7 @@ public class HftpFileSystem extends FileSystem
 
     @Override
     protected HttpURLConnection openConnection() throws IOException {
-      return (HttpURLConnection)url.openConnection();
+      return (HttpURLConnection)URLUtils.openConnection(url);
     }
 
     /** Use HTTP Range header for specifying offset. */

+ 5 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java

@@ -41,6 +41,7 @@ import javax.net.ssl.X509TrustManager;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.URLUtils;
 
 /**
  * An implementation of a protocol for accessing filesystems over HTTPS. The
@@ -137,15 +138,11 @@ public class HsftpFileSystem extends HftpFileSystem {
     query = addDelegationTokenParam(query);
     final URL url = new URL("https", nnAddr.getHostName(), 
         nnAddr.getPort(), path + '?' + query);
-    HttpsURLConnection conn = (HttpsURLConnection)url.openConnection();
+    HttpsURLConnection conn = (HttpsURLConnection)URLUtils.openConnection(url);
     // bypass hostname verification
-    try {
-      conn.setHostnameVerifier(new DummyHostnameVerifier());
-      conn.setRequestMethod("GET");
-      conn.connect();
-    } catch (IOException ioe) {
-      throwIOExceptionFromConnection(conn, ioe);
-    }
+    conn.setHostnameVerifier(new DummyHostnameVerifier());
+    conn.setRequestMethod("GET");
+    conn.connect();
 
     // check cert expiration date
     final int warnDays = ExpWarnDays;

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java

@@ -190,7 +190,7 @@ public class NameNodeProxies {
       InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
       throws IOException {
     JournalProtocolPB proxy = (JournalProtocolPB) createNameNodeProxy(address,
-        conf, ugi, JournalProtocolPB.class);
+        conf, ugi, JournalProtocolPB.class, 30000);
     return new JournalProtocolTranslatorPB(proxy);
   }
 
@@ -198,7 +198,7 @@ public class NameNodeProxies {
       createNNProxyWithRefreshAuthorizationPolicyProtocol(InetSocketAddress address,
           Configuration conf, UserGroupInformation ugi) throws IOException {
     RefreshAuthorizationPolicyProtocolPB proxy = (RefreshAuthorizationPolicyProtocolPB)
-        createNameNodeProxy(address, conf, ugi, RefreshAuthorizationPolicyProtocolPB.class);
+        createNameNodeProxy(address, conf, ugi, RefreshAuthorizationPolicyProtocolPB.class, 0);
     return new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(proxy);
   }
   
@@ -206,7 +206,7 @@ public class NameNodeProxies {
       createNNProxyWithRefreshUserMappingsProtocol(InetSocketAddress address,
           Configuration conf, UserGroupInformation ugi) throws IOException {
     RefreshUserMappingsProtocolPB proxy = (RefreshUserMappingsProtocolPB)
-        createNameNodeProxy(address, conf, ugi, RefreshUserMappingsProtocolPB.class);
+        createNameNodeProxy(address, conf, ugi, RefreshUserMappingsProtocolPB.class, 0);
     return new RefreshUserMappingsProtocolClientSideTranslatorPB(proxy);
   }
 
@@ -214,7 +214,7 @@ public class NameNodeProxies {
       InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
       throws IOException {
     GetUserMappingsProtocolPB proxy = (GetUserMappingsProtocolPB)
-        createNameNodeProxy(address, conf, ugi, GetUserMappingsProtocolPB.class);
+        createNameNodeProxy(address, conf, ugi, GetUserMappingsProtocolPB.class, 0);
     return new GetUserMappingsProtocolClientSideTranslatorPB(proxy);
   }
   
@@ -222,7 +222,7 @@ public class NameNodeProxies {
       InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
       boolean withRetries) throws IOException {
     NamenodeProtocolPB proxy = (NamenodeProtocolPB) createNameNodeProxy(
-        address, conf, ugi, NamenodeProtocolPB.class);
+        address, conf, ugi, NamenodeProtocolPB.class, 0);
     if (withRetries) { // create the proxy with retries
       RetryPolicy timeoutPolicy = RetryPolicies.exponentialBackoffRetry(5, 200,
           TimeUnit.MILLISECONDS);
@@ -244,7 +244,7 @@ public class NameNodeProxies {
       InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
       boolean withRetries) throws IOException {
     ClientNamenodeProtocolPB proxy = (ClientNamenodeProtocolPB) NameNodeProxies
-        .createNameNodeProxy(address, conf, ugi, ClientNamenodeProtocolPB.class);
+        .createNameNodeProxy(address, conf, ugi, ClientNamenodeProtocolPB.class, 0);
     if (withRetries) { // create the proxy with retries
       RetryPolicy createPolicy = RetryPolicies
           .retryUpToMaximumCountWithFixedSleep(5,
@@ -275,11 +275,11 @@ public class NameNodeProxies {
   
   @SuppressWarnings("unchecked")
   private static Object createNameNodeProxy(InetSocketAddress address,
-      Configuration conf, UserGroupInformation ugi, Class xface)
+      Configuration conf, UserGroupInformation ugi, Class xface, int rpcTimeout)
       throws IOException {
     RPC.setProtocolEngine(conf, xface, ProtobufRpcEngine.class);
     Object proxy = RPC.getProxy(xface, RPC.getProtocolVersion(xface), address,
-        ugi, conf, NetUtils.getDefaultSocketFactory(conf));
+        ugi, conf, NetUtils.getDefaultSocketFactory(conf), rpcTimeout);
     return proxy;
   }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -86,7 +86,7 @@ public class BlockMetadataHeader {
    * @return
    * @throws IOException
    */
-  static BlockMetadataHeader readHeader(File file) throws IOException {
+  public static BlockMetadataHeader readHeader(File file) throws IOException {
     DataInputStream in = null;
     try {
       in = new DataInputStream(new BufferedInputStream(
@@ -144,7 +144,7 @@ public class BlockMetadataHeader {
   /**
    * Returns the size of the header
    */
-  static int getHeaderSize() {
+  public static int getHeaderSize() {
     return Short.SIZE/Byte.SIZE + DataChecksum.getChecksumHeaderSize();
   }
 }

+ 10 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -540,8 +540,8 @@ public class DataNode extends Configured
     }
   }
   
-  // calls specific to BP
-  protected void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+  /** Notify the corresponding namenode to delete the block. */
+  public void notifyNamenodeDeletedBlock(ExtendedBlock block) {
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if (bpos != null) {
       bpos.notifyNamenodeDeletedBlock(block);
@@ -1134,9 +1134,8 @@ public class DataNode extends Configured
   
   /**
    *  Check if there is a disk failure and if so, handle the error
-   *
-   **/
-  protected void checkDiskError( ) {
+   */
+  public void checkDiskError() {
     try {
       data.checkDataDir();
     } catch (DiskErrorException de) {
@@ -1691,6 +1690,12 @@ public class DataNode extends Configured
     return data;
   }
 
+  /** @return the block scanner. */
+  public DataBlockScanner getBlockScanner() {
+    return blockScanner;
+  }
+
+
   public static void secureMain(String args[], SecureResources resources) {
     try {
       StringUtils.startupShutdownMessage(DataNode.class, args, LOG);

+ 10 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -30,28 +30,25 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Properties;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.HardLink;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
@@ -65,7 +62,7 @@ import org.apache.hadoop.util.DiskChecker;
 @InterfaceAudience.Private
 public class DataStorage extends Storage {
   // Constants
-  final static String BLOCK_SUBDIR_PREFIX = "subdir";
+  public final static String BLOCK_SUBDIR_PREFIX = "subdir";
   final static String BLOCK_FILE_PREFIX = "blk_";
   final static String COPY_FILE_PREFIX = "dncp_";
   final static String STORAGE_DIR_DETACHED = "detach";
@@ -98,15 +95,17 @@ public class DataStorage extends Storage {
     this.storageID = strgID;
   }
 
-  synchronized String getStorageID() {
+  /** @return storage ID. */
+  public synchronized String getStorageID() {
     return storageID;
   }
   
   synchronized void setStorageID(String newStorageID) {
     this.storageID = newStorageID;
   }
-  
-  synchronized void createStorageID(int datanodePort) {
+
+  /** Create an ID for this storage. */
+  public synchronized void createStorageID(int datanodePort) {
     if (storageID != null && !storageID.isEmpty()) {
       return;
     }

+ 16 - 41
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java

@@ -18,21 +18,17 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
-import java.io.FilenameFilter;
 import java.io.IOException;
-import java.util.Arrays;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 
 /** Provide utility methods for Datanode. */
 @InterfaceAudience.Private
-class DatanodeUtil {
-  static final String METADATA_EXTENSION = ".meta";
+public class DatanodeUtil {
+  public static final String UNLINK_BLOCK_SUFFIX = ".unlinked";
 
-  static final String UNLINK_BLOCK_SUFFIX = ".unlinked";
-
-  private static final String DISK_ERROR = "Possible disk error: ";
+  public static final String DISK_ERROR = "Possible disk error: ";
 
   /** Get the cause of an I/O exception if caused by a possible disk error
    * @param ioe an I/O exception
@@ -52,55 +48,34 @@ class DatanodeUtil {
    * @throws IOException 
    * if the file already exists or if the file cannot be created.
    */
-  static File createTmpFile(Block b, File f) throws IOException {
+  public static File createTmpFile(Block b, File f) throws IOException {
     if (f.exists()) {
-      throw new IOException("Unexpected problem in creating temporary file for "
-          + b + ".  File " + f + " should not be present, but is.");
+      throw new IOException("Failed to create temporary file for " + b
+          + ".  File " + f + " should not be present, but is.");
     }
     // Create the zero-length temp file
     final boolean fileCreated;
     try {
       fileCreated = f.createNewFile();
     } catch (IOException ioe) {
-      throw (IOException)new IOException(DISK_ERROR + f).initCause(ioe);
+      throw new IOException(DISK_ERROR + "Failed to create " + f, ioe);
     }
     if (!fileCreated) {
-      throw new IOException("Unexpected problem in creating temporary file for "
-          + b + ".  File " + f + " should be creatable, but is already present.");
+      throw new IOException("Failed to create temporary file for " + b
+          + ".  File " + f + " should be creatable, but is already present.");
     }
     return f;
   }
   
-  static String getMetaFileName(String blockFileName, long genStamp) {
-    return blockFileName + "_" + genStamp + METADATA_EXTENSION;
-  }
-  
-  static File getMetaFile(File f, long genStamp) {
-    return new File(getMetaFileName(f.getAbsolutePath(), genStamp));
+  /**
+   * @return the meta name given the block name and generation stamp.
+   */
+  public static String getMetaName(String blockName, long generationStamp) {
+    return blockName + "_" + generationStamp + Block.METADATA_EXTENSION; 
   }
 
-  /** Find the corresponding meta data file from a given block file */
-  static File findMetaFile(final File blockFile) throws IOException {
-    final String prefix = blockFile.getName() + "_";
-    final File parent = blockFile.getParentFile();
-    File[] matches = parent.listFiles(new FilenameFilter() {
-      public boolean accept(File dir, String name) {
-        return dir.equals(parent)
-            && name.startsWith(prefix) && name.endsWith(METADATA_EXTENSION);
-      }
-    });
-
-    if (matches == null || matches.length == 0) {
-      throw new IOException("Meta file not found, blockFile=" + blockFile);
-    }
-    else if (matches.length > 1) {
-      throw new IOException("Found more than one meta files: " 
-          + Arrays.asList(matches));
-    }
-    return matches[0];
-  }
-  
-  static File getUnlinkTmpFile(File f) {
+  /** @return the unlink file. */
+  public static File getUnlinkTmpFile(File f) {
     return new File(f.getParentFile(), f.getName()+UNLINK_BLOCK_SUFFIX);
   }
 }

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 /**
  * This class describes a replica that has been finalized.
  */
-class FinalizedReplica extends ReplicaInfo {
+public class FinalizedReplica extends ReplicaInfo {
   private boolean unlinked;      // copy-on-write done for block
 
   /**
@@ -37,7 +37,7 @@ class FinalizedReplica extends ReplicaInfo {
    * @param vol volume where replica is located
    * @param dir directory path where block and meta files are located
    */
-  FinalizedReplica(long blockId, long len, long genStamp,
+  public FinalizedReplica(long blockId, long len, long genStamp,
       FsVolumeSpi vol, File dir) {
     super(blockId, len, genStamp, vol, dir);
   }
@@ -48,7 +48,7 @@ class FinalizedReplica extends ReplicaInfo {
    * @param vol volume where replica is located
    * @param dir directory path where block and meta files are located
    */
-  FinalizedReplica(Block block, FsVolumeSpi vol, File dir) {
+  public FinalizedReplica(Block block, FsVolumeSpi vol, File dir) {
     super(block, vol, dir);
   }
 
@@ -56,7 +56,7 @@ class FinalizedReplica extends ReplicaInfo {
    * Copy constructor.
    * @param from
    */
-  FinalizedReplica(FinalizedReplica from) {
+  public FinalizedReplica(FinalizedReplica from) {
     super(from);
     this.unlinked = from.isUnlinked();
   }
@@ -67,12 +67,12 @@ class FinalizedReplica extends ReplicaInfo {
   }
   
   @Override // ReplicaInfo
-  boolean isUnlinked() {
+  public boolean isUnlinked() {
     return unlinked;
   }
 
   @Override  // ReplicaInfo
-  void setUnlinked() {
+  public void setUnlinked() {
     unlinked = true;
   }
   
@@ -99,6 +99,6 @@ class FinalizedReplica extends ReplicaInfo {
   @Override
   public String toString() {
     return super.toString()
-        + "\n  unlinked=" + unlinked;
+        + "\n  unlinked          =" + unlinked;
   }
 }

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

@@ -25,7 +25,7 @@ import java.io.IOException;
  * Exception indicating that the target block already exists 
  * and is not set to be recovered/overwritten.  
  */
-class ReplicaAlreadyExistsException extends IOException {
+public class ReplicaAlreadyExistsException extends IOException {
   private static final long serialVersionUID = 1L;
 
   public ReplicaAlreadyExistsException() {

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
  * Those are the replicas that
  * are created in a pipeline initiated by a dfs client.
  */
-class ReplicaBeingWritten extends ReplicaInPipeline {
+public class ReplicaBeingWritten extends ReplicaInPipeline {
   /**
    * Constructor for a zero length replica
    * @param blockId block id
@@ -35,7 +35,7 @@ class ReplicaBeingWritten extends ReplicaInPipeline {
    * @param vol volume where replica is located
    * @param dir directory path where block and meta files are located
    */
-  ReplicaBeingWritten(long blockId, long genStamp, 
+  public ReplicaBeingWritten(long blockId, long genStamp, 
         FsVolumeSpi vol, File dir) {
     super( blockId, genStamp, vol, dir);
   }
@@ -47,7 +47,7 @@ class ReplicaBeingWritten extends ReplicaInPipeline {
    * @param dir directory path where block and meta files are located
    * @param writer a thread that is writing to this replica
    */
-  ReplicaBeingWritten(Block block, 
+  public ReplicaBeingWritten(Block block, 
       FsVolumeSpi vol, File dir, Thread writer) {
     super( block, vol, dir, writer);
   }
@@ -61,7 +61,7 @@ class ReplicaBeingWritten extends ReplicaInPipeline {
    * @param dir directory path where block and meta files are located
    * @param writer a thread that is writing to this replica
    */
-  ReplicaBeingWritten(long blockId, long len, long genStamp,
+  public ReplicaBeingWritten(long blockId, long len, long genStamp,
       FsVolumeSpi vol, File dir, Thread writer ) {
     super( blockId, len, genStamp, vol, dir, writer);
   }
@@ -70,7 +70,7 @@ class ReplicaBeingWritten extends ReplicaInPipeline {
    * Copy constructor.
    * @param from
    */
-  ReplicaBeingWritten(ReplicaBeingWritten from) {
+  public ReplicaBeingWritten(ReplicaBeingWritten from) {
     super(from);
   }
 

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.util.DataChecksum;
  * 
  * The base class implements a temporary replica
  */
-class ReplicaInPipeline extends ReplicaInfo
+public class ReplicaInPipeline extends ReplicaInfo
                         implements ReplicaInPipelineInterface {
   private long bytesAcked;
   private long bytesOnDisk;
@@ -50,9 +50,8 @@ class ReplicaInPipeline extends ReplicaInfo
    * @param genStamp replica generation stamp
    * @param vol volume where replica is located
    * @param dir directory path where block and meta files are located
-   * @param state replica state
    */
-    ReplicaInPipeline(long blockId, long genStamp, 
+  public ReplicaInPipeline(long blockId, long genStamp, 
         FsVolumeSpi vol, File dir) {
     this( blockId, 0L, genStamp, vol, dir, Thread.currentThread());
   }
@@ -91,7 +90,7 @@ class ReplicaInPipeline extends ReplicaInfo
    * Copy constructor.
    * @param from
    */
-  ReplicaInPipeline(ReplicaInPipeline from) {
+  public ReplicaInPipeline(ReplicaInPipeline from) {
     super(from);
     this.bytesAcked = from.getBytesAcked();
     this.bytesOnDisk = from.getBytesOnDisk();
@@ -151,7 +150,7 @@ class ReplicaInPipeline extends ReplicaInfo
    * Interrupt the writing thread and wait until it dies
    * @throws IOException the waiting is interrupted
    */
-  void stopWriter() throws IOException {
+  public void stopWriter() throws IOException {
     if (writer != null && writer != Thread.currentThread() && writer.isAlive()) {
       writer.interrupt();
       try {

+ 9 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -84,20 +84,12 @@ abstract public class ReplicaInfo extends Block implements Replica {
   ReplicaInfo(ReplicaInfo from) {
     this(from, from.getVolume(), from.getDir());
   }
-
-  /**
-   * Get this replica's meta file name
-   * @return this replica's meta file name
-   */
-  private String getMetaFileName() {
-    return getBlockName() + "_" + getGenerationStamp() + METADATA_EXTENSION; 
-  }
   
   /**
    * Get the full path of this replica's data file
    * @return the full path of this replica's data file
    */
-  File getBlockFile() {
+  public File getBlockFile() {
     return new File(getDir(), getBlockName());
   }
   
@@ -105,15 +97,16 @@ abstract public class ReplicaInfo extends Block implements Replica {
    * Get the full path of this replica's meta file
    * @return the full path of this replica's meta file
    */
-  File getMetaFile() {
-    return new File(getDir(), getMetaFileName());
+  public File getMetaFile() {
+    return new File(getDir(),
+        DatanodeUtil.getMetaName(getBlockName(), getGenerationStamp()));
   }
   
   /**
    * Get the volume where this replica is located on disk
    * @return the volume where this replica is located on disk
    */
-  FsVolumeSpi getVolume() {
+  public FsVolumeSpi getVolume() {
     return volume;
   }
   
@@ -136,7 +129,7 @@ abstract public class ReplicaInfo extends Block implements Replica {
    * Set the parent directory where this replica is located
    * @param dir the parent directory where the replica is located
    */
-  void setDir(File dir) {
+  public void setDir(File dir) {
     this.dir = dir;
   }
 
@@ -145,14 +138,14 @@ abstract public class ReplicaInfo extends Block implements Replica {
    * @return true if the replica has already been unlinked 
    *         or no need to be detached; false otherwise
    */
-  boolean isUnlinked() {
+  public boolean isUnlinked() {
     return true;                // no need to be unlinked
   }
 
   /**
    * set that this replica is unlinked
    */
-  void setUnlinked() {
+  public void setUnlinked() {
     // no need to be unlinked
   }
   
@@ -201,7 +194,7 @@ abstract public class ReplicaInfo extends Block implements Replica {
    *         false if it is already detached or no need to be detached
    * @throws IOException if there is any copy error
    */
-  boolean unlinkBlock(int numLinks) throws IOException {
+  public boolean unlinkBlock(int numLinks) throws IOException {
     if (isUnlinked()) {
       return false;
     }

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

@@ -29,21 +29,21 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
  */
 public class ReplicaNotFoundException extends IOException {
   private static final long serialVersionUID = 1L;
-  final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
-  final static String UNFINALIZED_REPLICA = 
+  public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
+  public final static String UNFINALIZED_REPLICA = 
     "Cannot append to an unfinalized replica ";
-  final static String UNFINALIZED_AND_NONRBW_REPLICA = 
+  public final static String UNFINALIZED_AND_NONRBW_REPLICA = 
     "Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
-  final static String NON_EXISTENT_REPLICA =
+  public final static String NON_EXISTENT_REPLICA =
     "Cannot append to a non-existent replica ";
-  final static String UNEXPECTED_GS_REPLICA =
+  public final static String UNEXPECTED_GS_REPLICA =
     "Cannot append to a replica with unexpeted generation stamp ";
 
   public ReplicaNotFoundException() {
     super();
   }
 
-  ReplicaNotFoundException(ExtendedBlock b) {
+  public ReplicaNotFoundException(ExtendedBlock b) {
     super("Replica not found for " + b);
   }
   

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java

@@ -31,12 +31,12 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
  * A recovery with higher recovery id preempts recoveries with a lower id.
  *
  */
-class ReplicaUnderRecovery extends ReplicaInfo {
+public class ReplicaUnderRecovery extends ReplicaInfo {
   private ReplicaInfo original; // the original replica that needs to be recovered
   private long recoveryId; // recovery id; it is also the generation stamp 
                            // that the replica will be bumped to after recovery
 
-  ReplicaUnderRecovery(ReplicaInfo replica, long recoveryId) {
+  public ReplicaUnderRecovery(ReplicaInfo replica, long recoveryId) {
     super(replica.getBlockId(), replica.getNumBytes(), replica.getGenerationStamp(),
         replica.getVolume(), replica.getDir());
     if ( replica.getState() != ReplicaState.FINALIZED &&
@@ -52,7 +52,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
    * Copy constructor.
    * @param from
    */
-  ReplicaUnderRecovery(ReplicaUnderRecovery from) {
+  public ReplicaUnderRecovery(ReplicaUnderRecovery from) {
     super(from);
     this.original = from.getOriginalReplica();
     this.recoveryId = from.getRecoveryID();
@@ -62,7 +62,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
    * Get the recovery id
    * @return the generation stamp that the replica will be bumped to 
    */
-  long getRecoveryID() {
+  public long getRecoveryID() {
     return recoveryId;
   }
 
@@ -70,7 +70,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
    * Set the recovery id
    * @param recoveryId the new recoveryId
    */
-  void setRecoveryID(long recoveryId) {
+  public void setRecoveryID(long recoveryId) {
     if (recoveryId > this.recoveryId) {
       this.recoveryId = recoveryId;
     } else {
@@ -83,17 +83,17 @@ class ReplicaUnderRecovery extends ReplicaInfo {
    * Get the original replica that's under recovery
    * @return the original replica under recovery
    */
-  ReplicaInfo getOriginalReplica() {
+  public ReplicaInfo getOriginalReplica() {
     return original;
   }
 
   @Override //ReplicaInfo
-  boolean isUnlinked() {
+  public boolean isUnlinked() {
     return original.isUnlinked();
   }
 
   @Override //ReplicaInfo
-  void setUnlinked() {
+  public void setUnlinked() {
     original.setUnlinked();
   }
   
@@ -131,7 +131,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
   }
   
   @Override //ReplicaInfo
-  void setDir(File dir) {
+  public void setDir(File dir) {
     super.setDir(dir);
     original.setDir(dir);
   }
@@ -159,7 +159,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
         + "\n  original=" + original;
   }
 
-  ReplicaRecoveryInfo createInfo() {
+  public ReplicaRecoveryInfo createInfo() {
     return new ReplicaRecoveryInfo(original.getBlockId(), 
         original.getBytesOnDisk(), original.getGenerationStamp(),
         original.getState()); 

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

@@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
  * client continues to write or be recovered as a result of
  * lease recovery.
  */
-class ReplicaWaitingToBeRecovered extends ReplicaInfo {
+public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
   private boolean unlinked;      // copy-on-write done for block
 
   /**
@@ -43,7 +43,7 @@ class ReplicaWaitingToBeRecovered extends ReplicaInfo {
    * @param vol volume where replica is located
    * @param dir directory path where block and meta files are located
    */
-  ReplicaWaitingToBeRecovered(long blockId, long len, long genStamp,
+  public ReplicaWaitingToBeRecovered(long blockId, long len, long genStamp,
       FsVolumeSpi vol, File dir) {
     super(blockId, len, genStamp, vol, dir);
   }
@@ -54,7 +54,7 @@ class ReplicaWaitingToBeRecovered extends ReplicaInfo {
    * @param vol volume where replica is located
    * @param dir directory path where block and meta files are located
    */
-  ReplicaWaitingToBeRecovered(Block block, FsVolumeSpi vol, File dir) {
+  public ReplicaWaitingToBeRecovered(Block block, FsVolumeSpi vol, File dir) {
     super(block, vol, dir);
   }
   
@@ -62,7 +62,7 @@ class ReplicaWaitingToBeRecovered extends ReplicaInfo {
    * Copy constructor.
    * @param from
    */
-  ReplicaWaitingToBeRecovered(ReplicaWaitingToBeRecovered from) {
+  public ReplicaWaitingToBeRecovered(ReplicaWaitingToBeRecovered from) {
     super(from);
     this.unlinked = from.isUnlinked();
   }
@@ -73,12 +73,12 @@ class ReplicaWaitingToBeRecovered extends ReplicaInfo {
   }
   
   @Override //ReplicaInfo
-  boolean isUnlinked() {
+  public boolean isUnlinked() {
     return unlinked;
   }
 
   @Override //ReplicaInfo
-  void setUnlinked() {
+  public void setUnlinked() {
     unlinked = true;
   }
   

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

@@ -33,9 +33,9 @@ import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
 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;
@@ -58,7 +58,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
       @SuppressWarnings("rawtypes")
       final Class<? extends Factory> clazz = conf.getClass(
           DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
-          FSDataset.Factory.class,
+          FsDatasetFactory.class,
           Factory.class);
       return ReflectionUtils.newInstance(clazz, conf);
     }

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

@@ -0,0 +1,286 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DU;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+/**
+ * A block pool slice represents a portion of a block pool stored on a volume.  
+ * Taken together, all BlockPoolSlices sharing a block pool ID across a 
+ * cluster represent a single block pool.
+ * 
+ * This class is synchronized by {@link FsVolumeImpl}.
+ */
+class BlockPoolSlice {
+  private final String bpid;
+  private final FsVolumeImpl volume; // volume to which this BlockPool belongs to
+  private final File currentDir; // StorageDirectory/current/bpid/current
+  private final LDir finalizedDir; // directory store Finalized replica
+  private final File rbwDir; // directory store RBW replica
+  private final File tmpDir; // directory store Temporary replica
+  
+  // TODO:FEDERATION scalability issue - a thread per DU is needed
+  private final DU dfsUsage;
+
+  /**
+   * Create a blook pool slice 
+   * @param bpid Block pool Id
+   * @param volume {@link FsVolumeImpl} to which this BlockPool belongs to
+   * @param bpDir directory corresponding to the BlockPool
+   * @param conf
+   * @throws IOException
+   */
+  BlockPoolSlice(String bpid, FsVolumeImpl volume, File bpDir,
+      Configuration conf) throws IOException {
+    this.bpid = bpid;
+    this.volume = volume;
+    this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT); 
+    final File finalizedDir = new File(
+        currentDir, DataStorage.STORAGE_DIR_FINALIZED);
+
+    // Files that were being written when the datanode was last shutdown
+    // are now moved back to the data directory. It is possible that
+    // in the future, we might want to do some sort of datanode-local
+    // recovery for these blocks. For example, crc validation.
+    //
+    this.tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
+    if (tmpDir.exists()) {
+      FileUtil.fullyDelete(tmpDir);
+    }
+    this.rbwDir = new File(currentDir, DataStorage.STORAGE_DIR_RBW);
+    final boolean supportAppends = conf.getBoolean(
+        DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
+        DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
+    if (rbwDir.exists() && !supportAppends) {
+      FileUtil.fullyDelete(rbwDir);
+    }
+    final int maxBlocksPerDir = conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_KEY,
+        DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_DEFAULT);
+    this.finalizedDir = new LDir(finalizedDir, maxBlocksPerDir);
+    if (!rbwDir.mkdirs()) {  // create rbw directory if not exist
+      if (!rbwDir.isDirectory()) {
+        throw new IOException("Mkdirs failed to create " + rbwDir.toString());
+      }
+    }
+    if (!tmpDir.mkdirs()) {
+      if (!tmpDir.isDirectory()) {
+        throw new IOException("Mkdirs failed to create " + tmpDir.toString());
+      }
+    }
+    this.dfsUsage = new DU(bpDir, conf);
+    this.dfsUsage.start();
+  }
+
+  File getDirectory() {
+    return currentDir.getParentFile();
+  }
+
+  File getFinalizedDir() {
+    return finalizedDir.dir;
+  }
+  
+  File getRbwDir() {
+    return rbwDir;
+  }
+
+  /** Run DU on local drives.  It must be synchronized from caller. */
+  void decDfsUsed(long value) {
+    dfsUsage.decDfsUsed(value);
+  }
+  
+  long getDfsUsed() throws IOException {
+    return dfsUsage.getUsed();
+  }
+  
+  /**
+   * Temporary files. They get moved to the finalized block directory when
+   * the block is finalized.
+   */
+  File createTmpFile(Block b) throws IOException {
+    File f = new File(tmpDir, b.getBlockName());
+    return DatanodeUtil.createTmpFile(b, f);
+  }
+
+  /**
+   * RBW files. They get moved to the finalized block directory when
+   * the block is finalized.
+   */
+  File createRbwFile(Block b) throws IOException {
+    File f = new File(rbwDir, b.getBlockName());
+    return DatanodeUtil.createTmpFile(b, f);
+  }
+
+  File addBlock(Block b, File f) throws IOException {
+    File blockFile = finalizedDir.addBlock(b, f);
+    File metaFile = FsDatasetUtil.getMetaFile(blockFile, b.getGenerationStamp());
+    dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
+    return blockFile;
+  }
+    
+  void checkDirs() throws DiskErrorException {
+    finalizedDir.checkDirTree();
+    DiskChecker.checkDir(tmpDir);
+    DiskChecker.checkDir(rbwDir);
+  }
+    
+  void getVolumeMap(ReplicaMap volumeMap) throws IOException {
+    // add finalized replicas
+    finalizedDir.getVolumeMap(bpid, volumeMap, volume);
+    // add rbw replicas
+    addToReplicasMap(volumeMap, rbwDir, false);
+  }
+
+  /**
+   * Add replicas under the given directory to the volume map
+   * @param volumeMap the replicas map
+   * @param dir an input directory
+   * @param isFinalized true if the directory has finalized replicas;
+   *                    false if the directory has rbw replicas
+   */
+  void addToReplicasMap(ReplicaMap volumeMap, File dir, boolean isFinalized
+      ) throws IOException {
+    File blockFiles[] = FileUtil.listFiles(dir);
+    for (File blockFile : blockFiles) {
+      if (!Block.isBlockFilename(blockFile))
+        continue;
+      
+      long genStamp = FsDatasetUtil.getGenerationStampFromFile(
+          blockFiles, blockFile);
+      long blockId = Block.filename2id(blockFile.getName());
+      ReplicaInfo newReplica = null;
+      if (isFinalized) {
+        newReplica = new FinalizedReplica(blockId, 
+            blockFile.length(), genStamp, volume, blockFile.getParentFile());
+      } else {
+        newReplica = new ReplicaWaitingToBeRecovered(blockId,
+            validateIntegrity(blockFile, genStamp), 
+            genStamp, volume, blockFile.getParentFile());
+      }
+
+      ReplicaInfo oldReplica = volumeMap.add(bpid, newReplica);
+      if (oldReplica != null) {
+        FsDatasetImpl.LOG.warn("Two block files with the same block id exist " +
+            "on disk: " + oldReplica.getBlockFile() + " and " + blockFile );
+      }
+    }
+  }
+  
+  /**
+   * Find out the number of bytes in the block that match its crc.
+   * 
+   * This algorithm assumes that data corruption caused by unexpected 
+   * datanode shutdown occurs only in the last crc chunk. So it checks
+   * only the last chunk.
+   * 
+   * @param blockFile the block file
+   * @param genStamp generation stamp of the block
+   * @return the number of valid bytes
+   */
+  private long validateIntegrity(File blockFile, long genStamp) {
+    DataInputStream checksumIn = null;
+    InputStream blockIn = null;
+    try {
+      final File metaFile = FsDatasetUtil.getMetaFile(blockFile, genStamp);
+      long blockFileLen = blockFile.length();
+      long metaFileLen = metaFile.length();
+      int crcHeaderLen = DataChecksum.getChecksumHeaderSize();
+      if (!blockFile.exists() || blockFileLen == 0 ||
+          !metaFile.exists() || metaFileLen < crcHeaderLen) {
+        return 0;
+      }
+      checksumIn = new DataInputStream(
+          new BufferedInputStream(new FileInputStream(metaFile),
+              HdfsConstants.IO_FILE_BUFFER_SIZE));
+
+      // read and handle the common header here. For now just a version
+      BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
+      short version = header.getVersion();
+      if (version != BlockMetadataHeader.VERSION) {
+        FsDatasetImpl.LOG.warn("Wrong version (" + version + ") for metadata file "
+            + metaFile + " ignoring ...");
+      }
+      DataChecksum checksum = header.getChecksum();
+      int bytesPerChecksum = checksum.getBytesPerChecksum();
+      int checksumSize = checksum.getChecksumSize();
+      long numChunks = Math.min(
+          (blockFileLen + bytesPerChecksum - 1)/bytesPerChecksum, 
+          (metaFileLen - crcHeaderLen)/checksumSize);
+      if (numChunks == 0) {
+        return 0;
+      }
+      IOUtils.skipFully(checksumIn, (numChunks-1)*checksumSize);
+      blockIn = new FileInputStream(blockFile);
+      long lastChunkStartPos = (numChunks-1)*bytesPerChecksum;
+      IOUtils.skipFully(blockIn, lastChunkStartPos);
+      int lastChunkSize = (int)Math.min(
+          bytesPerChecksum, blockFileLen-lastChunkStartPos);
+      byte[] buf = new byte[lastChunkSize+checksumSize];
+      checksumIn.readFully(buf, lastChunkSize, checksumSize);
+      IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
+
+      checksum.update(buf, 0, lastChunkSize);
+      if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
+        return lastChunkStartPos + lastChunkSize;
+      } else { // last chunck is corrupt
+        return lastChunkStartPos;
+      }
+    } catch (IOException e) {
+      FsDatasetImpl.LOG.warn(e);
+      return 0;
+    } finally {
+      IOUtils.closeStream(checksumIn);
+      IOUtils.closeStream(blockIn);
+    }
+  }
+    
+  void clearPath(File f) {
+    finalizedDir.clearPath(f);
+  }
+    
+  @Override
+  public String toString() {
+    return currentDir.getAbsolutePath();
+  }
+  
+  void shutdown() {
+    dfsUsage.shutdown();
+  }
+}

+ 29 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
 import java.util.HashMap;
@@ -29,29 +29,25 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 
-/*
+/**
  * This class is a container of multiple thread pools, each for a volume,
  * so that we can schedule async disk operations easily.
  * 
- * Examples of async disk operations are deletion of block files in FSDataset.
+ * Examples of async disk operations are deletion of block files.
  * We don't want to create a new thread for each of the deletion request, and
  * we don't want to do all deletions in the heartbeat thread since deletion
  * can be slow, and we don't want to use a single thread pool because that
  * is inefficient when we have more than 1 volume.  AsyncDiskService is the
  * solution for these.
  * 
- * This class is used inside FSDataset.
- * 
- * In the future, we should extract AsyncDiskService and put it into common.
- * The FSDataset-specific logic should reside here. 
+ * This class and {@link org.apache.hadoop.util.AsyncDiskService} are similar.
+ * They should be combined.
  */
-class FSDatasetAsyncDiskService {
-  
-  final FSDataset dataset;
-  
-  public static final Log LOG = LogFactory.getLog(FSDatasetAsyncDiskService.class);
+class FsDatasetAsyncDiskService {
+  public static final Log LOG = LogFactory.getLog(FsDatasetAsyncDiskService.class);
   
   // ThreadPool core pool size
   private static final int CORE_THREADS_PER_VOLUME = 1;
@@ -60,9 +56,8 @@ class FSDatasetAsyncDiskService {
   // ThreadPool keep-alive time for threads over core pool size
   private static final long THREADS_KEEP_ALIVE_SECONDS = 60; 
   
-  private final ThreadGroup threadGroup = new ThreadGroup("async disk service");
-  
-  private HashMap<File, ThreadPoolExecutor> executors
+  private final DataNode datanode;
+  private Map<File, ThreadPoolExecutor> executors
       = new HashMap<File, ThreadPoolExecutor>();
   
   /**
@@ -74,8 +69,10 @@ class FSDatasetAsyncDiskService {
    * 
    * @param volumes The roots of the data volumes.
    */
-  FSDatasetAsyncDiskService(FSDataset dataset, File[] volumes) {
-    this.dataset = dataset;
+  FsDatasetAsyncDiskService(DataNode datanode, File[] volumes) {
+    this.datanode = datanode;
+
+    final ThreadGroup threadGroup = new ThreadGroup(getClass().getSimpleName());
     // Create one ThreadPool per volume
     for (int v = 0 ; v < volumes.length; v++) {
       final File vol = volumes[v];
@@ -136,16 +133,12 @@ class FSDatasetAsyncDiskService {
    * tasks to finish.
    */
   synchronized void shutdown() {
-    
     if (executors == null) {
-      
       LOG.warn("AsyncDiskService has already shut down.");
-      
     } else {
       LOG.info("Shutting down all async disk service threads...");
       
-      for (Map.Entry<File, ThreadPoolExecutor> e
-          : executors.entrySet()) {
+      for (Map.Entry<File, ThreadPoolExecutor> e : executors.entrySet()) {
         e.getValue().shutdown();
       }
       // clear the executor map so that calling execute again will fail.
@@ -159,11 +152,11 @@ class FSDatasetAsyncDiskService {
    * Delete the block file and meta file from the disk asynchronously, adjust
    * dfsUsed statistics accordingly.
    */
-  void deleteAsync(FSDataset.FSVolume volume, File blockFile, File metaFile,
+  void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile,
       ExtendedBlock block) {
-    DataNode.LOG.info("Scheduling block " + block.getLocalBlock().toString()
+    LOG.info("Scheduling block " + block.getLocalBlock()
         + " file " + blockFile + " for deletion");
-    ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(dataset,
+    ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
         volume, blockFile, metaFile, block);
     execute(volume.getCurrentDir(), deletionTask);
   }
@@ -171,51 +164,43 @@ class FSDatasetAsyncDiskService {
   /** A task for deleting a block file and its associated meta file, as well
    *  as decrement the dfs usage of the volume. 
    */
-  static class ReplicaFileDeleteTask implements Runnable {
-    final FSDataset dataset;
-    final FSDataset.FSVolume volume;
+  class ReplicaFileDeleteTask implements Runnable {
+    final FsVolumeImpl volume;
     final File blockFile;
     final File metaFile;
     final ExtendedBlock block;
     
-    ReplicaFileDeleteTask(FSDataset dataset, FSDataset.FSVolume volume, File blockFile,
+    ReplicaFileDeleteTask(FsVolumeImpl volume, File blockFile,
         File metaFile, ExtendedBlock block) {
-      this.dataset = dataset;
       this.volume = volume;
       this.blockFile = blockFile;
       this.metaFile = metaFile;
       this.block = block;
     }
-    
-    FSDataset.FSVolume getVolume() {
-      return volume;
-    }
 
     @Override
     public String toString() {
       // Called in AsyncDiskService.execute for displaying error messages.
       return "deletion of block " + block.getBlockPoolId() + " "
-          + block.getLocalBlock().toString() + " with block file " + blockFile
+          + block.getLocalBlock() + " with block file " + blockFile
           + " and meta file " + metaFile + " from volume " + volume;
     }
 
     @Override
     public void run() {
       long dfsBytes = blockFile.length() + metaFile.length();
-      if ( !blockFile.delete() || ( !metaFile.delete() && metaFile.exists() ) ) {
-        DataNode.LOG.warn("Unexpected error trying to delete block "
-            + block.getBlockPoolId() + " " + block.getLocalBlock().toString()
+      if (!blockFile.delete() || (!metaFile.delete() && metaFile.exists())) {
+        LOG.warn("Unexpected error trying to delete block "
+            + block.getBlockPoolId() + " " + block.getLocalBlock()
             + " at file " + blockFile + ". Ignored.");
       } else {
         if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){
-          dataset.notifyNamenodeDeletedBlock(block);
+          datanode.notifyNamenodeDeletedBlock(block);
         }
         volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
-        DataNode.LOG.info("Deleted block " + block.getBlockPoolId() + " "
-            + block.getLocalBlock().toString() + " at file " + blockFile);
+        LOG.info("Deleted block " + block.getBlockPoolId() + " "
+            + block.getLocalBlock() + " at file " + blockFile);
       }
     }
-  };
-  
-  
+  }
 }

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

@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+
+/**
+ * A factory for creating {@link FsDatasetImpl} objects.
+ */
+public class FsDatasetFactory extends FsDatasetSpi.Factory<FsDatasetImpl> {
+  @Override
+  public FsDatasetImpl newInstance(DataNode datanode,
+      DataStorage storage, Configuration conf) throws IOException {
+    return new FsDatasetImpl(datanode, storage, conf);
+  }
+}

Fichier diff supprimé car celui-ci est trop grand
+ 21 - 955
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java


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

@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+
+/** Utility methods. */
+@InterfaceAudience.Private
+public class FsDatasetUtil {
+  static boolean isUnlinkTmpFile(File f) {
+    return f.getName().endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX);
+  }
+
+  static File getOrigFile(File unlinkTmpFile) {
+    final String name = unlinkTmpFile.getName();
+    if (!name.endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX)) {
+      throw new IllegalArgumentException("unlinkTmpFile=" + unlinkTmpFile
+          + " does not end with " + DatanodeUtil.UNLINK_BLOCK_SUFFIX);
+    }
+    final int n = name.length() - DatanodeUtil.UNLINK_BLOCK_SUFFIX.length(); 
+    return new File(unlinkTmpFile.getParentFile(), name.substring(0, n));
+  }
+  
+  static File getMetaFile(File f, long gs) {
+    return new File(f.getParent(),
+        DatanodeUtil.getMetaName(f.getName(), gs));
+  }
+
+  /** Find the corresponding meta data file from a given block file */
+  static File findMetaFile(final File blockFile) throws IOException {
+    final String prefix = blockFile.getName() + "_";
+    final File parent = blockFile.getParentFile();
+    final File[] matches = parent.listFiles(new FilenameFilter() {
+      @Override
+      public boolean accept(File dir, String name) {
+        return dir.equals(parent) && name.startsWith(prefix)
+            && name.endsWith(Block.METADATA_EXTENSION);
+      }
+    });
+
+    if (matches == null || matches.length == 0) {
+      throw new IOException("Meta file not found, blockFile=" + blockFile);
+    }
+    if (matches.length > 1) {
+      throw new IOException("Found more than one meta files: " 
+          + Arrays.asList(matches));
+    }
+    return matches[0];
+  }
+
+  /**
+   * Find the meta-file for the specified block file
+   * and then return the generation stamp from the name of the meta-file.
+   */
+  static long getGenerationStampFromFile(File[] listdir, File blockFile) {
+    String blockName = blockFile.getName();
+    for (int j = 0; j < listdir.length; j++) {
+      String path = listdir[j].getName();
+      if (!path.startsWith(blockName)) {
+        continue;
+      }
+      if (blockFile == listdir[j]) {
+        continue;
+      }
+      return Block.getGenerationStamp(listdir[j].getName());
+    }
+    FsDatasetImpl.LOG.warn("Block " + blockFile + " does not have a metafile!");
+    return GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+  }
+
+  /** Find the corresponding meta data file from a given block file */
+  static long parseGenerationStamp(File blockFile, File metaFile
+      ) throws IOException {
+    final String metaname = metaFile.getName();
+    final String gs = metaname.substring(blockFile.getName().length() + 1,
+        metaname.length() - Block.METADATA_EXTENSION.length());
+    try {
+      return Long.parseLong(gs);
+    } catch(NumberFormatException nfe) {
+      throw new IOException("Failed to parse generation stamp: blockFile="
+          + blockFile + ", metaFile=" + metaFile, nfe);
+    }
+  }
+}

+ 288 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.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.server.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DF;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+/**
+ * The underlying volume used to store replica.
+ * 
+ * It uses the {@link FsDatasetImpl} object for synchronization.
+ */
+@InterfaceAudience.Private
+class FsVolumeImpl implements FsVolumeSpi {
+  private final FsDatasetImpl dataset;
+  private final String storageID;
+  private final Map<String, BlockPoolSlice> bpSlices
+      = new HashMap<String, BlockPoolSlice>();
+  private final File currentDir;    // <StorageDirectory>/current
+  private final DF usage;           
+  private final long reserved;
+  
+  FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
+      Configuration conf) throws IOException {
+    this.dataset = dataset;
+    this.storageID = storageID;
+    this.reserved = conf.getLong(
+        DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
+        DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT);
+    this.currentDir = currentDir; 
+    File parent = currentDir.getParentFile();
+    this.usage = new DF(parent, conf);
+  }
+  
+  File getCurrentDir() {
+    return currentDir;
+  }
+  
+  File getRbwDir(String bpid) throws IOException {
+    return getBlockPoolSlice(bpid).getRbwDir();
+  }
+  
+  void decDfsUsed(String bpid, long value) {
+    synchronized(dataset) {
+      BlockPoolSlice bp = bpSlices.get(bpid);
+      if (bp != null) {
+        bp.decDfsUsed(value);
+      }
+    }
+  }
+  
+  long getDfsUsed() throws IOException {
+    long dfsUsed = 0;
+    synchronized(dataset) {
+      for(BlockPoolSlice s : bpSlices.values()) {
+        dfsUsed += s.getDfsUsed();
+      }
+    }
+    return dfsUsed;
+  }
+
+  long getBlockPoolUsed(String bpid) throws IOException {
+    return getBlockPoolSlice(bpid).getDfsUsed();
+  }
+  
+  /**
+   * Calculate the capacity of the filesystem, after removing any
+   * reserved capacity.
+   * @return the unreserved number of bytes left in this filesystem. May be zero.
+   */
+  long getCapacity() {
+    long remaining = usage.getCapacity() - reserved;
+    return remaining > 0 ? remaining : 0;
+  }
+
+  @Override
+  public long getAvailable() throws IOException {
+    long remaining = getCapacity()-getDfsUsed();
+    long available = usage.getAvailable();
+    if (remaining > available) {
+      remaining = available;
+    }
+    return (remaining > 0) ? remaining : 0;
+  }
+    
+  long getReserved(){
+    return reserved;
+  }
+
+  BlockPoolSlice getBlockPoolSlice(String bpid) throws IOException {
+    BlockPoolSlice bp = bpSlices.get(bpid);
+    if (bp == null) {
+      throw new IOException("block pool " + bpid + " is not found");
+    }
+    return bp;
+  }
+
+  @Override
+  public String getPath(String bpid) throws IOException {
+    return getBlockPoolSlice(bpid).getDirectory().getAbsolutePath();
+  }
+
+  @Override
+  public File getFinalizedDir(String bpid) throws IOException {
+    return getBlockPoolSlice(bpid).getFinalizedDir();
+  }
+
+  /**
+   * Make a deep copy of the list of currently active BPIDs
+   */
+  @Override
+  public String[] getBlockPoolList() {
+    return bpSlices.keySet().toArray(new String[bpSlices.keySet().size()]);   
+  }
+    
+  /**
+   * Temporary files. They get moved to the finalized block directory when
+   * the block is finalized.
+   */
+  File createTmpFile(String bpid, Block b) throws IOException {
+    return getBlockPoolSlice(bpid).createTmpFile(b);
+  }
+
+  /**
+   * RBW files. They get moved to the finalized block directory when
+   * the block is finalized.
+   */
+  File createRbwFile(String bpid, Block b) throws IOException {
+    return getBlockPoolSlice(bpid).createRbwFile(b);
+  }
+
+  File addBlock(String bpid, Block b, File f) throws IOException {
+    return getBlockPoolSlice(bpid).addBlock(b, f);
+  }
+    
+  void checkDirs() throws DiskErrorException {
+    // TODO:FEDERATION valid synchronization
+    for(BlockPoolSlice s : bpSlices.values()) {
+      s.checkDirs();
+    }
+  }
+    
+  void getVolumeMap(ReplicaMap volumeMap) throws IOException {
+    for(BlockPoolSlice s : bpSlices.values()) {
+      s.getVolumeMap(volumeMap);
+    }
+  }
+  
+  void getVolumeMap(String bpid, ReplicaMap volumeMap) throws IOException {
+    getBlockPoolSlice(bpid).getVolumeMap(volumeMap);
+  }
+  
+  /**
+   * Add replicas under the given directory to the volume map
+   * @param volumeMap the replicas map
+   * @param dir an input directory
+   * @param isFinalized true if the directory has finalized replicas;
+   *                    false if the directory has rbw replicas
+   * @throws IOException 
+   */
+  void addToReplicasMap(String bpid, ReplicaMap volumeMap, 
+      File dir, boolean isFinalized) throws IOException {
+    BlockPoolSlice bp = getBlockPoolSlice(bpid);
+    // TODO move this up
+    // dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
+    bp.addToReplicasMap(volumeMap, dir, isFinalized);
+  }
+  
+  void clearPath(String bpid, File f) throws IOException {
+    getBlockPoolSlice(bpid).clearPath(f);
+  }
+
+  @Override
+  public String toString() {
+    return currentDir.getAbsolutePath();
+  }
+
+  void shutdown() {
+    Set<Entry<String, BlockPoolSlice>> set = bpSlices.entrySet();
+    for (Entry<String, BlockPoolSlice> entry : set) {
+      entry.getValue().shutdown();
+    }
+  }
+
+  void addBlockPool(String bpid, Configuration conf) throws IOException {
+    File bpdir = new File(currentDir, bpid);
+    BlockPoolSlice bp = new BlockPoolSlice(bpid, this, bpdir, conf);
+    bpSlices.put(bpid, bp);
+  }
+  
+  void shutdownBlockPool(String bpid) {
+    BlockPoolSlice bp = bpSlices.get(bpid);
+    if (bp != null) {
+      bp.shutdown();
+    }
+    bpSlices.remove(bpid);
+  }
+
+  boolean isBPDirEmpty(String bpid) throws IOException {
+    File volumeCurrentDir = this.getCurrentDir();
+    File bpDir = new File(volumeCurrentDir, bpid);
+    File bpCurrentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
+    File finalizedDir = new File(bpCurrentDir,
+        DataStorage.STORAGE_DIR_FINALIZED);
+    File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
+    if (finalizedDir.exists() && FileUtil.list(finalizedDir).length != 0) {
+      return false;
+    }
+    if (rbwDir.exists() && FileUtil.list(rbwDir).length != 0) {
+      return false;
+    }
+    return true;
+  }
+  
+  void deleteBPDirectories(String bpid, boolean force) throws IOException {
+    File volumeCurrentDir = this.getCurrentDir();
+    File bpDir = new File(volumeCurrentDir, bpid);
+    if (!bpDir.isDirectory()) {
+      // nothing to be deleted
+      return;
+    }
+    File tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP); 
+    File bpCurrentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
+    File finalizedDir = new File(bpCurrentDir,
+        DataStorage.STORAGE_DIR_FINALIZED);
+    File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
+    if (force) {
+      FileUtil.fullyDelete(bpDir);
+    } else {
+      if (!rbwDir.delete()) {
+        throw new IOException("Failed to delete " + rbwDir);
+      }
+      if (!finalizedDir.delete()) {
+        throw new IOException("Failed to delete " + finalizedDir);
+      }
+      FileUtil.fullyDelete(tmpDir);
+      for (File f : FileUtil.listFiles(bpCurrentDir)) {
+        if (!f.delete()) {
+          throw new IOException("Failed to delete " + f);
+        }
+      }
+      if (!bpCurrentDir.delete()) {
+        throw new IOException("Failed to delete " + bpCurrentDir);
+      }
+      for (File f : FileUtil.listFiles(bpDir)) {
+        if (!f.delete()) {
+          throw new IOException("Failed to delete " + f);
+        }
+      }
+      if (!bpDir.delete()) {
+        throw new IOException("Failed to delete " + bpDir);
+      }
+    }
+  }
+
+  String getStorageID() {
+    return storageID;
+  }
+}

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

@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+class FsVolumeList {
+  /**
+   * Read access to this unmodifiable list is not synchronized.
+   * This list is replaced on modification holding "this" lock.
+   */
+  volatile List<FsVolumeImpl> volumes = null;
+
+  private final VolumeChoosingPolicy<FsVolumeImpl> blockChooser;
+  private volatile int numFailedVolumes;
+
+  FsVolumeList(List<FsVolumeImpl> volumes, int failedVols,
+      VolumeChoosingPolicy<FsVolumeImpl> blockChooser) {
+    this.volumes = Collections.unmodifiableList(volumes);
+    this.blockChooser = blockChooser;
+    this.numFailedVolumes = failedVols;
+  }
+  
+  int numberOfFailedVolumes() {
+    return numFailedVolumes;
+  }
+  
+  /** 
+   * Get next volume. Synchronized to ensure {@link #curVolume} is updated
+   * by a single thread and next volume is chosen with no concurrent
+   * update to {@link #volumes}.
+   * @param blockSize free space needed on the volume
+   * @return next volume to store the block in.
+   */
+  synchronized FsVolumeImpl getNextVolume(long blockSize) throws IOException {
+    return blockChooser.chooseVolume(volumes, blockSize);
+  }
+    
+  long getDfsUsed() throws IOException {
+    long dfsUsed = 0L;
+    for (FsVolumeImpl v : volumes) {
+      dfsUsed += v.getDfsUsed();
+    }
+    return dfsUsed;
+  }
+
+  long getBlockPoolUsed(String bpid) throws IOException {
+    long dfsUsed = 0L;
+    for (FsVolumeImpl v : volumes) {
+      dfsUsed += v.getBlockPoolUsed(bpid);
+    }
+    return dfsUsed;
+  }
+
+  long getCapacity() {
+    long capacity = 0L;
+    for (FsVolumeImpl v : volumes) {
+      capacity += v.getCapacity();
+    }
+    return capacity;
+  }
+    
+  long getRemaining() throws IOException {
+    long remaining = 0L;
+    for (FsVolumeSpi vol : volumes) {
+      remaining += vol.getAvailable();
+    }
+    return remaining;
+  }
+    
+  void getVolumeMap(ReplicaMap volumeMap) throws IOException {
+    for (FsVolumeImpl v : volumes) {
+      v.getVolumeMap(volumeMap);
+    }
+  }
+  
+  void getVolumeMap(String bpid, ReplicaMap volumeMap) throws IOException {
+    for (FsVolumeImpl v : volumes) {
+      v.getVolumeMap(bpid, volumeMap);
+    }
+  }
+    
+  /**
+   * Calls {@link FsVolumeImpl#checkDirs()} on each volume, removing any
+   * volumes from the active list that result in a DiskErrorException.
+   * 
+   * This method is synchronized to allow only one instance of checkDirs() 
+   * call
+   * @return list of all the removed volumes.
+   */
+  synchronized List<FsVolumeImpl> checkDirs() {
+    ArrayList<FsVolumeImpl> removedVols = null;
+    
+    // Make a copy of volumes for performing modification 
+    final List<FsVolumeImpl> volumeList = new ArrayList<FsVolumeImpl>(volumes);
+
+    for(Iterator<FsVolumeImpl> i = volumeList.iterator(); i.hasNext(); ) {
+      final FsVolumeImpl fsv = i.next();
+      try {
+        fsv.checkDirs();
+      } catch (DiskErrorException e) {
+        FsDatasetImpl.LOG.warn("Removing failed volume " + fsv + ": ",e);
+        if (removedVols == null) {
+          removedVols = new ArrayList<FsVolumeImpl>(1);
+        }
+        removedVols.add(fsv);
+        fsv.shutdown(); 
+        i.remove(); // Remove the volume
+        numFailedVolumes++;
+      }
+    }
+    
+    if (removedVols != null && removedVols.size() > 0) {
+      // Replace volume list
+      volumes = Collections.unmodifiableList(volumeList);
+      FsDatasetImpl.LOG.info("Completed checkDirs. Removed " + removedVols.size()
+          + " volumes. Current volumes: " + this);
+    }
+
+    return removedVols;
+  }
+
+  @Override
+  public String toString() {
+    return volumes.toString();
+  }
+
+
+  void addBlockPool(String bpid, Configuration conf) throws IOException {
+    for (FsVolumeImpl v : volumes) {
+      v.addBlockPool(bpid, conf);
+    }
+  }
+  
+  void removeBlockPool(String bpid) {
+    for (FsVolumeImpl v : volumes) {
+      v.shutdownBlockPool(bpid);
+    }
+  }
+
+  void shutdown() {
+    for (FsVolumeImpl volume : volumes) {
+      if(volume != null) {
+        volume.shutdown();
+      }
+    }
+  }
+}

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

@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+/**
+ * A node type that can be built into a tree reflecting the
+ * hierarchy of replicas on the local disk.
+ */
+class LDir {
+  final File dir;
+  final int maxBlocksPerDir;
+
+  private int numBlocks = 0;
+  private LDir[] children = null;
+  private int lastChildIdx = 0;
+
+  LDir(File dir, int maxBlocksPerDir) throws IOException {
+    this.dir = dir;
+    this.maxBlocksPerDir = maxBlocksPerDir;
+
+    if (!dir.exists()) {
+      if (!dir.mkdirs()) {
+        throw new IOException("Failed to mkdirs " + dir);
+      }
+    } else {
+      File[] files = FileUtil.listFiles(dir); 
+      List<LDir> dirList = new ArrayList<LDir>();
+      for (int idx = 0; idx < files.length; idx++) {
+        if (files[idx].isDirectory()) {
+          dirList.add(new LDir(files[idx], maxBlocksPerDir));
+        } else if (Block.isBlockFilename(files[idx])) {
+          numBlocks++;
+        }
+      }
+      if (dirList.size() > 0) {
+        children = dirList.toArray(new LDir[dirList.size()]);
+      }
+    }
+  }
+      
+  File addBlock(Block b, File src) throws IOException {
+    //First try without creating subdirectories
+    File file = addBlock(b, src, false, false);          
+    return (file != null) ? file : addBlock(b, src, true, true);
+  }
+
+  private File addBlock(Block b, File src, boolean createOk, boolean resetIdx
+      ) throws IOException {
+    if (numBlocks < maxBlocksPerDir) {
+      final File dest = FsDatasetImpl.moveBlockFiles(b, src, dir);
+      numBlocks += 1;
+      return dest;
+    }
+          
+    if (lastChildIdx < 0 && resetIdx) {
+      //reset so that all children will be checked
+      lastChildIdx = DFSUtil.getRandom().nextInt(children.length);              
+    }
+          
+    if (lastChildIdx >= 0 && children != null) {
+      //Check if any child-tree has room for a block.
+      for (int i=0; i < children.length; i++) {
+        int idx = (lastChildIdx + i)%children.length;
+        File file = children[idx].addBlock(b, src, false, resetIdx);
+        if (file != null) {
+          lastChildIdx = idx;
+          return file; 
+        }
+      }
+      lastChildIdx = -1;
+    }
+          
+    if (!createOk) {
+      return null;
+    }
+          
+    if (children == null || children.length == 0) {
+      children = new LDir[maxBlocksPerDir];
+      for (int idx = 0; idx < maxBlocksPerDir; idx++) {
+        final File sub = new File(dir, DataStorage.BLOCK_SUBDIR_PREFIX+idx);
+        children[idx] = new LDir(sub, maxBlocksPerDir);
+      }
+    }
+          
+    //now pick a child randomly for creating a new set of subdirs.
+    lastChildIdx = DFSUtil.getRandom().nextInt(children.length);
+    return children[ lastChildIdx ].addBlock(b, src, true, false); 
+  }
+
+  void getVolumeMap(String bpid, ReplicaMap volumeMap, FsVolumeImpl volume
+      ) throws IOException {
+    if (children != null) {
+      for (int i = 0; i < children.length; i++) {
+        children[i].getVolumeMap(bpid, volumeMap, volume);
+      }
+    }
+
+    recoverTempUnlinkedBlock();
+    volume.addToReplicasMap(bpid, volumeMap, dir, true);
+  }
+      
+  /**
+   * Recover unlinked tmp files on datanode restart. If the original block
+   * does not exist, then the tmp file is renamed to be the
+   * original file name; otherwise the tmp file is deleted.
+   */
+  private void recoverTempUnlinkedBlock() throws IOException {
+    File files[] = FileUtil.listFiles(dir);
+    for (File file : files) {
+      if (!FsDatasetUtil.isUnlinkTmpFile(file)) {
+        continue;
+      }
+      File blockFile = FsDatasetUtil.getOrigFile(file);
+      if (blockFile.exists()) {
+        // If the original block file still exists, then no recovery  is needed.
+        if (!file.delete()) {
+          throw new IOException("Unable to cleanup unlinked tmp file " + file);
+        }
+      } else {
+        if (!file.renameTo(blockFile)) {
+          throw new IOException("Unable to cleanup detached file " + file);
+        }
+      }
+    }
+  }
+  
+  /**
+   * check if a data diretory is healthy
+   * @throws DiskErrorException
+   */
+  void checkDirTree() throws DiskErrorException {
+    DiskChecker.checkDir(dir);
+          
+    if (children != null) {
+      for (int i = 0; i < children.length; i++) {
+        children[i].checkDirTree();
+      }
+    }
+  }
+      
+  void clearPath(File f) {
+    String root = dir.getAbsolutePath();
+    String dir = f.getAbsolutePath();
+    if (dir.startsWith(root)) {
+      String[] dirNames = dir.substring(root.length()).
+        split(File.separator + DataStorage.BLOCK_SUBDIR_PREFIX);
+      if (clearPath(f, dirNames, 1))
+        return;
+    }
+    clearPath(f, null, -1);
+  }
+      
+  /**
+   * dirNames is an array of string integers derived from
+   * usual directory structure data/subdirN/subdirXY/subdirM ...
+   * If dirName array is non-null, we only check the child at 
+   * the children[dirNames[idx]]. This avoids iterating over
+   * children in common case. If directory structure changes 
+   * in later versions, we need to revisit this.
+   */
+  private boolean clearPath(File f, String[] dirNames, int idx) {
+    if ((dirNames == null || idx == dirNames.length) &&
+        dir.compareTo(f) == 0) {
+      numBlocks--;
+      return true;
+    }
+        
+    if (dirNames != null) {
+      //guess the child index from the directory name
+      if (idx > (dirNames.length - 1) || children == null) {
+        return false;
+      }
+      int childIdx; 
+      try {
+        childIdx = Integer.parseInt(dirNames[idx]);
+      } catch (NumberFormatException ignored) {
+        // layout changed? we could print a warning.
+        return false;
+      }
+      return (childIdx >= 0 && childIdx < children.length) ?
+        children[childIdx].clearPath(f, dirNames, idx+1) : false;
+    }
+
+    //guesses failed. back to blind iteration.
+    if (children != null) {
+      for(int i=0; i < children.length; i++) {
+        if (children[i].clearPath(f, null, -1)){
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return "FSDir{dir=" + dir + ", children="
+        + (children == null ? null : Arrays.asList(children)) + "}";
+  }
+}

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicasMap.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.util.Collection;
 import java.util.HashMap;
@@ -23,11 +23,12 @@ import java.util.Map;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 
 /**
- * Maintains the replicas map. 
+ * Maintains the replica map. 
  */
-class ReplicasMap {
+class ReplicaMap {
   // Object using which this class is synchronized
   private final Object mutex;
   
@@ -35,7 +36,7 @@ class ReplicasMap {
   private Map<String, Map<Long, ReplicaInfo>> map = 
     new HashMap<String, Map<Long, ReplicaInfo>>();
   
-  ReplicasMap(Object mutex) {
+  ReplicaMap(Object mutex) {
     if (mutex == null) {
       throw new HadoopIllegalArgumentException(
           "Object to synchronize on cannot be null");

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

@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
+
+class RollingLogsImpl implements RollingLogs {
+  private static final String CURR_SUFFIX = ".curr";
+  private static final String PREV_SUFFIX = ".prev";
+
+  static boolean isFilePresent(String dir, String filePrefix) {
+    return new File(dir, filePrefix + CURR_SUFFIX).exists() ||
+           new File(dir, filePrefix + PREV_SUFFIX).exists();
+  }
+
+  private final File curr;
+  private final File prev;
+  private PrintStream out; //require synchronized access
+
+  private Appender appender = new Appender() {
+    @Override
+    public Appendable append(CharSequence csq) {
+      synchronized(RollingLogsImpl.this) {
+        if (out == null) {
+          throw new IllegalStateException(RollingLogsImpl.this
+              + " is not yet opened.");
+        }
+        out.print(csq);
+      }
+      return this;
+    }
+
+    @Override
+    public Appendable append(char c) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Appendable append(CharSequence csq, int start, int end) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void close() {
+      synchronized(RollingLogsImpl.this) {
+        if (out != null) {
+          out.close();
+          out = null;
+        }
+      }
+    }
+  };
+
+
+  private final AtomicInteger numReaders = new AtomicInteger();
+
+  RollingLogsImpl(String dir, String filePrefix) throws FileNotFoundException{
+    curr = new File(dir, filePrefix + CURR_SUFFIX);
+    prev = new File(dir, filePrefix + PREV_SUFFIX);
+    out = new PrintStream(new FileOutputStream(curr, true));
+  }
+
+  @Override
+  public Reader iterator(boolean skipPrevFile) throws IOException {
+    numReaders.incrementAndGet(); 
+    return new Reader(skipPrevFile);
+  }
+
+  @Override
+  public Appender appender() {
+    return appender;
+  }
+
+  @Override
+  public boolean roll() throws IOException {
+    if (numReaders.get() > 0) {
+      return false;
+    }
+    if (!prev.delete() && prev.exists()) {
+      throw new IOException("Failed to delete " + prev);
+    }
+
+    synchronized(this) {
+      appender.close();
+      final boolean renamed = curr.renameTo(prev);
+      out = new PrintStream(new FileOutputStream(curr, true));
+      if (!renamed) {
+        throw new IOException("Failed to rename " + curr + " to " + prev);
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return curr.toString();
+  }
+  
+  /**
+   * This is used to read the lines in order.
+   * If the data is not read completely (i.e, untill hasNext() returns
+   * false), it needs to be explicitly 
+   */
+  private class Reader implements RollingLogs.LineIterator {
+    private File file;
+    private BufferedReader reader;
+    private String line;
+    private boolean closed = false;
+    
+    private Reader(boolean skipPrevFile) throws IOException {
+      reader = null;
+      file = skipPrevFile? curr : prev;
+      readNext();        
+    }
+
+    @Override
+    public boolean isPrevious() {
+      return file == prev;
+    }
+
+    private boolean openFile() throws IOException {
+
+      for(int i=0; i<2; i++) {
+        if (reader != null || i > 0) {
+          // move to next file
+          file = isPrevious()? curr : null;
+        }
+        if (file == null) {
+          return false;
+        }
+        if (file.exists()) {
+          break;
+        }
+      }
+      
+      if (reader != null ) {
+        reader.close();
+        reader = null;
+      }
+      
+      reader = new BufferedReader(new FileReader(file));
+      return true;
+    }
+    
+    // read next line if possible.
+    private void readNext() throws IOException {
+      line = null;
+      try {
+        if (reader != null && (line = reader.readLine()) != null) {
+          return;
+        }
+        if (line == null) {
+          // move to the next file.
+          if (openFile()) {
+            readNext();
+          }
+        }
+      } finally {
+        if (!hasNext()) {
+          close();
+        }
+      }
+    }
+    
+    @Override
+    public boolean hasNext() {
+      return line != null;
+    }
+
+    @Override
+    public String next() {
+      String curLine = line;
+      try {
+        readNext();
+      } catch (IOException e) {
+        DataBlockScanner.LOG.warn("Failed to read next line.", e);
+      }
+      return curLine;
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        try {
+          if (reader != null) {
+            reader.close();
+          }
+        } finally {
+          file = null;
+          reader = null;
+          closed = true;
+          final int n = numReaders.decrementAndGet();
+          assert(n >= 0);
+        }
+      }
+    }
+  }
+}

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.BlockingService;
 
 /**
@@ -171,6 +172,12 @@ public class BackupNode extends NameNode {
 
   @Override // NameNode
   public void stop() {
+    stop(true);
+  }
+  
+  @VisibleForTesting
+  void stop(boolean reportError) {
+   
     if(checkpointManager != null) {
       // Prevent from starting a new checkpoint.
       // Checkpoints that has already been started may proceed until 
@@ -180,7 +187,10 @@ public class BackupNode extends NameNode {
       // ClosedByInterruptException.
       checkpointManager.shouldRun = false;
     }
-    if(namenode != null && getRegistration() != null) {
+    
+    // reportError is a test hook to simulate backupnode crashing and not
+    // doing a clean exit w.r.t active namenode
+    if (reportError && namenode != null && getRegistration() != null) {
       // Exclude this node from the list of backup streams on the name-node
       try {
         namenode.errorReport(getRegistration(), NamenodeProtocol.FATAL,

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

@@ -1020,7 +1020,7 @@ public class FSEditLog  {
     
     LOG.info("Registering new backup node: " + bnReg);
     BackupJournalManager bjm = new BackupJournalManager(bnReg, nnReg);
-    journalSet.add(bjm, true);
+    journalSet.add(bjm, false);
   }
   
   synchronized void releaseBackupStream(NamenodeRegistration registration)

+ 9 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -300,8 +300,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   NameNodeResourceChecker nnResourceChecker;
 
   private FsServerDefaults serverDefaults;
-  // allow appending to hdfs files
-  private boolean supportAppends = true;
+
+  private boolean supportAppends;
   private ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure = 
       ReplaceDatanodeOnFailure.DEFAULT;
 
@@ -1813,9 +1813,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws AccessControlException, SafeModeException,
       FileAlreadyExistsException, FileNotFoundException,
       ParentNotDirectoryException, IOException {
-    if (supportAppends == false) {
-      throw new UnsupportedOperationException("Append to hdfs not supported." +
-                            " Please refer to dfs.support.append configuration parameter.");
+    if (!supportAppends) {
+      throw new UnsupportedOperationException(
+          "Append is not enabled on this NameNode. Use the " +
+          DFS_SUPPORT_APPEND_KEY + " configuration option to enable it.");
     }
     LocatedBlock lb = null;
     writeLock();
@@ -2896,9 +2897,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
         //remove lease, close file
         finalizeINodeFileUnderConstruction(src, pendingFile);
-      } else if (supportAppends) {
+      } else {
         // If this commit does not want to close the file, persist blocks
-        // only if append is supported or we're explicitly told to
         dir.persistBlocks(src, pendingFile);
       }
     } finally {
@@ -4481,9 +4481,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } finally {
       writeUnlock();
     }
-    if (supportAppends || persistBlocks) {
-      getEditLog().logSync();
-    }
+    getEditLog().logSync();
     LOG.info("updatePipeline(" + oldBlock + ") successfully to " + newBlock);
   }
 
@@ -4522,11 +4520,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     blockinfo.setExpectedLocations(descriptors);
 
-    // persist blocks only if append is supported
     String src = leaseManager.findPath(pendingFile);
-    if (supportAppends) {
-      dir.persistBlocks(src, pendingFile);
-    }
+    dir.persistBlocks(src, pendingFile);
   }
 
   // rename was successful. If any part of the renamed subtree had

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.server.namenode.CancelDelegationTokenServlet;
 import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
 import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
+import org.apache.hadoop.hdfs.web.URLUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
@@ -224,8 +225,7 @@ public class DelegationTokenFetcher {
       
       URL remoteURL = new URL(url.toString());
       SecurityUtil.fetchServiceTicket(remoteURL);
-      URLConnection connection = remoteURL.openConnection();
-
+      URLConnection connection = URLUtils.openConnection(remoteURL);
       InputStream in = connection.getInputStream();
       Credentials ts = new Credentials();
       dis = new DataInputStream(in);
@@ -265,7 +265,7 @@ public class DelegationTokenFetcher {
     try {
       URL url = new URL(buf.toString());
       SecurityUtil.fetchServiceTicket(url);
-      connection = (HttpURLConnection) url.openConnection();
+      connection = (HttpURLConnection)URLUtils.openConnection(url);
       if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
         throw new IOException("Error renewing token: " + 
             connection.getResponseMessage());
@@ -359,7 +359,7 @@ public class DelegationTokenFetcher {
     try {
       URL url = new URL(buf.toString());
       SecurityUtil.fetchServiceTicket(url);
-      connection = (HttpURLConnection) url.openConnection();
+      connection = (HttpURLConnection)URLUtils.openConnection(url);
       if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
         throw new IOException("Error cancelling token: " + 
             connection.getResponseMessage());

+ 6 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -513,18 +513,13 @@ public class JsonUtil {
     final byte[] bytes = StringUtils.hexStringToByte((String)m.get("bytes"));
 
     final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
-    final int bytesPerCRC = in.readInt();
-    final long crcPerBlock = in.readLong();
-    final MD5Hash md5 = MD5Hash.read(in);
-    final MD5MD5CRC32FileChecksum checksum = new MD5MD5CRC32FileChecksum(
-        bytesPerCRC, crcPerBlock, md5);
+    final MD5MD5CRC32FileChecksum checksum = new MD5MD5CRC32FileChecksum();
+    checksum.readFields(in);
 
     //check algorithm name
-    final String alg = "MD5-of-" + crcPerBlock + "MD5-of-" + bytesPerCRC + "CRC32";
-    if (!alg.equals(algorithm)) {
-      throw new IOException("Algorithm not matched: algorithm=" + algorithm
-          + ", crcPerBlock=" + crcPerBlock
-          + ", bytesPerCRC=" + bytesPerCRC);
+    if (!checksum.getAlgorithmName().equals(algorithm)) {
+      throw new IOException("Algorithm not matched. Expected " + algorithm
+          + ", Received " + checksum.getAlgorithmName());
     }
     //check length
     if (length != checksum.getLength()) {
@@ -534,4 +529,4 @@ public class JsonUtil {
 
     return checksum;
   }
-}
+}

+ 51 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java

@@ -0,0 +1,51 @@
+/**
+ * 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.web;
+
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLConnection;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Utilities for handling URLs
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+@InterfaceStability.Unstable
+public class URLUtils {
+  /**
+   * Timeout for socket connects and reads
+   */
+  public static int SOCKET_TIMEOUT = 1*60*1000; // 1 minute
+
+  /**
+   * Opens a url with read and connect timeouts
+   * @param url to open
+   * @return URLConnection
+   * @throws IOException
+   */
+  public static URLConnection openConnection(URL url) throws IOException {
+    URLConnection connection = url.openConnection();
+    connection.setConnectTimeout(SOCKET_TIMEOUT);
+    connection.setReadTimeout(SOCKET_TIMEOUT);
+    return connection;    
+  }
+}

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

@@ -836,4 +836,26 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.support.append</name>
+  <value>true</value>
+  <description>
+    Does HDFS allow appends to files?
+  </description>
+</property>
+
+<property>
+  <name>dfs.client.local.interfaces</name>
+  <value></value>
+  <description>A comma separated list of network interface names to use
+    for data transfer between the client and datanodes. When creating
+    a connection to read from or write to a datanode, the client
+    chooses one of the specified interfaces at random and binds its
+    socket to the IP of that interface. Individual names may be
+    specified as either an interface name (eg "eth0"), a subinterface
+    name (eg "eth0:0"), or an IP address (which may be specified using
+    CIDR notation to match a range of IPs).
+  </description>
+</property>
+
 </configuration>

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java

@@ -230,7 +230,6 @@ public class TestFiPipelines {
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, customBlockSize / 2);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 0);
   }
 

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java

@@ -163,7 +163,6 @@ public class TestStickyBit extends TestCase {
     try {
       Configuration conf = new HdfsConfiguration();
       conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-      conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
 
       FileSystem hdfs = cluster.getFileSystem();

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

@@ -55,7 +55,6 @@ public class FileAppendTest4 {
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, BYTES_PER_CHECKSUM);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, PACKET_SIZE);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
   }
   
   @BeforeClass

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

@@ -40,7 +40,6 @@ public class TestClientProtocolForPipelineRecovery {
   @Test public void testGetNewStamp() throws IOException {
     int numDataNodes = 1;
     Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
       cluster.waitActive();

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

@@ -207,7 +207,6 @@ public class TestDataTransferProtocol extends TestCase {
   @Test public void testOpWrite() throws IOException {
     int numDataNodes = 1;
     Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
       cluster.waitActive();

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java

@@ -85,7 +85,6 @@ public class TestFileAppend2 extends TestCase {
       SimulatedFSDataset.setFactory(conf);
     }
     conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
@@ -338,7 +337,6 @@ public class TestFileAppend2 extends TestCase {
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 30000);
     conf.setInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY, 30000);
     conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
                                                .numDataNodes(numDatanodes)

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

@@ -70,7 +70,6 @@ public class TestFileAppend3 extends junit.framework.TestCase {
         AppendTestUtil.LOG.info("setUp()");
         conf = new HdfsConfiguration();
         conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
-        conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
         buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
         cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
         fs = (DistributedFileSystem)cluster.getFileSystem();

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

@@ -79,7 +79,6 @@ public class TestFileAppend4 {
     if (simulatedStorage) {
       SimulatedFSDataset.setFactory(conf);
     }
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
 
     // lower heartbeat interval for fast recognition of DN death
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,

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

@@ -311,7 +311,6 @@ public class TestFileConcurrentReader extends junit.framework.TestCase {
     final int writeSize,
     Configuration conf
   ) throws IOException {
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, syncType == SyncType.APPEND);
     conf.setBoolean(DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY,
         transferToAllowed);
     init(conf);

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

@@ -38,6 +38,7 @@ import java.io.FileNotFoundException;
 import java.io.FileReader;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
 import java.util.EnumSet;
 
 import org.apache.commons.logging.LogFactory;
@@ -67,6 +68,7 @@ import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.log4j.Level;
 
+import static org.junit.Assume.assumeTrue;
 
 /**
  * This class tests various cases during file creation.
@@ -140,11 +142,34 @@ public class TestFileCreation extends junit.framework.TestCase {
     }
   }
 
+  public void testFileCreation() throws IOException {
+    checkFileCreation(null);
+  }
+
+  /** Same test but the client should bind to a local interface */
+  public void testFileCreationSetLocalInterface() throws IOException {
+    assumeTrue(System.getProperty("os.name").startsWith("Linux"));
+
+    // The mini cluster listens on the loopback so we can use it here
+    checkFileCreation("lo");
+
+    try {
+      checkFileCreation("bogus-interface");
+      fail("Able to specify a bogus interface");
+    } catch (UnknownHostException e) {
+      assertEquals("No such interface bogus-interface", e.getMessage());
+    }
+  }
+
   /**
    * Test if file creation and disk space consumption works right
+   * @param netIf the local interface, if any, clients should use to access DNs
    */
-  public void testFileCreation() throws IOException {
+  public void checkFileCreation(String netIf) throws IOException {
     Configuration conf = new HdfsConfiguration();
+    if (netIf != null) {
+      conf.set(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES, netIf);
+    }
     if (simulatedStorage) {
       SimulatedFSDataset.setFactory(conf);
     }

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

@@ -43,7 +43,6 @@ public class TestFileCreationDelete extends junit.framework.TestCase {
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
 
     // create cluster
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();

+ 128 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.InetAddress;
+import java.net.ServerSocket;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.net.URLConnection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.web.URLUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestHftpURLTimeouts {
+  @BeforeClass
+  public static void setup() {
+    URLUtils.SOCKET_TIMEOUT = 1;
+  }
+  
+  @Test
+  public void testHftpSocketTimeout() throws Exception {
+    Configuration conf = new Configuration();
+    ServerSocket socket = new ServerSocket(0,1);
+    URI uri = new URI("hftp", null,
+        InetAddress.getByName(null).getHostAddress(),
+        socket.getLocalPort(),
+        null, null, null);
+    boolean timedout = false;
+
+    HftpFileSystem fs = (HftpFileSystem)FileSystem.get(uri, conf);
+    HttpURLConnection conn = fs.openConnection("/", "");
+    timedout = false;
+    try {
+      // this will consume the only slot in the backlog
+      conn.getInputStream();
+    } catch (SocketTimeoutException ste) {
+      timedout = true;
+      assertEquals("Read timed out", ste.getMessage());
+    } finally {
+      if (conn != null) conn.disconnect();
+    }
+    assertTrue("read timedout", timedout);
+    assertTrue("connect timedout", checkConnectTimeout(fs, false));
+  }
+
+  @Test
+  public void testHsftpSocketTimeout() throws Exception {
+    Configuration conf = new Configuration();
+    ServerSocket socket = new ServerSocket(0,1);
+    URI uri = new URI("hsftp", null,
+        InetAddress.getByName(null).getHostAddress(),
+        socket.getLocalPort(),
+        null, null, null);
+    boolean timedout = false;
+
+    HsftpFileSystem fs = (HsftpFileSystem)FileSystem.get(uri, conf);
+    HttpURLConnection conn = null;
+    timedout = false;
+    try {
+      // this will consume the only slot in the backlog
+      conn = fs.openConnection("/", "");
+    } catch (SocketTimeoutException ste) {
+      // SSL expects a negotiation, so it will timeout on read, unlike hftp
+      timedout = true;
+      assertEquals("Read timed out", ste.getMessage());
+    } finally {
+      if (conn != null) conn.disconnect();
+    }
+    assertTrue("ssl read connect timedout", timedout);
+    assertTrue("connect timedout", checkConnectTimeout(fs, true));
+  }
+  
+  private boolean checkConnectTimeout(HftpFileSystem fs, boolean ignoreReadTimeout)
+      throws IOException {
+    boolean timedout = false;
+    List<HttpURLConnection> conns = new LinkedList<HttpURLConnection>();
+    try {
+      // with a listen backlog of 1, should only have to make one connection
+      // to trigger a connection timeout.  however... linux doesn't honor the
+      // socket's listen backlog so we have to try a bunch of times
+      for (int n=32; !timedout && n > 0; n--) {
+        try {
+          conns.add(fs.openConnection("/", ""));
+        } catch (SocketTimeoutException ste) {
+          String message = ste.getMessage();
+          // https will get a read timeout due to SSL negotiation, but
+          // a normal http will not, so need to ignore SSL read timeouts
+          // until a connect timeout occurs
+          if (!(ignoreReadTimeout && message.equals("Read timed out"))) {
+            timedout = true;
+            assertEquals("connect timed out", message);
+          }
+        }
+      }
+    } finally {
+      for (HttpURLConnection conn : conns) {
+        conn.disconnect();
+      }
+    }
+    return timedout;
+  }
+}

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

@@ -28,7 +28,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 
@@ -70,7 +70,6 @@ public class TestLeaseRecovery extends junit.framework.TestCase {
     final int ORG_FILE_SIZE = 3000; 
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = null;
 
     try {

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

@@ -155,7 +155,6 @@ public class TestPipelines {
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, customBlockSize / 2);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 0);
   }
 

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java

@@ -84,7 +84,6 @@ public class TestQuota {
     // Space quotas
     final int DEFAULT_BLOCK_SIZE = 512;
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
@@ -533,7 +532,6 @@ public class TestQuota {
     // set a smaller block size so that we can test with smaller 
     // diskspace quotas
     conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "512");
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java

@@ -56,8 +56,6 @@ public class TestReadWhileWriting {
   @Test
   public void pipeline_02_03() throws Exception {
     final Configuration conf = new HdfsConfiguration();
-    //enable append
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
 
     // create cluster

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

@@ -54,7 +54,6 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
 
     // create cluster
     System.out.println("Test 1*****************************");
@@ -140,7 +139,6 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     System.out.println("Test 2************************************");
 
     // create cluster
@@ -215,7 +213,6 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     System.out.println("Test 3************************************");
 
     // create cluster
@@ -280,7 +277,6 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     System.out.println("Test 4************************************");
 
     // create cluster

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

@@ -182,8 +182,6 @@ public class TestBlockTokenWithDFS {
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, numDataNodes);
     conf.setInt("ipc.client.connect.max.retries", 0);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
-        DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
     return conf;
   }
 

+ 23 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java

@@ -22,12 +22,16 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.mockito.Mockito;
 
 import com.google.common.base.Preconditions;
@@ -100,6 +104,19 @@ public class DataNodeTestUtils {
     return spy;
   }
 
+  public static InterDatanodeProtocol createInterDatanodeProtocolProxy(
+      DataNode dn, DatanodeID datanodeid, final Configuration conf
+      ) throws IOException {
+    return DataNode.createInterDataNodeProtocolProxy(datanodeid, conf,
+        dn.getDnConf().socketTimeout);
+  }
+  
+  public static void shutdownBlockScanner(DataNode dn) {
+    if (dn.blockScanner != null) {
+      dn.blockScanner.shutdown();
+    }
+  }
+
   /**
    * This method is used for testing. 
    * Examples are adding and deleting blocks directly.
@@ -111,26 +128,22 @@ public class DataNodeTestUtils {
     return dn.getFSDataset();
   }
 
-  public static FSDataset getFsDatasetImpl(DataNode dn) {
-    return (FSDataset)dn.getFSDataset();
-  }
-
   public static File getFile(DataNode dn, String bpid, long bid) {
-    return getFsDatasetImpl(dn).getFile(bpid, bid);
+    return FsDatasetTestUtil.getFile(dn.getFSDataset(), bpid, bid);
   }
 
   public static File getBlockFile(DataNode dn, String bpid, Block b
       ) throws IOException {
-    return getFsDatasetImpl(dn).getBlockFile(bpid, b);
+    return FsDatasetTestUtil.getBlockFile(dn.getFSDataset(), bpid, b);
   }
 
-  public static boolean unlinkBlock(DataNode dn, ExtendedBlock block, int numLinks
+  public static boolean unlinkBlock(DataNode dn, ExtendedBlock bk, int numLinks
       ) throws IOException {
-    return getFsDatasetImpl(dn).getReplicaInfo(block).unlinkBlock(numLinks);
+    return FsDatasetTestUtil.unlinkBlock(dn.getFSDataset(), bk, numLinks);
   }
 
   public static long getPendingAsyncDeletions(DataNode dn) {
-    return getFsDatasetImpl(dn).asyncDiskService.countPendingDeletions();
+    return FsDatasetTestUtil.getPendingAsyncDeletions(dn.getFSDataset());
   }
 
   /**
@@ -142,6 +155,6 @@ public class DataNodeTestUtils {
    */
   public static ReplicaInfo fetchReplicaInfo(final DataNode dn,
       final String bpid, final long blkId) {
-    return getFsDatasetImpl(dn).fetchReplicaInfo(bpid, blkId);
+    return FsDatasetTestUtil.fetchReplicaInfo(dn.getFSDataset(), bpid, blkId);
   }
 }

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java

@@ -613,14 +613,13 @@ public class TestBlockReport {
     
     // Look about specified DN for the replica of the block from 1st DN
     final DataNode dn1 = cluster.getDataNodes().get(DN_N1);
-    final FSDataset dataset1 = (FSDataset)DataNodeTestUtils.getFSDataset(dn1);
     String bpid = cluster.getNamesystem().getBlockPoolId();
-    Replica r = dataset1.fetchReplicaInfo(bpid, bl.getBlockId());
+    Replica r = DataNodeTestUtils.fetchReplicaInfo(dn1, bpid, bl.getBlockId());
     long start = System.currentTimeMillis();
     int count = 0;
     while (r == null) {
       waitTil(5);
-      r = dataset1.fetchReplicaInfo(bpid, bl.getBlockId());
+      r = DataNodeTestUtils.fetchReplicaInfo(dn1, bpid, bl.getBlockId());
       long waiting_period = System.currentTimeMillis() - start;
       if (count++ % 100 == 0)
         if(LOG.isDebugEnabled()) {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java

@@ -37,6 +37,7 @@ 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.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -376,7 +377,7 @@ public class TestDataNodeVolumeFailure {
         new FilenameFilter() {
           public boolean accept(File dir, String name) {
             return name.startsWith("blk_") &&
-            name.endsWith(DatanodeUtil.METADATA_EXTENSION);
+            name.endsWith(Block.METADATA_EXTENSION);
           }
         }
     );

+ 17 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -38,7 +38,9 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 
 /**
  * Tests {@link DirectoryScanner} handling of differences
@@ -51,7 +53,7 @@ public class TestDirectoryScanner extends TestCase {
 
   private MiniDFSCluster cluster;
   private String bpid;
-  private FSDataset fds = null;
+  private FsDatasetSpi<? extends FsVolumeSpi> fds = null;
   private DirectoryScanner scanner = null;
   private Random rand = new Random();
   private Random r = new Random();
@@ -72,7 +74,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Truncate a block file */
   private long truncateBlockFile() throws IOException {
     synchronized (fds) {
-      for (ReplicaInfo b : fds.volumeMap.replicas(bpid)) {
+      for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
         File f = b.getBlockFile();
         File mf = b.getMetaFile();
         // Truncate a block file that has a corresponding metadata file
@@ -91,7 +93,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Delete a block file */
   private long deleteBlockFile() {
     synchronized(fds) {
-      for (ReplicaInfo b : fds.volumeMap.replicas(bpid)) {
+      for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
         File f = b.getBlockFile();
         File mf = b.getMetaFile();
         // Delete a block file that has corresponding metadata file
@@ -107,7 +109,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Delete block meta file */
   private long deleteMetaFile() {
     synchronized(fds) {
-      for (ReplicaInfo b : fds.volumeMap.replicas(bpid)) {
+      for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
         File file = b.getMetaFile();
         // Delete a metadata file
         if (file.exists() && file.delete()) {
@@ -124,7 +126,7 @@ public class TestDirectoryScanner extends TestCase {
     long id = rand.nextLong();
     while (true) {
       id = rand.nextLong();
-      if (fds.fetchReplicaInfo(bpid, id) == null) {
+      if (FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, id) == null) {
         break;
       }
     }
@@ -142,7 +144,7 @@ public class TestDirectoryScanner extends TestCase {
 
   /** Create a block file in a random volume*/
   private long createBlockFile() throws IOException {
-    List<FSVolume> volumes = fds.getVolumes();
+    List<? extends FsVolumeSpi> volumes = fds.getVolumes();
     int index = rand.nextInt(volumes.size() - 1);
     long id = getFreeBlockId();
     File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
@@ -155,7 +157,7 @@ public class TestDirectoryScanner extends TestCase {
 
   /** Create a metafile in a random volume*/
   private long createMetaFile() throws IOException {
-    List<FSVolume> volumes = fds.getVolumes();
+    List<? extends FsVolumeSpi> volumes = fds.getVolumes();
     int index = rand.nextInt(volumes.size() - 1);
     long id = getFreeBlockId();
     File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
@@ -168,7 +170,7 @@ public class TestDirectoryScanner extends TestCase {
 
   /** Create block file and corresponding metafile in a rondom volume */
   private long createBlockMetaFile() throws IOException {
-    List<FSVolume> volumes = fds.getVolumes();
+    List<? extends FsVolumeSpi> volumes = fds.getVolumes();
     int index = rand.nextInt(volumes.size() - 1);
     long id = getFreeBlockId();
     File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
@@ -228,8 +230,7 @@ public class TestDirectoryScanner extends TestCase {
     try {
       cluster.waitActive();
       bpid = cluster.getNamesystem().getBlockPoolId();
-      fds = (FSDataset)DataNodeTestUtils.getFSDataset(
-          cluster.getDataNodes().get(0));
+      fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
       CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
                   parallelism);
       DataNode dn = cluster.getDataNodes().get(0);
@@ -348,12 +349,13 @@ public class TestDirectoryScanner extends TestCase {
 
   private void verifyAddition(long blockId, long genStamp, long size) {
     final ReplicaInfo replicainfo;
-    replicainfo = fds.fetchReplicaInfo(bpid, blockId);
+    replicainfo = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
     assertNotNull(replicainfo);
 
     // Added block has the same file as the one created by the test
     File file = new File(getBlockFile(blockId));
-    assertEquals(file.getName(), fds.getFile(bpid, blockId).getName());
+    assertEquals(file.getName(),
+        FsDatasetTestUtil.getFile(fds, bpid, blockId).getName());
 
     // Generation stamp is same as that of created file
     assertEquals(genStamp, replicainfo.getGenerationStamp());
@@ -364,12 +366,12 @@ public class TestDirectoryScanner extends TestCase {
 
   private void verifyDeletion(long blockId) {
     // Ensure block does not exist in memory
-    assertNull(fds.fetchReplicaInfo(bpid, blockId));
+    assertNull(FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId));
   }
 
   private void verifyGenStamp(long blockId, long genStamp) {
     final ReplicaInfo memBlock;
-    memBlock = fds.fetchReplicaInfo(bpid, blockId);
+    memBlock = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
     assertNotNull(memBlock);
     assertEquals(genStamp, memBlock.getGenerationStamp());
   }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
 import org.apache.hadoop.util.DataChecksum;
 
 /**
@@ -90,7 +91,7 @@ public class TestSimulatedFSDataset extends TestCase {
   public void testFSDatasetFactory() {
     final Configuration conf = new Configuration();
     FsDatasetSpi.Factory<?> f = FsDatasetSpi.Factory.getFactory(conf);
-    assertEquals(FSDataset.Factory.class, f.getClass());
+    assertEquals(FsDatasetFactory.class, f.getClass());
     assertFalse(f.isSimulated());
 
     SimulatedFSDataset.setFactory(conf);
@@ -243,7 +244,7 @@ public class TestSimulatedFSDataset extends TestCase {
     }
   }
 
-  public void checkInvalidBlock(ExtendedBlock b) throws IOException {
+  public void checkInvalidBlock(ExtendedBlock b) {
     final SimulatedFSDataset fsdataset = getSimulatedFSDataset();
     assertFalse(fsdataset.isValidBlock(b));
     try {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.log4j.Level;
 import org.junit.Assert;
@@ -58,8 +59,8 @@ public class TestTransferRbw {
   }
   private static ReplicaInPipeline getReplica(final DataNode datanode,
       final String bpid, final ReplicaState expectedState) throws InterruptedException {
-    final FSDataset dataset = ((FSDataset)datanode.data);
-    final Collection<ReplicaInfo> replicas = dataset.volumeMap.replicas(bpid);
+    final Collection<ReplicaInfo> replicas = FsDatasetTestUtil.getReplicas(
+        datanode.getFSDataset(), bpid);
     for(int i = 0; i < 5 && replicas.size() == 0; i++) {
       LOG.info("wait since replicas.size() == 0; i=" + i);
       Thread.sleep(1000);

+ 59 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.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.server.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+
+public class FsDatasetTestUtil {
+
+  public static File getFile(FsDatasetSpi<?> fsd, String bpid, long bid) {
+    return ((FsDatasetImpl)fsd).getFile(bpid, bid);
+  }
+
+  public static File getBlockFile(FsDatasetSpi<?> fsd, String bpid, Block b
+      ) throws IOException {
+    return ((FsDatasetImpl)fsd).getBlockFile(bpid, b);
+  }
+
+  public static boolean unlinkBlock(FsDatasetSpi<?> fsd,
+      ExtendedBlock block, int numLinks) throws IOException {
+    final ReplicaInfo info = ((FsDatasetImpl)fsd).getReplicaInfo(block);
+    return info.unlinkBlock(numLinks);
+  }
+
+  public static ReplicaInfo fetchReplicaInfo (final FsDatasetSpi<?> fsd,
+      final String bpid, final long blockId) {
+    return ((FsDatasetImpl)fsd).fetchReplicaInfo(bpid, blockId);
+  }
+
+  public static long getPendingAsyncDeletions(FsDatasetSpi<?> fsd) {
+    return ((FsDatasetImpl)fsd).asyncDiskService.countPendingDeletions();
+  }
+  
+  public static Collection<ReplicaInfo> getReplicas(FsDatasetSpi<?> fsd,
+      String bpid) {
+    return ((FsDatasetImpl)fsd).volumeMap.replicas(bpid);
+  }
+}

+ 15 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -36,7 +36,10 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
@@ -73,7 +76,6 @@ public class TestDatanodeRestart {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     cluster.waitActive();
     try {
@@ -98,8 +100,8 @@ public class TestDatanodeRestart {
       out.write(writeBuf);
       out.hflush();
       DataNode dn = cluster.getDataNodes().get(0);
-      for (FsVolumeSpi v : dn.data.getVolumes()) {
-        FSVolume volume = (FSVolume)v;
+      for (FsVolumeSpi v : dataset(dn).getVolumes()) {
+        final FsVolumeImpl volume = (FsVolumeImpl)v;
         File currentDir = volume.getCurrentDir().getParentFile().getParentFile();
         File rbwDir = new File(currentDir, "rbw");
         for (File file : rbwDir.listFiles()) {
@@ -114,7 +116,7 @@ public class TestDatanodeRestart {
 
       // check volumeMap: one rwr replica
       String bpid = cluster.getNamesystem().getBlockPoolId();
-      ReplicasMap replicas = ((FSDataset)(dn.data)).volumeMap;
+      ReplicaMap replicas = dataset(dn).volumeMap;
       Assert.assertEquals(1, replicas.size(bpid));
       ReplicaInfo replica = replicas.replicas(bpid).iterator().next();
       Assert.assertEquals(ReplicaState.RWR, replica.getState());
@@ -123,7 +125,7 @@ public class TestDatanodeRestart {
       } else {
         Assert.assertEquals(fileLen, replica.getNumBytes());
       }
-      dn.data.invalidate(bpid, new Block[]{replica});
+      dataset(dn).invalidate(bpid, new Block[]{replica});
     } finally {
       IOUtils.closeStream(out);
       if (fs.exists(src)) {
@@ -138,7 +140,6 @@ public class TestDatanodeRestart {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
-    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitActive();
     try {
@@ -151,7 +152,7 @@ public class TestDatanodeRestart {
       String bpid = cluster.getNamesystem().getBlockPoolId();
       DataNode dn = cluster.getDataNodes().get(0);
       Iterator<ReplicaInfo> replicasItor = 
-        ((FSDataset)dn.data).volumeMap.replicas(bpid).iterator();
+          dataset(dn).volumeMap.replicas(bpid).iterator();
       ReplicaInfo replica = replicasItor.next();
       createUnlinkTmpFile(replica, true, true); // rename block file
       createUnlinkTmpFile(replica, false, true); // rename meta file
@@ -167,8 +168,7 @@ public class TestDatanodeRestart {
       dn = cluster.getDataNodes().get(0);
 
       // check volumeMap: 4 finalized replica
-      Collection<ReplicaInfo> replicas = 
-        ((FSDataset)(dn.data)).volumeMap.replicas(bpid);
+      Collection<ReplicaInfo> replicas = dataset(dn).volumeMap.replicas(bpid);
       Assert.assertEquals(4, replicas.size());
       replicasItor = replicas.iterator();
       while (replicasItor.hasNext()) {
@@ -180,6 +180,10 @@ public class TestDatanodeRestart {
     }
   }
 
+  private static FsDatasetImpl dataset(DataNode dn) {
+    return (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
+  }
+
   private static void createUnlinkTmpFile(ReplicaInfo replicaInfo, 
       boolean changeBlockFile, 
       boolean isRename) throws IOException {

+ 36 - 32
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java

@@ -15,22 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.util.List;
 import java.net.InetSocketAddress;
-
 import java.net.SocketTimeoutException;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -41,16 +34,28 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+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;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -103,14 +108,14 @@ public class TestInterDatanodeProtocol {
   }
 
   public static void checkMetaInfo(ExtendedBlock b, DataNode dn) throws IOException {
-    Block metainfo = dn.data.getStoredBlock(b.getBlockPoolId(), b.getBlockId());
+    Block metainfo = DataNodeTestUtils.getFSDataset(dn).getStoredBlock(
+        b.getBlockPoolId(), b.getBlockId());
     Assert.assertEquals(b.getBlockId(), metainfo.getBlockId());
     Assert.assertEquals(b.getNumBytes(), metainfo.getNumBytes());
   }
 
   public static LocatedBlock getLastLocatedBlock(
-      ClientProtocol namenode, String src
-  ) throws IOException {
+      ClientProtocol namenode, String src) throws IOException {
     //get block info for the last block
     LocatedBlocks locations = namenode.getBlockLocations(src, 0, Long.MAX_VALUE);
     List<LocatedBlock> blocks = locations.getLocatedBlocks();
@@ -148,13 +153,11 @@ public class TestInterDatanodeProtocol {
 
       //connect to a data node
       DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
-      InterDatanodeProtocol idp = DataNode.createInterDataNodeProtocolProxy(
-          datanodeinfo[0], conf, datanode.getDnConf().socketTimeout);
+      InterDatanodeProtocol idp = DataNodeTestUtils.createInterDatanodeProtocolProxy(
+          datanode, datanodeinfo[0], conf);
       
       //stop block scanner, so we could compare lastScanTime
-      if (datanode.blockScanner != null) {
-        datanode.blockScanner.shutdown();
-      }
+      DataNodeTestUtils.shutdownBlockScanner(datanode);
 
       //verify BlockMetaDataInfo
       ExtendedBlock b = locatedblock.getBlock();
@@ -187,14 +190,14 @@ public class TestInterDatanodeProtocol {
   }
 
   /** Test 
-   * {@link FSDataset#initReplicaRecovery(String, ReplicasMap, Block, long)}
+   * {@link FsDatasetImpl#initReplicaRecovery(String, ReplicaMap, Block, long)}
    */
   @Test
   public void testInitReplicaRecovery() throws IOException {
     final long firstblockid = 10000L;
     final long gs = 7777L;
     final long length = 22L;
-    final ReplicasMap map = new ReplicasMap(this);
+    final ReplicaMap map = new ReplicaMap(this);
     String bpid = "BP-TEST";
     final Block[] blocks = new Block[5];
     for(int i = 0; i < blocks.length; i++) {
@@ -208,7 +211,8 @@ public class TestInterDatanodeProtocol {
       final ReplicaInfo originalInfo = map.get(bpid, b);
 
       final long recoveryid = gs + 1;
-      final ReplicaRecoveryInfo recoveryInfo = FSDataset.initReplicaRecovery(bpid, map, blocks[0], recoveryid);
+      final ReplicaRecoveryInfo recoveryInfo = FsDatasetImpl.initReplicaRecovery(
+          bpid, map, blocks[0], recoveryid);
       assertEquals(originalInfo, recoveryInfo);
 
       final ReplicaUnderRecovery updatedInfo = (ReplicaUnderRecovery)map.get(bpid, b);
@@ -217,7 +221,7 @@ public class TestInterDatanodeProtocol {
 
       //recover one more time 
       final long recoveryid2 = gs + 2;
-      final ReplicaRecoveryInfo recoveryInfo2 = FSDataset.initReplicaRecovery(bpid, map, blocks[0], recoveryid2);
+      final ReplicaRecoveryInfo recoveryInfo2 = FsDatasetImpl.initReplicaRecovery(bpid, map, blocks[0], recoveryid2);
       assertEquals(originalInfo, recoveryInfo2);
 
       final ReplicaUnderRecovery updatedInfo2 = (ReplicaUnderRecovery)map.get(bpid, b);
@@ -226,7 +230,7 @@ public class TestInterDatanodeProtocol {
       
       //case RecoveryInProgressException
       try {
-        FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
+        FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
         Assert.fail();
       }
       catch(RecoveryInProgressException ripe) {
@@ -237,7 +241,7 @@ public class TestInterDatanodeProtocol {
     { // BlockRecoveryFI_01: replica not found
       final long recoveryid = gs + 1;
       final Block b = new Block(firstblockid - 1, length, gs);
-      ReplicaRecoveryInfo r = FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
+      ReplicaRecoveryInfo r = FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
       Assert.assertNull("Data-node should not have this replica.", r);
     }
     
@@ -245,7 +249,7 @@ public class TestInterDatanodeProtocol {
       final long recoveryid = gs - 1;
       final Block b = new Block(firstblockid + 1, length, gs);
       try {
-        FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
+        FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
         Assert.fail();
       }
       catch(IOException ioe) {
@@ -258,7 +262,7 @@ public class TestInterDatanodeProtocol {
       final long recoveryid = gs + 1;
       final Block b = new Block(firstblockid, length, gs+1);
       try {
-        FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
+        FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
         fail("InitReplicaRecovery should fail because replica's " +
         		"gs is less than the block's gs");
       } catch (IOException e) {
@@ -270,7 +274,7 @@ public class TestInterDatanodeProtocol {
 
   /** 
    * Test  for
-   * {@link FSDataset#updateReplicaUnderRecovery(ExtendedBlock, long, long)} 
+   * {@link FsDatasetImpl#updateReplicaUnderRecovery(ExtendedBlock, long, long)} 
    * */
   @Test
   public void testUpdateReplicaUnderRecovery() throws IOException {
@@ -296,22 +300,22 @@ public class TestInterDatanodeProtocol {
       //get DataNode and FSDataset objects
       final DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
       Assert.assertTrue(datanode != null);
-      Assert.assertTrue(datanode.data instanceof FSDataset);
-      final FSDataset fsdataset = (FSDataset)datanode.data;
 
       //initReplicaRecovery
       final ExtendedBlock b = locatedblock.getBlock();
       final long recoveryid = b.getGenerationStamp() + 1;
       final long newlength = b.getNumBytes() - 1;
+      final FsDatasetSpi<?> fsdataset = DataNodeTestUtils.getFSDataset(datanode);
       final ReplicaRecoveryInfo rri = fsdataset.initReplicaRecovery(
           new RecoveringBlock(b, null, recoveryid));
 
       //check replica
-      final ReplicaInfo replica = fsdataset.fetchReplicaInfo(bpid, b.getBlockId());
+      final ReplicaInfo replica = FsDatasetTestUtil.fetchReplicaInfo(
+          fsdataset, bpid, b.getBlockId());
       Assert.assertEquals(ReplicaState.RUR, replica.getState());
 
       //check meta data before update
-      FSDataset.checkReplicaFiles(replica);
+      FsDatasetImpl.checkReplicaFiles(replica);
 
       //case "THIS IS NOT SUPPOSED TO HAPPEN"
       //with (block length) != (stored replica's on disk length). 

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReplicasMap.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java

@@ -15,21 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaMap;
 import org.junit.Before;
 import org.junit.Test;
 
 /**
  * Unit test for ReplicasMap class
  */
-public class TestReplicasMap {
-  private final ReplicasMap map = new ReplicasMap(TestReplicasMap.class);
+public class TestReplicaMap {
+  private final ReplicaMap map = new ReplicaMap(TestReplicaMap.class);
   private final String bpid = "BP-TEST";
   private final  Block block = new Block(1234, 1234, 1234);
   

+ 23 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestWriteToReplica.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java

@@ -15,14 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.junit.Assert;
 import org.junit.Test;
@@ -45,7 +54,7 @@ public class TestWriteToReplica {
     try {
       cluster.waitActive();
       DataNode dn = cluster.getDataNodes().get(0);
-      FSDataset dataSet = (FSDataset)dn.data;
+      FsDatasetImpl dataSet = (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
 
       // set up replicasMap
       String bpid = cluster.getNamesystem().getBlockPoolId();
@@ -66,7 +75,7 @@ public class TestWriteToReplica {
     try {
       cluster.waitActive();
       DataNode dn = cluster.getDataNodes().get(0);
-      FSDataset dataSet = (FSDataset)dn.data;
+      FsDatasetImpl dataSet = (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
 
       // set up replicasMap
       String bpid = cluster.getNamesystem().getBlockPoolId();
@@ -86,7 +95,7 @@ public class TestWriteToReplica {
     try {
       cluster.waitActive();
       DataNode dn = cluster.getDataNodes().get(0);
-      FSDataset dataSet = (FSDataset)dn.data;
+      FsDatasetImpl dataSet = (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
 
       // set up replicasMap
       String bpid = cluster.getNamesystem().getBlockPoolId();
@@ -106,7 +115,7 @@ public class TestWriteToReplica {
     try {
       cluster.waitActive();
       DataNode dn = cluster.getDataNodes().get(0);
-      FSDataset dataSet = (FSDataset)dn.data;
+      FsDatasetImpl dataSet = (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
 
       // set up replicasMap
       String bpid = cluster.getNamesystem().getBlockPoolId();
@@ -128,7 +137,7 @@ public class TestWriteToReplica {
    * @return Contrived blocks for further testing.
    * @throws IOException
    */
-  private ExtendedBlock[] setup(String bpid, FSDataset dataSet) throws IOException {
+  private ExtendedBlock[] setup(String bpid, FsDatasetImpl dataSet) throws IOException {
     // setup replicas map
     
     ExtendedBlock[] blocks = new ExtendedBlock[] {
@@ -137,8 +146,8 @@ public class TestWriteToReplica {
         new ExtendedBlock(bpid, 5, 1, 2005), new ExtendedBlock(bpid, 6, 1, 2006)
     };
     
-    ReplicasMap replicasMap = dataSet.volumeMap;
-    FSVolume vol = dataSet.volumes.getNextVolume(0);
+    ReplicaMap replicasMap = dataSet.volumeMap;
+    FsVolumeImpl vol = dataSet.volumes.getNextVolume(0);
     ReplicaInfo replicaInfo = new FinalizedReplica(
         blocks[FINALIZED].getLocalBlock(), vol, vol.getCurrentDir().getParentFile());
     replicasMap.add(bpid, replicaInfo);
@@ -165,9 +174,9 @@ public class TestWriteToReplica {
     return blocks;
   }
   
-  private void testAppend(String bpid, FSDataset dataSet, ExtendedBlock[] blocks) throws IOException {
+  private void testAppend(String bpid, FsDatasetImpl dataSet, ExtendedBlock[] blocks) throws IOException {
     long newGS = blocks[FINALIZED].getGenerationStamp()+1;
-    final FSVolume v = (FSVolume)dataSet.volumeMap.get(
+    final FsVolumeImpl v = (FsVolumeImpl)dataSet.volumeMap.get(
         bpid, blocks[FINALIZED].getLocalBlock()).getVolume();
     long available = v.getCapacity()-v.getDfsUsed();
     long expectedLen = blocks[FINALIZED].getNumBytes();
@@ -285,7 +294,7 @@ public class TestWriteToReplica {
     }
   }
 
-  private void testClose(FSDataset dataSet, ExtendedBlock [] blocks) throws IOException {
+  private void testClose(FsDatasetImpl dataSet, ExtendedBlock [] blocks) throws IOException {
     long newGS = blocks[FINALIZED].getGenerationStamp()+1;
     dataSet.recoverClose(blocks[FINALIZED], newGS, 
         blocks[FINALIZED].getNumBytes());  // successful
@@ -335,7 +344,7 @@ public class TestWriteToReplica {
     }
   }
   
-  private void testWriteToRbw(FSDataset dataSet, ExtendedBlock[] blocks) throws IOException {
+  private void testWriteToRbw(FsDatasetImpl dataSet, ExtendedBlock[] blocks) throws IOException {
     try {
       dataSet.recoverRbw(blocks[FINALIZED],
           blocks[FINALIZED].getGenerationStamp()+1,
@@ -428,7 +437,7 @@ public class TestWriteToReplica {
     dataSet.createRbw(blocks[NON_EXISTENT]);
   }
   
-  private void testWriteToTemporary(FSDataset dataSet, ExtendedBlock[] blocks) throws IOException {
+  private void testWriteToTemporary(FsDatasetImpl dataSet, ExtendedBlock[] blocks) throws IOException {
     try {
       dataSet.createTemporary(blocks[FINALIZED]);
       Assert.fail("Should not have created a temporary replica that was " +

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

@@ -179,12 +179,22 @@ public class TestBackupNode {
       
       // do some edits
       assertTrue(fileSys.mkdirs(new Path("/edit-while-bn-down")));
-      
+  
       // start a new backup node
       backup = startBackupNode(conf, StartupOption.BACKUP, 1);
 
       testBNInSync(cluster, backup, 4);
       assertNotNull(backup.getNamesystem().getFileInfo("/edit-while-bn-down", false));
+      
+      // Trigger an unclean shutdown of the backup node. Backup node will not
+      // unregister from the active when this is done simulating a node crash.
+      backup.stop(false);
+           
+      // do some edits on the active. This should go through without failing.
+      // This will verify that active is still up and can add entries to
+      // master editlog.
+      assertTrue(fileSys.mkdirs(new Path("/edit-while-bn-down-2")));
+      
     } finally {
       LOG.info("Shutting down...");
       if (backup != null) backup.stop();

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

@@ -195,6 +195,51 @@ Release 2.0.0 - UNRELEASED
 
     MAPREDUCE-4091. tools testcases failing because of MAPREDUCE-4082 (tucu)
 
+    MAPREDUCE-4095. TestJobInProgress#testLocality uses a bogus topology.
+    (Colin Patrick McCabe via eli)
+    
+Release 0.23.3 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+    MAPREDUCE-4072. User set java.library.path seems to overwrite default
+    creating problems native lib loading (Anupam Seth via bobby)
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    MAPREDUCE-4092.  commitJob Exception does not fail job (Jon Eagles via
+    bobby)
+
+    MAPREDUCE-4089. Hung Tasks never time out. (Robert Evans via tgraves)
+
+    MAPREDUCE-4024. RM webservices can't query on finalStatus (Tom Graves
+    via bobby)
+
+    MAPREDUCE-4060. Multiple SLF4J binding warning (Jason Lowe via bobby)
+
+    MAPREDUCE-3983. TestTTResourceReporting can fail, and should just be
+    deleted (Ravi Prakash via bobby)
+
+    MAPREDUCE-4012 Hadoop Job setup error leaves no useful info to users 
+    (when LinuxTaskController is used). (tgraves)
+
+    MAPREDUCE-4062. AM Launcher thread can hang forever (tgraves via bobby)
+
+    MAPREDUCE-3988. mapreduce.job.local.dir doesn't point to a single directory
+    on a node. (Eric Payne via bobby)
+
+    MAPREDUCE-3999. Tracking link gives an error if the AppMaster hasn't
+    started yet (Ravi Prakash via bobby)
+
+    MAPREDUCE-4020. Web services returns incorrect JSON for deep queue tree
+    (Anupam Seth via tgraves)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 7 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java

@@ -175,7 +175,7 @@ public class TaskAttemptListenerImpl extends CompositeService
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
         TypeConverter.toYarn(taskAttemptID);
 
-    taskHeartbeatHandler.receivedPing(attemptID);
+    taskHeartbeatHandler.progressing(attemptID);
 
     Job job = context.getJob(attemptID.getTaskId().getJobId());
     Task task = job.getTask(attemptID.getTaskId());
@@ -203,7 +203,7 @@ public class TaskAttemptListenerImpl extends CompositeService
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
         TypeConverter.toYarn(taskAttemptID);
 
-    taskHeartbeatHandler.receivedPing(attemptID);
+    taskHeartbeatHandler.progressing(attemptID);
     //Ignorable TaskStatus? - since a task will send a LastStatusUpdate
     context.getEventHandler().handle(
         new TaskAttemptEvent(attemptID, 
@@ -217,7 +217,7 @@ public class TaskAttemptListenerImpl extends CompositeService
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
         TypeConverter.toYarn(taskAttemptID);
 
-    taskHeartbeatHandler.receivedPing(attemptID);
+    taskHeartbeatHandler.progressing(attemptID);
 
     context.getEventHandler().handle(
         new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_DONE));
@@ -270,7 +270,7 @@ public class TaskAttemptListenerImpl extends CompositeService
         context.getJob(attemptID.getTaskId().getJobId()).getTaskAttemptCompletionEvents(
             fromEventId, maxEvents);
 
-    taskHeartbeatHandler.receivedPing(attemptID);
+    taskHeartbeatHandler.progressing(attemptID);
 
     // filter the events to return only map completion events in old format
     List<TaskCompletionEvent> mapEvents = new ArrayList<TaskCompletionEvent>();
@@ -287,7 +287,7 @@ public class TaskAttemptListenerImpl extends CompositeService
   @Override
   public boolean ping(TaskAttemptID taskAttemptID) throws IOException {
     LOG.info("Ping from " + taskAttemptID.toString());
-    taskHeartbeatHandler.receivedPing(TypeConverter.toYarn(taskAttemptID));
+    taskHeartbeatHandler.pinged(TypeConverter.toYarn(taskAttemptID));
     return true;
   }
 
@@ -299,7 +299,7 @@ public class TaskAttemptListenerImpl extends CompositeService
 
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
       TypeConverter.toYarn(taskAttemptID);
-    taskHeartbeatHandler.receivedPing(attemptID);
+    taskHeartbeatHandler.progressing(attemptID);
 
     // This is mainly used for cases where we want to propagate exception traces
     // of tasks that fail.
@@ -317,7 +317,7 @@ public class TaskAttemptListenerImpl extends CompositeService
     LOG.info("Status update from " + taskAttemptID.toString());
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId yarnAttemptID =
         TypeConverter.toYarn(taskAttemptID);
-    taskHeartbeatHandler.receivedPing(yarnAttemptID);
+    taskHeartbeatHandler.progressing(yarnAttemptID);
     TaskAttemptStatus taskAttemptStatus =
         new TaskAttemptStatus();
     taskAttemptStatus.id = yarnAttemptID;

+ 28 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java

@@ -54,6 +54,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.log4j.LogManager;
@@ -236,11 +237,33 @@ class YarnChild {
     job.setStrings(MRConfig.LOCAL_DIR, localSysDirs);
     LOG.info(MRConfig.LOCAL_DIR + " for child: " + job.get(MRConfig.LOCAL_DIR));
     LocalDirAllocator lDirAlloc = new LocalDirAllocator(MRConfig.LOCAL_DIR);
-    Path workDir = lDirAlloc.getLocalPathForWrite("work", job);
-    FileSystem lfs = FileSystem.getLocal(job).getRaw();
-    if (!lfs.mkdirs(workDir)) {
-        throw new IOException("Mkdirs failed to create "
-            + workDir.toString());
+    Path workDir = null;
+    // First, try to find the JOB_LOCAL_DIR on this host.
+    try {
+      workDir = lDirAlloc.getLocalPathToRead("work", job);
+    } catch (DiskErrorException e) {
+      // DiskErrorException means dir not found. If not found, it will
+      // be created below.
+    }
+    if (workDir == null) {
+      // JOB_LOCAL_DIR doesn't exist on this host -- Create it.
+      workDir = lDirAlloc.getLocalPathForWrite("work", job);
+      FileSystem lfs = FileSystem.getLocal(job).getRaw();
+      boolean madeDir = false;
+      try {
+        madeDir = lfs.mkdirs(workDir);
+      } catch (FileAlreadyExistsException e) {
+        // Since all tasks will be running in their own JVM, the race condition
+        // exists where multiple tasks could be trying to create this directory
+        // at the same time. If this task loses the race, it's okay because
+        // the directory already exists.
+        madeDir = true;
+        workDir = lDirAlloc.getLocalPathToRead("work", job);
+      }
+      if (!madeDir) {
+          throw new IOException("Mkdirs failed to create "
+              + workDir.toString());
+      }
     }
     job.set(MRJobConfig.JOB_LOCAL_DIR,workDir.toString());
   }

+ 62 - 26
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java

@@ -44,9 +44,36 @@ import org.apache.hadoop.yarn.service.AbstractService;
  */
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class TaskHeartbeatHandler extends AbstractService {
-
+  
+  private static class ReportTime {
+    private long lastPing;
+    private long lastProgress;
+    
+    public ReportTime(long time) {
+      setLastProgress(time);
+    }
+    
+    public synchronized void setLastPing(long time) {
+      lastPing = time;
+    }
+    
+    public synchronized void setLastProgress(long time) {
+      lastProgress = time;
+      lastPing = time;
+    }
+    
+    public synchronized long getLastPing() {
+      return lastPing;
+    }
+    
+    public synchronized long getLastProgress() {
+      return lastProgress;
+    }
+  }
+  
   private static final Log LOG = LogFactory.getLog(TaskHeartbeatHandler.class);
-
+  private static final int PING_TIMEOUT = 5 * 60 * 1000;
+  
   //thread which runs periodically to see the last time since a heartbeat is
   //received from a task.
   private Thread lostTaskCheckerThread;
@@ -56,8 +83,8 @@ public class TaskHeartbeatHandler extends AbstractService {
 
   private final EventHandler eventHandler;
   private final Clock clock;
-
-  private ConcurrentMap<TaskAttemptId, Long> runningAttempts;
+  
+  private ConcurrentMap<TaskAttemptId, ReportTime> runningAttempts;
 
   public TaskHeartbeatHandler(EventHandler eventHandler, Clock clock,
       int numThreads) {
@@ -65,7 +92,7 @@ public class TaskHeartbeatHandler extends AbstractService {
     this.eventHandler = eventHandler;
     this.clock = clock;
     runningAttempts =
-      new ConcurrentHashMap<TaskAttemptId, Long>(16, 0.75f, numThreads);
+      new ConcurrentHashMap<TaskAttemptId, ReportTime>(16, 0.75f, numThreads);
   }
 
   @Override
@@ -91,14 +118,26 @@ public class TaskHeartbeatHandler extends AbstractService {
     super.stop();
   }
 
-  public void receivedPing(TaskAttemptId attemptID) {
+  public void progressing(TaskAttemptId attemptID) {
   //only put for the registered attempts
     //TODO throw an exception if the task isn't registered.
-    runningAttempts.replace(attemptID, clock.getTime());
+    ReportTime time = runningAttempts.get(attemptID);
+    if(time != null) {
+      time.setLastProgress(clock.getTime());
+    }
   }
 
+  public void pinged(TaskAttemptId attemptID) {
+    //only put for the registered attempts
+      //TODO throw an exception if the task isn't registered.
+      ReportTime time = runningAttempts.get(attemptID);
+      if(time != null) {
+        time.setLastPing(clock.getTime());
+      }
+    }
+  
   public void register(TaskAttemptId attemptID) {
-    runningAttempts.put(attemptID, clock.getTime());
+    runningAttempts.put(attemptID, new ReportTime(clock.getTime()));
   }
 
   public void unregister(TaskAttemptId attemptID) {
@@ -110,30 +149,27 @@ public class TaskHeartbeatHandler extends AbstractService {
     @Override
     public void run() {
       while (!stopped && !Thread.currentThread().isInterrupted()) {
-        Iterator<Map.Entry<TaskAttemptId, Long>> iterator =
+        Iterator<Map.Entry<TaskAttemptId, ReportTime>> iterator =
             runningAttempts.entrySet().iterator();
 
         // avoid calculating current time everytime in loop
         long currentTime = clock.getTime();
 
         while (iterator.hasNext()) {
-          Map.Entry<TaskAttemptId, Long> entry = iterator.next();
-          if (currentTime > entry.getValue() + taskTimeOut) {
-
-            //In case the iterator isn't picking up the latest.
-            // Extra lookup outside of the iterator - but only if the task
-            // is considered to be timed out.
-            Long taskTime = runningAttempts.get(entry.getKey());
-            if (taskTime != null && currentTime > taskTime + taskTimeOut) {
-              // task is lost, remove from the list and raise lost event
-              iterator.remove();
-              eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry
-                  .getKey(), "AttemptID:" + entry.getKey().toString()
-                  + " Timed out after " + taskTimeOut / 1000 + " secs"));
-              eventHandler.handle(new TaskAttemptEvent(entry.getKey(),
-                  TaskAttemptEventType.TA_TIMED_OUT));
-            }
-
+          Map.Entry<TaskAttemptId, ReportTime> entry = iterator.next();
+          boolean taskTimedOut = (taskTimeOut > 0) && 
+              (currentTime > (entry.getValue().getLastProgress() + taskTimeOut));
+          boolean pingTimedOut =
+              (currentTime > (entry.getValue().getLastPing() + PING_TIMEOUT));
+              
+          if(taskTimedOut || pingTimedOut) {
+            // task is lost, remove from the list and raise lost event
+            iterator.remove();
+            eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry
+                .getKey(), "AttemptID:" + entry.getKey().toString()
+                + " Timed out after " + taskTimeOut / 1000 + " secs"));
+            eventHandler.handle(new TaskAttemptEvent(entry.getKey(),
+                TaskAttemptEventType.TA_TIMED_OUT));
           }
         }
         try {

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

@@ -727,7 +727,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
         // Commit job & do cleanup
         job.getCommitter().commitJob(job.getJobContext());
       } catch (IOException e) {
-        LOG.warn("Could not do commit for Job", e);
+        LOG.error("Could not do commit for Job", e);
+        job.logJobHistoryFinishedEvent();
+        return job.finished(JobState.FAILED);
       }
       job.logJobHistoryFinishedEvent();
       return job.finished(JobState.SUCCEEDED);

+ 0 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java

@@ -30,11 +30,4 @@ public interface ContainerLauncher
     CONTAINER_REMOTE_CLEANUP
   }
 
-  // Not a documented config. Only used for tests
-  static final String MR_AM_NM_COMMAND_TIMEOUT = MRJobConfig.MR_AM_PREFIX
-      + "nm-command-timeout";
-  /**
-   *  Maximum of 1 minute timeout for a Node to react to the command
-   */
-  static final int DEFAULT_NM_COMMAND_TIMEOUT = 60000;
 }

+ 2 - 104
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java

@@ -23,8 +23,6 @@ import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
 import java.util.HashSet;
 import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -72,8 +70,6 @@ public class ContainerLauncherImpl extends AbstractService implements
 
   static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
 
-  int nmTimeOut;
-
   private ConcurrentHashMap<ContainerId, Container> containers = 
     new ConcurrentHashMap<ContainerId, Container>(); 
   private AppContext context;
@@ -83,7 +79,6 @@ public class ContainerLauncherImpl extends AbstractService implements
   private Thread eventHandlingThread;
   protected BlockingQueue<ContainerLauncherEvent> eventQueue =
       new LinkedBlockingQueue<ContainerLauncherEvent>();
-  final Timer commandTimer = new Timer(true);
   YarnRPC rpc;
 
   private Container getContainer(ContainerId id) {
@@ -130,30 +125,18 @@ public class ContainerLauncherImpl extends AbstractService implements
             "Container was killed before it was launched");
         return;
       }
-      CommandTimerTask timerTask = new CommandTimerTask(Thread
-          .currentThread(), event);
       
+
       final String containerManagerBindAddr = event.getContainerMgrAddress();
       ContainerId containerID = event.getContainerID();
       ContainerToken containerToken = event.getContainerToken();
 
       ContainerManager proxy = null;
       try {
-        commandTimer.schedule(timerTask, nmTimeOut);
 
         proxy = getCMProxy(containerID, containerManagerBindAddr,
             containerToken);
 
-        // Interrupted during getProxy, but that didn't throw exception
-        if (Thread.interrupted()) {
-          // The timer canceled the command in the mean while.
-          String message = "Container launch failed for " + containerID
-              + " : Start-container for " + event.getContainerID()
-              + " got interrupted. Returning.";
-          this.state = ContainerState.FAILED;
-          sendContainerLaunchFailedMsg(taskAttemptID, message);
-          return;
-        }
         // Construct the actual Container
         ContainerLaunchContext containerLaunchContext =
           event.getContainer();
@@ -164,19 +147,6 @@ public class ContainerLauncherImpl extends AbstractService implements
         startRequest.setContainerLaunchContext(containerLaunchContext);
         StartContainerResponse response = proxy.startContainer(startRequest);
 
-        // container started properly. Stop the timer
-        timerTask.cancel();
-        if (Thread.interrupted()) {
-          // The timer canceled the command in the mean while, but
-          // startContainer didn't throw exception
-          String message = "Container launch failed for " + containerID
-              + " : Start-container for " + event.getContainerID()
-              + " got interrupted. Returning.";
-          this.state = ContainerState.FAILED;
-          sendContainerLaunchFailedMsg(taskAttemptID, message);
-          return;
-        }
-
         ByteBuffer portInfo = response
           .getServiceResponse(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID);
         int port = -1;
@@ -198,17 +168,11 @@ public class ContainerLauncherImpl extends AbstractService implements
             new TaskAttemptContainerLaunchedEvent(taskAttemptID, port));
         this.state = ContainerState.RUNNING;
       } catch (Throwable t) {
-        if (Thread.interrupted()) {
-          // The timer canceled the command in the mean while.
-          LOG.info("Start-container for " + event.getContainerID()
-              + " got interrupted.");
-        }
         String message = "Container launch failed for " + containerID + " : "
             + StringUtils.stringifyException(t);
         this.state = ContainerState.FAILED;
         sendContainerLaunchFailedMsg(taskAttemptID, message);
       } finally {
-        timerTask.cancel();
         if (proxy != null) {
           ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
         }
@@ -220,41 +184,24 @@ public class ContainerLauncherImpl extends AbstractService implements
       if(this.state == ContainerState.PREP) {
         this.state = ContainerState.KILLED_BEFORE_LAUNCH;
       } else {
-        CommandTimerTask timerTask = new CommandTimerTask(Thread
-            .currentThread(), event);
-
         final String containerManagerBindAddr = event.getContainerMgrAddress();
         ContainerId containerID = event.getContainerID();
         ContainerToken containerToken = event.getContainerToken();
         TaskAttemptId taskAttemptID = event.getTaskAttemptID();
         LOG.info("KILLING " + taskAttemptID);
-        commandTimer.schedule(timerTask, nmTimeOut);
 
         ContainerManager proxy = null;
         try {
           proxy = getCMProxy(containerID, containerManagerBindAddr,
               containerToken);
 
-          if (Thread.interrupted()) {
-            // The timer canceled the command in the mean while. No need to
-            // return, send cleaned up event anyways.
-            LOG.info("Stop-container for " + event.getContainerID()
-                + " got interrupted.");
-          } else {
             // kill the remote container if already launched
             StopContainerRequest stopRequest = Records
               .newRecord(StopContainerRequest.class);
             stopRequest.setContainerId(event.getContainerID());
             proxy.stopContainer(stopRequest);
-          }
-        } catch (Throwable t) {
 
-          if (Thread.interrupted()) {
-            // The timer canceled the command in the mean while, clear the
-            // interrupt flag
-            LOG.info("Stop-container for " + event.getContainerID()
-                + " got interrupted.");
-          }
+        } catch (Throwable t) {
 
           // ignore the cleanup failure
           String message = "cleanup failed for container "
@@ -264,15 +211,6 @@ public class ContainerLauncherImpl extends AbstractService implements
             new TaskAttemptDiagnosticsUpdateEvent(taskAttemptID, message));
           LOG.warn(message);
         } finally {
-          timerTask.cancel();
-          if (Thread.interrupted()) {
-            LOG.info("Stop-container for " + event.getContainerID()
-                + " got interrupted.");
-            // ignore the cleanup failure
-            context.getEventHandler().handle(
-              new TaskAttemptDiagnosticsUpdateEvent(taskAttemptID,
-                "cleanup failed for container " + event.getContainerID()));
-          }
           if (proxy != null) {
             ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
           }
@@ -303,8 +241,6 @@ public class ContainerLauncherImpl extends AbstractService implements
         MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT,
         MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
     LOG.info("Upper limit on the thread pool size is " + this.limitOnPoolSize);
-    this.nmTimeOut = conf.getInt(ContainerLauncher.MR_AM_NM_COMMAND_TIMEOUT,
-        ContainerLauncher.DEFAULT_NM_COMMAND_TIMEOUT);
     this.rpc = createYarnRPC(conf);
     super.init(conf);
   }
@@ -409,44 +345,6 @@ public class ContainerLauncherImpl extends AbstractService implements
     return proxy;
   }
 
-  private static class CommandTimerTask extends TimerTask {
-    private final Thread commandThread;
-    protected final String message;
-    private boolean cancelled = false;
-
-    public CommandTimerTask(Thread thread, ContainerLauncherEvent event) {
-      super();
-      this.commandThread = thread;
-      this.message = "Couldn't complete " + event.getType() + " on "
-          + event.getContainerID() + "/" + event.getTaskAttemptID()
-          + ". Interrupting and returning";
-    }
-
-    @Override
-    public void run() {
-      synchronized (this) {
-        if (this.cancelled) {
-          return;
-        }
-        LOG.warn(this.message);
-        StackTraceElement[] trace = this.commandThread.getStackTrace();
-        StringBuilder logMsg = new StringBuilder();
-        for (int i = 0; i < trace.length; i++) {
-          logMsg.append("\n\tat " + trace[i]);
-        }
-        LOG.info("Stack trace of the command-thread: \n" + logMsg.toString());
-        this.commandThread.interrupt();
-      }
-    }
-
-    @Override
-    public boolean cancel() {
-      synchronized (this) {
-        this.cancelled = true;
-        return super.cancel();
-      }
-    }
-  }
 
   /**
    * Setup and start the container on remote nodemanager.

+ 69 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java

@@ -0,0 +1,69 @@
+/**
+* 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.yarn.Clock;
+import org.apache.hadoop.yarn.SystemClock;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.junit.Test;
+
+import static org.mockito.Mockito.*;
+
+
+public class TestTaskHeartbeatHandler {
+  
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  @Test
+  public void testTimeout() throws InterruptedException {
+    EventHandler mockHandler = mock(EventHandler.class);
+    Clock clock = new SystemClock();
+    TaskHeartbeatHandler hb = new TaskHeartbeatHandler(mockHandler, clock, 1);
+    
+    
+    Configuration conf = new Configuration();
+    conf.setInt(MRJobConfig.TASK_TIMEOUT, 10); //10 ms
+    conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 10); //10 ms
+    
+    hb.init(conf);
+    hb.start();
+    try {
+      ApplicationId appId = BuilderUtils.newApplicationId(0l, 5);
+      JobId jobId = MRBuilderUtils.newJobId(appId, 4);
+      TaskId tid = MRBuilderUtils.newTaskId(jobId, 3, TaskType.MAP);
+      TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 2);
+      hb.register(taid);
+      Thread.sleep(100);
+      //Events only happen when the task is canceled
+      verify(mockHandler, times(2)).handle(any(Event.class));
+    } finally {
+      hb.stop();
+    }
+  }
+
+}

+ 33 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -72,6 +73,37 @@ public class TestJobImpl {
         JobState.ERROR, state);
   }
 
+  @Test
+  public void testCommitJobFailsJob() {
+
+    JobImpl mockJob = mock(JobImpl.class);
+    mockJob.tasks = new HashMap<TaskId, Task>();
+    OutputCommitter mockCommitter = mock(OutputCommitter.class);
+    EventHandler mockEventHandler = mock(EventHandler.class);
+    JobContext mockJobContext = mock(JobContext.class);
+
+    when(mockJob.getCommitter()).thenReturn(mockCommitter);
+    when(mockJob.getEventHandler()).thenReturn(mockEventHandler);
+    when(mockJob.getJobContext()).thenReturn(mockJobContext);
+    doNothing().when(mockJob).setFinishTime();
+    doNothing().when(mockJob).logJobHistoryFinishedEvent();
+    when(mockJob.finished(JobState.KILLED)).thenReturn(JobState.KILLED);
+    when(mockJob.finished(JobState.FAILED)).thenReturn(JobState.FAILED);
+    when(mockJob.finished(JobState.SUCCEEDED)).thenReturn(JobState.SUCCEEDED);
+
+    try {
+      doThrow(new IOException()).when(mockCommitter).commitJob(any(JobContext.class));
+    } catch (IOException e) {
+      // commitJob stubbed out, so this can't happen
+    }
+    doNothing().when(mockEventHandler).handle(any(JobHistoryEvent.class));
+    Assert.assertNotNull("checkJobCompleteSuccess incorrectly returns null " +
+      "for successful job",
+      JobImpl.checkJobCompleteSuccess(mockJob));
+    Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
+        JobState.FAILED, JobImpl.checkJobCompleteSuccess(mockJob));
+  }
+
   @Test
   public void testCheckJobCompleteSuccess() {
     
@@ -98,9 +130,7 @@ public class TestJobImpl {
       "for successful job",
       JobImpl.checkJobCompleteSuccess(mockJob));
     Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
-        JobImpl.checkJobCompleteSuccess(mockJob), JobState.SUCCEEDED);
-
-    
+        JobState.SUCCEEDED, JobImpl.checkJobCompleteSuccess(mockJob));
   }
 
   @Test

+ 106 - 49
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.mapreduce.v2.app.launcher;
 import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -30,6 +32,7 @@ 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.ipc.Server;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
@@ -44,18 +47,39 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.ContainerManager;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.factory.providers.YarnRemoteExceptionFactoryProvider;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
 
 public class TestContainerLauncher {
 
-  static final Log LOG = LogFactory
-      .getLog(TestContainerLauncher.class);
+  private static final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+  Configuration conf;
+  Server server;
+
+  static final Log LOG = LogFactory.getLog(TestContainerLauncher.class);
 
   @Test
   public void testPoolSize() throws InterruptedException {
@@ -104,10 +128,10 @@ public class TestContainerLauncher {
     Assert.assertEquals(10, containerLauncher.numEventsProcessed.get());
     containerLauncher.finishEventHandling = false;
     for (int i = 0; i < 10; i++) {
-      ContainerId containerId =
-          BuilderUtils.newContainerId(appAttemptId, i + 10);
-      TaskAttemptId taskAttemptId =
-          MRBuilderUtils.newTaskAttemptId(taskId, i + 10);
+      ContainerId containerId = BuilderUtils.newContainerId(appAttemptId,
+          i + 10);
+      TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId,
+          i + 10);
       containerLauncher.handle(new ContainerLauncherEvent(taskAttemptId,
         containerId, "host" + i + ":1234", null,
         ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
@@ -119,8 +143,7 @@ public class TestContainerLauncher {
     // Different hosts, there should be an increase in core-thread-pool size to
     // 21(11hosts+10buffer)
     // Core pool size should be 21 but the live pool size should be only 11.
-    containerLauncher.expectedCorePoolSize =
-        11 + ContainerLauncherImpl.INITIAL_POOL_SIZE;
+    containerLauncher.expectedCorePoolSize = 11 + ContainerLauncherImpl.INITIAL_POOL_SIZE;
     containerLauncher.finishEventHandling = false;
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 21);
     TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, 21);
@@ -200,26 +223,28 @@ public class TestContainerLauncher {
 
   @Test
   public void testSlowNM() throws Exception {
-    test(false);
-  }
-
-  @Test
-  public void testSlowNMWithInterruptsSwallowed() throws Exception {
-    test(true);
+    test();
   }
 
-  private void test(boolean swallowInterrupts) throws Exception {
+  private void test() throws Exception {
 
-    MRApp app = new MRAppWithSlowNM(swallowInterrupts);
-
-    Configuration conf = new Configuration();
+    conf = new Configuration();
     int maxAttempts = 1;
     conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, maxAttempts);
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
-
-    // Set low timeout for NM commands
-    conf.setInt(ContainerLauncher.MR_AM_NM_COMMAND_TIMEOUT, 3000);
-
+    // set timeout low for the test
+    conf.setInt("yarn.rpc.nm-command-timeout", 3000);
+    conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class.getName());
+    YarnRPC rpc = YarnRPC.create(conf);
+    String bindAddr = "localhost:0";
+    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+    server = rpc.getServer(ContainerManager.class, new DummyContainerManager(),
+        addr, conf, null, 1);
+    server.start();
+
+    MRApp app = new MRAppWithSlowNM();
+
+    try {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
 
@@ -231,8 +256,8 @@ public class TestContainerLauncher {
 
     Map<TaskAttemptId, TaskAttempt> attempts = tasks.values().iterator()
         .next().getAttempts();
-    Assert.assertEquals("Num attempts is not correct", maxAttempts, attempts
-        .size());
+      Assert.assertEquals("Num attempts is not correct", maxAttempts,
+          attempts.size());
 
     TaskAttempt attempt = attempts.values().iterator().next();
     app.waitForState(attempt, TaskAttemptState.ASSIGNED);
@@ -241,20 +266,18 @@ public class TestContainerLauncher {
 
     String diagnostics = attempt.getDiagnostics().toString();
     LOG.info("attempt.getDiagnostics: " + diagnostics);
-    if (swallowInterrupts) {
-      Assert.assertEquals("[Container launch failed for "
-          + "container_0_0000_01_000000 : Start-container for "
-          + "container_0_0000_01_000000 got interrupted. Returning.]",
-          diagnostics);
-    } else {
+
       Assert.assertTrue(diagnostics.contains("Container launch failed for "
           + "container_0_0000_01_000000 : "));
-      Assert.assertTrue(diagnostics
-          .contains(": java.lang.InterruptedException"));
-    }
+      Assert
+          .assertTrue(diagnostics
+              .contains("java.net.SocketTimeoutException: 3000 millis timeout while waiting for channel"));
 
+    } finally {
+      server.stop();
     app.stop();
   }
+  }
 
   private final class CustomContainerLauncher extends ContainerLauncherImpl {
 
@@ -317,13 +340,10 @@ public class TestContainerLauncher {
     }
   }
 
-  private static class MRAppWithSlowNM extends MRApp {
+  private class MRAppWithSlowNM extends MRApp {
 
-    final boolean swallowInterrupts;
-
-    public MRAppWithSlowNM(boolean swallowInterrupts) {
+    public MRAppWithSlowNM() {
       super(1, 0, false, "TestContainerLauncher", true);
-      this.swallowInterrupts = swallowInterrupts;
     }
 
     @Override
@@ -333,20 +353,57 @@ public class TestContainerLauncher {
         protected ContainerManager getCMProxy(ContainerId containerID,
             String containerManagerBindAddr, ContainerToken containerToken)
             throws IOException {
+          // make proxy connect to our local containerManager server
+          ContainerManager proxy = (ContainerManager) rpc.getProxy(
+              ContainerManager.class,
+              NetUtils.createSocketAddr("localhost:" + server.getPort()), conf);
+          return proxy;
+        }
+      };
+
+    };
+  }
+
+  public class DummyContainerManager implements ContainerManager {
+
+    private ContainerStatus status = null;
+
+    @Override
+    public GetContainerStatusResponse getContainerStatus(
+        GetContainerStatusRequest request) throws YarnRemoteException {
+      GetContainerStatusResponse response = recordFactory
+          .newRecordInstance(GetContainerStatusResponse.class);
+      response.setStatus(status);
+      return response;
+    }
+
+    @Override
+    public StartContainerResponse startContainer(StartContainerRequest request)
+        throws YarnRemoteException {
+      ContainerLaunchContext container = request.getContainerLaunchContext();
+      StartContainerResponse response = recordFactory
+          .newRecordInstance(StartContainerResponse.class);
+      status = recordFactory.newRecordInstance(ContainerStatus.class);
           try {
-            synchronized (this) {
-              wait(); // Just hang the thread simulating a very slow NM.
+        // make the thread sleep to look like its not going to respond
+        Thread.sleep(15000);
+      } catch (Exception e) {
+        LOG.error(e);
+        throw new UndeclaredThrowableException(e);
             }
-          } catch (InterruptedException e) {
-            LOG.info(e);
-            if (!MRAppWithSlowNM.this.swallowInterrupts) {
-              throw new IOException(e);
+      status.setState(ContainerState.RUNNING);
+      status.setContainerId(container.getContainerId());
+      status.setExitStatus(0);
+      return response;
             }
-            Thread.currentThread().interrupt();
+
+    @Override
+    public StopContainerResponse stopContainer(StopContainerRequest request)
+        throws YarnRemoteException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw YarnRemoteExceptionFactoryProvider.getYarnRemoteExceptionFactory(
+          null).createYarnRemoteException(e);
           }
-          return null;
         }
-      };
-    };
   }
-}

+ 7 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -351,7 +351,7 @@
   <value>600000</value>
   <description>The number of milliseconds before a task will be
   terminated if it neither reads an input, writes an output, nor
-  updates its status string.
+  updates its status string.  A value of 0 disables the timeout.
   </description>
 </property>
 
@@ -412,7 +412,12 @@
         -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc
   
   The configuration variable mapred.child.ulimit can be used to control the
-  maximum virtual memory of the child processes. 
+  maximum virtual memory of the child processes.
+
+  Usage of -Djava.library.path can cause programs to no longer function if
+  hadoop native libraries are used. These values should instead be set as part 
+  of LD_LIBRARY_PATH in the map / reduce JVM env using the mapreduce.map.env and 
+  mapreduce.reduce.env config settings. 
   </description>
 </property>
 

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java

@@ -31,6 +31,7 @@ import javax.ws.rs.core.UriInfo;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
+import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
@@ -96,6 +97,7 @@ public class HsWebServices {
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public JobsInfo getJobs(@QueryParam("user") String userQuery,
       @QueryParam("limit") String count,
+      @QueryParam("state") String stateQuery,
       @QueryParam("queue") String queueQuery,
       @QueryParam("startedTimeBegin") String startedBegin,
       @QueryParam("startedTimeEnd") String startedEnd,
@@ -185,6 +187,13 @@ public class HsWebServices {
         break;
       }
 
+      if (stateQuery != null && !stateQuery.isEmpty()) {
+        JobState.valueOf(stateQuery);
+        if (!job.getState().toString().equalsIgnoreCase(stateQuery)) {
+          continue;
+        }
+      }
+
       // can't really validate queue is a valid one since queues could change
       if (queueQuery != null && !queueQuery.isEmpty()) {
         if (!job.getQueueName().equals(queueQuery)) {

+ 70 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java

@@ -32,6 +32,7 @@ import javax.ws.rs.core.MediaType;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockJobs;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
@@ -120,7 +121,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     public Job getPartialJob(JobId jobID) {
       return partialJobs.get(jobID);
     }
-    
+
     @Override
     public Map<JobId, Job> getAllJobs() {
       return partialJobs; // OK
@@ -195,6 +196,72 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .contextPath("jersey-guice-filter").servletPath("/").build());
   }
 
+  @Test
+  public void testJobsQueryStateNone() throws JSONException, Exception {
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("history")
+        .path("mapreduce").path("jobs").queryParam("state", JobState.KILL_WAIT.toString())
+        .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
+  }
+
+  @Test
+  public void testJobsQueryState() throws JSONException, Exception {
+    WebResource r = resource();
+    // we only create 3 jobs and it cycles through states so we should have 3 unique states
+    Map<JobId, Job> jobsMap = appContext.getAllJobs();
+    String queryState = "BOGUS";
+    JobId jid = null;
+    for (Map.Entry<JobId, Job> entry : jobsMap.entrySet()) {
+      jid = entry.getValue().getID();
+      queryState = entry.getValue().getState().toString();
+      break;
+    }
+    ClientResponse response = r.path("ws").path("v1").path("history")
+        .path("mapreduce").path("jobs").queryParam("state", queryState)
+        .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    assertEquals("incorrect number of elements", 1, json.length());
+    JSONObject jobs = json.getJSONObject("jobs");
+    JSONArray arr = jobs.getJSONArray("job");
+    assertEquals("incorrect number of elements", 1, arr.length());
+    JSONObject info = arr.getJSONObject(0);
+    Job job = appContext.getPartialJob(jid);
+    VerifyJobsUtils.verifyHsJobPartial(info, job);
+  }
+
+  @Test
+  public void testJobsQueryStateInvalid() throws JSONException, Exception {
+    WebResource r = resource();
+
+    ClientResponse response = r.path("ws").path("v1").path("history")
+        .path("mapreduce").path("jobs").queryParam("state", "InvalidState")
+        .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+
+    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject msg = response.getEntity(JSONObject.class);
+    JSONObject exception = msg.getJSONObject("RemoteException");
+    assertEquals("incorrect number of elements", 3, exception.length());
+    String message = exception.getString("message");
+    String type = exception.getString("exception");
+    String classname = exception.getString("javaClassName");
+    WebServicesTestUtils
+        .checkStringMatch(
+            "exception message",
+            "No enum const class org.apache.hadoop.mapreduce.v2.api.records.JobState.InvalidState",
+            message);
+    WebServicesTestUtils.checkStringMatch("exception type",
+        "IllegalArgumentException", type);
+    WebServicesTestUtils.checkStringMatch("exception classname",
+        "java.lang.IllegalArgumentException", classname);
+  }
+
+
   @Test
   public void testJobsQueryUserNone() throws JSONException, Exception {
     WebResource r = resource();
@@ -215,6 +282,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
+    System.out.println(json.toString());
+
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject jobs = json.getJSONObject("jobs");
     JSONArray arr = jobs.getJSONArray("job");

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
@@ -228,6 +229,10 @@ public class TestMiniMRChildTask {
       // check if X=$(tt's X var):/tmp for an old env variable inherited from 
       // the tt
       checkEnv("PATH",  path + ":/tmp", "noappend");
+
+      String jobLocalDir = job.get(MRJobConfig.JOB_LOCAL_DIR);
+      assertNotNull(MRJobConfig.JOB_LOCAL_DIR + " is null",
+                    jobLocalDir);
     }
 
     public void map(WritableComparable key, Writable value,

+ 17 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java

@@ -24,6 +24,8 @@ import java.net.InetSocketAddress;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.ContainerManagerPB;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
@@ -38,6 +40,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestP
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
@@ -48,12 +51,25 @@ import com.google.protobuf.ServiceException;
 
 public class ContainerManagerPBClientImpl implements ContainerManager {
 
+  // Not a documented config. Only used for tests
+  static final String NM_COMMAND_TIMEOUT = YarnConfiguration.YARN_PREFIX
+      + "rpc.nm-command-timeout";
+
+  /**
+   *  Maximum of 1 minute timeout for a Node to react to the command
+   */
+  static final int DEFAULT_COMMAND_TIMEOUT = 60000;
+
   private ContainerManagerPB proxy;
   
   public ContainerManagerPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
     RPC.setProtocolEngine(conf, ContainerManagerPB.class, ProtobufRpcEngine.class);
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
+    int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT);
     proxy = (ContainerManagerPB)RPC.getProxy(
-        ContainerManagerPB.class, clientVersion, addr, conf);
+        ContainerManagerPB.class, clientVersion, addr, ugi, conf,
+        NetUtils.getDefaultSocketFactory(conf), expireIntvl);
   }
 
   public void close() {

+ 170 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java

@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn;
+
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetSocketAddress;
+
+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.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ContainerManager;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.factory.providers.YarnRemoteExceptionFactoryProvider;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.junit.Test;
+
+/*
+ * Test that the container launcher rpc times out properly. This is used
+ * by both RM to launch an AM as well as an AM to launch containers.
+ */
+public class TestContainerLaunchRPC {
+
+  static final Log LOG = LogFactory.getLog(TestContainerLaunchRPC.class);
+
+  private static final String EXCEPTION_CAUSE = "java.net.SocketTimeoutException";
+  private static final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+
+  @Test
+  public void testHadoopProtoRPCTimeout() throws Exception {
+    testRPCTimeout(HadoopYarnProtoRPC.class.getName());
+  }
+
+  private void testRPCTimeout(String rpcClass) throws Exception {
+    Configuration conf = new Configuration();
+    // set timeout low for the test
+    conf.setInt("yarn.rpc.nm-command-timeout", 3000);
+
+    conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass);
+    YarnRPC rpc = YarnRPC.create(conf);
+    String bindAddr = "localhost:0";
+    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+    Server server = rpc.getServer(ContainerManager.class,
+        new DummyContainerManager(), addr, conf, null, 1);
+    server.start();
+    try {
+
+      ContainerManager proxy = (ContainerManager) rpc.getProxy(
+          ContainerManager.class,
+          NetUtils.createSocketAddr("localhost:" + server.getPort()), conf);
+      ContainerLaunchContext containerLaunchContext = recordFactory
+          .newRecordInstance(ContainerLaunchContext.class);
+      containerLaunchContext.setUser("dummy-user");
+      ContainerId containerId = recordFactory
+          .newRecordInstance(ContainerId.class);
+      ApplicationId applicationId = recordFactory
+          .newRecordInstance(ApplicationId.class);
+      ApplicationAttemptId applicationAttemptId = recordFactory
+          .newRecordInstance(ApplicationAttemptId.class);
+      applicationId.setClusterTimestamp(0);
+      applicationId.setId(0);
+      applicationAttemptId.setApplicationId(applicationId);
+      applicationAttemptId.setAttemptId(0);
+      containerId.setApplicationAttemptId(applicationAttemptId);
+      containerId.setId(100);
+      containerLaunchContext.setContainerId(containerId);
+      containerLaunchContext.setResource(recordFactory
+          .newRecordInstance(Resource.class));
+
+      StartContainerRequest scRequest = recordFactory
+          .newRecordInstance(StartContainerRequest.class);
+      scRequest.setContainerLaunchContext(containerLaunchContext);
+      try {
+        proxy.startContainer(scRequest);
+      } catch (Exception e) {
+        LOG.info(StringUtils.stringifyException(e));
+        Assert.assertTrue("Error, exception does not contain: "
+            + EXCEPTION_CAUSE,
+            e.getCause().getMessage().contains(EXCEPTION_CAUSE));
+
+        return;
+      }
+    } finally {
+      server.stop();
+    }
+
+    Assert.fail("timeout exception should have occurred!");
+  }
+
+  public class DummyContainerManager implements ContainerManager {
+
+    private ContainerStatus status = null;
+
+    @Override
+    public GetContainerStatusResponse getContainerStatus(
+        GetContainerStatusRequest request) throws YarnRemoteException {
+      GetContainerStatusResponse response = recordFactory
+          .newRecordInstance(GetContainerStatusResponse.class);
+      response.setStatus(status);
+      return response;
+    }
+
+    @Override
+    public StartContainerResponse startContainer(StartContainerRequest request)
+        throws YarnRemoteException {
+      ContainerLaunchContext container = request.getContainerLaunchContext();
+      StartContainerResponse response = recordFactory
+          .newRecordInstance(StartContainerResponse.class);
+      status = recordFactory.newRecordInstance(ContainerStatus.class);
+      try {
+        // make the thread sleep to look like its not going to respond
+        Thread.sleep(10000);
+      } catch (Exception e) {
+        LOG.error(e);
+        throw new UndeclaredThrowableException(e);
+      }
+      status.setState(ContainerState.RUNNING);
+      status.setContainerId(container.getContainerId());
+      status.setExitStatus(0);
+      return response;
+    }
+
+    @Override
+    public StopContainerResponse stopContainer(StopContainerRequest request)
+        throws YarnRemoteException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw YarnRemoteExceptionFactoryProvider.getYarnRemoteExceptionFactory(
+          null).createYarnRemoteException(e);
+    }
+  }
+}

+ 2 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -172,7 +172,8 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       int exitCode = shExec.getExitCode();
       LOG.warn("Exit code from container is : " + exitCode);
       logOutput(shExec.getOutput());
-      throw new IOException("App initialization failed (" + exitCode + ")", e);
+      throw new IOException("App initialization failed (" + exitCode + 
+          ") with output: " + shExec.getOutput(), e);
     }
   }
 

+ 3 - 3
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java

@@ -100,8 +100,8 @@ class CapacitySchedulerPage extends RmView {
     @Override
     public void render(Block html) {
       ArrayList<CapacitySchedulerQueueInfo> subQueues =
-          (csqinfo.qinfo == null) ? csqinfo.csinfo.getSubQueues()
-              : csqinfo.qinfo.getSubQueues();
+          (csqinfo.qinfo == null) ? csqinfo.csinfo.getQueues().getQueueInfoList()
+              : csqinfo.qinfo.getQueues().getQueueInfoList();
       UL<Hamlet> ul = html.ul("#pq");
       for (CapacitySchedulerQueueInfo info : subQueues) {
         float used = info.getUsedCapacity() / 100;
@@ -122,7 +122,7 @@ class CapacitySchedulerPage extends RmView {
               _(join(percent(used), " used"))._();
 
         csqinfo.qinfo = info;
-        if (info.getSubQueues() == null) {
+        if (info.getQueues() == null) {
           li.ul("#lq").li()._(LeafQueueInfoBlock.class)._()._();
         } else {
           li._(QueueBlock.class);

+ 2 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfoList;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FifoSchedulerInfo;
@@ -57,7 +58,7 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
       SchedulerTypeInfo.class, NodeInfo.class, UserMetricsInfo.class,
       CapacitySchedulerInfo.class, ClusterMetricsInfo.class,
       SchedulerInfo.class, AppsInfo.class, NodesInfo.class,
-      RemoteExceptionData.class};
+      RemoteExceptionData.class, CapacitySchedulerQueueInfoList.class};
 
   public JAXBContextResolver() throws Exception {
     this.types = new HashSet<Class>(Arrays.asList(cTypes));

+ 16 - 7
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java

@@ -36,6 +36,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -225,6 +226,7 @@ public class RMWebServices {
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public AppsInfo getApps(@Context HttpServletRequest hsr,
       @QueryParam("state") String stateQuery,
+      @QueryParam("finalStatus") String finalStatusQuery,
       @QueryParam("user") String userQuery,
       @QueryParam("queue") String queueQuery,
       @QueryParam("limit") String count,
@@ -294,19 +296,25 @@ public class RMWebServices {
         .getRMApps();
     AppsInfo allApps = new AppsInfo();
     for (RMApp rmapp : apps.values()) {
+
       if (checkCount && num == countNum) {
         break;
       }
-      AppInfo app = new AppInfo(rmapp, hasAccess(rmapp, hsr));
-
       if (stateQuery != null && !stateQuery.isEmpty()) {
         RMAppState.valueOf(stateQuery);
-        if (!app.getState().equalsIgnoreCase(stateQuery)) {
+        if (!rmapp.getState().toString().equalsIgnoreCase(stateQuery)) {
+          continue;
+        }
+      }
+      if (finalStatusQuery != null && !finalStatusQuery.isEmpty()) {
+        FinalApplicationStatus.valueOf(finalStatusQuery);
+        if (!rmapp.getFinalApplicationStatus().toString()
+            .equalsIgnoreCase(finalStatusQuery)) {
           continue;
         }
       }
       if (userQuery != null && !userQuery.isEmpty()) {
-        if (!app.getUser().equals(userQuery)) {
+        if (!rmapp.getUser().equals(userQuery)) {
           continue;
         }
       }
@@ -321,19 +329,20 @@ public class RMWebServices {
             throw new BadRequestException(e.getMessage());
           }
         }
-        if (!app.getQueue().equals(queueQuery)) {
+        if (!rmapp.getQueue().equals(queueQuery)) {
           continue;
         }
       }
 
       if (checkStart
-          && (app.getStartTime() < sBegin || app.getStartTime() > sEnd)) {
+          && (rmapp.getStartTime() < sBegin || rmapp.getStartTime() > sEnd)) {
         continue;
       }
       if (checkEnd
-          && (app.getFinishTime() < fBegin || app.getFinishTime() > fEnd)) {
+          && (rmapp.getFinishTime() < fBegin || rmapp.getFinishTime() > fEnd)) {
         continue;
       }
+      AppInfo app = new AppInfo(rmapp, hasAccess(rmapp, hsr));
 
       allApps.add(app);
       num++;

+ 6 - 8
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 
-import java.util.ArrayList;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
@@ -38,7 +36,7 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
   protected float usedCapacity;
   protected float maxCapacity;
   protected String queueName;
-  protected ArrayList<CapacitySchedulerQueueInfo> queues;
+  protected CapacitySchedulerQueueInfoList queues;
 
   @XmlTransient
   static final float EPSILON = 1e-8f;
@@ -74,22 +72,22 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
     return this.queueName;
   }
 
-  public ArrayList<CapacitySchedulerQueueInfo> getSubQueues() {
+  public CapacitySchedulerQueueInfoList getQueues() {
     return this.queues;
   }
 
-  protected ArrayList<CapacitySchedulerQueueInfo> getQueues(CSQueue parent) {
+  protected CapacitySchedulerQueueInfoList getQueues(CSQueue parent) {
     CSQueue parentQueue = parent;
-    ArrayList<CapacitySchedulerQueueInfo> queuesInfo = new ArrayList<CapacitySchedulerQueueInfo>();
+    CapacitySchedulerQueueInfoList queuesInfo = new CapacitySchedulerQueueInfoList();
     for (CSQueue queue : parentQueue.getChildQueues()) {
       CapacitySchedulerQueueInfo info;
       if (queue instanceof LeafQueue) {
         info = new CapacitySchedulerLeafQueueInfo((LeafQueue)queue);
       } else {
         info = new CapacitySchedulerQueueInfo(queue);
-        info.subQueues = getQueues(queue);
+        info.queues = getQueues(queue);
       }
-      queuesInfo.add(info);
+      queuesInfo.addToQueueInfoList(info);
     }
     return queuesInfo;
   }

+ 3 - 5
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 
-import java.util.ArrayList;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
@@ -49,7 +47,7 @@ public class CapacitySchedulerQueueInfo {
   protected String usedResources;
   protected String queueName;
   protected QueueState state;
-  protected ArrayList<CapacitySchedulerQueueInfo> subQueues;
+  protected CapacitySchedulerQueueInfoList queues;
 
   CapacitySchedulerQueueInfo() {
   };
@@ -117,8 +115,8 @@ public class CapacitySchedulerQueueInfo {
     return this.queuePath;
   }
 
-  public ArrayList<CapacitySchedulerQueueInfo> getSubQueues() {
-    return this.subQueues;
+  public CapacitySchedulerQueueInfoList getQueues() {
+    return this.queues;
   }
 
   /**

Certains fichiers n'ont pas été affichés car il y a eu trop de fichiers modifiés dans ce diff