Преглед изворни кода

Merge r1569890 through r1572417 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1572418 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze пре 11 година
родитељ
комит
58bccdbc04
37 измењених фајлова са 557 додато и 305 уклоњено
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 5 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
  3. 0 51
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java
  4. 71 54
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
  5. 12 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  6. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  7. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  8. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
  9. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  10. 23 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsBlocksMetadata.java
  11. 6 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
  12. 7 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  13. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  14. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  15. 17 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  16. 10 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
  17. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  18. 1 38
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestVolumeId.java
  19. 9 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
  21. 120 24
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  22. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
  24. 6 0
      hadoop-mapreduce-project/CHANGES.txt
  25. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  26. 5 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  27. 8 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  28. 7 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
  29. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr
  30. 1 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java
  31. 48 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
  32. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
  33. 130 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
  34. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1393307629410_0001-1393307687476-user-Sleep+job-1393307723835-0-0-FAILED-default-1393307693920.jhist
  35. 6 0
      hadoop-yarn-project/CHANGES.txt
  36. 17 2
      hadoop-yarn-project/hadoop-yarn/bin/yarn
  37. 8 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

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

@@ -362,6 +362,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-6018. Exception recorded in LOG when IPCLoggerChannel#close is called.
     (jing9)
 
+    HDFS-3969. Small bug fixes and improvements for disk locations API.
+    (Todd Lipcon and Andrew Wang)
+
   OPTIMIZATIONS
 
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery

+ 5 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java

@@ -17,18 +17,18 @@
  */
 package org.apache.hadoop.fs;
 
-import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.base.Preconditions;
 
 /**
  * HDFS-specific volume identifier which implements {@link VolumeId}. Can be
  * used to differentiate between the data directories on a single datanode. This
  * identifier is only unique on a per-datanode basis.
- * 
- * Note that invalid IDs are represented by {@link VolumeId#INVALID_VOLUME_ID}.
  */
 @InterfaceStability.Unstable
 @InterfaceAudience.Public
@@ -37,28 +37,15 @@ public class HdfsVolumeId implements VolumeId {
   private final byte[] id;
 
   public HdfsVolumeId(byte[] id) {
-    if (id == null) {
-      throw new NullPointerException("A valid Id can only be constructed " +
-      		"with a non-null byte array.");
-    }
+    Preconditions.checkNotNull(id, "id cannot be null");
     this.id = id;
   }
 
-  @Override
-  public final boolean isValid() {
-    return true;
-  }
-
   @Override
   public int compareTo(VolumeId arg0) {
     if (arg0 == null) {
       return 1;
     }
-    if (!arg0.isValid()) {
-      // any valid ID is greater 
-      // than any invalid ID: 
-      return 1;
-    }
     return hashCode() - arg0.hashCode();
   }
 
@@ -76,14 +63,11 @@ public class HdfsVolumeId implements VolumeId {
       return true;
     }
     HdfsVolumeId that = (HdfsVolumeId) obj;
-    // NB: if (!obj.isValid()) { return false; } check is not necessary
-    // because we have class identity checking above, and for this class
-    // isValid() is always true.
     return new EqualsBuilder().append(this.id, that.id).isEquals();
   }
 
   @Override
   public String toString() {
-    return Base64.encodeBase64String(id);
+    return StringUtils.byteToHexString(id);
   }
 }

+ 0 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java

@@ -28,57 +28,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Public
 public interface VolumeId extends Comparable<VolumeId> {
 
-  /**
-   * Represents an invalid Volume ID (ID for unknown content).
-   */
-  public static final VolumeId INVALID_VOLUME_ID = new VolumeId() {
-    
-    @Override
-    public int compareTo(VolumeId arg0) {
-      // This object is equal only to itself;
-      // It is greater than null, and
-      // is always less than any other VolumeId:
-      if (arg0 == null) {
-        return 1;
-      }
-      if (arg0 == this) {
-        return 0;
-      } else {
-        return -1;
-      }
-    }
-    
-    @Override
-    public boolean equals(Object obj) {
-      // this object is equal only to itself:
-      return (obj == this);
-    }
-    
-    @Override
-    public int hashCode() {
-      return Integer.MIN_VALUE;
-    }
-    
-    @Override
-    public boolean isValid() {
-      return false;
-    }
-    
-    @Override
-    public String toString() {
-      return "Invalid VolumeId";
-    }
-  };
-  
-  /**
-   * Indicates if the disk identifier is valid. Invalid identifiers indicate
-   * that the block was not present, or the location could otherwise not be
-   * determined.
-   * 
-   * @return true if the disk identifier is valid
-   */
-  public boolean isValid();
-
   @Override
   abstract public int compareTo(VolumeId arg0);
 

+ 71 - 54
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java

@@ -19,10 +19,8 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
@@ -43,7 +41,6 @@ import org.apache.hadoop.fs.HdfsVolumeId;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -51,16 +48,20 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.token.Token;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 class BlockStorageLocationUtil {
   
-  private static final Log LOG = LogFactory
+  static final Log LOG = LogFactory
       .getLog(BlockStorageLocationUtil.class);
   
   /**
    * Create a list of {@link VolumeBlockLocationCallable} corresponding to a set
-   * of datanodes and blocks.
+   * of datanodes and blocks. The blocks must all correspond to the same
+   * block pool.
    * 
    * @param datanodeBlocks
    *          Map of datanodes to block replicas at each datanode
@@ -70,6 +71,11 @@ class BlockStorageLocationUtil {
   private static List<VolumeBlockLocationCallable> createVolumeBlockLocationCallables(
       Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
       int timeout, boolean connectToDnViaHostname) {
+    
+    if (datanodeBlocks.isEmpty()) {
+      return Lists.newArrayList();
+    }
+    
     // Construct the callables, one per datanode
     List<VolumeBlockLocationCallable> callables = 
         new ArrayList<VolumeBlockLocationCallable>();
@@ -78,17 +84,32 @@ class BlockStorageLocationUtil {
       // Construct RPC parameters
       DatanodeInfo datanode = entry.getKey();
       List<LocatedBlock> locatedBlocks = entry.getValue();
-      List<ExtendedBlock> extendedBlocks = 
-          new ArrayList<ExtendedBlock>(locatedBlocks.size());
+      if (locatedBlocks.isEmpty()) {
+        continue;
+      }
+      
+      // Ensure that the blocks all are from the same block pool.
+      String poolId = locatedBlocks.get(0).getBlock().getBlockPoolId();
+      for (LocatedBlock lb : locatedBlocks) {
+        if (!poolId.equals(lb.getBlock().getBlockPoolId())) {
+          throw new IllegalArgumentException(
+              "All blocks to be queried must be in the same block pool: " +
+              locatedBlocks.get(0).getBlock() + " and " + lb +
+              " are from different pools.");
+        }
+      }
+      
+      long[] blockIds = new long[locatedBlocks.size()];
+      int i = 0;
       List<Token<BlockTokenIdentifier>> dnTokens = 
           new ArrayList<Token<BlockTokenIdentifier>>(
           locatedBlocks.size());
       for (LocatedBlock b : locatedBlocks) {
-        extendedBlocks.add(b.getBlock());
+        blockIds[i++] = b.getBlock().getBlockId();
         dnTokens.add(b.getBlockToken());
       }
       VolumeBlockLocationCallable callable = new VolumeBlockLocationCallable(
-          conf, datanode, extendedBlocks, dnTokens, timeout, 
+          conf, datanode, poolId, blockIds, dnTokens, timeout, 
           connectToDnViaHostname);
       callables.add(callable);
     }
@@ -102,18 +123,17 @@ class BlockStorageLocationUtil {
    * 
    * @param datanodeBlocks
    *          Map of datanodes to the blocks present on the DN
-   * @return metadatas List of block metadata for each datanode, specifying
-   *         volume locations for each block
+   * @return metadatas Map of datanodes to block metadata of the DN
    * @throws InvalidBlockTokenException
    *           if client does not have read access on a requested block
    */
-  static List<HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata(
+  static Map<DatanodeInfo, HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata(
       Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
-      int poolsize, int timeout, boolean connectToDnViaHostname)
+      int poolsize, int timeoutMs, boolean connectToDnViaHostname)
       throws InvalidBlockTokenException {
 
     List<VolumeBlockLocationCallable> callables = 
-        createVolumeBlockLocationCallables(conf, datanodeBlocks, timeout, 
+        createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs, 
             connectToDnViaHostname);
     
     // Use a thread pool to execute the Callables in parallel
@@ -121,27 +141,24 @@ class BlockStorageLocationUtil {
         new ArrayList<Future<HdfsBlocksMetadata>>();
     ExecutorService executor = new ScheduledThreadPoolExecutor(poolsize);
     try {
-      futures = executor.invokeAll(callables, timeout, TimeUnit.SECONDS);
+      futures = executor.invokeAll(callables, timeoutMs,
+          TimeUnit.MILLISECONDS);
     } catch (InterruptedException e) {
       // Swallow the exception here, because we can return partial results
     }
     executor.shutdown();
     
-    // Initialize metadatas list with nulls
-    // This is used to later indicate if we didn't get a response from a DN
-    List<HdfsBlocksMetadata> metadatas = new ArrayList<HdfsBlocksMetadata>();
-    for (int i = 0; i < futures.size(); i++) {
-      metadatas.add(null);
-    }
+    Map<DatanodeInfo, HdfsBlocksMetadata> metadatas =
+        Maps.newHashMapWithExpectedSize(datanodeBlocks.size());
     // Fill in metadatas with results from DN RPCs, where possible
     for (int i = 0; i < futures.size(); i++) {
+      VolumeBlockLocationCallable callable = callables.get(i);
+      DatanodeInfo datanode = callable.getDatanodeInfo();
       Future<HdfsBlocksMetadata> future = futures.get(i);
       try {
         HdfsBlocksMetadata metadata = future.get();
-        metadatas.set(i, metadata);
+        metadatas.put(callable.getDatanodeInfo(), metadata);
       } catch (ExecutionException e) {
-        VolumeBlockLocationCallable callable = callables.get(i);
-        DatanodeInfo datanode = callable.getDatanodeInfo();
         Throwable t = e.getCause();
         if (t instanceof InvalidBlockTokenException) {
           LOG.warn("Invalid access token when trying to retrieve "
@@ -153,8 +170,8 @@ class BlockStorageLocationUtil {
               + " required #getHdfsBlocksMetadata() API");
           throw (UnsupportedOperationException) t;
         } else {
-          LOG.info("Failed to connect to datanode " +
-              datanode.getIpcAddr(false));
+          LOG.info("Failed to query block locations on datanode " +
+              datanode.getIpcAddr(false) + ": " + t);
         }
         if (LOG.isDebugEnabled()) {
           LOG.debug("Could not fetch information from datanode", t);
@@ -175,23 +192,21 @@ class BlockStorageLocationUtil {
    * 
    * @param blocks
    *          Original LocatedBlock array
-   * @param datanodeBlocks
-   *          Mapping from datanodes to the list of replicas on each datanode
    * @param metadatas
    *          VolumeId information for the replicas on each datanode
    * @return blockVolumeIds per-replica VolumeId information associated with the
    *         parent LocatedBlock
    */
   static Map<LocatedBlock, List<VolumeId>> associateVolumeIdsWithBlocks(
-      List<LocatedBlock> blocks, Map<DatanodeInfo, 
-      List<LocatedBlock>> datanodeBlocks, List<HdfsBlocksMetadata> metadatas) {
+      List<LocatedBlock> blocks,
+      Map<DatanodeInfo, HdfsBlocksMetadata> metadatas) {
     
     // Initialize mapping of ExtendedBlock to LocatedBlock. 
     // Used to associate results from DN RPCs to the parent LocatedBlock
-    Map<ExtendedBlock, LocatedBlock> extBlockToLocBlock = 
-        new HashMap<ExtendedBlock, LocatedBlock>();
+    Map<Long, LocatedBlock> blockIdToLocBlock = 
+        new HashMap<Long, LocatedBlock>();
     for (LocatedBlock b : blocks) {
-      extBlockToLocBlock.put(b.getBlock(), b);
+      blockIdToLocBlock.put(b.getBlock().getBlockId(), b);
     }
     
     // Initialize the mapping of blocks -> list of VolumeIds, one per replica
@@ -200,9 +215,8 @@ class BlockStorageLocationUtil {
         new HashMap<LocatedBlock, List<VolumeId>>();
     for (LocatedBlock b : blocks) {
       ArrayList<VolumeId> l = new ArrayList<VolumeId>(b.getLocations().length);
-      // Start off all IDs as invalid, fill it in later with results from RPCs
       for (int i = 0; i < b.getLocations().length; i++) {
-        l.add(VolumeId.INVALID_VOLUME_ID);
+        l.add(null);
       }
       blockVolumeIds.put(b, l);
     }
@@ -210,27 +224,28 @@ class BlockStorageLocationUtil {
     // Iterate through the list of metadatas (one per datanode). 
     // For each metadata, if it's valid, insert its volume location information 
     // into the Map returned to the caller 
-    Iterator<HdfsBlocksMetadata> metadatasIter = metadatas.iterator();
-    Iterator<DatanodeInfo> datanodeIter = datanodeBlocks.keySet().iterator();
-    while (metadatasIter.hasNext()) {
-      HdfsBlocksMetadata metadata = metadatasIter.next();
-      DatanodeInfo datanode = datanodeIter.next();
+    for (Map.Entry<DatanodeInfo, HdfsBlocksMetadata> entry : metadatas.entrySet()) {
+      DatanodeInfo datanode = entry.getKey();
+      HdfsBlocksMetadata metadata = entry.getValue();
       // Check if metadata is valid
       if (metadata == null) {
         continue;
       }
-      ExtendedBlock[] metaBlocks = metadata.getBlocks();
+      long[] metaBlockIds = metadata.getBlockIds();
       List<byte[]> metaVolumeIds = metadata.getVolumeIds();
       List<Integer> metaVolumeIndexes = metadata.getVolumeIndexes();
       // Add VolumeId for each replica in the HdfsBlocksMetadata
-      for (int j = 0; j < metaBlocks.length; j++) {
+      for (int j = 0; j < metaBlockIds.length; j++) {
         int volumeIndex = metaVolumeIndexes.get(j);
-        ExtendedBlock extBlock = metaBlocks[j];
+        long blockId = metaBlockIds[j];
         // Skip if block wasn't found, or not a valid index into metaVolumeIds
         // Also skip if the DN responded with a block we didn't ask for
         if (volumeIndex == Integer.MAX_VALUE
             || volumeIndex >= metaVolumeIds.size()
-            || !extBlockToLocBlock.containsKey(extBlock)) {
+            || !blockIdToLocBlock.containsKey(blockId)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("No data for block " + blockId);
+          }
           continue;
         }
         // Get the VolumeId by indexing into the list of VolumeIds
@@ -238,7 +253,7 @@ class BlockStorageLocationUtil {
         byte[] volumeId = metaVolumeIds.get(volumeIndex);
         HdfsVolumeId id = new HdfsVolumeId(volumeId);
         // Find out which index we are in the LocatedBlock's replicas
-        LocatedBlock locBlock = extBlockToLocBlock.get(extBlock);
+        LocatedBlock locBlock = blockIdToLocBlock.get(blockId);
         DatanodeInfo[] dnInfos = locBlock.getLocations();
         int index = -1;
         for (int k = 0; k < dnInfos.length; k++) {
@@ -292,21 +307,23 @@ class BlockStorageLocationUtil {
   private static class VolumeBlockLocationCallable implements 
     Callable<HdfsBlocksMetadata> {
     
-    private Configuration configuration;
-    private int timeout;
-    private DatanodeInfo datanode;
-    private List<ExtendedBlock> extendedBlocks;
-    private List<Token<BlockTokenIdentifier>> dnTokens;
-    private boolean connectToDnViaHostname;
+    private final Configuration configuration;
+    private final int timeout;
+    private final DatanodeInfo datanode;
+    private final String poolId;
+    private final long[] blockIds;
+    private final List<Token<BlockTokenIdentifier>> dnTokens;
+    private final boolean connectToDnViaHostname;
     
     VolumeBlockLocationCallable(Configuration configuration,
-        DatanodeInfo datanode, List<ExtendedBlock> extendedBlocks,
+        DatanodeInfo datanode, String poolId, long []blockIds,
         List<Token<BlockTokenIdentifier>> dnTokens, int timeout, 
         boolean connectToDnViaHostname) {
       this.configuration = configuration;
       this.timeout = timeout;
       this.datanode = datanode;
-      this.extendedBlocks = extendedBlocks;
+      this.poolId = poolId;
+      this.blockIds = blockIds;
       this.dnTokens = dnTokens;
       this.connectToDnViaHostname = connectToDnViaHostname;
     }
@@ -323,7 +340,7 @@ class BlockStorageLocationUtil {
       try {
         cdp = DFSUtil.createClientDatanodeProtocolProxy(datanode, configuration,
             timeout, connectToDnViaHostname);
-        metadata = cdp.getHdfsBlocksMetadata(extendedBlocks, dnTokens);
+        metadata = cdp.getHdfsBlocksMetadata(poolId, blockIds, dnTokens);
       } catch (IOException e) {
         // Bubble this up to the caller, handle with the Future
         throw e;

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

@@ -241,6 +241,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
    */
   public static class Conf {
     final int hdfsTimeout;    // timeout value for a DFS operation.
+
     final int maxFailoverAttempts;
     final int maxRetryAttempts;
     final int failoverSleepBaseMillis;
@@ -267,7 +268,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     final boolean connectToDnViaHostname;
     final boolean getHdfsBlocksMetadataEnabled;
     final int getFileBlockStorageLocationsNumThreads;
-    final int getFileBlockStorageLocationsTimeout;
+    final int getFileBlockStorageLocationsTimeoutMs;
     final int retryTimesForGetLastBlockLength;
     final int retryIntervalForGetLastBlockLength;
     final long datanodeRestartTimeout;
@@ -290,7 +291,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     public Conf(Configuration conf) {
       // The hdfsTimeout is currently the same as the ipc timeout 
       hdfsTimeout = Client.getTimeout(conf);
-
       maxFailoverAttempts = conf.getInt(
           DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
           DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
@@ -349,9 +349,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
       getFileBlockStorageLocationsNumThreads = conf.getInt(
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS,
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS_DEFAULT);
-      getFileBlockStorageLocationsTimeout = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT,
-          DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_DEFAULT);
+      getFileBlockStorageLocationsTimeoutMs = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS,
+          DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT);
       retryTimesForGetLastBlockLength = conf.getInt(
           DFSConfigKeys.DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH,
           DFSConfigKeys.DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
@@ -1209,16 +1209,20 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     }
         
     // Make RPCs to the datanodes to get volume locations for its replicas
-    List<HdfsBlocksMetadata> metadatas = BlockStorageLocationUtil
+    Map<DatanodeInfo, HdfsBlocksMetadata> metadatas = BlockStorageLocationUtil
         .queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
             getConf().getFileBlockStorageLocationsNumThreads,
-            getConf().getFileBlockStorageLocationsTimeout,
+            getConf().getFileBlockStorageLocationsTimeoutMs,
             getConf().connectToDnViaHostname);
     
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("metadata returned: " + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
+    }
+    
     // Regroup the returned VolumeId metadata to again be grouped by
     // LocatedBlock rather than by datanode
     Map<LocatedBlock, List<VolumeId>> blockVolumeIds = BlockStorageLocationUtil
-        .associateVolumeIdsWithBlocks(blocks, datanodeBlocks, metadatas);
+        .associateVolumeIdsWithBlocks(blocks, metadatas);
     
     // Combine original BlockLocations with new VolumeId information
     BlockStorageLocation[] volumeBlockLocations = BlockStorageLocationUtil

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

@@ -66,8 +66,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
   public static final String  DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS = "dfs.client.file-block-storage-locations.num-threads";
   public static final int     DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS_DEFAULT = 10;
-  public static final String  DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT = "dfs.client.file-block-storage-locations.timeout";
-  public static final int     DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_DEFAULT = 60;
+  public static final String  DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS = "dfs.client.file-block-storage-locations.timeout.millis";
+  public static final int     DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT = 1000;
   public static final String  DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH = "dfs.client.retry.times.get-last-block-length";
   public static final int     DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT = 3;
   public static final String  DFS_CLIENT_RETRY_INTERVAL_GET_LAST_BLOCK_LENGTH = "dfs.client.retry.interval-ms.get-last-block-length";

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

@@ -234,9 +234,9 @@ public class DistributedFileSystem extends FileSystem {
    * The returned array of {@link BlockStorageLocation} augments
    * {@link BlockLocation} with a {@link VolumeId} per block replica. The
    * VolumeId specifies the volume on the datanode on which the replica resides.
-   * The VolumeId has to be checked via {@link VolumeId#isValid()} before being
-   * used because volume information can be unavailable if the corresponding
-   * datanode is down or if the requested block is not found.
+   * The VolumeId associated with a replica may be null because volume
+   * information can be unavailable if the corresponding datanode is down or
+   * if the requested block is not found.
    * 
    * This API is unstable, and datanode-side support is disabled by default. It
    * can be enabled by setting "dfs.datanode.hdfs-blocks-metadata.enabled" to

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

@@ -137,7 +137,9 @@ public class HdfsConfiguration extends Configuration {
       new DeprecationDelta("dfs.federation.nameservices",
         DFSConfigKeys.DFS_NAMESERVICES),
       new DeprecationDelta("dfs.federation.nameservice.id",
-        DFSConfigKeys.DFS_NAMESERVICE_ID)
+        DFSConfigKeys.DFS_NAMESERVICE_ID),
+      new DeprecationDelta("dfs.client.file-block-storage-locations.timeout",
+        DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS)
     });
   }
 

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -113,7 +113,8 @@ public interface ClientDatanodeProtocol {
    * This is in the form of an opaque {@link VolumeId} for each configured
    * data directory, which is not guaranteed to be the same across DN restarts.
    * 
-   * @param blocks
+   * @param blockPoolId the pool to query
+   * @param blockIds
    *          list of blocks on the local datanode
    * @param tokens
    *          block access tokens corresponding to the requested blocks
@@ -122,8 +123,8 @@ public interface ClientDatanodeProtocol {
    * @throws IOException
    *           if datanode is unreachable, or replica is not found on datanode
    */
-  HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks,
-      List<Token<BlockTokenIdentifier>> tokens) throws IOException; 
+  HdfsBlocksMetadata getHdfsBlocksMetadata(String blockPoolId,
+      long []blockIds, List<Token<BlockTokenIdentifier>> tokens) throws IOException; 
 
   /**
    * Shuts down a datanode.

+ 23 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsBlocksMetadata.java

@@ -22,6 +22,10 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Longs;
+
 /**
  * Augments an array of blocks on a datanode with additional information about
  * where the block is stored.
@@ -30,10 +34,13 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 public class HdfsBlocksMetadata {
   
+  /** The block pool that was queried */
+  private final String blockPoolId;
+  
   /**
    * List of blocks
    */
-  private final ExtendedBlock[] blocks;
+  private final long[] blockIds;
   
   /**
    * List of volumes
@@ -50,7 +57,7 @@ public class HdfsBlocksMetadata {
   /**
    * Constructs HdfsBlocksMetadata.
    * 
-   * @param blocks
+   * @param blockIds
    *          List of blocks described
    * @param volumeIds
    *          List of potential volume identifiers, specifying volumes where 
@@ -58,9 +65,13 @@ public class HdfsBlocksMetadata {
    * @param volumeIndexes
    *          Indexes into the list of volume identifiers, one per block
    */
-  public HdfsBlocksMetadata(ExtendedBlock[] blocks, List<byte[]> volumeIds, 
+  public HdfsBlocksMetadata(String blockPoolId,
+      long[] blockIds, List<byte[]> volumeIds, 
       List<Integer> volumeIndexes) {
-    this.blocks = blocks;
+    Preconditions.checkArgument(blockIds.length == volumeIndexes.size(),
+        "Argument lengths should match");
+    this.blockPoolId = blockPoolId;
+    this.blockIds = blockIds;
     this.volumeIds = volumeIds;
     this.volumeIndexes = volumeIndexes;
   }
@@ -70,8 +81,8 @@ public class HdfsBlocksMetadata {
    * 
    * @return array of blocks
    */
-  public ExtendedBlock[] getBlocks() {
-    return blocks;
+  public long[] getBlockIds() {
+    return blockIds;
   }
   
   /**
@@ -91,4 +102,10 @@ public class HdfsBlocksMetadata {
   public List<Integer> getVolumeIndexes() {
     return volumeIndexes;
   }
+
+  @Override
+  public String toString() {
+    return "Metadata for " + blockIds.length + " blocks in " +
+        blockPoolId + ": " + Joiner.on(",").join(Longs.asList(blockIds));
+  }
 }

+ 6 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.VersionInfo;
 
+import com.google.common.primitives.Longs;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -132,19 +133,17 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     HdfsBlocksMetadata resp;
     try {
-      // Construct the Lists to make the actual call
-      List<ExtendedBlock> blocks = 
-          new ArrayList<ExtendedBlock>(request.getBlocksCount());
-      for (ExtendedBlockProto b : request.getBlocksList()) {
-        blocks.add(PBHelper.convert(b));
-      }
+      String poolId = request.getBlockPoolId();
+
       List<Token<BlockTokenIdentifier>> tokens = 
           new ArrayList<Token<BlockTokenIdentifier>>(request.getTokensCount());
       for (TokenProto b : request.getTokensList()) {
         tokens.add(PBHelper.convert(b));
       }
+      long[] blockIds = Longs.toArray(request.getBlockIdsList());
+      
       // Call the real implementation
-      resp = impl.getHdfsBlocksMetadata(blocks, tokens);
+      resp = impl.getHdfsBlocksMetadata(poolId, blockIds, tokens);
     } catch (IOException e) {
       throw new ServiceException(e);
     }

+ 7 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import javax.net.SocketFactory;
@@ -61,6 +62,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 
+import com.google.common.primitives.Longs;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -223,23 +225,19 @@ public class ClientDatanodeProtocolTranslatorPB implements
   }
 
   @Override
-  public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks,
+  public HdfsBlocksMetadata getHdfsBlocksMetadata(String blockPoolId,
+      long[] blockIds,
       List<Token<BlockTokenIdentifier>> tokens) throws IOException {
-    // Convert to proto objects
-    List<ExtendedBlockProto> blocksProtos = 
-        new ArrayList<ExtendedBlockProto>(blocks.size());
     List<TokenProto> tokensProtos = 
         new ArrayList<TokenProto>(tokens.size());
-    for (ExtendedBlock b : blocks) {
-      blocksProtos.add(PBHelper.convert(b));
-    }
     for (Token<BlockTokenIdentifier> t : tokens) {
       tokensProtos.add(PBHelper.convert(t));
     }
     // Build the request
     GetHdfsBlockLocationsRequestProto request = 
         GetHdfsBlockLocationsRequestProto.newBuilder()
-        .addAllBlocks(blocksProtos)
+        .setBlockPoolId(blockPoolId)
+        .addAllBlockIds(Longs.asList(blockIds))
         .addAllTokens(tokensProtos)
         .build();
     // Send the RPC
@@ -258,7 +256,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
     // Array of indexes into the list of volumes, one per block
     List<Integer> volumeIndexes = response.getVolumeIndexesList();
     // Parsed HdfsVolumeId values, one per block
-    return new HdfsBlocksMetadata(blocks.toArray(new ExtendedBlock[] {}), 
+    return new HdfsBlocksMetadata(blockPoolId, blockIds,
         volumeIds, volumeIndexes);
   }
 

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

@@ -1136,22 +1136,23 @@ public class DataNode extends Configured
   }
 
   @Override
-  public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks,
+  public HdfsBlocksMetadata getHdfsBlocksMetadata(
+      String bpId, long[] blockIds,
       List<Token<BlockTokenIdentifier>> tokens) throws IOException, 
       UnsupportedOperationException {
     if (!getHdfsBlockLocationsEnabled) {
       throw new UnsupportedOperationException("Datanode#getHdfsBlocksMetadata "
           + " is not enabled in datanode config");
     }
-    if (blocks.size() != tokens.size()) {
+    if (blockIds.length != tokens.size()) {
       throw new IOException("Differing number of blocks and tokens");
     }
     // Check access for each block
-    for (int i = 0; i < blocks.size(); i++) {
-      checkBlockToken(blocks.get(i), tokens.get(i), 
-          BlockTokenSecretManager.AccessMode.READ);
+    for (int i = 0; i < blockIds.length; i++) {
+      checkBlockToken(new ExtendedBlock(bpId, blockIds[i]),
+          tokens.get(i), BlockTokenSecretManager.AccessMode.READ);
     }
-    return data.getHdfsBlocksMetadata(blocks);
+    return data.getHdfsBlocksMetadata(bpId, blockIds);
   }
   
   private void checkBlockToken(ExtendedBlock block, Token<BlockTokenIdentifier> token,

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

@@ -405,12 +405,13 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * Get a {@link HdfsBlocksMetadata} corresponding to the list of blocks in 
    * <code>blocks</code>.
    * 
-   * @param blocks List of blocks for which to return metadata
+   * @param bpid pool to query
+   * @param blockIds List of block ids for which to return metadata
    * @return metadata Metadata for the list of blocks
    * @throws IOException
    */
-  public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
-      throws IOException;
+  public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
+      long[] blockIds) throws IOException;
 
   /**
    * Enable 'trash' for the given dataset. When trash is enabled, files are

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

@@ -1822,31 +1822,35 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
 
   @Override // FsDatasetSpi
-  public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
-      throws IOException {
+  public HdfsBlocksMetadata getHdfsBlocksMetadata(String poolId,
+      long[] blockIds) throws IOException {
     // List of VolumeIds, one per volume on the datanode
     List<byte[]> blocksVolumeIds = new ArrayList<byte[]>(volumes.volumes.size());
     // List of indexes into the list of VolumeIds, pointing at the VolumeId of
     // the volume that the block is on
-    List<Integer> blocksVolumeIndexes = new ArrayList<Integer>(blocks.size());
+    List<Integer> blocksVolumeIndexes = new ArrayList<Integer>(blockIds.length);
     // Initialize the list of VolumeIds simply by enumerating the volumes
     for (int i = 0; i < volumes.volumes.size(); i++) {
       blocksVolumeIds.add(ByteBuffer.allocate(4).putInt(i).array());
     }
     // Determine the index of the VolumeId of each block's volume, by comparing 
     // the block's volume against the enumerated volumes
-    for (int i = 0; i < blocks.size(); i++) {
-      ExtendedBlock block = blocks.get(i);
-      FsVolumeSpi blockVolume = getReplicaInfo(block).getVolume();
+    for (int i = 0; i < blockIds.length; i++) {
+      long blockId = blockIds[i];
       boolean isValid = false;
+
+      ReplicaInfo info = volumeMap.get(poolId, blockId);
       int volumeIndex = 0;
-      for (FsVolumeImpl volume : volumes.volumes) {
-        // This comparison of references should be safe
-        if (blockVolume == volume) {
-          isValid = true;
-          break;
+      if (info != null) {
+        FsVolumeSpi blockVolume = info.getVolume();
+        for (FsVolumeImpl volume : volumes.volumes) {
+          // This comparison of references should be safe
+          if (blockVolume == volume) {
+            isValid = true;
+            break;
+          }
+          volumeIndex++;
         }
-        volumeIndex++;
       }
       // Indicates that the block is not present, or not found in a data dir
       if (!isValid) {
@@ -1854,7 +1858,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       }
       blocksVolumeIndexes.add(volumeIndex);
     }
-    return new HdfsBlocksMetadata(blocks.toArray(new ExtendedBlock[] {}), 
+    return new HdfsBlocksMetadata(poolId, blockIds,
         blocksVolumeIds, blocksVolumeIndexes);
   }
 

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

@@ -102,12 +102,18 @@ message GetBlockLocalPathInfoResponseProto {
 }
 
 /**
- * blocks - list of ExtendedBlocks on which we are querying additional info
- * tokens - list of access tokens corresponding to list of ExtendedBlocks
+ * Query for the disk locations of a number of blocks on this DN.
+ * blockPoolId - the pool to query
+ * blockIds - list of block IDs to query
+ * tokens - list of access tokens corresponding to list of block IDs
  */
 message GetHdfsBlockLocationsRequestProto {
-  repeated ExtendedBlockProto blocks = 1;
+  // Removed: HDFS-3969
+  // repeated ExtendedBlockProto blocks = 1;
   repeated hadoop.common.TokenProto tokens = 2;
+
+  required string blockPoolId = 3;
+  repeated sfixed64 blockIds = 4 [ packed = true ];
 }
 
 /**
@@ -118,7 +124,7 @@ message GetHdfsBlockLocationsRequestProto {
  */
 message GetHdfsBlockLocationsResponseProto {
   repeated bytes volumeIds = 1;
-  repeated uint32 volumeIndexes = 2;
+  repeated uint32 volumeIndexes = 2 [ packed = true ];
 }
 
 /**

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

@@ -1321,10 +1321,10 @@
 </property>
 
 <property>
-  <name>dfs.client.file-block-storage-locations.timeout</name>
-  <value>60</value>
+  <name>dfs.client.file-block-storage-locations.timeout.millis</name>
+  <value>1000</value>
   <description>
-    Timeout (in seconds) for the parallel RPCs made in DistributedFileSystem#getFileBlockStorageLocations().
+    Timeout (in milliseconds) for the parallel RPCs made in DistributedFileSystem#getFileBlockStorageLocations().
   </description>
 </property>
 

+ 1 - 38
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestVolumeId.java

@@ -123,56 +123,19 @@ public class TestVolumeId {
   @Test
   public void testIdEmptyBytes() {
     final VolumeId idEmpty1   = new HdfsVolumeId(new byte[0]);
-    assertTrue(idEmpty1.isValid());
     final VolumeId idEmpty2   = new HdfsVolumeId(new byte[0]);
-    assertTrue(idEmpty2.isValid());
     final VolumeId idNotEmpty = new HdfsVolumeId(new byte[] { (byte)1 });
-    assertTrue(idNotEmpty.isValid());
     
     testEq(true, idEmpty1, idEmpty2);
     testEq(false, idEmpty1, idNotEmpty);
     testEq(false, idEmpty2, idNotEmpty);
   }
-  
-  /*
-   * Test the VolumeId.INVALID_VOLUME_ID singleton.
-   */
-  @Test
-  public void testInvalidId() {
-    try {
-      new HdfsVolumeId(null);
-      assertTrue("NPE expected.", false);
-    } catch (NullPointerException npe) {
-      // okay
-    }
-    final VolumeId idEmpty   = new HdfsVolumeId(new byte[] {});
-    final VolumeId idNotEmpty = new HdfsVolumeId(new byte[] { (byte)1 });
-    
-    testEq(false, VolumeId.INVALID_VOLUME_ID, idNotEmpty);
-    testEq(false, VolumeId.INVALID_VOLUME_ID, idEmpty);
-    
-    testEqMany(true, 
-        new VolumeId[] { 
-          VolumeId.INVALID_VOLUME_ID, 
-          VolumeId.INVALID_VOLUME_ID, 
-          VolumeId.INVALID_VOLUME_ID } );
-    testEqMany(false, 
-        new VolumeId[] {
-          VolumeId.INVALID_VOLUME_ID, 
-          idEmpty, 
-          idNotEmpty });
-  }
-  
+
   /*
    * test #toString() for typical VolumeId equality classes
    */
   @Test
   public void testToString() {
-    // The #toString() return value is only checked for != null.
-    // We cannot assert more.
-    String strInvalid = VolumeId.INVALID_VOLUME_ID.toString();
-    assertNotNull(strInvalid);
-    
     String strEmpty = new HdfsVolumeId(new byte[] {}).toString();
     assertNotNull(strEmpty);
     

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

@@ -399,13 +399,15 @@ public class MiniDFSCluster {
     Configuration conf;
     String[] dnArgs;
     SecureResources secureResources;
+    int ipcPort;
 
     DataNodeProperties(DataNode node, Configuration conf, String[] args,
-                       SecureResources secureResources) {
+                       SecureResources secureResources, int ipcPort) {
       this.datanode = node;
       this.conf = conf;
       this.dnArgs = args;
       this.secureResources = secureResources;
+      this.ipcPort = ipcPort;
     }
 
     public void setDnArgs(String ... args) {
@@ -1301,7 +1303,8 @@ public class MiniDFSCluster {
                                   racks[i-curDatanodesNum]);
       }
       dn.runDatanodeDaemon();
-      dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs, secureResources));
+      dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs,
+          secureResources, dn.getIpcPort()));
     }
     curDatanodesNum += numDataNodes;
     this.numDataNodes += numDataNodes;
@@ -1758,10 +1761,12 @@ public class MiniDFSCluster {
       InetSocketAddress addr = dnprop.datanode.getXferAddress();
       conf.set(DFS_DATANODE_ADDRESS_KEY, 
           addr.getAddress().getHostAddress() + ":" + addr.getPort());
+      conf.set(DFS_DATANODE_IPC_ADDRESS_KEY,
+          addr.getAddress().getHostAddress() + ":" + dnprop.ipcPort); 
     }
+    DataNode newDn = DataNode.createDataNode(args, conf, secureResources);
     dataNodes.add(new DataNodeProperties(
-        DataNode.createDataNode(args, conf, secureResources),
-        newconf, args, secureResources));
+        newDn, newconf, args, secureResources, newDn.getIpcPort()));
     numDataNodes++;
     return true;
   }

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

@@ -178,7 +178,7 @@ public class MiniDFSClusterWithNodeGroup extends MiniDFSCluster {
         }
       }
       dn.runDatanodeDaemon();
-      dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs, secureResources));
+      dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs, secureResources, dn.getIpcPort()));
     }
     curDatanodesNum += numDataNodes;
     this.numDataNodes += numDataNodes;

+ 120 - 24
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.eq;
@@ -43,6 +45,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -56,15 +59,22 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.junit.Test;
 import org.mockito.InOrder;
 
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+
 public class TestDistributedFileSystem {
   private static final Random RAN = new Random();
 
@@ -684,20 +694,47 @@ public class TestDistributedFileSystem {
    * {@link DistributedFileSystem#getFileBlockStorageLocations(java.util.List)}
    * call
    */
-  @Test
+  @Test(timeout=60000)
   public void testGetFileBlockStorageLocationsBatching() throws Exception {
     final Configuration conf = getTestConfiguration();
+    ((Log4JLogger)ProtobufRpcEngine.LOG).getLogger().setLevel(Level.TRACE);
+    ((Log4JLogger)BlockStorageLocationUtil.LOG).getLogger().setLevel(Level.TRACE);
+    ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.TRACE);
+
     conf.setBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED,
         true);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(2).build();
     try {
-      DistributedFileSystem fs = cluster.getFileSystem();
+      final DistributedFileSystem fs = cluster.getFileSystem();
       // Create two files
-      Path tmpFile1 = new Path("/tmpfile1.dat");
-      Path tmpFile2 = new Path("/tmpfile2.dat");
+      final Path tmpFile1 = new Path("/tmpfile1.dat");
+      final Path tmpFile2 = new Path("/tmpfile2.dat");
       DFSTestUtil.createFile(fs, tmpFile1, 1024, (short) 2, 0xDEADDEADl);
       DFSTestUtil.createFile(fs, tmpFile2, 1024, (short) 2, 0xDEADDEADl);
+      // Make sure files are fully replicated before continuing
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            List<BlockLocation> list = Lists.newArrayList();
+            list.addAll(Arrays.asList(fs.getFileBlockLocations(tmpFile1, 0,
+                1024)));
+            list.addAll(Arrays.asList(fs.getFileBlockLocations(tmpFile2, 0,
+                1024)));
+            int totalRepl = 0;
+            for (BlockLocation loc : list) {
+              totalRepl += loc.getHosts().length;
+            }
+            if (totalRepl == 4) {
+              return true;
+            }
+          } catch(IOException e) {
+            // swallow
+          }
+          return false;
+        }
+      }, 500, 30000);
       // Get locations of blocks of both files and concat together
       BlockLocation[] blockLocs1 = fs.getFileBlockLocations(tmpFile1, 0, 1024);
       BlockLocation[] blockLocs2 = fs.getFileBlockLocations(tmpFile2, 0, 1024);
@@ -728,7 +765,7 @@ public class TestDistributedFileSystem {
           VolumeId id = l.getVolumeIds()[i];
           String name = l.getNames()[i];
           assertTrue("Expected block to be valid on datanode " + name,
-              id.isValid());
+              id != null);
         }
       }
     } finally {
@@ -740,38 +777,97 @@ public class TestDistributedFileSystem {
    * Tests error paths for
    * {@link DistributedFileSystem#getFileBlockStorageLocations(java.util.List)}
    */
-  @Test
+  @Test(timeout=60000)
   public void testGetFileBlockStorageLocationsError() throws Exception {
     final Configuration conf = getTestConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED,
         true);
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(2).build();
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
+    MiniDFSCluster cluster = null;
     try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       cluster.getDataNodes();
-      DistributedFileSystem fs = cluster.getFileSystem();
-      // Create a file
-      Path tmpFile = new Path("/tmpfile1.dat");
-      DFSTestUtil.createFile(fs, tmpFile, 1024, (short) 2, 0xDEADDEADl);
-      // Get locations of blocks of the file
-      BlockLocation[] blockLocs = fs.getFileBlockLocations(tmpFile, 0, 1024);
-      // Stop a datanode to simulate a failure
-      cluster.stopDataNode(0);
-      // Fetch VolumeBlockLocations
-      BlockStorageLocation[] locs = fs.getFileBlockStorageLocations(Arrays
-          .asList(blockLocs));
+      final DistributedFileSystem fs = cluster.getFileSystem();
+      
+      // Create a few files and add together their block locations into
+      // a list.
+      final Path tmpFile1 = new Path("/errorfile1.dat");
+      final Path tmpFile2 = new Path("/errorfile2.dat");
 
-      assertEquals("Expected one HdfsBlockLocation for one 1-block file", 1,
-          locs.length);
+      DFSTestUtil.createFile(fs, tmpFile1, 1024, (short) 2, 0xDEADDEADl);
+      DFSTestUtil.createFile(fs, tmpFile2, 1024, (short) 2, 0xDEADDEADl);
+
+      // Make sure files are fully replicated before continuing
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            List<BlockLocation> list = Lists.newArrayList();
+            list.addAll(Arrays.asList(fs.getFileBlockLocations(tmpFile1, 0,
+                1024)));
+            list.addAll(Arrays.asList(fs.getFileBlockLocations(tmpFile2, 0,
+                1024)));
+            int totalRepl = 0;
+            for (BlockLocation loc : list) {
+              totalRepl += loc.getHosts().length;
+            }
+            if (totalRepl == 4) {
+              return true;
+            }
+          } catch(IOException e) {
+            // swallow
+          }
+          return false;
+        }
+      }, 500, 30000);
+      
+      BlockLocation[] blockLocs1 = fs.getFileBlockLocations(tmpFile1, 0, 1024);
+      BlockLocation[] blockLocs2 = fs.getFileBlockLocations(tmpFile2, 0, 1024);
 
+      List<BlockLocation> allLocs = Lists.newArrayList();
+      allLocs.addAll(Arrays.asList(blockLocs1));
+      allLocs.addAll(Arrays.asList(blockLocs2));
+          
+      // Stop a datanode to simulate a failure.
+      DataNodeProperties stoppedNode = cluster.stopDataNode(0);
+      
+      // Fetch VolumeBlockLocations
+      BlockStorageLocation[] locs = fs.getFileBlockStorageLocations(allLocs);
+      assertEquals("Expected two HdfsBlockLocation for two 1-block files", 2,
+          locs.length);
+  
       for (BlockStorageLocation l : locs) {
         assertEquals("Expected two replicas for each block", 2,
+            l.getHosts().length);
+        assertEquals("Expected two VolumeIDs for each block", 2,
             l.getVolumeIds().length);
-        assertTrue("Expected one valid and one invalid replica",
-            (l.getVolumeIds()[0].isValid()) ^ (l.getVolumeIds()[1].isValid()));
+        assertTrue("Expected one valid and one invalid volume",
+            (l.getVolumeIds()[0] == null) ^ (l.getVolumeIds()[1] == null));
       }
+      
+      // Start the datanode again, and remove one of the blocks.
+      // This is a different type of failure where the block itself
+      // is invalid.
+      cluster.restartDataNode(stoppedNode, true /*keepPort*/);
+      cluster.waitActive();
+      
+      fs.delete(tmpFile2, true);
+      HATestUtil.waitForNNToIssueDeletions(cluster.getNameNode());
+      cluster.triggerHeartbeats();
+      HATestUtil.waitForDNDeletions(cluster);
+  
+      locs = fs.getFileBlockStorageLocations(allLocs);
+      assertEquals("Expected two HdfsBlockLocations for two 1-block files", 2,
+          locs.length);
+      assertNotNull(locs[0].getVolumeIds()[0]);
+      assertNotNull(locs[0].getVolumeIds()[1]);
+      assertNull(locs[1].getVolumeIds()[0]);
+      assertNull(locs[1].getVolumeIds()[1]);
     } finally {
-      cluster.shutdown();
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 

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

@@ -1049,7 +1049,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
+  public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid, long[] blockIds)
       throws IOException {
     throw new UnsupportedOperationException();
   }

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

@@ -111,7 +111,7 @@ public abstract class HATestUtil {
    * Wait for the NameNode to issue any deletions that are already
    * pending (i.e. for the pendingDeletionBlocksCount to go to 0)
    */
-  static void waitForNNToIssueDeletions(final NameNode nn)
+  public static void waitForNNToIssueDeletions(final NameNode nn)
       throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override

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

@@ -176,6 +176,12 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5761. Added a simple log message to denote when encrypted shuffle
     is on in the shuffle-handler. (Jian He via vinodkv) 
 
+    MAPREDUCE-5754. Preserve Job diagnostics in history (Gera Shegalov via
+    jlowe)
+
+    MAPREDUCE-5766. Moved ping messages from TaskAttempts to be at DEBUG level
+    inside the ApplicationMaster log. (Jian He via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -360,12 +360,13 @@ public class TaskAttemptListenerImpl extends CompositeService
 
     if (taskStatus == null) {
       //We are using statusUpdate only as a simple ping
-      LOG.info("Ping from " + taskAttemptID.toString());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Ping from " + taskAttemptID.toString());
+      }
       return feedback;
     }
 
     // if we are here there is an actual status update to be processed
-    LOG.info("Status update from " + taskAttemptID.toString());
 
     taskHeartbeatHandler.progressing(yarnAttemptID);
     TaskAttemptStatus taskAttemptStatus =
@@ -453,7 +454,7 @@ public class TaskAttemptListenerImpl extends CompositeService
 
     JVMId jvmId = context.jvmId;
     LOG.info("JVM with ID : " + jvmId + " asked for a task");
-    
+
     JvmTask jvmTask = null;
     // TODO: Is it an authorized container to get a task? Otherwise return null.
 

+ 5 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.mapreduce.TypeConverter;
 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.job.Job;
 import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
@@ -343,11 +344,12 @@ public class JobHistoryEventHandler extends AbstractService
           LOG.warn("Found jobId " + toClose
             + " to have not been closed. Will close");
           //Create a JobFinishEvent so that it is written to the job history
+          final Job job = context.getJob(toClose);
           JobUnsuccessfulCompletionEvent jucEvent =
             new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(toClose),
-              System.currentTimeMillis(), context.getJob(toClose)
-              .getCompletedMaps(), context.getJob(toClose).getCompletedReduces(),
-              JobState.KILLED.toString());
+                System.currentTimeMillis(), job.getCompletedMaps(),
+                job.getCompletedReduces(), JobState.KILLED.toString(),
+                job.getDiagnostics());
           JobHistoryEvent jfEvent = new JobHistoryEvent(toClose, jucEvent);
           //Bypass the queue mechanism which might wait. Call the method directly
           handleEvent(jfEvent);

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

@@ -149,6 +149,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
 
   // Maximum no. of fetch-failure notifications after which map task is failed
   private static final int MAX_FETCH_FAILURES_NOTIFICATIONS = 3;
+
+  public static final String JOB_KILLED_DIAG =
+      "Job received Kill while in RUNNING state.";
   
   //final fields
   private final ApplicationAttemptId applicationAttemptId;
@@ -1617,7 +1620,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
               finishTime,
               succeededMapTaskCount,
               succeededReduceTaskCount,
-              finalState.toString());
+              finalState.toString(),
+              diagnostics);
       eventHandler.handle(new JobHistoryEvent(jobId,
           unsuccessfulJobEvent));
       finished(finalState);
@@ -1730,7 +1734,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       JobUnsuccessfulCompletionEvent failedEvent =
           new JobUnsuccessfulCompletionEvent(job.oldJobId,
               job.finishTime, 0, 0,
-              JobStateInternal.KILLED.toString());
+              JobStateInternal.KILLED.toString(), job.diagnostics);
       job.eventHandler.handle(new JobHistoryEvent(job.jobId, failedEvent));
       job.finished(JobStateInternal.KILLED);
     }
@@ -1763,7 +1767,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       implements SingleArcTransition<JobImpl, JobEvent> {
     @Override
     public void transition(JobImpl job, JobEvent event) {
-      job.addDiagnostic("Job received Kill while in RUNNING state.");
+      job.addDiagnostic(JOB_KILLED_DIAG);
       for (Task task : job.tasks.values()) {
         job.eventHandler.handle(
             new TaskEvent(task.getID(), TaskEventType.T_KILL));
@@ -2127,7 +2131,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       JobUnsuccessfulCompletionEvent failedEvent =
           new JobUnsuccessfulCompletionEvent(job.oldJobId,
               job.finishTime, 0, 0,
-              jobHistoryString);
+              jobHistoryString, job.diagnostics);
       job.eventHandler.handle(new JobHistoryEvent(job.jobId, failedEvent));
       job.finished(terminationState);
     }

+ 7 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.junit.Test;
 
 public class TestEvents {
@@ -334,11 +335,12 @@ public class TestEvents {
   private FakeEvent getJobKilledEvent() {
     FakeEvent result = new FakeEvent(EventType.JOB_KILLED);
     JobUnsuccessfulCompletion datum = new JobUnsuccessfulCompletion();
-    datum.finishedMaps = 1;
-    datum.finishedReduces = 2;
-    datum.finishTime = 3;
-    datum.jobid = "ID";
-    datum.jobStatus = "STATUS";
+    datum.setFinishedMaps(1);
+    datum.setFinishedReduces(2);
+    datum.setFinishTime(3L);
+    datum.setJobid("ID");
+    datum.setJobStatus("STATUS");
+    datum.setDiagnostics(JobImpl.JOB_KILLED_DIAG);
     result.setDatum(datum);
     return result;
   }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr

@@ -135,7 +135,8 @@
           {"name": "finishTime", "type": "long"},
           {"name": "finishedMaps", "type": "int"},
           {"name": "finishedReduces", "type": "int"},
-          {"name": "jobStatus", "type": "string"}
+          {"name": "jobStatus", "type": "string"},
+          {"name": "diagnostics", "type": "string"}
       ]
      },
 

+ 1 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java

@@ -353,10 +353,6 @@ public class JobHistoryParser implements HistoryEventHandler {
     taskInfo.error = StringInterner.weakIntern(event.getError());
     taskInfo.failedDueToAttemptId = event.getFailedAttemptID();
     taskInfo.counters = event.getCounters();
-    if (info.errorInfo.isEmpty()) {
-      info.errorInfo = "Task " + taskInfo.taskId + " failed " +
-          taskInfo.attemptsMap.size() + " times ";
-    }
   }
 
   private void handleTaskStartedEvent(TaskStartedEvent event) {
@@ -373,6 +369,7 @@ public class JobHistoryParser implements HistoryEventHandler {
     info.finishedMaps = event.getFinishedMaps();
     info.finishedReduces = event.getFinishedReduces();
     info.jobStatus = StringInterner.weakIntern(event.getStatus());
+    info.errorInfo = StringInterner.weakIntern(event.getDiagnostics());
   }
 
   private void handleJobFinishedEvent(JobFinishedEvent event) {

+ 48 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java

@@ -18,11 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import com.google.common.base.Joiner;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
 
+import java.util.Collections;
+
 /**
  * Event to record Failed and Killed completion of jobs
  *
@@ -30,6 +34,10 @@ import org.apache.hadoop.mapreduce.JobID;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
+  private static final String NODIAGS = "";
+  private static final Iterable<String> NODIAGS_LIST =
+      Collections.singletonList(NODIAGS);
+
   private JobUnsuccessfulCompletion datum
     = new JobUnsuccessfulCompletion();
 
@@ -44,11 +52,33 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
   public JobUnsuccessfulCompletionEvent(JobID id, long finishTime,
       int finishedMaps,
       int finishedReduces, String status) {
-    datum.jobid = new Utf8(id.toString());
-    datum.finishTime = finishTime;
-    datum.finishedMaps = finishedMaps;
-    datum.finishedReduces = finishedReduces;
-    datum.jobStatus = new Utf8(status);
+    this(id, finishTime, finishedMaps, finishedReduces, status, NODIAGS_LIST);
+  }
+
+  /**
+   * Create an event to record unsuccessful completion (killed/failed) of jobs
+   * @param id Job ID
+   * @param finishTime Finish time of the job
+   * @param finishedMaps Number of finished maps
+   * @param finishedReduces Number of finished reduces
+   * @param status Status of the job
+   * @param diagnostics job runtime diagnostics
+   */
+  public JobUnsuccessfulCompletionEvent(JobID id, long finishTime,
+      int finishedMaps,
+      int finishedReduces,
+      String status,
+      Iterable<String> diagnostics) {
+    datum.setJobid(new Utf8(id.toString()));
+    datum.setFinishTime(finishTime);
+    datum.setFinishedMaps(finishedMaps);
+    datum.setFinishedReduces(finishedReduces);
+    datum.setJobStatus(new Utf8(status));
+    if (diagnostics == null) {
+      diagnostics = NODIAGS_LIST;
+    }
+    datum.setDiagnostics(new Utf8(Joiner.on('\n').skipNulls()
+        .join(diagnostics)));
   }
 
   JobUnsuccessfulCompletionEvent() {}
@@ -61,13 +91,13 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
   /** Get the Job ID */
   public JobID getJobId() { return JobID.forName(datum.jobid.toString()); }
   /** Get the job finish time */
-  public long getFinishTime() { return datum.finishTime; }
+  public long getFinishTime() { return datum.getFinishTime(); }
   /** Get the number of finished maps */
-  public int getFinishedMaps() { return datum.finishedMaps; }
+  public int getFinishedMaps() { return datum.getFinishedMaps(); }
   /** Get the number of finished reduces */
-  public int getFinishedReduces() { return datum.finishedReduces; }
+  public int getFinishedReduces() { return datum.getFinishedReduces(); }
   /** Get the status */
-  public String getStatus() { return datum.jobStatus.toString(); }
+  public String getStatus() { return datum.getJobStatus().toString(); }
   /** Get the event type */
   public EventType getEventType() {
     if ("FAILED".equals(getStatus())) {
@@ -78,4 +108,13 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
       return EventType.JOB_KILLED;
   }
 
+  /**
+   * Retrieves diagnostics information preserved in the history file
+   *
+   * @return diagnostics as of the time of job termination
+   */
+  public String getDiagnostics() {
+    final CharSequence diagnostics = datum.getDiagnostics();
+    return diagnostics == null ? NODIAGS : diagnostics.toString();
+  }
 }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml

@@ -73,7 +73,7 @@
         <configuration>
           <excludes>
             <exclude>src/test/resources/job_1329348432655_0001_conf.xml</exclude>
-            <exclude>src/test/resources/job_1329348432655_0001-1329348443227-user-Sleep+job-1329348468601-10-1-SUCCEEDED-default.jhist</exclude>
+            <exclude>src/test/resources/*.jhist</exclude>
           </excludes>
         </configuration>
       </plugin>

+ 130 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.mapreduce.v2.hs;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
+    .NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
@@ -25,6 +29,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -36,9 +41,9 @@ import junit.framework.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobID;
@@ -53,6 +58,7 @@ import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
+import org.apache.hadoop.mapreduce.jobhistory.JobUnsuccessfulCompletionEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskFailedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskStartedEvent;
@@ -66,8 +72,11 @@ import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.JobIdPBImpl;
 import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.TaskIdPBImpl;
 import org.apache.hadoop.mapreduce.v2.app.MRApp;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.hs.HistoryFileManager.HistoryFileInfo;
@@ -149,7 +158,7 @@ public class TestJobHistoryParsing {
     conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
     long amStartTimeEst = System.currentTimeMillis();
     conf.setClass(
-        CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+        NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
         MyResolver.class, DNSToSwitchMapping.class);
     RackResolver.init(conf);
     MRApp app = new MRAppWithHistory(numMaps, numReduces, true, this.getClass()
@@ -390,7 +399,7 @@ public class TestJobHistoryParsing {
     try {
       Configuration conf = new Configuration();
       conf.setClass(
-          CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
           MyResolver.class, DNSToSwitchMapping.class);
       RackResolver.init(conf);
       MRApp app = new MRAppWithHistoryWithFailedAttempt(2, 1, true, this
@@ -455,7 +464,7 @@ public class TestJobHistoryParsing {
     try {
       Configuration conf = new Configuration();
       conf.setClass(
-          CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
           MyResolver.class, DNSToSwitchMapping.class);
       RackResolver.init(conf);
       MRApp app = new MRAppWithHistoryWithFailedTask(2, 1, true, this
@@ -499,18 +508,85 @@ public class TestJobHistoryParsing {
         Assert.assertNotNull("completed task report has null counters", ct
             .getReport().getCounters());
       }
+      final List<String> originalDiagnostics = job.getDiagnostics();
+      final String historyError = jobInfo.getErrorInfo();
+      assertTrue("No original diagnostics for a failed job",
+          originalDiagnostics != null && !originalDiagnostics.isEmpty());
+      assertNotNull("No history error info for a failed job ", historyError);
+      for (String diagString : originalDiagnostics) {
+        assertTrue(historyError.contains(diagString));
+      }
     } finally {
       LOG.info("FINISHED testCountersForFailedTask");
     }
   }
 
+  @Test(timeout = 60000)
+  public void testDiagnosticsForKilledJob() throws Exception {
+    LOG.info("STARTING testDiagnosticsForKilledJob");
+    try {
+      final Configuration conf = new Configuration();
+      conf.setClass(
+          NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          MyResolver.class, DNSToSwitchMapping.class);
+      RackResolver.init(conf);
+      MRApp app = new MRAppWithHistoryWithJobKilled(2, 1, true, this
+          .getClass().getName(), true);
+      app.submit(conf);
+      Job job = app.getContext().getAllJobs().values().iterator().next();
+      JobId jobId = job.getID();
+      app.waitForState(job, JobState.KILLED);
+
+      // make sure all events are flushed
+      app.waitForState(Service.STATE.STOPPED);
+
+      JobHistory jobHistory = new JobHistory();
+      jobHistory.init(conf);
+
+      HistoryFileInfo fileInfo = jobHistory.getJobFileInfo(jobId);
+
+      JobHistoryParser parser;
+      JobInfo jobInfo;
+      synchronized (fileInfo) {
+        Path historyFilePath = fileInfo.getHistoryFile();
+        FSDataInputStream in = null;
+        FileContext fc = null;
+        try {
+          fc = FileContext.getFileContext(conf);
+          in = fc.open(fc.makeQualified(historyFilePath));
+        } catch (IOException ioe) {
+          LOG.info("Can not open history file: " + historyFilePath, ioe);
+          throw (new Exception("Can not open History File"));
+        }
+
+        parser = new JobHistoryParser(in);
+        jobInfo = parser.parse();
+      }
+      Exception parseException = parser.getParseException();
+      assertNull("Caught an expected exception " + parseException,
+          parseException);
+      final List<String> originalDiagnostics = job.getDiagnostics();
+      final String historyError = jobInfo.getErrorInfo();
+      assertTrue("No original diagnostics for a failed job",
+          originalDiagnostics != null && !originalDiagnostics.isEmpty());
+      assertNotNull("No history error info for a failed job ", historyError);
+      for (String diagString : originalDiagnostics) {
+        assertTrue(historyError.contains(diagString));
+      }
+      assertTrue("No killed message in diagnostics",
+        historyError.contains(JobImpl.JOB_KILLED_DIAG));
+    } finally {
+      LOG.info("FINISHED testDiagnosticsForKilledJob");
+    }
+  }
+
   @Test(timeout = 50000)
   public void testScanningOldDirs() throws Exception {
     LOG.info("STARTING testScanningOldDirs");
     try {
       Configuration conf = new Configuration();
       conf.setClass(
-          CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
           MyResolver.class, DNSToSwitchMapping.class);
       RackResolver.init(conf);
       MRApp app = new MRAppWithHistory(1, 1, true, this.getClass().getName(),
@@ -590,6 +666,27 @@ public class TestJobHistoryParsing {
     }
   }
 
+  static class MRAppWithHistoryWithJobKilled extends MRAppWithHistory {
+
+    public MRAppWithHistoryWithJobKilled(int maps, int reduces,
+        boolean autoComplete, String testName, boolean cleanOnStart) {
+      super(maps, reduces, autoComplete, testName, cleanOnStart);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected void attemptLaunched(TaskAttemptId attemptID) {
+      if (attemptID.getTaskId().getId() == 0) {
+        getContext().getEventHandler().handle(
+            new JobEvent(attemptID.getTaskId().getJobId(),
+                JobEventType.JOB_KILL));
+      } else {
+        getContext().getEventHandler().handle(
+            new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_DONE));
+      }
+    }
+  }
+
   static class HistoryFileManagerForTest extends HistoryFileManager {
     void deleteJobFromJobListCache(HistoryFileInfo fileInfo) {
       jobListCache.delete(fileInfo);
@@ -613,7 +710,7 @@ public class TestJobHistoryParsing {
       Configuration conf = new Configuration();
 
       conf.setClass(
-          CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
           MyResolver.class, DNSToSwitchMapping.class);
 
       RackResolver.init(conf);
@@ -668,7 +765,7 @@ public class TestJobHistoryParsing {
       Configuration configuration = new Configuration();
       configuration
           .setClass(
-              CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+              NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
               MyResolver.class, DNSToSwitchMapping.class);
 
       RackResolver.init(configuration);
@@ -743,7 +840,7 @@ public class TestJobHistoryParsing {
     final org.apache.hadoop.mapreduce.TaskType taskType =
         org.apache.hadoop.mapreduce.TaskType.MAP;
     final TaskID[] tids = new TaskID[2];
-    JobID jid = new JobID("1", 1);
+    final JobID jid = new JobID("1", 1);
     tids[0] = new TaskID(jid, taskType, 0);
     tids[1] = new TaskID(jid, taskType, 1);
     Mockito.when(reader.getNextEvent()).thenAnswer(
@@ -762,6 +859,13 @@ public class TestJobHistoryParsing {
               tfe.setDatum(tfe.getDatum());
               return tfe;
             }
+            if (eventId < 5) {
+              JobUnsuccessfulCompletionEvent juce =
+                  new JobUnsuccessfulCompletionEvent(jid, 100L, 2, 0,
+                      "JOB_FAILED", Collections.singletonList(
+                          "Task failed: " + tids[0].toString()));
+              return juce;
+            }
             return null;
           }
         });
@@ -769,4 +873,22 @@ public class TestJobHistoryParsing {
     assertTrue("Task 0 not implicated",
         info.getErrorInfo().contains(tids[0].toString()));
   }
+
+  @Test
+  public void testFailedJobHistoryWithoutDiagnostics() throws Exception {
+    final Path histPath = new Path(getClass().getClassLoader().getResource(
+        "job_1393307629410_0001-1393307687476-user-Sleep+job-1393307723835-0-0-FAILED-default-1393307693920.jhist")
+        .getFile());
+    final FileSystem lfs = FileSystem.getLocal(new Configuration());
+    final FSDataInputStream fsdis = lfs.open(histPath);
+    try {
+      JobHistoryParser parser = new JobHistoryParser(fsdis);
+      JobInfo info = parser.parse();
+      assertEquals("History parsed jobId incorrectly",
+          info.getJobId(), JobID.forName("job_1393307629410_0001") );
+      assertEquals("Default diagnostics incorrect ", "", info.getErrorInfo());
+    } finally {
+      fsdis.close();
+    }
+  }
 }

Разлика између датотеке није приказан због своје велике величине
+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1393307629410_0001-1393307687476-user-Sleep+job-1393307723835-0-0-FAILED-default-1393307693920.jhist


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

@@ -32,6 +32,9 @@ Release 2.5.0 - UNRELEASED
 
     YARN-1561. Fix a generic type warning in FairScheduler. (Chen He via junping_du)
 
+    YARN-1429. *nix: Allow a way for users to augment classpath of YARN daemons.
+    (Jarek Jarcec Cecho via kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES 
@@ -235,6 +238,9 @@ Release 2.4.0 - UNRELEASED
     YARN-1749. Updated application-history related configs to reflect the latest
     reality and to be consistently named. (Zhijie Shen via vinodkv)
 
+    YARN-1301. Added the INFO level log of the non-empty blacklist additions
+    and removals inside ApplicationMasterService. (Tsuyoshi Ozawa via zjshen)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 17 - 2
hadoop-yarn-project/hadoop-yarn/bin/yarn

@@ -22,7 +22,12 @@
 #
 #   JAVA_HOME        The java implementation to use.  Overrides JAVA_HOME.
 #
-#   YARN_CLASSPATH Extra Java CLASSPATH entries.
+#   YARN_USER_CLASSPATH Additional user CLASSPATH entries.
+#
+#   YARN_USER_CLASSPATH_FIRST  If set to non empty value then the user classpath
+#                              specified in YARN_USER_CLASSPATH will be
+#                              appended at the beginning of YARN's final
+#                              classpath instead of at the end.
 #
 #   YARN_HEAPSIZE  The maximum amount of heap to use, in MB. 
 #                    Default is 1000.
@@ -163,6 +168,17 @@ fi
 CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/${YARN_DIR}/*
 CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/${YARN_LIB_JARS_DIR}/*
 
+# Add user defined YARN_USER_CLASSPATH to the class path (if defined)
+if [ -n "$YARN_USER_CLASSPATH" ]; then
+  if [ -n "$YARN_USER_CLASSPATH_FIRST" ]; then
+    # User requested to add the custom entries at the beginning
+    CLASSPATH=${YARN_USER_CLASSPATH}:${CLASSPATH}
+  else
+    # By default we will just append the extra entries at the end
+    CLASSPATH=${CLASSPATH}:${YARN_USER_CLASSPATH}
+  fi
+fi
+
 # so that filenames w/ spaces are handled correctly in loops below
 IFS=
 
@@ -249,4 +265,3 @@ if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
 fi  
 
 exec "$JAVA" -Dproc_$COMMAND $JAVA_HEAP_MAX $YARN_OPTS -classpath "$CLASSPATH" $CLASS "$@"
-fi

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

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -447,10 +448,10 @@ public class ApplicationMasterService extends AbstractService implements
           request.getResourceBlacklistRequest();
       List<String> blacklistAdditions =
           (blacklistRequest != null) ?
-              blacklistRequest.getBlacklistAdditions() : null;
+              blacklistRequest.getBlacklistAdditions() : Collections.EMPTY_LIST;
       List<String> blacklistRemovals =
           (blacklistRequest != null) ?
-              blacklistRequest.getBlacklistRemovals() : null;
+              blacklistRequest.getBlacklistRemovals() : Collections.EMPTY_LIST;
 
       // sanity check
       try {
@@ -487,6 +488,11 @@ public class ApplicationMasterService extends AbstractService implements
           this.rScheduler.allocate(appAttemptId, ask, release, 
               blacklistAdditions, blacklistRemovals);
 
+      if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
+        LOG.info("blacklist are updated in Scheduler." +
+            "blacklistAdditions: " + blacklistAdditions + ", " +
+            "blacklistRemovals: " + blacklistRemovals);
+      }
       RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
       AllocateResponse allocateResponse =
           recordFactory.newRecordInstance(AllocateResponse.class);

Неке датотеке нису приказане због велике количине промена