Просмотр исходного кода

Merge trunk into HA branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1213867 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 лет назад
Родитель
Сommit
a0fe4f476a
63 измененных файлов с 989 добавлено и 355 удалено
  1. 31 0
      dev-support/test-patch.sh
  2. 8 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  4. 7 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  5. 21 8
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  6. 67 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  7. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java
  8. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/CorruptFileBlockIterator.java
  9. 7 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  10. 3 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  11. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  12. 11 29
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  13. 6 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java
  14. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  15. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  17. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  18. 31 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  19. 20 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  20. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  21. 20 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
  22. 49 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  23. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  24. 10 71
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  25. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  26. 81 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
  27. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  28. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
  30. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
  31. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
  32. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  33. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  34. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  35. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  37. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  38. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberManager.java
  39. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
  40. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeRegistrationWritable.java
  41. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  42. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
  43. 14 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  44. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  45. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java
  46. 135 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  47. 9 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
  48. 12 0
      hadoop-mapreduce-project/CHANGES.txt
  49. 13 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
  50. 33 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java
  51. 0 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
  52. 10 17
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java
  53. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java
  54. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
  55. 54 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobQueueClient.java
  56. 9 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  57. 63 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java
  58. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
  59. 20 6
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
  60. 2 5
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  61. 54 4
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
  62. 73 8
      hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/JobClientUnitTest.java
  63. 0 7
      hadoop-project/pom.xml

+ 31 - 0
dev-support/test-patch.sh

@@ -585,6 +585,35 @@ $JIRA_COMMENT_FOOTER"
   return 0
 }
 
+###############################################################################
+### Verify eclipse:eclipse works
+checkEclipseGeneration () {
+  echo ""
+  echo ""
+  echo "======================================================================"
+  echo "======================================================================"
+  echo "    Running mvn eclipse:eclipse."
+  echo "======================================================================"
+  echo "======================================================================"
+  echo ""
+  echo ""
+
+  echo "$MVN eclipse:eclipse -D${PROJECT_NAME}PatchProcess"
+  $MVN eclipse:eclipse -D${PROJECT_NAME}PatchProcess
+  if [[ $? != 0 ]] ; then
+      JIRA_COMMENT="$JIRA_COMMENT
+
+    -1 eclipse:eclipse.  The patch failed to build with eclipse:eclipse."
+    return 1
+  fi
+  JIRA_COMMENT="$JIRA_COMMENT
+
+    +1 eclipse:eclipse.  The patch built with eclipse:eclipse."
+  return 0
+}
+
+
+
 ###############################################################################
 ### Run the tests
 runTests () {
@@ -790,6 +819,8 @@ checkJavadocWarnings
 (( RESULT = RESULT + $? ))
 checkJavacWarnings
 (( RESULT = RESULT + $? ))
+checkEclipseGeneration
+(( RESULT = RESULT + $? ))
 ### Checkstyle not implemented yet
 #checkStyle
 #(( RESULT = RESULT + $? ))

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

@@ -134,6 +134,8 @@ Trunk (unreleased changes)
     HADOOP-7902. skipping name rules setting (if already set) should be done 
     on UGI initialization only. (tucu)
 
+    HADOOP-7913 Fix bug in ProtoBufRpcEngine  (sanjay)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -172,6 +174,9 @@ Release 0.23.1 - Unreleased
     HADOOP-6886. LocalFileSystem Needs createNonRecursive API.
     (Nicolas Spiegelberg and eli via eli)
 
+    HADOOP-7912. test-patch should run eclipse:eclipse to verify that it does
+    not break again. (Robert Joseph Evans via tomwhite)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -200,6 +205,9 @@ Release 0.23.1 - Unreleased
    HADOOP-7878  Regression: HADOOP-7777 switch changes break HDFS tests when the
    isSingleSwitch() predicate is used. (stevel)
 
+   HADOOP-7914. Remove the duplicated declaration of hadoop-hdfs test-jar in
+   hadoop-project/pom.xml.  (szetszwo)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -325,7 +325,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         int numReaders, int queueSizePerHandler, boolean verbose,
         SecretManager<? extends TokenIdentifier> secretManager)
         throws IOException {
-      super(bindAddress, port, RpcRequestWritable.class, numHandlers,
+      super(bindAddress, port, null, numHandlers,
           numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
               .getClass().getName()), secretManager);
       this.verbose = verbose;  

+ 7 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -62,13 +62,13 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.ipc.RpcPayloadHeader.*;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.ipc.RPC.RpcInvoker;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.RpcPayloadHeader.RpcPayloadOperation;
 import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
 import org.apache.hadoop.ipc.metrics.RpcMetrics;
@@ -76,18 +76,18 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
-import org.apache.hadoop.security.SaslRpcServer.SaslStatus;
 import org.apache.hadoop.security.SaslRpcServer.SaslDigestCallbackHandler;
 import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.apache.hadoop.security.SaslRpcServer.SaslStatus;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 
@@ -1504,7 +1504,8 @@ public abstract class Server {
         rpcRequest.readFields(dis);
       } catch (Throwable t) {
         LOG.warn("Unable to read call parameters for client " +
-                 getHostAddress(), t);
+                 getHostAddress() + "on connection protocol " +
+            this.protocolName + " for rpcKind " + header.getkind(),  t);
         final Call readParamsFailedCall = 
             new Call(header.getCallId(), null, this);
         ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();

+ 21 - 8
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -32,6 +32,10 @@ Trunk (unreleased changes)
     HDFS-2647. Used protobuf based RPC for InterDatanodeProtocol, 
     ClientDatanodeProtocol, JournalProtocol, NamenodeProtocol. (suresh)
 
+    HDFS-2663. Handle protobuf optional parameters correctly. (suresh)
+
+    HDFS-2666. Fix TestBackupNode failure. (suresh)
+
   IMPROVEMENTS
 
     HADOOP-7524 Change RPC to allow multiple protocols including multuple 
@@ -88,8 +92,8 @@ Trunk (unreleased changes)
     (suresh)
 
     HADOOP-7862   Hdfs changes to work with HADOOP 7862: 
-                  Move the support for multiple protocols to lower layer so that Writable,
-                  PB and Avro can all use it (Sanjay)
+    Move the support for multiple protocols to lower layer so that Writable,
+    PB and Avro can all use it (Sanjay)
 
     HDFS-1580. Add interface for generic Write Ahead Logging mechanisms.
     (Ivan Kelly via jitendra)
@@ -100,16 +104,17 @@ Trunk (unreleased changes)
 
     HDFS-2651 ClientNameNodeProtocol Translators for Protocol Buffers (sanjay)
 
+    HDFS-2650. Replace @inheritDoc with @Override. (Hari Mankude via suresh).
+
   OPTIMIZATIONS
     HDFS-2477. Optimize computing the diff between a block report and the
-               namenode state. (Tomasz Nykiel via hairong)
+    namenode state. (Tomasz Nykiel via hairong)
 
     HDFS-2495. Increase granularity of write operations in ReplicationMonitor
     thus reducing contention for write lock. (Tomasz Nykiel via hairong)
 
     HDFS-2476. More CPU efficient data structure for under-replicated,
-               over-replicated, and invalidated blocks.
-               (Tomasz Nykiel via todd)
+    over-replicated, and invalidated blocks. (Tomasz Nykiel via todd)
 
   BUG FIXES
     HDFS-2299. TestOfflineEditsViewer is failing on trunk. (Uma Maheswara Rao G
@@ -132,7 +137,7 @@ Trunk (unreleased changes)
     info on the console with security enabled. (Arpit Gupta via suresh)
 
     HDFS-2349. Corruption detected during block transfers between DNs
-               should log a WARN instead of INFO. (harsh)
+    should log a WARN instead of INFO. (harsh)
 
     HDFS-2188. Make FSEditLog create its journals from a list of URIs rather 
     than NNStorage. (Ivan Kelly via jitendra)
@@ -143,14 +148,17 @@ Trunk (unreleased changes)
     HDFS-2497 Fix TestBackupNode failure. (suresh)
 
     HDFS-2526. (Client)NamenodeProtocolTranslatorR23 do not need to keep a
-               reference to rpcProxyWithoutRetry (atm)
+    reference to rpcProxyWithoutRetry (atm)
 
     HDFS-2532. TestDfsOverAvroRpc timing out in trunk (Uma Maheswara Rao G
-               via todd)
+    via todd)
 
     HDFS-2606. webhdfs client filesystem impl must set the content-type 
     header for create/append. (tucu)
 
+    HDFS-1765. Block Replication should respect under-replication
+    block priority. (Uma Maheswara Rao G via eli)
+
 Release 0.23.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -199,6 +207,8 @@ Release 0.23.1 - UNRELEASED
     HDFS-2604. Add a log message to show if WebHDFS is enabled and a
     configuration section in the forrest doc.  (szetszwo)
 
+    HDFS-2654. Make BlockReaderLocal not extend RemoteBlockReader2. (eli)
+
   OPTIMIZATIONS
 
     HDFS-2130. Switch default checksum to CRC32C. (todd)
@@ -231,6 +241,9 @@ Release 0.23.1 - UNRELEASED
     HDFS-2590. Fix the missing links in the WebHDFS forrest doc.  (szetszwo)
 
     HDFS-2596. TestDirectoryScanner doesn't test parallel scans. (eli)
+
+    HDFS-2653. DFSClient should cache whether addrs are non-local when
+    short-circuiting is enabled. (eli)
     
 Release 0.23.0 - 2011-11-01 
 

+ 67 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -21,6 +21,7 @@ import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
+import java.net.Socket;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.HashMap;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.FSDataset;
+import org.apache.hadoop.hdfs.util.DirectBufferPool;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -57,8 +59,8 @@ import org.apache.hadoop.util.DataChecksum;
  * if security is enabled.</li>
  * </ul>
  */
-class BlockReaderLocal extends RemoteBlockReader2 {
-  public static final Log LOG = LogFactory.getLog(DFSClient.class);
+class BlockReaderLocal implements BlockReader {
+  private static final Log LOG = LogFactory.getLog(DFSClient.class);
 
   //Stores the cache and proxy for a local datanode.
   private static class LocalDatanodeInfo {
@@ -117,13 +119,24 @@ class BlockReaderLocal extends RemoteBlockReader2 {
   private static Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
 
   private final FileInputStream dataIn; // reader for the data file
-
   private FileInputStream checksumIn;   // reader for the checksum file
 
   private int offsetFromChunkBoundary;
   
-  ByteBuffer dataBuff = null;
-  ByteBuffer checksumBuff = null;
+  private byte[] skipBuf = null;
+  private ByteBuffer dataBuff = null;
+  private ByteBuffer checksumBuff = null;
+  private DataChecksum checksum;
+  private final boolean verifyChecksum;
+
+  private static DirectBufferPool bufferPool = new DirectBufferPool();
+
+  private int bytesPerChecksum;
+  private int checksumSize;
+
+  /** offset in block where reader wants to actually read */
+  private long startOffset;
+  private final String filename;
   
   /**
    * The only way this object can be instantiated.
@@ -256,9 +269,14 @@ class BlockReaderLocal extends RemoteBlockReader2 {
       long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
       boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
       FileInputStream checksumIn) throws IOException {
-    super(hdfsfile, block.getBlockPoolId(), block.getBlockId(), dataIn
-        .getChannel(), checksum, verifyChecksum, startOffset, firstChunkOffset,
-        length, null);
+    this.filename = hdfsfile;
+    this.checksum = checksum;
+    this.verifyChecksum = verifyChecksum;
+    this.startOffset = Math.max(startOffset, 0);
+
+    bytesPerChecksum = this.checksum.getBytesPerChecksum();
+    checksumSize = this.checksum.getChecksumSize();
+
     this.dataIn = dataIn;
     this.checksumIn = checksumIn;
     this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
@@ -322,10 +340,8 @@ class BlockReaderLocal extends RemoteBlockReader2 {
         readIntoBuffer(checksumIn, checksumBuff);
         checksumBuff.flip();
         dataBuff.flip();
-        if (verifyChecksum) {
-          checksum.verifyChunkedSums(dataBuff, checksumBuff, filename,
-              this.startOffset);
-        }
+        checksum.verifyChunkedSums(dataBuff, checksumBuff, filename,
+            this.startOffset);
       } else {
         dataRead = dataBuff.remaining();
       }
@@ -356,9 +372,24 @@ class BlockReaderLocal extends RemoteBlockReader2 {
     }
     if (!verifyChecksum) {
       return dataIn.skip(n);
-    } else {
-     return super.skip(n);
     }
+    // Skip by reading the data so we stay in sync with checksums.
+    // This could be implemented more efficiently in the future to
+    // skip to the beginning of the appropriate checksum chunk
+    // and then only read to the middle of that chunk.
+    if (skipBuf == null) {
+      skipBuf = new byte[bytesPerChecksum];
+    }
+    long nSkipped = 0;
+    while ( nSkipped < n ) {
+      int toSkip = (int)Math.min(n-nSkipped, skipBuf.length);
+      int ret = read(skipBuf, 0, toSkip);
+      if ( ret <= 0 ) {
+        return nSkipped;
+      }
+      nSkipped += ret;
+    }
+    return nSkipped;
   }
 
   @Override
@@ -375,6 +406,27 @@ class BlockReaderLocal extends RemoteBlockReader2 {
       bufferPool.returnBuffer(checksumBuff);
       checksumBuff = null;
     }
-    super.close();
+    startOffset = -1;
+    checksum = null;
+  }
+
+  @Override
+  public int readAll(byte[] buf, int offset, int len) throws IOException {
+    return BlockReaderUtil.readAll(this, buf, offset, len);
+  }
+
+  @Override
+  public void readFully(byte[] buf, int off, int len) throws IOException {
+    BlockReaderUtil.readFully(this, buf, off, len);
+  }
+
+  @Override
+  public Socket takeSocket() {
+    return null;
+  }
+
+  @Override
+  public boolean hasSentStatusCode() {
+    return false;
   }
 }

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java

@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+/**
+ * For sharing between the local and remote block reader implementations.
+ */
+class BlockReaderUtil {
+
+  /* See {@link BlockReader#readAll(byte[], int, int)} */
+  public static int readAll(BlockReader reader,
+      byte[] buf, int offset, int len) throws IOException {
+    int n = 0;
+    for (;;) {
+      int nread = reader.read(buf, offset + n, len - n);
+      if (nread <= 0)
+        return (n == 0) ? nread : n;
+      n += nread;
+      if (n >= len)
+        return n;
+    }
+  }
+
+  /* See {@link BlockReader#readFully(byte[], int, int)} */
+  public static void readFully(BlockReader reader,
+      byte[] buf, int off, int len) throws IOException {
+    int toRead = len;
+    while (toRead > 0) {
+      int ret = reader.read(buf, off, toRead);
+      if (ret < 0) {
+        throw new IOException("Premature EOF from inputStream");
+      }
+      toRead -= ret;
+      off += ret;
+    }
+  }
+}

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

@@ -81,17 +81,13 @@ public class CorruptFileBlockIterator implements RemoteIterator<Path> {
     }
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
   @Override
   public boolean hasNext() {
     return nextPath != null;
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
   @Override
   public Path next() throws IOException {
     if (!hasNext()) {

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

@@ -33,10 +33,8 @@ import java.net.URI;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import javax.net.SocketFactory;
 
@@ -575,12 +573,13 @@ public class DFSClient implements java.io.Closeable {
     }
   }
   
-  private static Set<String> localIpAddresses = Collections
-      .synchronizedSet(new HashSet<String>());
+  private static Map<String, Boolean> localAddrMap = Collections
+      .synchronizedMap(new HashMap<String, Boolean>());
   
   private static boolean isLocalAddress(InetSocketAddress targetAddr) {
     InetAddress addr = targetAddr.getAddress();
-    if (localIpAddresses.contains(addr.getHostAddress())) {
+    Boolean cached = localAddrMap.get(addr.getHostAddress());
+    if (cached != null && cached) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Address " + targetAddr + " is local");
       }
@@ -601,9 +600,7 @@ public class DFSClient implements java.io.Closeable {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Address " + targetAddr + " is local");
     }
-    if (local == true) {
-      localIpAddresses.add(addr.getHostAddress());
-    }
+    localAddrMap.put(addr.getHostAddress(), local);
     return local;
   }
   
@@ -1735,8 +1732,7 @@ public class DFSClient implements java.io.Closeable {
     }
   }
   
-  boolean shouldTryShortCircuitRead(InetSocketAddress targetAddr)
-      throws IOException {
+  boolean shouldTryShortCircuitRead(InetSocketAddress targetAddr) {
     if (shortCircuitLocalReads && isLocalAddress(targetAddr)) {
       return true;
     }
@@ -1759,7 +1755,7 @@ public class DFSClient implements java.io.Closeable {
     }
   }
 
-  /** {@inheritDoc} */
+  @Override
   public String toString() {
     return getClass().getSimpleName() + "[clientName=" + clientName
         + ", ugi=" + ugi + "]"; 

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

@@ -179,7 +179,7 @@ public class DistributedFileSystem extends FileSystem {
     workingDir = makeAbsolute(dir);
   }
 
-  /** {@inheritDoc} */
+  
   @Override
   public Path getHomeDirectory() {
     return makeQualified(new Path("/user/" + dfs.ugi.getShortUserName()));
@@ -306,7 +306,7 @@ public class DistributedFileSystem extends FileSystem {
     dfs.concat(getPathName(trg), srcs);
   }
 
-  /** {@inheritDoc} */
+  
   @SuppressWarnings("deprecation")
   @Override
   public boolean rename(Path src, Path dst) throws IOException {
@@ -315,7 +315,6 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /** 
-   * {@inheritDoc}
    * This rename operation is guaranteed to be atomic.
    */
   @SuppressWarnings("deprecation")
@@ -331,7 +330,6 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.delete(getPathName(f), recursive);
   }
   
-  /** {@inheritDoc} */
   @Override
   public ContentSummary getContentSummary(Path f) throws IOException {
     statistics.incrementReadOps(1);
@@ -512,7 +510,7 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.primitiveMkdir(getPathName(f), absolutePermission);
   }
 
-  /** {@inheritDoc} */
+ 
   @Override
   public void close() throws IOException {
     try {
@@ -552,7 +550,6 @@ public class DistributedFileSystem extends FileSystem {
     }
   }
   
-  /** {@inheritDoc} */
   @Override
   public FsStatus getStatus(Path p) throws IOException {
     statistics.incrementReadOps(1);
@@ -614,9 +611,6 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.getCorruptBlocksCount();
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public RemoteIterator<Path> listCorruptFileBlocks(Path path)
     throws IOException {
@@ -694,7 +688,6 @@ public class DistributedFileSystem extends FileSystem {
     dfs.metaSave(pathname);
   }
 
-  /** {@inheritDoc} */
   @Override
   public FsServerDefaults getServerDefaults() throws IOException {
     return dfs.getServerDefaults();
@@ -765,14 +758,12 @@ public class DistributedFileSystem extends FileSystem {
     }
   }
 
-  /** {@inheritDoc} */
   @Override
   public MD5MD5CRC32FileChecksum getFileChecksum(Path f) throws IOException {
     statistics.incrementReadOps(1);
     return dfs.getFileChecksum(getPathName(f));
   }
 
-  /** {@inheritDoc }*/
   @Override
   public void setPermission(Path p, FsPermission permission
       ) throws IOException {
@@ -780,7 +771,6 @@ public class DistributedFileSystem extends FileSystem {
     dfs.setPermission(getPathName(p), permission);
   }
 
-  /** {@inheritDoc }*/
   @Override
   public void setOwner(Path p, String username, String groupname
       ) throws IOException {
@@ -791,7 +781,6 @@ public class DistributedFileSystem extends FileSystem {
     dfs.setOwner(getPathName(p), username, groupname);
   }
 
-  /** {@inheritDoc }*/
   @Override
   public void setTimes(Path p, long mtime, long atime
       ) throws IOException {

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

@@ -529,7 +529,7 @@ public class HftpFileSystem extends FileSystem
   private class ChecksumParser extends DefaultHandler {
     private FileChecksum filechecksum;
 
-    /** {@inheritDoc} */
+    @Override
     public void startElement(String ns, String localname, String qname,
                 Attributes attrs) throws SAXException {
       if (!MD5MD5CRC32FileChecksum.class.getName().equals(qname)) {
@@ -563,7 +563,7 @@ public class HftpFileSystem extends FileSystem
     }
   }
 
-  /** {@inheritDoc} */
+  @Override
   public FileChecksum getFileChecksum(Path f) throws IOException {
     final String s = makeQualified(f).toUri().getPath();
     return new ChecksumParser().getFileChecksum(s);
@@ -611,7 +611,7 @@ public class HftpFileSystem extends FileSystem
   private class ContentSummaryParser extends DefaultHandler {
     private ContentSummary contentsummary;
 
-    /** {@inheritDoc} */
+    @Override
     public void startElement(String ns, String localname, String qname,
                 Attributes attrs) throws SAXException {
       if (!ContentSummary.class.getName().equals(qname)) {
@@ -697,7 +697,7 @@ public class HftpFileSystem extends FileSystem
     }
   }
 
-  /** {@inheritDoc} */
+  @Override
   public ContentSummary getContentSummary(Path f) throws IOException {
     final String s = makeQualified(f).toUri().getPath();
     final ContentSummary cs = new ContentSummaryParser().getContentSummary(s);

+ 11 - 29
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java

@@ -85,7 +85,7 @@ public class RemoteBlockReader2  implements BlockReader {
   
   Socket dnSock; //for now just sending the status code (e.g. checksumOk) after the read.
   private ReadableByteChannel in;
-  protected DataChecksum checksum;
+  private DataChecksum checksum;
   
   private PacketHeader curHeader;
   private ByteBuffer curPacketBuf = null;
@@ -96,25 +96,24 @@ public class RemoteBlockReader2  implements BlockReader {
   private long lastSeqNo = -1;
 
   /** offset in block where reader wants to actually read */
-  protected long startOffset;
-  protected final String filename;
+  private long startOffset;
+  private final String filename;
 
-  protected static DirectBufferPool bufferPool =
-    new DirectBufferPool();
+  private static DirectBufferPool bufferPool = new DirectBufferPool();
   private ByteBuffer headerBuf = ByteBuffer.allocate(
       PacketHeader.PKT_HEADER_LEN);
 
-  protected int bytesPerChecksum;
-  protected int checksumSize;
+  private int bytesPerChecksum;
+  private int checksumSize;
 
   /**
    * The total number of bytes we need to transfer from the DN.
    * This is the amount that the user has requested plus some padding
    * at the beginning so that the read can begin on a chunk boundary.
    */
-  protected long bytesNeededToFinish;
+  private long bytesNeededToFinish;
 
-  protected final boolean verifyChecksum;
+  private final boolean verifyChecksum;
 
   private boolean sentStatusCode = false;
   
@@ -389,29 +388,12 @@ public class RemoteBlockReader2  implements BlockReader {
 
   @Override
   public int readAll(byte[] buf, int offset, int len) throws IOException {
-    int n = 0;
-    for (;;) {
-      int nread = read(buf, offset + n, len - n);
-      if (nread <= 0) 
-        return (n == 0) ? nread : n;
-      n += nread;
-      if (n >= len)
-        return n;
-    }
+    return BlockReaderUtil.readAll(this, buf, offset, len);
   }
 
   @Override
-  public void readFully(byte[] buf, int off, int len)
-      throws IOException {
-    int toRead = len;
-    while (toRead > 0) {
-      int ret = read(buf, off, toRead);
-      if (ret < 0) {
-        throw new IOException("Premature EOF from inputStream");
-      }
-      toRead -= ret;
-      off += ret;
-    }    
+  public void readFully(byte[] buf, int off, int len) throws IOException {
+    BlockReaderUtil.readFully(this, buf, off, len);
   }
   
   /**

+ 6 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java

@@ -53,9 +53,7 @@ public class CorruptFileBlocks implements Writable {
     return cookie;
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
   @Override
   public void readFields(DataInput in) throws IOException {
     int fileCount = in.readInt();
@@ -66,9 +64,7 @@ public class CorruptFileBlocks implements Writable {
     cookie = Text.readString(in);
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
   @Override
   public void write(DataOutput out) throws IOException {
     out.writeInt(files.length);
@@ -78,9 +74,8 @@ public class CorruptFileBlocks implements Writable {
     Text.writeString(out, cookie);
   }
 
-  /**
-   * {@inheritDoc}
-   */
+ 
+  @Override
   public boolean equals(Object obj) {
     if (this == obj) {
       return true;
@@ -93,9 +88,8 @@ public class CorruptFileBlocks implements Writable {
       Arrays.equals(files, other.files);
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
+  @Override
   public int hashCode() {
     int result = cookie.hashCode();
 

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

@@ -185,14 +185,14 @@ public class DatanodeID implements WritableComparable<DatanodeID> {
   /////////////////////////////////////////////////
   // Writable
   /////////////////////////////////////////////////
-  /** {@inheritDoc} */
+  @Override
   public void write(DataOutput out) throws IOException {
     DeprecatedUTF8.writeString(out, name);
     DeprecatedUTF8.writeString(out, storageID);
     out.writeShort(infoPort);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void readFields(DataInput in) throws IOException {
     name = DeprecatedUTF8.readString(in);
     storageID = DeprecatedUTF8.readString(in);

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

@@ -382,7 +382,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
        });
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
 
@@ -400,7 +400,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     WritableUtils.writeEnum(out, getAdminState());
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
 

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

@@ -159,7 +159,7 @@ public class LocatedBlock implements Writable {
     return lb;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public String toString() {
     return getClass().getSimpleName() + "{" + b
         + "; getBlockSize()=" + getBlockSize()

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

@@ -225,7 +225,6 @@ public class LocatedBlocks implements Writable {
     }
   }
 
-  /** {@inheritDoc} */
   @Override
   public String toString() {
     final StringBuilder b = new StringBuilder(getClass().getSimpleName());

+ 31 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -24,6 +24,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
@@ -52,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdd
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
@@ -124,6 +128,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.UpgradeStatusReportProto;
 import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
@@ -208,11 +213,16 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, GetBlockLocationsRequestProto req)
       throws ServiceException {
     try {
-      return GetBlockLocationsResponseProto
-          .newBuilder()
-          .setLocations(
-              PBHelper.convert(server.getBlockLocations(req.getSrc(),
-                  req.getOffset(), req.getLength()))).build();
+      LocatedBlocks b = server.getBlockLocations(req.getSrc(), req.getOffset(),
+          req.getLength());
+      Builder builder = GetBlockLocationsResponseProto
+          .newBuilder();
+      if (b != null) {
+        builder.setLocations(
+            PBHelper.convert(server.getBlockLocations(req.getSrc(),
+                req.getOffset(), req.getLength()))).build();
+      }
+      return builder.build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -325,7 +335,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       return AddBlockResponseProto.newBuilder().setBlock(
           PBHelper.convert(
           server.addBlock(req.getSrc(), req.getClientName(), 
-                PBHelper.convert(req.getPrevious()), 
+                req.hasPrevious() ? PBHelper.convert(req.getPrevious()) : null, 
                 PBHelper.convert(
                   (DatanodeInfoProto[]) req.getExcludeNodesList().toArray()))))
            .build();
@@ -594,10 +604,14 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, DistributedUpgradeProgressRequestProto req)
       throws ServiceException {
     try {
-      UpgradeStatusReportProto result = PBHelper.convert(server
-          .distributedUpgradeProgress(PBHelper.convert(req.getAction())));
-      return DistributedUpgradeProgressResponseProto.newBuilder()
-          .setReport(result).build();
+      UpgradeStatusReport result = server.distributedUpgradeProgress(PBHelper
+          .convert(req.getAction()));
+      DistributedUpgradeProgressResponseProto.Builder builder = 
+          DistributedUpgradeProgressResponseProto.newBuilder();
+      if (result != null) {
+        builder.setReport(PBHelper.convert(result));
+      }
+      return builder.build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -636,9 +650,13 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public GetFileInfoResponseProto getFileInfo(RpcController controller,
       GetFileInfoRequestProto req) throws ServiceException {
     try {
-      HdfsFileStatusProto result = 
-          PBHelper.convert(server.getFileInfo(req.getSrc()));
-      return GetFileInfoResponseProto.newBuilder().setFs(result).build();
+      HdfsFileStatus res = server.getFileInfo(req.getSrc());
+      GetFileInfoResponseProto.Builder builder = 
+          GetFileInfoResponseProto.newBuilder();
+      if (res != null) {
+        builder.setFs(PBHelper.convert(res));
+      }
+      return builder.build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }

+ 20 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -83,14 +83,17 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Create
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DistributedUpgradeProgressRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DistributedUpgradeProgressResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDelegationTokenRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
@@ -205,7 +208,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setLength(length)
         .build();
     try {
-      return PBHelper.convert(rpcProxy.getBlockLocations(null, req).getLocations());
+      GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
+          req);
+      return resp.hasLocations() ? 
+        PBHelper.convert(resp.getLocations()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -329,12 +335,15 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws AccessControlException, FileNotFoundException,
       NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
       IOException {
-    AddBlockRequestProto req = AddBlockRequestProto.newBuilder().setSrc(src)
-        .setClientName(clientName).setPrevious(PBHelper.convert(previous))
-        .addAllExcludeNodes(Arrays.asList(PBHelper.convert(excludeNodes)))
-        .build();
+    AddBlockRequestProto.Builder builder = AddBlockRequestProto.newBuilder();
+    builder.setSrc(src)
+        .setClientName(clientName)
+        .addAllExcludeNodes(Arrays.asList(PBHelper.convert(excludeNodes)));
+    if (previous != null) {
+      builder.setPrevious(PBHelper.convert(previous));
+    }
     try {
-      return PBHelper.convert(rpcProxy.addBlock(null, req).getBlock());
+      return PBHelper.convert(rpcProxy.addBlock(null, builder.build()).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -615,8 +624,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
         DistributedUpgradeProgressRequestProto.newBuilder().
         setAction(PBHelper.convert(action)).build();
     try {
-      return PBHelper.convert(
-          rpcProxy.distributedUpgradeProgress(null, req).getReport());
+      DistributedUpgradeProgressResponseProto res = rpcProxy
+          .distributedUpgradeProgress(null, req);
+      return res.hasReport() ? PBHelper.convert(res.getReport()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -653,7 +663,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
     GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      return PBHelper.convert(rpcProxy.getFileInfo(null, req).getFs());
+      GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
+      return res.hasFs() ? PBHelper.convert(res.getFs()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

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

@@ -206,7 +206,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
     } catch (ServiceException se) {
       throw ProtobufHelper.getRemoteException(se);
     }
-    return PBHelper.convert(resp.getCmd());
+    return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
   }
 
   @Override
@@ -264,7 +264,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
     } catch (ServiceException se) {
       throw ProtobufHelper.getRemoteException(se);
     }
-    return PBHelper.convert(resp.getCmd());
+    return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
   }
 
   @Override

+ 20 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportR
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
@@ -110,7 +111,9 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     DatanodeCommand[] cmds = response.getCommands();
     if (cmds != null) {
       for (int i = 0; i < cmds.length; i++) {
-        builder.addCmds(i, PBHelper.convert(cmds[i]));
+        if (cmds[i] != null) {
+          builder.addCmds(PBHelper.convert(cmds[i]));
+        }
       }
     }
     return builder.build();
@@ -131,8 +134,12 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     } catch (IOException e) {
       throw new ServiceException(e);
     }
-    return BlockReportResponseProto.newBuilder().setCmd(PBHelper.convert(cmd))
-        .build();
+    BlockReportResponseProto.Builder builder = 
+        BlockReportResponseProto.newBuilder();
+    if (cmd != null) {
+      builder.setCmd(PBHelper.convert(cmd));
+    }
+    return builder.build();
   }
 
   @Override
@@ -182,14 +189,20 @@ public class DatanodeProtocolServerSideTranslatorPB implements
   @Override
   public ProcessUpgradeResponseProto processUpgrade(RpcController controller,
       ProcessUpgradeRequestProto request) throws ServiceException {
-    UpgradeCommand cmd;
+    UpgradeCommand ret;
     try {
-      cmd = impl.processUpgradeCommand(PBHelper.convert(request.getCmd()));
+      UpgradeCommand cmd = request.hasCmd() ? PBHelper
+          .convert(request.getCmd()) : null;
+      ret = impl.processUpgradeCommand(cmd);
     } catch (IOException e) {
       throw new ServiceException(e);
     }
-    return ProcessUpgradeResponseProto.newBuilder()
-        .setCmd(PBHelper.convert(cmd)).build();
+    ProcessUpgradeResponseProto.Builder builder = 
+        ProcessUpgradeResponseProto.newBuilder();
+    if (ret != null) {
+      builder.setCmd(PBHelper.convert(ret));
+    }
+    return builder.build();
   }
 
   @Override

+ 49 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -129,6 +129,10 @@ import com.google.protobuf.ByteString;
 
 /**
  * Utilities for converting protobuf classes to and from implementation classes.
+ * 
+ * Note that when converting from an internal type to protobuf type, the
+ * converter never return null for protobuf type. The check for internal type
+ * being null must be done before calling the convert() method.
  */
 public class PBHelper {
   private static final RegisterCommandProto REG_CMD_PROTO = 
@@ -341,16 +345,19 @@ public class PBHelper {
 
   public static CheckpointCommandProto convert(CheckpointCommand cmd) {
     return CheckpointCommandProto.newBuilder()
-        .setSignature(convert(cmd.getSignature())).build();
+        .setSignature(convert(cmd.getSignature()))
+        .setNeedToReturnImage(cmd.needToReturnImage()).build();
   }
 
   public static NamenodeCommandProto convert(NamenodeCommand cmd) {
     if (cmd instanceof CheckpointCommand) {
       return NamenodeCommandProto.newBuilder().setAction(cmd.getAction())
-          .setType(NamenodeCommandProto.Type.NamenodeCommand)
+          .setType(NamenodeCommandProto.Type.CheckPointCommand)
           .setCheckpointCmd(convert((CheckpointCommand) cmd)).build();
     }
-    return NamenodeCommandProto.newBuilder().setAction(cmd.getAction()).build();
+    return NamenodeCommandProto.newBuilder()
+        .setType(NamenodeCommandProto.Type.NamenodeCommand)
+        .setAction(cmd.getAction()).build();
   }
 
   public static BlockKey[] convertBlockKeys(List<BlockKeyProto> list) {
@@ -369,6 +376,7 @@ public class PBHelper {
   }
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
+    if (cmd == null) return null;
     switch (cmd.getType()) {
     case CheckPointCommand:
       CheckpointCommandProto chkPt = cmd.getCheckpointCmd();
@@ -425,7 +433,8 @@ public class PBHelper {
     if (di == null) return null;
     return new DatanodeInfo(
         PBHelper.convert(di.getId()),
-        di.getLocation(), di.getHostName(),
+        di.hasLocation() ? di.getLocation() : null , 
+        di.hasHostName() ? di.getHostName() : null,
         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
         di.getBlockPoolUsed()  ,  di.getLastUpdate() , di.getXceiverCount() ,
         PBHelper.convert(di.getAdminState())); 
@@ -433,10 +442,16 @@ public class PBHelper {
   
   static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) {
     if (di == null) return null;
-    return DatanodeInfoProto.newBuilder().
+    DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
+    if (di.getHostName() != null) {
+      builder.setHostName(di.getHostName());
+    }
+    if (di.getNetworkLocation() != null) {
+      builder.setLocation(di.getNetworkLocation());
+    }
+        
+    return builder.
      setId(PBHelper.convert((DatanodeID) di)).
-     setLocation(di.getNetworkLocation()).
-     setHostName(di.getHostName()).
      setCapacity(di.getCapacity()).
      setDfsUsed(di.getDfsUsed()).
      setRemaining(di.getRemaining()).
@@ -776,9 +791,14 @@ public class PBHelper {
 
   public static ReceivedDeletedBlockInfoProto convert(
       ReceivedDeletedBlockInfo receivedDeletedBlockInfo) {
-    return ReceivedDeletedBlockInfoProto.newBuilder()
-        .setBlock(PBHelper.convert(receivedDeletedBlockInfo.getBlock()))
-        .setDeleteHint(receivedDeletedBlockInfo.getDelHints()).build();
+    ReceivedDeletedBlockInfoProto.Builder builder = 
+        ReceivedDeletedBlockInfoProto.newBuilder();
+    
+    if (receivedDeletedBlockInfo.getDelHints() != null) {
+      builder.setDeleteHint(receivedDeletedBlockInfo.getDelHints());
+    }
+    return builder.setBlock(PBHelper.convert(receivedDeletedBlockInfo.getBlock()))
+        .build();
   }
 
   public static UpgradeCommandProto convert(UpgradeCommand comm) {
@@ -802,7 +822,7 @@ public class PBHelper {
   public static ReceivedDeletedBlockInfo convert(
       ReceivedDeletedBlockInfoProto proto) {
     return new ReceivedDeletedBlockInfo(PBHelper.convert(proto.getBlock()),
-        proto.getDeleteHint());
+        proto.hasDeleteHint() ? proto.getDeleteHint() : null);
   }
   
   public static NamespaceInfoProto convert(NamespaceInfo info) {
@@ -862,13 +882,10 @@ public class PBHelper {
   
   // LocatedBlocks
   public static LocatedBlocks convert(LocatedBlocksProto lb) {
-    if (lb == null) {
-      return null;
-    }
     return new LocatedBlocks(
         lb.getFileLength(), lb.getUnderConstruction(),
         PBHelper.convertLocatedBlock(lb.getBlocksList()),
-        PBHelper.convert(lb.getLastBlock()),
+        lb.hasLastBlock() ? PBHelper.convert(lb.getLastBlock()) : null,
         lb.getIsLastBlockComplete());
   }
   
@@ -876,11 +893,15 @@ public class PBHelper {
     if (lb == null) {
       return null;
     }
-    return LocatedBlocksProto.newBuilder().
-      setFileLength(lb.getFileLength()).
-      setUnderConstruction(lb.isUnderConstruction()).
-      addAllBlocks(PBHelper.convertLocatedBlock2(lb.getLocatedBlocks())).
-      setLastBlock(PBHelper.convert(lb.getLastLocatedBlock())).setIsLastBlockComplete(lb.isLastBlockComplete()).build();
+    LocatedBlocksProto.Builder builder = 
+        LocatedBlocksProto.newBuilder();
+    if (lb.getLastLocatedBlock() != null) {
+      builder.setLastBlock(PBHelper.convert(lb.getLastLocatedBlock()));
+    }
+    return builder.setFileLength(lb.getFileLength())
+        .setUnderConstruction(lb.isUnderConstruction())
+        .addAllBlocks(PBHelper.convertLocatedBlock2(lb.getLocatedBlocks()))
+        .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
   }
   
   public static FsServerDefaults convert(FsServerDefaultsProto fs) {
@@ -981,11 +1002,16 @@ public class PBHelper {
       setPermission(PBHelper.convert(fs.getPermission())).
       setOwner(fs.getOwner()).
       setGroup(fs.getGroup()).
-      setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes())).
       setPath(ByteString.copyFrom(fs.getLocalNameInBytes()));
-    LocatedBlocks locations = null;
+    
+    if (fs.getSymlink() != null) {
+      builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
+    }
     if (fs instanceof HdfsLocatedFileStatus) {
-      builder.setLocations(PBHelper.convert(locations));
+      LocatedBlocks locations = ((HdfsLocatedFileStatus)fs).getBlockLocations();
+      if (locations != null) {
+        builder.setLocations(PBHelper.convert(locations));
+      }
     }
     return builder.build();
   }

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -112,7 +112,6 @@ public class BlockInfoUnderConstruction extends BlockInfo {
       return (this == obj) || super.equals(obj);
     }
 
-    /** {@inheritDoc} */
     @Override
     public String toString() {
       final StringBuilder b = new StringBuilder(getClass().getSimpleName());
@@ -271,7 +270,6 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     return (this == obj) || super.equals(obj);
   }
 
-  /** {@inheritDoc} */
   @Override
   public String toString() {
     final StringBuilder b = new StringBuilder(super.toString());

+ 10 - 71
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -168,9 +168,6 @@ public class BlockManager {
   /** variable to enable check for enough racks */
   final boolean shouldCheckForEnoughRacks;
 
-  /** Last block index used for replication work. */
-  private int replIndex = 0;
-
   /** for block replicas placement */
   private BlockPlacementPolicy blockplacement;
   
@@ -923,74 +920,16 @@ public class BlockManager {
    * @return number of blocks scheduled for replication during this iteration.
    */
   private int computeReplicationWork(int blocksToProcess) throws IOException {
-    // Choose the blocks to be replicated
-    List<List<Block>> blocksToReplicate =
-      chooseUnderReplicatedBlocks(blocksToProcess);
-
-    // replicate blocks
-    return computeReplicationWorkForBlocks(blocksToReplicate);
-  }
-
-  /**
-   * Get a list of block lists to be replicated The index of block lists
-   * represents the
-   *
-   * @param blocksToProcess
-   * @return Return a list of block lists to be replicated. The block list index
-   *         represents its replication priority.
-   */
-  private List<List<Block>> chooseUnderReplicatedBlocks(int blocksToProcess) {
-    // initialize data structure for the return value
-    List<List<Block>> blocksToReplicate = new ArrayList<List<Block>>(
-        UnderReplicatedBlocks.LEVEL);
-    for (int i = 0; i < UnderReplicatedBlocks.LEVEL; i++) {
-      blocksToReplicate.add(new ArrayList<Block>());
-    }
+    List<List<Block>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
-      synchronized (neededReplications) {
-        if (neededReplications.size() == 0) {
-          return blocksToReplicate;
-        }
-
-        // Go through all blocks that need replications.
-        UnderReplicatedBlocks.BlockIterator neededReplicationsIterator = 
-            neededReplications.iterator();
-        // skip to the first unprocessed block, which is at replIndex
-        for (int i = 0; i < replIndex && neededReplicationsIterator.hasNext(); i++) {
-          neededReplicationsIterator.next();
-        }
-        // # of blocks to process equals either twice the number of live
-        // data-nodes or the number of under-replicated blocks whichever is less
-        blocksToProcess = Math.min(blocksToProcess, neededReplications.size());
-
-        for (int blkCnt = 0; blkCnt < blocksToProcess; blkCnt++, replIndex++) {
-          if (!neededReplicationsIterator.hasNext()) {
-            // start from the beginning
-            replIndex = 0;
-            blocksToProcess = Math.min(blocksToProcess, neededReplications
-                .size());
-            if (blkCnt >= blocksToProcess)
-              break;
-            neededReplicationsIterator = neededReplications.iterator();
-            assert neededReplicationsIterator.hasNext() : "neededReplications should not be empty.";
-          }
-
-          Block block = neededReplicationsIterator.next();
-          int priority = neededReplicationsIterator.getPriority();
-          if (priority < 0 || priority >= blocksToReplicate.size()) {
-            LOG.warn("Unexpected replication priority: "
-                + priority + " " + block);
-          } else {
-            blocksToReplicate.get(priority).add(block);
-          }
-        } // end for
-      } // end synchronized neededReplication
+      // Choose the blocks to be replicated
+      blocksToReplicate = neededReplications
+          .chooseUnderReplicatedBlocks(blocksToProcess);
     } finally {
       namesystem.writeUnlock();
     }
-
-    return blocksToReplicate;
+    return computeReplicationWorkForBlocks(blocksToReplicate);
   }
 
   /** Replicate a set of blocks
@@ -1019,7 +958,7 @@ public class BlockManager {
             // abandoned block or block reopened for append
             if(fileINode == null || fileINode.isUnderConstruction()) {
               neededReplications.remove(block, priority); // remove from neededReplications
-              replIndex--;
+              neededReplications.decrementReplicationIndex(priority);
               continue;
             }
 
@@ -1043,7 +982,7 @@ public class BlockManager {
               if ( (pendingReplications.getNumReplicas(block) > 0) ||
                    (blockHasEnoughRacks(block)) ) {
                 neededReplications.remove(block, priority); // remove from neededReplications
-                replIndex--;
+                neededReplications.decrementReplicationIndex(priority);
                 NameNode.stateChangeLog.info("BLOCK* "
                     + "Removing block " + block
                     + " from neededReplications as it has enough replicas.");
@@ -1104,7 +1043,7 @@ public class BlockManager {
           if(fileINode == null || fileINode.isUnderConstruction()) {
             neededReplications.remove(block, priority); // remove from neededReplications
             rw.targets = null;
-            replIndex--;
+            neededReplications.decrementReplicationIndex(priority);
             continue;
           }
           requiredReplication = fileINode.getReplication();
@@ -1118,7 +1057,7 @@ public class BlockManager {
             if ( (pendingReplications.getNumReplicas(block) > 0) ||
                  (blockHasEnoughRacks(block)) ) {
               neededReplications.remove(block, priority); // remove from neededReplications
-              replIndex--;
+              neededReplications.decrementReplicationIndex(priority);
               rw.targets = null;
               NameNode.stateChangeLog.info("BLOCK* "
                   + "Removing block " + block
@@ -1156,7 +1095,7 @@ public class BlockManager {
           // remove from neededReplications
           if(numEffectiveReplicas + targets.length >= requiredReplication) {
             neededReplications.remove(block, priority); // remove from neededReplications
-            replIndex--;
+            neededReplications.decrementReplicationIndex(priority);
           }
         }
       }

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -63,7 +63,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   BlockPlacementPolicyDefault() {
   }
     
-  /** {@inheritDoc} */
+  @Override
   public void initialize(Configuration conf,  FSClusterStats stats,
                          NetworkTopology clusterMap) {
     this.considerLoad = conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, true);
@@ -79,7 +79,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     }
   };
 
-  /** {@inheritDoc} */
+  @Override
   public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
@@ -89,7 +89,6 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         null, blocksize);
   }
 
-  /** {@inheritDoc} */
   @Override
   public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
@@ -525,7 +524,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     return nodes;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public int verifyBlockPlacement(String srcPath,
                                   LocatedBlock lBlk,
                                   int minRacks) {
@@ -544,7 +543,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     return minRacks - racks.size();
   }
 
-  /** {@inheritDoc} */
+  @Override
   public DatanodeDescriptor chooseReplicaToDelete(FSInodeInfo inode,
                                                  Block block,
                                                  short replicationFactor,

+ 81 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java

@@ -18,8 +18,11 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -81,10 +84,14 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   private List<LightWeightLinkedSet<Block>> priorityQueues
       = new ArrayList<LightWeightLinkedSet<Block>>();
 
+  /** Stores the replication index for each priority */
+  private Map<Integer, Integer> priorityToReplIdx = new HashMap<Integer, Integer>(LEVEL);
+  
   /** Create an object. */
   UnderReplicatedBlocks() {
     for (int i = 0; i < LEVEL; i++) {
       priorityQueues.add(new LightWeightLinkedSet<Block>());
+      priorityToReplIdx.put(i, 0);
     }
   }
 
@@ -300,6 +307,70 @@ class UnderReplicatedBlocks implements Iterable<Block> {
       }
     }
   }
+  
+  /**
+   * Get a list of block lists to be replicated. The index of block lists
+   * represents its replication priority. Replication index will be tracked for
+   * each priority list separately in priorityToReplIdx map. Iterates through
+   * all priority lists and find the elements after replication index. Once the
+   * last priority lists reaches to end, all replication indexes will be set to
+   * 0 and start from 1st priority list to fulfill the blockToProces count.
+   * 
+   * @param blocksToProcess - number of blocks to fetch from underReplicated blocks.
+   * @return Return a list of block lists to be replicated. The block list index
+   *         represents its replication priority.
+   */
+  public synchronized List<List<Block>> chooseUnderReplicatedBlocks(
+      int blocksToProcess) {
+    // initialize data structure for the return value
+    List<List<Block>> blocksToReplicate = new ArrayList<List<Block>>(LEVEL);
+    for (int i = 0; i < LEVEL; i++) {
+      blocksToReplicate.add(new ArrayList<Block>());
+    }
+
+    if (size() == 0) { // There are no blocks to collect.
+      return blocksToReplicate;
+    }
+    
+    int blockCount = 0;
+    for (int priority = 0; priority < LEVEL; priority++) { 
+      // Go through all blocks that need replications with current priority.
+      BlockIterator neededReplicationsIterator = iterator(priority);
+      Integer replIndex = priorityToReplIdx.get(priority);
+      
+      // skip to the first unprocessed block, which is at replIndex
+      for (int i = 0; i < replIndex && neededReplicationsIterator.hasNext(); i++) {
+        neededReplicationsIterator.next();
+      }
+
+      blocksToProcess = Math.min(blocksToProcess, size());
+      
+      if (blockCount == blocksToProcess) {
+        break;  // break if already expected blocks are obtained
+      }
+      
+      // Loop through all remaining blocks in the list.
+      while (blockCount < blocksToProcess
+          && neededReplicationsIterator.hasNext()) {
+        Block block = neededReplicationsIterator.next();
+        blocksToReplicate.get(priority).add(block);
+        replIndex++;
+        blockCount++;
+      }
+      
+      if (!neededReplicationsIterator.hasNext()
+          && neededReplicationsIterator.getPriority() == LEVEL - 1) {
+        // reset all priorities replication index to 0 because there is no
+        // recently added blocks in any list.
+        for (int i = 0; i < LEVEL; i++) {
+          priorityToReplIdx.put(i, 0);
+        }
+        break;
+      }
+      priorityToReplIdx.put(priority, replIndex); 
+    }
+    return blocksToReplicate;
+  }
 
   /** returns an iterator of all blocks in a given priority queue */
   synchronized BlockIterator iterator(int level) {
@@ -380,4 +451,14 @@ class UnderReplicatedBlocks implements Iterable<Block> {
       return level;
     }
   }
+
+  /**
+   * This method is to decrement the replication index for the given priority
+   * 
+   * @param priority  - int priority level
+   */
+  public void decrementReplicationIndex(int priority) {
+    Integer replIdx = priorityToReplIdx.get(priority);
+    priorityToReplIdx.put(priority, --replIdx); 
+  }
 }

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

@@ -1788,7 +1788,7 @@ public class DataNode extends Configured
     return new ExtendedBlock(oldBlock.getBlockPoolId(), r);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public long getProtocolVersion(String protocol, long clientVersion
       ) throws IOException {
     if (protocol.equals(InterDatanodeProtocol.class.getName())) {
@@ -1821,7 +1821,7 @@ public class DataNode extends Configured
       this.rInfo = rInfo;
     }
 
-    /** {@inheritDoc} */
+    @Override
     public String toString() {
       return "block:" + rInfo + " node:" + id;
     }
@@ -2006,7 +2006,6 @@ public class DataNode extends Configured
   }
 
   // ClientDataNodeProtocol implementation
-  /** {@inheritDoc} */
   @Override // ClientDataNodeProtocol
   public long getReplicaVisibleLength(final ExtendedBlock block) throws IOException {
     checkWriteAccess(block);

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

@@ -193,7 +193,7 @@ public interface FSDatasetInterface extends FSDatasetMBean {
       this.checksumIn = checksumIn;
     }
 
-    /** {@inheritDoc} */
+    @Override
     public void close() {
       IOUtils.closeStream(dataIn);
       IOUtils.closeStream(checksumIn);

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

@@ -40,7 +40,7 @@ public class ContentSummaryServlet extends DfsServlet {
   /** For java.io.Serializable */
   private static final long serialVersionUID = 1L;
   
-  /** {@inheritDoc} */
+  @Override
   public void doGet(final HttpServletRequest request,
       final HttpServletResponse response) throws ServletException, IOException {
     final Configuration conf = 

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

@@ -71,7 +71,6 @@ class EditLogFileOutputStream extends EditLogOutputStream {
     fc.position(fc.size());
   }
 
-  /** {@inheritDoc} */
   @Override
   public void write(FSEditLogOp op) throws IOException {
     doubleBuf.writeOp(op);

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

@@ -80,7 +80,7 @@ public class FileChecksumServlets {
           dtParam + addrParam);
     }
 
-    /** {@inheritDoc} */
+    @Override
     public void doGet(HttpServletRequest request, HttpServletResponse response
         ) throws ServletException, IOException {
       final ServletContext context = getServletContext();
@@ -104,7 +104,7 @@ public class FileChecksumServlets {
     /** For java.io.Serializable */
     private static final long serialVersionUID = 1L;
     
-    /** {@inheritDoc} */
+    @Override
     public void doGet(HttpServletRequest request, HttpServletResponse response
         ) throws ServletException, IOException {
       final PrintWriter out = response.getWriter();

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

@@ -261,13 +261,13 @@ public abstract class INode implements Comparable<byte[]>, FSInodeInfo {
     this.name = name;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public String getFullPathName() {
     // Get the full path name of this inode.
     return FSDirectory.getFullPathName(this);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public String toString() {
     return "\"" + getFullPathName() + "\":"
     + getUserName() + ":" + getGroupName() + ":"

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

@@ -372,7 +372,7 @@ class INodeDirectory extends INode {
     return parent;
   }
 
-  /** {@inheritDoc} */
+  @Override
   DirCounts spaceConsumedInTree(DirCounts counts) {
     counts.nsCount += 1;
     if (children != null) {
@@ -383,7 +383,7 @@ class INodeDirectory extends INode {
     return counts;    
   }
 
-  /** {@inheritDoc} */
+  @Override
   long[] computeContentSummary(long[] summary) {
     // Walk through the children of this node, using a new summary array
     // for the (sub)tree rooted at this node

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

@@ -168,7 +168,7 @@ public class INodeFile extends INode {
     return 1;
   }
 
-  /** {@inheritDoc} */
+  @Override
   long[] computeContentSummary(long[] summary) {
     summary[0] += computeFileSize(true);
     summary[1]++;

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

@@ -252,13 +252,13 @@ public class LeaseManager {
       return paths.remove(src);
     }
 
-    /** {@inheritDoc} */
+    @Override
     public String toString() {
       return "[Lease.  Holder: " + holder
           + ", pendingcreates: " + paths.size() + "]";
     }
   
-    /** {@inheritDoc} */
+    @Override
     public int compareTo(Lease o) {
       Lease l1 = this;
       Lease l2 = o;
@@ -273,7 +273,7 @@ public class LeaseManager {
       }
     }
   
-    /** {@inheritDoc} */
+    @Override
     public boolean equals(Object o) {
       if (!(o instanceof Lease)) {
         return false;
@@ -286,7 +286,7 @@ public class LeaseManager {
       return false;
     }
   
-    /** {@inheritDoc} */
+    @Override
     public int hashCode() {
       return holder.hashCode();
     }
@@ -436,7 +436,7 @@ public class LeaseManager {
     }
   }
 
-  /** {@inheritDoc} */
+  @Override
   public synchronized String toString() {
     return getClass().getSimpleName() + "= {"
         + "\n leases=" + leases

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

@@ -647,7 +647,7 @@ public class NamenodeFsck {
       return (float) (totalReplicas) / (float) totalBlocks;
     }
     
-    /** {@inheritDoc} */
+    @Override
     public String toString() {
       StringBuilder res = new StringBuilder();
       res.append("Status: ").append((isHealthy() ? "HEALTHY" : "CORRUPT"))

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

@@ -129,7 +129,7 @@ public class SecondaryNameNode implements Runnable {
   private FSNamesystem namesystem;
 
 
-  /** {@inheritDoc} */
+  @Override
   public String toString() {
     return getClass().getSimpleName() + " Status" 
       + "\nName Node Address    : " + nameNodeAddr   

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

@@ -64,7 +64,7 @@ class SerialNumberManager {
       return i2t.get(i);
     }
 
-    /** {@inheritDoc} */
+    @Override
     public String toString() {
       return "max=" + max + ",\n  t2i=" + t2i + ",\n  i2t=" + i2t;
     }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java

@@ -113,7 +113,7 @@ implements Writable, NodeRegistration {
   /////////////////////////////////////////////////
   // Writable
   /////////////////////////////////////////////////
-  /** {@inheritDoc} */
+  @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
 
@@ -124,7 +124,7 @@ implements Writable, NodeRegistration {
     exportedKeys.write(out);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeRegistrationWritable.java

@@ -74,7 +74,7 @@ public class DatanodeRegistrationWritable implements Writable {
   /////////////////////////////////////////////////
   // Writable
   /////////////////////////////////////////////////
-  /** {@inheritDoc} */
+  @Override
   public void write(DataOutput out) throws IOException {
     datanodeId.write(out);
 
@@ -85,7 +85,7 @@ public class DatanodeRegistrationWritable implements Writable {
     exportedKeys.write(out);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void readFields(DataInput in) throws IOException {
     datanodeId.readFields(in);
 

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -39,7 +39,7 @@ message GetBlockLocationsRequestProto {
 }
 
 message GetBlockLocationsResponseProto {
-  required LocatedBlocksProto locations = 1;
+  optional LocatedBlocksProto locations = 1;
 }
 
 message GetServerDefaultsRequestProto { // No parameters
@@ -115,7 +115,7 @@ message AbandonBlockResponseProto { // void response
 message AddBlockRequestProto {
   required string src = 1;
   required string clientName = 2;
-  required ExtendedBlockProto previous = 3;
+  optional ExtendedBlockProto previous = 3;
   repeated DatanodeInfoProto excludeNodes = 4;
 }
 
@@ -306,7 +306,7 @@ message DistributedUpgradeProgressRequestProto {
   required UpgradeActionProto action = 1;
 }
 message DistributedUpgradeProgressResponseProto {
-  required UpgradeStatusReportProto report = 1;
+  optional UpgradeStatusReportProto report = 1;
 }
 
 message ListCorruptFileBlocksRequestProto {
@@ -330,7 +330,7 @@ message GetFileInfoRequestProto {
 }
 
 message GetFileInfoResponseProto {
-  required HdfsFileStatusProto fs = 1;
+  optional HdfsFileStatusProto fs = 1;
 }
 
 message GetFileLinkInfoRequestProto {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto

@@ -184,7 +184,7 @@ message NNHAStatusHeartbeatProto {
  * haStatus - Status (from an HA perspective) of the NN sending this response
  */
 message HeartbeatResponseProto {
-  repeated DatanodeCommandProto cmds = 1;
+  repeated DatanodeCommandProto cmds = 1; // Returned commands can be null
   required NNHAStatusHeartbeatProto haStatus = 2;
 }
 

+ 14 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -30,7 +30,8 @@ message ExtendedBlockProto {
   required string poolId = 1;   // Block pool id - gloablly unique across clusters
   required uint64 blockId = 2;  // the local id within a pool
   required uint64 generationStamp = 3;
-  optional uint64 numBytes = 4;  // block len does not belong in ebid - here for historical reasons
+  optional uint64 numBytes = 4 [default = 0];  // len does not belong in ebid 
+                                               // here for historical reasons
 }
 
 /**
@@ -65,12 +66,12 @@ message DatanodeInfosProto {
  */
 message DatanodeInfoProto {
   required DatanodeIDProto id = 1;
-  optional uint64 capacity = 2;
-  optional uint64 dfsUsed = 3;
-  optional uint64 remaining = 4;
-  optional uint64 blockPoolUsed = 5;
-  optional uint64 lastUpdate = 6;
-  optional uint32 xceiverCount = 7;
+  optional uint64 capacity = 2 [default = 0];
+  optional uint64 dfsUsed = 3 [default = 0];
+  optional uint64 remaining = 4 [default = 0];
+  optional uint64 blockPoolUsed = 5 [default = 0];
+  optional uint64 lastUpdate = 6 [default = 0];
+  optional uint32 xceiverCount = 7 [default = 0];
   optional string location = 8;
   optional string hostName = 9;
   enum AdminState {
@@ -79,7 +80,7 @@ message DatanodeInfoProto {
     DECOMMISSIONED = 2;
   }
 
-  optional AdminState adminState = 10;
+  optional AdminState adminState = 10 [default = NORMAL];
 }
 
 /**
@@ -162,8 +163,8 @@ message HdfsFileStatusProto {
   optional bytes symlink = 9;             // if symlink, target encoded java UTF8 
 
   // Optional fields for file
-  optional uint32 block_replication = 10; // Actually a short - only 16bits used
-  optional uint64 blocksize = 11;
+  optional uint32 block_replication = 10 [default = 0]; // only 16bits used
+  optional uint64 blocksize = 11 [default = 0];
   optional LocatedBlocksProto locations = 12;  // suppled only if asked by client
 } 
 
@@ -218,7 +219,7 @@ message NamenodeRegistrationProto {
     CHECKPOINT = 3;
   }
   required StorageInfoProto storageInfo = 3;  // Node information
-  optional NamenodeRoleProto role = 4;        // Namenode role
+  optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
 }
 
 /**
@@ -264,7 +265,7 @@ message CheckpointCommandProto {
 message BlockProto {
   required uint64 blockId = 1;
   required uint64 genStamp = 2;
-  optional uint64 numBytes = 3;
+  optional uint64 numBytes = 3 [default = 0];
 }
 
 /**
@@ -313,7 +314,7 @@ message NamespaceInfoProto {
 message BlockKeyProto {
   required uint32 keyId = 1;      // Key identifier
   required uint64 expiryDate = 2; // Expiry time in milliseconds
-  required bytes keyBytes = 3;    // Key secret
+  optional bytes keyBytes = 3;    // Key secret
 }
 
 /**

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -515,6 +515,11 @@ public class MiniDFSCluster {
     this.waitSafeMode = waitSafeMode;
     
     // use alternate RPC engine if spec'd
+    /*
+    Turned off - see HDFS-2647 and HDFS-2660 for related comments.
+    This test can be turned on when Avro RPC is enabled using mechanism
+    similar to protobuf.
+    
     String rpcEngineName = System.getProperty("hdfs.rpc.engine");
     if (rpcEngineName != null && !"".equals(rpcEngineName)) {
       
@@ -538,6 +543,7 @@ public class MiniDFSCluster {
       conf.setBoolean(HADOOP_SECURITY_AUTHORIZATION,
                       false);
     }
+    */
     
     int replication = conf.getInt(DFS_REPLICATION_KEY, 3);
     conf.setInt(DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java

@@ -28,9 +28,16 @@ public class TestDfsOverAvroRpc extends TestLocalDFS {
 
   @Test(timeout=20000)
   public void testWorkingDirectory() throws IOException {
+    /*
+    Test turned off - see HDFS-2647 and HDFS-2660 for related comments.
+    This test can be turned on when Avro RPC is enabled using mechanism
+    similar to protobuf.
+    */
+    /*
     System.setProperty("hdfs.rpc.engine",
                        "org.apache.hadoop.ipc.AvroRpcEngine");
     super.testWorkingDirectory();
+    */
   }
 
 }

+ 135 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -17,26 +17,32 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
-
-import junit.framework.TestCase;
+import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
 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.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
+import org.junit.Test;
 
-public class TestReplicationPolicy extends TestCase {
+public class TestReplicationPolicy {
+  private Random random= DFSUtil.getRandom();
   private static final int BLOCK_SIZE = 1024;
   private static final int NUM_OF_DATANODES = 6;
   private static final Configuration CONF = new HdfsConfiguration();
@@ -90,6 +96,7 @@ public class TestReplicationPolicy extends TestCase {
    * the 1st is on dataNodes[0] and the 2nd is on a different rack.
    * @throws Exception
    */
+  @Test
   public void testChooseTarget1() throws Exception {
     dataNodes[0].updateHeartbeat(
         2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 
@@ -150,6 +157,7 @@ public class TestReplicationPolicy extends TestCase {
    * should be placed on a third rack.
    * @throws Exception
    */
+  @Test
   public void testChooseTarget2() throws Exception { 
     HashMap<Node, Node> excludedNodes;
     DatanodeDescriptor[] targets;
@@ -225,6 +233,7 @@ public class TestReplicationPolicy extends TestCase {
    * and the rest should be placed on the third rack.
    * @throws Exception
    */
+  @Test
   public void testChooseTarget3() throws Exception {
     // make data node 0 to be not qualified to choose
     dataNodes[0].updateHeartbeat(
@@ -278,6 +287,7 @@ public class TestReplicationPolicy extends TestCase {
    * the 3rd replica should be placed on the same rack as the 1st replica,
    * @throws Exception
    */
+  @Test
   public void testChoooseTarget4() throws Exception {
     // make data node 0 & 1 to be not qualified to choose: not enough disk space
     for(int i=0; i<2; i++) {
@@ -325,6 +335,7 @@ public class TestReplicationPolicy extends TestCase {
    * the 3rd replica should be placed on the same rack as the 2nd replica,
    * @throws Exception
    */
+  @Test
   public void testChooseTarget5() throws Exception {
     DatanodeDescriptor[] targets;
     targets = replicator.chooseTarget(filename,
@@ -354,6 +365,7 @@ public class TestReplicationPolicy extends TestCase {
    * the 1st replica. The 3rd replica can be placed randomly.
    * @throws Exception
    */
+  @Test
   public void testRereplicate1() throws Exception {
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
     chosenNodes.add(dataNodes[0]);    
@@ -388,6 +400,7 @@ public class TestReplicationPolicy extends TestCase {
    * the rest replicas can be placed randomly,
    * @throws Exception
    */
+  @Test
   public void testRereplicate2() throws Exception {
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
     chosenNodes.add(dataNodes[0]);
@@ -417,6 +430,7 @@ public class TestReplicationPolicy extends TestCase {
    * the rest replicas can be placed randomly,
    * @throws Exception
    */
+  @Test
   public void testRereplicate3() throws Exception {
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
     chosenNodes.add(dataNodes[0]);
@@ -450,4 +464,122 @@ public class TestReplicationPolicy extends TestCase {
     assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
   }
   
+  /**
+   * Test for the high priority blocks are processed before the low priority
+   * blocks.
+   */
+  @Test(timeout = 60000)
+  public void testReplicationWithPriority() throws Exception {
+    int DFS_NAMENODE_REPLICATION_INTERVAL = 1000;
+    int HIGH_PRIORITY = 0;
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
+        .format(true).build();
+    try {
+      cluster.waitActive();
+      final UnderReplicatedBlocks neededReplications = (UnderReplicatedBlocks) cluster
+          .getNameNode().getNamesystem().getBlockManager().neededReplications;
+      for (int i = 0; i < 100; i++) {
+        // Adding the blocks directly to normal priority
+        neededReplications.add(new Block(random.nextLong()), 2, 0, 3);
+      }
+      // Lets wait for the replication interval, to start process normal
+      // priority blocks
+      Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
+      
+      // Adding the block directly to high priority list
+      neededReplications.add(new Block(random.nextLong()), 1, 0, 3);
+      
+      // Lets wait for the replication interval
+      Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
+
+      // Check replication completed successfully. Need not wait till it process
+      // all the 100 normal blocks.
+      assertFalse("Not able to clear the element from high priority list",
+          neededReplications.iterator(HIGH_PRIORITY).hasNext());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Test for the ChooseUnderReplicatedBlocks are processed based on priority
+   */
+  @Test
+  public void testChooseUnderReplicatedBlocks() throws Exception {
+    UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
+
+    for (int i = 0; i < 5; i++) {
+      // Adding QUEUE_HIGHEST_PRIORITY block
+      underReplicatedBlocks.add(new Block(random.nextLong()), 1, 0, 3);
+
+      // Adding QUEUE_VERY_UNDER_REPLICATED block
+      underReplicatedBlocks.add(new Block(random.nextLong()), 2, 0, 7);
+
+      // Adding QUEUE_UNDER_REPLICATED block
+      underReplicatedBlocks.add(new Block(random.nextLong()), 6, 0, 6);
+
+      // Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block
+      underReplicatedBlocks.add(new Block(random.nextLong()), 5, 0, 6);
+
+      // Adding QUEUE_WITH_CORRUPT_BLOCKS block
+      underReplicatedBlocks.add(new Block(random.nextLong()), 0, 0, 3);
+    }
+
+    // Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks
+    // from
+    // QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_UNDER_REPLICATED.
+    List<List<Block>> chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
+    assertTheChosenBlocks(chosenBlocks, 5, 1, 0, 0, 0);
+
+    // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 4 blocks from
+    // QUEUE_VERY_UNDER_REPLICATED, 5 blocks from QUEUE_UNDER_REPLICATED and 1
+    // block from QUEUE_REPLICAS_BADLY_DISTRIBUTED.
+    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(10);
+    assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0);
+
+    // Adding QUEUE_HIGHEST_PRIORITY
+    underReplicatedBlocks.add(new Block(random.nextLong()), 1, 0, 3);
+
+    // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from
+    // QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED
+    // and 5 blocks from QUEUE_WITH_CORRUPT_BLOCKS.
+    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(10);
+    assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 4, 5);
+
+    // Since it is reached to end of all lists,
+    // should start picking the blocks from start.
+    // Choose 7 blocks from UnderReplicatedBlocks. Then it should pick 6 blocks from
+    // QUEUE_HIGHEST_PRIORITY, 1 block from QUEUE_VERY_UNDER_REPLICATED.
+    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(7);
+    assertTheChosenBlocks(chosenBlocks, 6, 1, 0, 0, 0);
+  }
+  
+  /** asserts the chosen blocks with expected priority blocks */
+  private void assertTheChosenBlocks(
+      List<List<Block>> chosenBlocks, int firstPrioritySize,
+      int secondPrioritySize, int thirdPrioritySize, int fourthPrioritySize,
+      int fifthPrioritySize) {
+    assertEquals(
+        "Not returned the expected number of QUEUE_HIGHEST_PRIORITY blocks",
+        firstPrioritySize, chosenBlocks.get(
+            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY).size());
+    assertEquals(
+        "Not returned the expected number of QUEUE_VERY_UNDER_REPLICATED blocks",
+        secondPrioritySize, chosenBlocks.get(
+            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED).size());
+    assertEquals(
+        "Not returned the expected number of QUEUE_UNDER_REPLICATED blocks",
+        thirdPrioritySize, chosenBlocks.get(
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED).size());
+    assertEquals(
+        "Not returned the expected number of QUEUE_REPLICAS_BADLY_DISTRIBUTED blocks",
+        fourthPrioritySize, chosenBlocks.get(
+            UnderReplicatedBlocks.QUEUE_REPLICAS_BADLY_DISTRIBUTED).size());
+    assertEquals(
+        "Not returned the expected number of QUEUE_WITH_CORRUPT_BLOCKS blocks",
+        fifthPrioritySize, chosenBlocks.get(
+            UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS).size());
+  }
 }

+ 9 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

@@ -145,9 +145,7 @@ public class TestNameNodeMetrics extends TestCase {
     fs.delete(file, true);
     filesTotal--; // reduce the filecount for deleted file
     
-    // Wait for more than DATANODE_COUNT replication intervals to ensure all 
-    // the blocks pending deletion are sent for deletion to the datanodes.
-    Thread.sleep(DFS_REPLICATION_INTERVAL * (DATANODE_COUNT + 1) * 1000);
+    waitForDeletion();
     updateMetrics();
     rb = getMetrics(NS_METRICS);
     assertGauge("FilesTotal", filesTotal, rb);
@@ -176,7 +174,7 @@ public class TestNameNodeMetrics extends TestCase {
     assertGauge("PendingReplicationBlocks", 1L, rb);
     assertGauge("ScheduledReplicationBlocks", 1L, rb);
     fs.delete(file, true);
-    updateMetrics();
+    waitForDeletion();
     rb = getMetrics(NS_METRICS);
     assertGauge("CorruptBlocks", 0L, rb);
     assertGauge("PendingReplicationBlocks", 0L, rb);
@@ -212,9 +210,15 @@ public class TestNameNodeMetrics extends TestCase {
     assertGauge("UnderReplicatedBlocks", 1L, rb);
     assertGauge("MissingBlocks", 1L, rb);
     fs.delete(file, true);
-    updateMetrics();
+    waitForDeletion();
     assertGauge("UnderReplicatedBlocks", 0L, getMetrics(NS_METRICS));
   }
+
+  private void waitForDeletion() throws InterruptedException {
+    // Wait for more than DATANODE_COUNT replication intervals to ensure all
+    // the blocks pending deletion are sent for deletion to the datanodes.
+    Thread.sleep(DFS_REPLICATION_INTERVAL * (DATANODE_COUNT + 1) * 1000);
+  }
   
   public void testRenameMetrics() throws Exception {
     Path src = getTestPath("src");

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

@@ -144,6 +144,9 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3369. Migrate MR1 tests to run on MR2 using the new interfaces
     introduced in MAPREDUCE-3169. (Ahmed Radwan via tomwhite)
 
+    MAPREDUCE-3518. mapred queue -info <queue> -showJobs throws NPE. 
+    (Jonathan Eagles via mahadev)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -264,6 +267,15 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3527. Fix minor API incompatibilities between 1.0 and 0.23.
     (tomwhite)
 
+    MAPREDUCE-3328. mapred queue -list output inconsistent and missing child 
+    queues. (Ravi Prakash via mahadev)
+
+    MAPREDUCE-3510. Capacity Scheduler inherited ACLs not displayed by mapred queue 
+    -showacls (Jonathan Eagles via mahadev)
+
+    MAPREDUCE-3537. Fix race condition in DefaultContainerExecutor which led
+    to container localization occuring in wrong directories. (acmurthy) 
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 13 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java

@@ -450,9 +450,19 @@ public class TypeConverter {
 
   public static QueueInfo fromYarn(org.apache.hadoop.yarn.api.records.QueueInfo
       queueInfo, Configuration conf) {
-    return new QueueInfo(queueInfo.getQueueName(),queueInfo.toString(),
-        fromYarn(queueInfo.getQueueState()), TypeConverter.fromYarnApps(
-        queueInfo.getApplications(), conf));
+    QueueInfo toReturn = new QueueInfo(queueInfo.getQueueName(), "Capacity: " +
+      queueInfo.getCapacity() * 100 + ", MaximumCapacity: " +
+      (queueInfo.getMaximumCapacity() < 0 ? "UNDEFINED" :
+        queueInfo.getMaximumCapacity()) + ", CurrentCapacity: " +
+      queueInfo.getCurrentCapacity() * 100, fromYarn(queueInfo.getQueueState()),
+      TypeConverter.fromYarnApps(queueInfo.getApplications(), conf));
+    List<QueueInfo> childQueues = new ArrayList<QueueInfo>();
+    for(org.apache.hadoop.yarn.api.records.QueueInfo childQueue :
+      queueInfo.getChildQueues()) {
+      childQueues.add(fromYarn(childQueue, conf));
+    }
+    toReturn.setQueueChildren(childQueues);
+    return toReturn;
   }
 
   public static QueueInfo[] fromYarnQueueInfo(

+ 33 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java

@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.mapreduce;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -36,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 public class TestTypeConverter {
   @Test
@@ -134,4 +138,33 @@ public class TestTypeConverter {
     Assert.assertEquals("queueInfo translation didn't work.",
       returned.getState().toString(), queueInfo.getQueueState().toString().toLowerCase());
   }
+
+  /**
+   * Test that child queues are converted too during conversion of the parent
+   * queue
+   */
+  @Test
+  public void testFromYarnQueue() {
+    //Define child queue
+    org.apache.hadoop.yarn.api.records.QueueInfo child =
+      Mockito.mock(org.apache.hadoop.yarn.api.records.QueueInfo.class);
+    Mockito.when(child.getQueueState()).thenReturn(QueueState.RUNNING);
+
+    //Define parent queue
+    org.apache.hadoop.yarn.api.records.QueueInfo queueInfo =
+      Mockito.mock(org.apache.hadoop.yarn.api.records.QueueInfo.class);
+    List<org.apache.hadoop.yarn.api.records.QueueInfo> children =
+      new ArrayList<org.apache.hadoop.yarn.api.records.QueueInfo>();
+    children.add(child); //Add one child
+    Mockito.when(queueInfo.getChildQueues()).thenReturn(children);
+    Mockito.when(queueInfo.getQueueState()).thenReturn(QueueState.RUNNING);
+
+    //Call the function we're testing
+    org.apache.hadoop.mapreduce.QueueInfo returned =
+      TypeConverter.fromYarn(queueInfo, new Configuration());
+
+    //Verify that the converted queue has the 1 child we had added
+    Assert.assertEquals("QueueInfo children weren't properly converted",
+      returned.getQueueChildren().size(), 1);
+  }
 }

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java

@@ -432,7 +432,6 @@ public class JobClient extends CLI {
 
   }
 
-  Cluster cluster;
   /**
    * Ugi of the client. We store this ugi when the client is created and 
    * then make sure that the same ugi is used to run the various protocols.

+ 10 - 17
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java

@@ -110,40 +110,33 @@ class JobQueueClient extends Configured implements Tool {
   }
 
   // format and print information about the passed in job queue.
-  void printJobQueueInfo(JobQueueInfo jobQueueInfo, Writer writer)
-      throws IOException {
+  void printJobQueueInfo(JobQueueInfo jobQueueInfo, Writer writer,
+    String prefix) throws IOException {
     if (jobQueueInfo == null) {
       writer.write("No queue found.\n");
       writer.flush();
       return;
     }
-    writer.write(String.format("Queue Name : %s \n",
+    writer.write(String.format(prefix + "======================\n"));
+    writer.write(String.format(prefix + "Queue Name : %s \n",
         jobQueueInfo.getQueueName()));
-    writer.write(String.format("Queue State : %s \n",
+    writer.write(String.format(prefix + "Queue State : %s \n",
         jobQueueInfo.getQueueState()));
-    writer.write(String.format("Scheduling Info : %s \n",
+    writer.write(String.format(prefix + "Scheduling Info : %s \n",
         jobQueueInfo.getSchedulingInfo()));
     List<JobQueueInfo> childQueues = jobQueueInfo.getChildren();
     if (childQueues != null && childQueues.size() > 0) {
-      writer.write(String.format("Child Queues : "));
       for (int i = 0; i < childQueues.size(); i++) {
-        JobQueueInfo childQueue = childQueues.get(i);
-        writer.write(String.format("%s", childQueue.getQueueName()));
-        if (i != childQueues.size() - 1) {
-          writer.write(String.format(", "));
-        }
+	  printJobQueueInfo(childQueues.get(i), writer, "    " + prefix);
       }
-      writer.write("\n");
     }
-    writer.write(String.format("======================\n"));
     writer.flush();
   }
   
   private void displayQueueList() throws IOException {
     JobQueueInfo[] rootQueues = jc.getRootQueues();
-    List<JobQueueInfo> allQueues = expandQueueList(rootQueues);
-    for (JobQueueInfo queue : allQueues) {
-      printJobQueueInfo(queue, new PrintWriter(System.out));
+    for (JobQueueInfo queue : rootQueues) {
+      printJobQueueInfo(queue, new PrintWriter(System.out), "");
     }
   }
   
@@ -181,7 +174,7 @@ class JobQueueClient extends Configured implements Tool {
       System.out.println("Queue \"" + queue + "\" does not exist.");
       return;
     }
-    printJobQueueInfo(jobQueueInfo, new PrintWriter(System.out));
+    printJobQueueInfo(jobQueueInfo, new PrintWriter(System.out), "");
     if (showJobs && (jobQueueInfo.getChildren() == null ||
         jobQueueInfo.getChildren().size() == 0)) {
       JobStatus[] jobs = jc.getJobsFromQueue(queue);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java

@@ -105,7 +105,7 @@ public class JobQueueInfo extends QueueInfo {
   public List<JobQueueInfo> getChildren() {
     List<JobQueueInfo> list = new ArrayList<JobQueueInfo>();
     for (QueueInfo q : super.getQueueChildren()) {
-      list.add((JobQueueInfo)q);
+      list.add(new JobQueueInfo(q));
     }
     return list;
   }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java

@@ -55,7 +55,7 @@ import org.apache.hadoop.yarn.logaggregation.LogDumper;
 @InterfaceStability.Stable
 public class CLI extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(CLI.class);
-  private Cluster cluster;
+  protected Cluster cluster;
 
   public CLI() {
   }

+ 54 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobQueueClient.java

@@ -0,0 +1,54 @@
+/**
+ * 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.mapred;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+
+public class TestJobQueueClient {
+  /**
+   * Test that print job queue recursively prints child queues
+   */
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testPrintJobQueueInfo() throws IOException {
+    JobQueueClient queueClient = new JobQueueClient();
+    JobQueueInfo parent = new JobQueueInfo();
+    JobQueueInfo child = new JobQueueInfo();
+    JobQueueInfo grandChild = new JobQueueInfo();
+    child.addChild(grandChild);
+    parent.addChild(child);
+    grandChild.setQueueName("GrandChildQueue");
+
+    ByteArrayOutputStream bbos = new ByteArrayOutputStream();
+    PrintWriter writer = new PrintWriter(bbos);
+    queueClient.printJobQueueInfo(parent, writer, "");
+
+    Assert.assertTrue("printJobQueueInfo did not print grandchild's name",
+      bbos.toString().contains("GrandChildQueue"));
+  }
+
+}

+ 9 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java

@@ -198,13 +198,16 @@ public class ResourceMgrDelegate {
   }
   
   private void getChildQueues(org.apache.hadoop.yarn.api.records.QueueInfo parent, 
-      List<org.apache.hadoop.yarn.api.records.QueueInfo> queues) {
+      List<org.apache.hadoop.yarn.api.records.QueueInfo> queues,
+      boolean recursive) {
     List<org.apache.hadoop.yarn.api.records.QueueInfo> childQueues = 
       parent.getChildQueues();
 
     for (org.apache.hadoop.yarn.api.records.QueueInfo child : childQueues) {
       queues.add(child);
-      getChildQueues(child, queues);
+      if(recursive) {
+        getChildQueues(child, queues, recursive);
+      }
     }
   }
 
@@ -226,7 +229,7 @@ public class ResourceMgrDelegate {
     org.apache.hadoop.yarn.api.records.QueueInfo rootQueue = 
       applicationsManager.getQueueInfo(
           getQueueInfoRequest(ROOT, false, true, true)).getQueueInfo();
-    getChildQueues(rootQueue, queues);
+    getChildQueues(rootQueue, queues, true);
 
     return TypeConverter.fromYarnQueueInfo(queues, this.conf);
   }
@@ -238,8 +241,8 @@ public class ResourceMgrDelegate {
 
     org.apache.hadoop.yarn.api.records.QueueInfo rootQueue = 
       applicationsManager.getQueueInfo(
-          getQueueInfoRequest(ROOT, false, true, false)).getQueueInfo();
-    getChildQueues(rootQueue, queues);
+          getQueueInfoRequest(ROOT, false, true, true)).getQueueInfo();
+    getChildQueues(rootQueue, queues, false);
 
     return TypeConverter.fromYarnQueueInfo(queues, this.conf);
   }
@@ -252,7 +255,7 @@ public class ResourceMgrDelegate {
         org.apache.hadoop.yarn.api.records.QueueInfo parentQueue = 
           applicationsManager.getQueueInfo(
               getQueueInfoRequest(parent, false, true, false)).getQueueInfo();
-        getChildQueues(parentQueue, queues);
+        getChildQueues(parentQueue, queues, true);
         
         return TypeConverter.fromYarnQueueInfo(queues, this.conf);
   }

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

@@ -0,0 +1,63 @@
+/**
+* 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.mapred;
+
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.yarn.api.ClientRMProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class TestResourceMgrDelegate {
+
+  /**
+   * Tests that getRootQueues makes a request for the (recursive) child queues
+   */
+@Test
+  public void testGetRootQueues() throws IOException, InterruptedException {
+    ClientRMProtocol applicationsManager = Mockito.mock(ClientRMProtocol.class);
+    GetQueueInfoResponse response = Mockito.mock(GetQueueInfoResponse.class);
+    org.apache.hadoop.yarn.api.records.QueueInfo queueInfo =
+      Mockito.mock(org.apache.hadoop.yarn.api.records.QueueInfo.class);
+    Mockito.when(response.getQueueInfo()).thenReturn(queueInfo);
+    Mockito.when(applicationsManager.getQueueInfo(Mockito.any(
+      GetQueueInfoRequest.class))).thenReturn(response);
+
+    ResourceMgrDelegate delegate = new ResourceMgrDelegate(
+      new YarnConfiguration(), applicationsManager);
+    delegate.getRootQueues();
+
+    ArgumentCaptor<GetQueueInfoRequest> argument =
+      ArgumentCaptor.forClass(GetQueueInfoRequest.class);
+    Mockito.verify(delegate.applicationsManager).getQueueInfo(
+      argument.capture());
+
+    Assert.assertTrue("Children of root queue not requested",
+      argument.getValue().getIncludeChildQueues());
+    Assert.assertTrue("Request wasn't to recurse through children",
+      argument.getValue().getRecursive());
+  }
+
+}

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

@@ -75,7 +75,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
   }
   
   @Override
-  public void startLocalizer(Path nmPrivateContainerTokensPath,
+  public synchronized void startLocalizer(Path nmPrivateContainerTokensPath,
       InetSocketAddress nmAddr, String user, String appId, String locId,
       List<String> localDirs, List<String> logDirs)
       throws IOException, InterruptedException {

+ 20 - 6
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java

@@ -198,19 +198,33 @@ public class CapacitySchedulerConfiguration extends Configuration {
   private static String getAclKey(QueueACL acl) {
     return "acl_" + acl.toString().toLowerCase();
   }
-  
+
+  public AccessControlList getAcl(String queue, QueueACL acl) {
+    String queuePrefix = getQueuePrefix(queue);
+    String aclString = get(queuePrefix + getAclKey(acl), DEFAULT_ACL);
+    return new AccessControlList(aclString);
+  }
+
+  public void setAcl(String queue, QueueACL acl, String aclString) {
+    String queuePrefix = getQueuePrefix(queue);
+    set(queuePrefix + getAclKey(acl), aclString);
+  }
+
   public Map<QueueACL, AccessControlList> getAcls(String queue) {
-    Map<QueueACL, AccessControlList> acls = 
+    Map<QueueACL, AccessControlList> acls =
       new HashMap<QueueACL, AccessControlList>();
-    String queuePrefix = getQueuePrefix(queue);
     for (QueueACL acl : QueueACL.values()) {
-      acls.put(acl, 
-          new AccessControlList(get(queuePrefix + getAclKey(acl), 
-              DEFAULT_ACL)));
+      acls.put(acl, getAcl(queue, acl));
     }
     return acls;
   }
 
+  public void setAcls(String queue, Map<QueueACL, AccessControlList> acls) {
+    for (Map.Entry<QueueACL, AccessControlList> e : acls.entrySet()) {
+      setAcl(queue, e.getKey(), e.getValue().getAclString());
+    }
+  }
+
   public String[] getQueues(String queue) {
     LOG.info("CSConf - getQueues called for: queuePrefix=" + getQueuePrefix(queue));
     String[] queues = getStrings(getQueuePrefix(queue) + QUEUES);

+ 2 - 5
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -492,11 +492,8 @@ public class LeafQueue implements CSQueue {
     QueueUserACLInfo userAclInfo = 
       recordFactory.newRecordInstance(QueueUserACLInfo.class);
     List<QueueACL> operations = new ArrayList<QueueACL>();
-    for (Map.Entry<QueueACL, AccessControlList> e : acls.entrySet()) {
-      QueueACL operation = e.getKey();
-      AccessControlList acl = e.getValue();
-
-      if (acl.isUserAllowed(user)) {
+    for (QueueACL operation : QueueACL.values()) {
+      if (hasAccess(operation, user)) {
         operations.add(operation);
       }
     }

+ 54 - 4
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -30,11 +31,14 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -102,20 +106,29 @@ public class TestLeafQueue {
   
   private static final String A = "a";
   private static final String B = "b";
+  private static final String C = "c";
   private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
     
     // Define top-level queues
-    conf.setQueues(CapacityScheduler.ROOT, new String[] {A, B});
+    conf.setQueues(CapacityScheduler.ROOT, new String[] {A, B, C});
     conf.setCapacity(CapacityScheduler.ROOT, 100);
     conf.setMaximumCapacity(CapacityScheduler.ROOT, 100);
+    conf.setAcl(CapacityScheduler.ROOT, QueueACL.SUBMIT_APPLICATIONS, " ");
     
     final String Q_A = CapacityScheduler.ROOT + "." + A;
-    conf.setCapacity(Q_A, 10);
+    conf.setCapacity(Q_A, 9);
     conf.setMaximumCapacity(Q_A, 20);
+    conf.setAcl(Q_A, QueueACL.SUBMIT_APPLICATIONS, "*");
     
     final String Q_B = CapacityScheduler.ROOT + "." + B;
     conf.setCapacity(Q_B, 90);
     conf.setMaximumCapacity(Q_B, 99);
+    conf.setAcl(Q_B, QueueACL.SUBMIT_APPLICATIONS, "*");
+
+    final String Q_C = CapacityScheduler.ROOT + "." + C;
+    conf.setCapacity(Q_C, 1);
+    conf.setMaximumCapacity(Q_C, 10);
+    conf.setAcl(Q_C, QueueACL.SUBMIT_APPLICATIONS, " ");
     
     LOG.info("Setup top-level queues a and b");
   }
@@ -167,8 +180,8 @@ public class TestLeafQueue {
 	  //can add more sturdy test with 3-layer queues 
 	  //once MAPREDUCE:3410 is resolved
 	  LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
-	  assertEquals(0.1, a.getCapacity(), epsilon);
-	  assertEquals(0.1, a.getAbsoluteCapacity(), epsilon);
+	  assertEquals(0.09, a.getCapacity(), epsilon);
+	  assertEquals(0.09, a.getAbsoluteCapacity(), epsilon);
 	  assertEquals(0.2, a.getMaximumCapacity(), epsilon);
 	  assertEquals(0.2, a.getAbsoluteMaximumCapacity(), epsilon);
 	  
@@ -177,6 +190,12 @@ public class TestLeafQueue {
 	  assertEquals(0.9, b.getAbsoluteCapacity(), epsilon);
 	  assertEquals(0.99, b.getMaximumCapacity(), epsilon);
 	  assertEquals(0.99, b.getAbsoluteMaximumCapacity(), epsilon);
+
+	  LeafQueue c = stubLeafQueue((LeafQueue)queues.get(C));
+	  assertEquals(0.01, c.getCapacity(), epsilon);
+	  assertEquals(0.01, c.getAbsoluteCapacity(), epsilon);
+	  assertEquals(0.1, c.getMaximumCapacity(), epsilon);
+	  assertEquals(0.1, c.getAbsoluteMaximumCapacity(), epsilon);
   }
  
   @Test
@@ -1082,6 +1101,37 @@ public class TestLeafQueue {
     assertEquals(0, app_0.getTotalRequiredResources(priority));
 
   }
+
+  public boolean hasQueueACL(List<QueueUserACLInfo> aclInfos, QueueACL acl) {
+    for (QueueUserACLInfo aclInfo : aclInfos) {
+      if (aclInfo.getUserAcls().contains(acl)) {
+        return true;
+      }
+    }    
+    return false;
+  }
+
+  @Test
+  public void testInheritedQueueAcls() throws IOException {
+    UserGroupInformation user = UserGroupInformation.getCurrentUser();
+
+    LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
+    LeafQueue b = stubLeafQueue((LeafQueue)queues.get(B));
+    LeafQueue c = stubLeafQueue((LeafQueue)queues.get(C));
+
+    assertFalse(root.hasAccess(QueueACL.SUBMIT_APPLICATIONS, user));
+    assertTrue(a.hasAccess(QueueACL.SUBMIT_APPLICATIONS, user));
+    assertTrue(b.hasAccess(QueueACL.SUBMIT_APPLICATIONS, user));
+    assertFalse(c.hasAccess(QueueACL.SUBMIT_APPLICATIONS, user));
+
+    assertTrue(hasQueueACL(
+          a.getQueueUserAclInfo(user), QueueACL.SUBMIT_APPLICATIONS));
+    assertTrue(hasQueueACL(
+          b.getQueueUserAclInfo(user), QueueACL.SUBMIT_APPLICATIONS));
+    assertFalse(hasQueueACL(
+          c.getQueueUserAclInfo(user), QueueACL.SUBMIT_APPLICATIONS));
+
+  }
   
   @After
   public void tearDown() throws Exception {

+ 73 - 8
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/JobClientUnitTest.java

@@ -19,21 +19,41 @@
 package org.apache.hadoop.mapred;
 
 import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.atLeastOnce;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobPriority;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.TaskReport;
 import org.junit.Test;
 
 public class JobClientUnitTest {
   
+  public class TestJobClient extends JobClient {
+
+    TestJobClient(JobConf jobConf) throws IOException {
+      super(jobConf);
+    }
+
+    void setCluster(Cluster cluster) {
+      this.cluster = cluster;
+    }
+  }
+
   @SuppressWarnings("deprecation")
   @Test
   public void testMapTaskReportsWithNullJob() throws Exception {
-    JobClient client = new JobClient();
+    TestJobClient client = new TestJobClient(new JobConf());
     Cluster mockCluster = mock(Cluster.class);
-    client.cluster = mockCluster;
+    client.setCluster(mockCluster);
     JobID id = new JobID("test",0);
     
     when(mockCluster.getJob(id)).thenReturn(null);
@@ -47,9 +67,9 @@ public class JobClientUnitTest {
   @SuppressWarnings("deprecation")
   @Test
   public void testReduceTaskReportsWithNullJob() throws Exception {
-    JobClient client = new JobClient();
+    TestJobClient client = new TestJobClient(new JobConf());
     Cluster mockCluster = mock(Cluster.class);
-    client.cluster = mockCluster;
+    client.setCluster(mockCluster);
     JobID id = new JobID("test",0);
     
     when(mockCluster.getJob(id)).thenReturn(null);
@@ -63,9 +83,9 @@ public class JobClientUnitTest {
   @SuppressWarnings("deprecation")
   @Test
   public void testSetupTaskReportsWithNullJob() throws Exception {
-    JobClient client = new JobClient();
+    TestJobClient client = new TestJobClient(new JobConf());
     Cluster mockCluster = mock(Cluster.class);
-    client.cluster = mockCluster;
+    client.setCluster(mockCluster);
     JobID id = new JobID("test",0);
     
     when(mockCluster.getJob(id)).thenReturn(null);
@@ -79,9 +99,9 @@ public class JobClientUnitTest {
   @SuppressWarnings("deprecation")
   @Test
   public void testCleanupTaskReportsWithNullJob() throws Exception {
-    JobClient client = new JobClient();
+    TestJobClient client = new TestJobClient(new JobConf());
     Cluster mockCluster = mock(Cluster.class);
-    client.cluster = mockCluster;
+    client.setCluster(mockCluster);
     JobID id = new JobID("test",0);
     
     when(mockCluster.getJob(id)).thenReturn(null);
@@ -91,4 +111,49 @@ public class JobClientUnitTest {
     
     verify(mockCluster).getJob(id);
   }
+
+  @Test
+  public void testShowJob() throws Exception {
+    TestJobClient client = new TestJobClient(new JobConf());
+    JobID jobID = new JobID("test", 0);
+
+    JobStatus mockJobStatus = mock(JobStatus.class);
+    when(mockJobStatus.getJobID()).thenReturn(jobID);
+    when(mockJobStatus.getState()).thenReturn(JobStatus.State.RUNNING);
+    when(mockJobStatus.getStartTime()).thenReturn(0L);
+    when(mockJobStatus.getUsername()).thenReturn("mockuser");
+    when(mockJobStatus.getQueue()).thenReturn("mockqueue");
+    when(mockJobStatus.getPriority()).thenReturn(JobPriority.NORMAL);
+    when(mockJobStatus.getNumUsedSlots()).thenReturn(1);
+    when(mockJobStatus.getNumReservedSlots()).thenReturn(1);
+    when(mockJobStatus.getUsedMem()).thenReturn(1024);
+    when(mockJobStatus.getReservedMem()).thenReturn(512);
+    when(mockJobStatus.getNeededMem()).thenReturn(2048);
+    when(mockJobStatus.getSchedulingInfo()).thenReturn("NA");
+
+    Job mockJob = mock(Job.class);
+    when(mockJob.getTaskReports(isA(TaskType.class))).thenReturn(new TaskReport[0]);
+
+    Cluster mockCluster = mock(Cluster.class);
+    when(mockCluster.getJob(jobID)).thenReturn(mockJob);
+
+    client.setCluster(mockCluster);
+    
+    
+    client.displayJobList(new JobStatus[] {mockJobStatus});
+    verify(mockJobStatus, atLeastOnce()).getJobID();
+    verify(mockJob, atLeastOnce()).getTaskReports(isA(TaskType.class));
+    verify(mockCluster, atLeastOnce()).getJob(jobID);
+    verify(mockJobStatus).getState();
+    verify(mockJobStatus).getStartTime();
+    verify(mockJobStatus).getUsername();
+    verify(mockJobStatus).getQueue();
+    verify(mockJobStatus).getPriority();
+    verify(mockJobStatus).getNumUsedSlots();
+    verify(mockJobStatus).getNumReservedSlots();
+    verify(mockJobStatus).getUsedMem();
+    verify(mockJobStatus).getReservedMem();
+    verify(mockJobStatus).getNeededMem();
+    verify(mockJobStatus).getSchedulingInfo();
+  }
 }

+ 0 - 7
hadoop-project/pom.xml

@@ -137,13 +137,6 @@
         <version>${project.version}</version>
       </dependency>
 
-      <dependency>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-hdfs</artifactId>
-        <version>${project.version}</version>
-        <type>test-jar</type>
-      </dependency>
-
       <dependency>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-mapreduce-examples</artifactId>