You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zj...@apache.org on 2015/04/18 00:35:29 UTC

[01/50] [abbrv] hadoop git commit: HDFS-8103. Move BlockTokenSecretManager.AccessMode into BlockTokenIdentifier. Contributed by Haohui Mai.

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2928 a3ebc7b7b -> 4459349c8


HDFS-8103. Move BlockTokenSecretManager.AccessMode into BlockTokenIdentifier. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b5e18f00
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b5e18f00
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b5e18f00

Branch: refs/heads/YARN-2928
Commit: b5e18f00ce99ae0239273074249f383b7d793d6f
Parents: ffe4801
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Apr 8 17:31:30 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:39 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../block/BlockPoolTokenSecretManager.java      |  6 +++---
 .../token/block/BlockTokenIdentifier.java       |  7 +++++--
 .../token/block/BlockTokenSecretManager.java    | 12 +++++-------
 .../hadoop/hdfs/server/balancer/KeyManager.java |  3 +--
 .../server/blockmanagement/BlockManager.java    | 11 ++++++-----
 .../hadoop/hdfs/server/datanode/DataNode.java   | 13 ++++++-------
 .../hdfs/server/datanode/DataXceiver.java       | 17 ++++++++---------
 .../hdfs/server/namenode/FSNamesystem.java      |  9 ++++-----
 .../security/token/block/TestBlockToken.java    | 20 ++++++++++----------
 .../blockmanagement/TestBlockTokenWithDFS.java  | 10 +++++-----
 11 files changed, 56 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c2f0363..be5c238 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -418,6 +418,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8100. Refactor DFSClient.Conf to a standalone class and separates
     short-circuit related conf to ShortCircuitConf.  (szetszwo)
 
+    HDFS-8103. Move BlockTokenSecretManager.AccessMode into
+    BlockTokenIdentifier. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockPoolTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockPoolTokenSecretManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockPoolTokenSecretManager.java
index 0df7067..cdfe7ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockPoolTokenSecretManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockPoolTokenSecretManager.java
@@ -23,7 +23,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier.AccessMode;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 
@@ -81,7 +81,7 @@ public class BlockPoolTokenSecretManager extends
 
   /**
    * See {@link BlockTokenSecretManager#checkAccess(BlockTokenIdentifier, 
-   *                String, ExtendedBlock, AccessMode)}
+   *                String, ExtendedBlock, BlockTokenIdentifier.AccessMode)}
    */
   public void checkAccess(BlockTokenIdentifier id, String userId,
       ExtendedBlock block, AccessMode mode) throws InvalidToken {
@@ -90,7 +90,7 @@ public class BlockPoolTokenSecretManager extends
 
   /**
    * See {@link BlockTokenSecretManager#checkAccess(Token, String, 
-   *                ExtendedBlock, AccessMode)}
+   *                ExtendedBlock, BlockTokenIdentifier.AccessMode)}
    */
   public void checkAccess(Token<BlockTokenIdentifier> token,
       String userId, ExtendedBlock block, AccessMode mode) throws InvalidToken {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java
index 67b1fe9..e293dcc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -35,6 +34,10 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 public class BlockTokenIdentifier extends TokenIdentifier {
   static final Text KIND_NAME = new Text("HDFS_BLOCK_TOKEN");
 
+  public enum AccessMode {
+    READ, WRITE, COPY, REPLACE
+  }
+
   private long expiryDate;
   private int keyId;
   private String userId;
@@ -175,7 +178,7 @@ public class BlockTokenIdentifier extends TokenIdentifier {
     
     return cache;
   }
-  
+
   @InterfaceAudience.Private
   public static class Renewer extends Token.TrivialRenewer {
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
index a3685ca..b103c1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
@@ -80,9 +80,7 @@ public class BlockTokenSecretManager extends
   
   private final SecureRandom nonceGenerator = new SecureRandom();
 
-  public static enum AccessMode {
-    READ, WRITE, COPY, REPLACE
-  };
+  ;
   
   /**
    * Constructor for slaves.
@@ -239,7 +237,7 @@ public class BlockTokenSecretManager extends
 
   /** Generate an block token for current user */
   public Token<BlockTokenIdentifier> generateToken(ExtendedBlock block,
-      EnumSet<AccessMode> modes) throws IOException {
+      EnumSet<BlockTokenIdentifier.AccessMode> modes) throws IOException {
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     String userID = (ugi == null ? null : ugi.getShortUserName());
     return generateToken(userID, block, modes);
@@ -247,7 +245,7 @@ public class BlockTokenSecretManager extends
 
   /** Generate a block token for a specified user */
   public Token<BlockTokenIdentifier> generateToken(String userId,
-      ExtendedBlock block, EnumSet<AccessMode> modes) throws IOException {
+      ExtendedBlock block, EnumSet<BlockTokenIdentifier.AccessMode> modes) throws IOException {
     BlockTokenIdentifier id = new BlockTokenIdentifier(userId, block
         .getBlockPoolId(), block.getBlockId(), modes);
     return new Token<BlockTokenIdentifier>(id, this);
@@ -259,7 +257,7 @@ public class BlockTokenSecretManager extends
    * when token password has already been verified (e.g., in the RPC layer).
    */
   public void checkAccess(BlockTokenIdentifier id, String userId,
-      ExtendedBlock block, AccessMode mode) throws InvalidToken {
+      ExtendedBlock block, BlockTokenIdentifier.AccessMode mode) throws InvalidToken {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Checking access for user=" + userId + ", block=" + block
           + ", access mode=" + mode + " using " + id.toString());
@@ -288,7 +286,7 @@ public class BlockTokenSecretManager extends
 
   /** Check if access should be allowed. userID is not checked if null */
   public void checkAccess(Token<BlockTokenIdentifier> token, String userId,
-      ExtendedBlock block, AccessMode mode) throws InvalidToken {
+      ExtendedBlock block, BlockTokenIdentifier.AccessMode mode) throws InvalidToken {
     BlockTokenIdentifier id = new BlockTokenIdentifier();
     try {
       id.readFields(new DataInputStream(new ByteArrayInputStream(token

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/KeyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/KeyManager.java
index 2ac8f48..1c6b352 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/KeyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/KeyManager.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@@ -100,7 +99,7 @@ public class KeyManager implements Closeable, DataEncryptionKeyFactory {
             "Cannot get access token since BlockKeyUpdater is not running");
       }
       return blockTokenSecretManager.generateToken(null, eb,
-          EnumSet.of(AccessMode.REPLACE, AccessMode.COPY));
+          EnumSet.of(BlockTokenIdentifier.AccessMode.REPLACE, BlockTokenIdentifier.AccessMode.COPY));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 9a6535e..e2c9b89 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -55,8 +55,9 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
@@ -747,7 +748,7 @@ public class BlockManager {
 
     final long fileLength = bc.computeContentSummary(getStoragePolicySuite()).getLength();
     final long pos = fileLength - ucBlock.getNumBytes();
-    return createLocatedBlock(ucBlock, pos, AccessMode.WRITE);
+    return createLocatedBlock(ucBlock, pos, BlockTokenIdentifier.AccessMode.WRITE);
   }
 
   /**
@@ -813,7 +814,7 @@ public class BlockManager {
   }
   
   private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos,
-    final BlockTokenSecretManager.AccessMode mode) throws IOException {
+    final AccessMode mode) throws IOException {
     final LocatedBlock lb = createLocatedBlock(blk, pos);
     if (mode != null) {
       setBlockToken(lb, mode);
@@ -886,7 +887,7 @@ public class BlockManager {
       if (LOG.isDebugEnabled()) {
         LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
       }
-      final AccessMode mode = needBlockToken? AccessMode.READ: null;
+      final AccessMode mode = needBlockToken? BlockTokenIdentifier.AccessMode.READ: null;
       final List<LocatedBlock> locatedblocks = createLocatedBlockList(
           blocks, offset, length, Integer.MAX_VALUE, mode);
 
@@ -918,7 +919,7 @@ public class BlockManager {
 
   /** Generate a block token for the located block. */
   public void setBlockToken(final LocatedBlock b,
-      final BlockTokenSecretManager.AccessMode mode) throws IOException {
+      final AccessMode mode) throws IOException {
     if (isBlockTokenEnabled()) {
       // Use cached UGI if serving RPC calls.
       b.setBlockToken(blockTokenSecretManager.generateToken(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 8c08871..b0e79e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -143,7 +143,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
@@ -157,7 +157,6 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.web.DatanodeHttpServer;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
@@ -1540,7 +1539,7 @@ public class DataNode extends ReconfigurableBase
   public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
       Token<BlockTokenIdentifier> token) throws IOException {
     checkBlockLocalPathAccess();
-    checkBlockToken(block, token, BlockTokenSecretManager.AccessMode.READ);
+    checkBlockToken(block, token, BlockTokenIdentifier.AccessMode.READ);
     Preconditions.checkNotNull(data, "Storage not yet initialized");
     BlockLocalPathInfo info = data.getBlockLocalPathInfo(block);
     if (LOG.isDebugEnabled()) {
@@ -1585,7 +1584,7 @@ public class DataNode extends ReconfigurableBase
       throw new ShortCircuitFdsUnsupportedException(
           fileDescriptorPassingDisabledReason);
     }
-    checkBlockToken(blk, token, BlockTokenSecretManager.AccessMode.READ);
+    checkBlockToken(blk, token, BlockTokenIdentifier.AccessMode.READ);
     int blkVersion = CURRENT_BLOCK_FORMAT_VERSION;
     if (maxVersion < blkVersion) {
       throw new ShortCircuitFdsVersionException("Your client is too old " +
@@ -1622,7 +1621,7 @@ public class DataNode extends ReconfigurableBase
     // Check access for each block
     for (int i = 0; i < blockIds.length; i++) {
       checkBlockToken(new ExtendedBlock(bpId, blockIds[i]),
-          tokens.get(i), BlockTokenSecretManager.AccessMode.READ);
+          tokens.get(i), BlockTokenIdentifier.AccessMode.READ);
     }
 
     DataNodeFaultInjector.get().getHdfsBlocksMetadata();
@@ -2124,7 +2123,7 @@ public class DataNode extends ReconfigurableBase
         Token<BlockTokenIdentifier> accessToken = BlockTokenSecretManager.DUMMY_TOKEN;
         if (isBlockTokenEnabled) {
           accessToken = blockPoolTokenSecretManager.generateToken(b, 
-              EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE));
+              EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
         }
 
         long writeTimeout = dnConf.socketWriteTimeout + 
@@ -2847,7 +2846,7 @@ public class DataNode extends ReconfigurableBase
           LOG.debug("Got: " + id.toString());
         }
         blockPoolTokenSecretManager.checkAccess(id, null, block,
-            BlockTokenSecretManager.AccessMode.READ);
+            BlockTokenIdentifier.AccessMode.READ);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index cf1b6be..83d6449 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -70,7 +70,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmR
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException;
 import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException;
@@ -507,7 +506,7 @@ class DataXceiver extends Receiver implements Runnable {
     DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
         baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
     checkAccess(out, true, block, blockToken,
-        Op.READ_BLOCK, BlockTokenSecretManager.AccessMode.READ);
+        Op.READ_BLOCK, BlockTokenIdentifier.AccessMode.READ);
   
     // send the block
     BlockSender blockSender = null;
@@ -651,7 +650,7 @@ class DataXceiver extends Receiver implements Runnable {
             getOutputStream(),
             HdfsConstants.SMALL_BUFFER_SIZE));
     checkAccess(replyOut, isClient, block, blockToken,
-        Op.WRITE_BLOCK, BlockTokenSecretManager.AccessMode.WRITE);
+        Op.WRITE_BLOCK, BlockTokenIdentifier.AccessMode.WRITE);
 
     DataOutputStream mirrorOut = null;  // stream to next target
     DataInputStream mirrorIn = null;    // reply from next target
@@ -849,7 +848,7 @@ class DataXceiver extends Receiver implements Runnable {
       final DatanodeInfo[] targets,
       final StorageType[] targetStorageTypes) throws IOException {
     checkAccess(socketOut, true, blk, blockToken,
-        Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
+        Op.TRANSFER_BLOCK, BlockTokenIdentifier.AccessMode.COPY);
     previousOpClientName = clientName;
     updateCurrentThreadName(Op.TRANSFER_BLOCK + " " + blk);
 
@@ -911,7 +910,7 @@ class DataXceiver extends Receiver implements Runnable {
     final DataOutputStream out = new DataOutputStream(
         getOutputStream());
     checkAccess(out, true, block, blockToken,
-        Op.BLOCK_CHECKSUM, BlockTokenSecretManager.AccessMode.READ);
+        Op.BLOCK_CHECKSUM, BlockTokenIdentifier.AccessMode.READ);
     // client side now can specify a range of the block for checksum
     long requestLength = block.getNumBytes();
     Preconditions.checkArgument(requestLength >= 0);
@@ -976,7 +975,7 @@ class DataXceiver extends Receiver implements Runnable {
     if (datanode.isBlockTokenEnabled) {
       try {
         datanode.blockPoolTokenSecretManager.checkAccess(blockToken, null, block,
-            BlockTokenSecretManager.AccessMode.COPY);
+            BlockTokenIdentifier.AccessMode.COPY);
       } catch (InvalidToken e) {
         LOG.warn("Invalid access token in request from " + remoteAddress
             + " for OP_COPY_BLOCK for block " + block + " : "
@@ -1064,7 +1063,7 @@ class DataXceiver extends Receiver implements Runnable {
     if (datanode.isBlockTokenEnabled) {
       try {
         datanode.blockPoolTokenSecretManager.checkAccess(blockToken, null, block,
-            BlockTokenSecretManager.AccessMode.REPLACE);
+            BlockTokenIdentifier.AccessMode.REPLACE);
       } catch (InvalidToken e) {
         LOG.warn("Invalid access token in request from " + remoteAddress
             + " for OP_REPLACE_BLOCK for block " + block + " : "
@@ -1251,7 +1250,7 @@ class DataXceiver extends Receiver implements Runnable {
       final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> t,
       final Op op,
-      final BlockTokenSecretManager.AccessMode mode) throws IOException {
+      final BlockTokenIdentifier.AccessMode mode) throws IOException {
     if (datanode.isBlockTokenEnabled) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Checking block access token for block '" + blk.getBlockId()
@@ -1264,7 +1263,7 @@ class DataXceiver extends Receiver implements Runnable {
           if (reply) {
             BlockOpResponseProto.Builder resp = BlockOpResponseProto.newBuilder()
               .setStatus(ERROR_ACCESS_TOKEN);
-            if (mode == BlockTokenSecretManager.AccessMode.WRITE) {
+            if (mode == BlockTokenIdentifier.AccessMode.WRITE) {
               DatanodeRegistration dnR = 
                 datanode.getDNRegistrationForBP(blk.getBlockPoolId());
               // NB: Unconditionally using the xfer addr w/o hostname

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index f7d8878..449b7e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -197,8 +197,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
@@ -3288,7 +3287,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     LocatedBlock lBlk = new LocatedBlock(
         getExtendedBlock(blk), locs, offset, false);
     getBlockManager().setBlockToken(
-        lBlk, BlockTokenSecretManager.AccessMode.WRITE);
+        lBlk, BlockTokenIdentifier.AccessMode.WRITE);
     return lBlk;
   }
 
@@ -3350,7 +3349,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         src, numAdditionalNodes, clientnode, chosen, 
         excludes, preferredblocksize, storagePolicyID);
     final LocatedBlock lb = new LocatedBlock(blk, targets, -1, false);
-    blockManager.setBlockToken(lb, AccessMode.COPY);
+    blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY);
     return lb;
   }
 
@@ -6272,7 +6271,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       // get a new generation stamp and an access token
       block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock())));
       locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
-      blockManager.setBlockToken(locatedBlock, AccessMode.WRITE);
+      blockManager.setBlockToken(locatedBlock, BlockTokenIdentifier.AccessMode.WRITE);
     } finally {
       writeUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
index 1fe7ba8..d5a9426 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
@@ -139,7 +139,7 @@ public class TestBlockToken {
         LOG.info("Got: " + id.toString());
         assertTrue("Received BlockTokenIdentifier is wrong", ident.equals(id));
         sm.checkAccess(id, null, PBHelper.convert(req.getBlock()),
-            BlockTokenSecretManager.AccessMode.WRITE);
+            BlockTokenIdentifier.AccessMode.WRITE);
         result = id.getBlockId();
       }
       return GetReplicaVisibleLengthResponseProto.newBuilder()
@@ -149,7 +149,7 @@ public class TestBlockToken {
 
   private BlockTokenIdentifier generateTokenId(BlockTokenSecretManager sm,
       ExtendedBlock block,
-      EnumSet<BlockTokenSecretManager.AccessMode> accessModes)
+      EnumSet<BlockTokenIdentifier.AccessMode> accessModes)
       throws IOException {
     Token<BlockTokenIdentifier> token = sm.generateToken(block, accessModes);
     BlockTokenIdentifier id = sm.createIdentifier();
@@ -164,17 +164,17 @@ public class TestBlockToken {
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
         blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
     TestWritable.testWritable(generateTokenId(sm, block1,
-        EnumSet.allOf(BlockTokenSecretManager.AccessMode.class)));
+        EnumSet.allOf(BlockTokenIdentifier.AccessMode.class)));
     TestWritable.testWritable(generateTokenId(sm, block2,
-        EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE)));
+        EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE)));
     TestWritable.testWritable(generateTokenId(sm, block3,
-        EnumSet.noneOf(BlockTokenSecretManager.AccessMode.class)));
+        EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class)));
   }
 
   private void tokenGenerationAndVerification(BlockTokenSecretManager master,
       BlockTokenSecretManager slave) throws Exception {
     // single-mode tokens
-    for (BlockTokenSecretManager.AccessMode mode : BlockTokenSecretManager.AccessMode
+    for (BlockTokenIdentifier.AccessMode mode : BlockTokenIdentifier.AccessMode
         .values()) {
       // generated by master
       Token<BlockTokenIdentifier> token1 = master.generateToken(block1,
@@ -189,8 +189,8 @@ public class TestBlockToken {
     }
     // multi-mode tokens
     Token<BlockTokenIdentifier> mtoken = master.generateToken(block3,
-        EnumSet.allOf(BlockTokenSecretManager.AccessMode.class));
-    for (BlockTokenSecretManager.AccessMode mode : BlockTokenSecretManager.AccessMode
+        EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
+    for (BlockTokenIdentifier.AccessMode mode : BlockTokenIdentifier.AccessMode
         .values()) {
       master.checkAccess(mtoken, null, block3, mode);
       slave.checkAccess(mtoken, null, block3, mode);
@@ -246,7 +246,7 @@ public class TestBlockToken {
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
         blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
     Token<BlockTokenIdentifier> token = sm.generateToken(block3,
-        EnumSet.allOf(BlockTokenSecretManager.AccessMode.class));
+        EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
 
     final Server server = createMockDatanode(sm, token, conf);
 
@@ -285,7 +285,7 @@ public class TestBlockToken {
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
         blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
     Token<BlockTokenIdentifier> token = sm.generateToken(block3,
-        EnumSet.allOf(BlockTokenSecretManager.AccessMode.class));
+        EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
 
     final Server server = createMockDatanode(sm, token, conf);
     server.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5e18f00/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index c280027..43f2992 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@ -413,21 +413,21 @@ public class TestBlockTokenWithDFS {
       tryRead(conf, lblock, false);
       // use a valid new token
       lblock.setBlockToken(sm.generateToken(lblock.getBlock(),
-              EnumSet.of(BlockTokenSecretManager.AccessMode.READ)));
+              EnumSet.of(BlockTokenIdentifier.AccessMode.READ)));
       // read should succeed
       tryRead(conf, lblock, true);
       // use a token with wrong blockID
       ExtendedBlock wrongBlock = new ExtendedBlock(lblock.getBlock()
           .getBlockPoolId(), lblock.getBlock().getBlockId() + 1);
       lblock.setBlockToken(sm.generateToken(wrongBlock,
-          EnumSet.of(BlockTokenSecretManager.AccessMode.READ)));
+          EnumSet.of(BlockTokenIdentifier.AccessMode.READ)));
       // read should fail
       tryRead(conf, lblock, false);
       // use a token with wrong access modes
       lblock.setBlockToken(sm.generateToken(lblock.getBlock(),
-          EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE,
-                     BlockTokenSecretManager.AccessMode.COPY,
-                     BlockTokenSecretManager.AccessMode.REPLACE)));
+          EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE,
+                     BlockTokenIdentifier.AccessMode.COPY,
+                     BlockTokenIdentifier.AccessMode.REPLACE)));
       // read should fail
       tryRead(conf, lblock, false);
 


[40/50] [abbrv] hadoop git commit: MAPREDUCE-6300. Task list sort by task id broken. Contributed by Siqi Li.

Posted by zj...@apache.org.
MAPREDUCE-6300. Task list sort by task id broken. Contributed by Siqi Li.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1e2ab614
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1e2ab614
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1e2ab614

Branch: refs/heads/YARN-2928
Commit: 1e2ab61415596664a7063faa71f843255c1b074e
Parents: 223c80e
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Apr 16 10:04:09 2015 -0500
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:46 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                             | 2 ++
 .../org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java     | 2 +-
 .../org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java    | 2 +-
 .../src/main/resources/webapps/static/yarn.dt.plugins.js         | 3 +++
 .../server/applicationhistoryservice/webapp/AppAttemptPage.java  | 4 ++--
 .../yarn/server/applicationhistoryservice/webapp/AppPage.java    | 2 +-
 .../org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java    | 2 --
 .../main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java | 2 --
 .../java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java     | 1 -
 .../server/resourcemanager/webapp/FairSchedulerAppsBlock.java    | 1 -
 .../hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java    | 2 --
 .../hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java   | 1 -
 12 files changed, 10 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index a0854c1..c81868d 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -346,6 +346,8 @@ Release 2.7.1 - UNRELEASED
 
   BUG FIXES
 
+    MAPREDUCE-6300. Task list sort by task id broken. (Siqi Li via aajisaka)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
index 0212ae4..9648527 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
@@ -43,7 +43,7 @@ public class TasksPage extends AppView {
       .append(", bProcessing: true")
 
       .append("\n, aoColumnDefs: [\n")
-      .append("{'sType':'numeric', 'aTargets': [0]")
+      .append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }")
 
       .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets': [1]")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
index 4e2b687..2619dff 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
@@ -76,7 +76,7 @@ public class HsTasksPage extends HsView {
     .append(", bProcessing: true")
 
     .append("\n, aoColumnDefs: [\n")
-    .append("{'sType':'numeric', 'aTargets': [ 0 ]")
+    .append("{'sType':'string', 'aTargets': [ 0 ]")
     .append(", 'mRender': parseHadoopID }")
 
     .append(", {'sType':'numeric', 'aTargets': [ 4")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js
index 7b069df..c9416fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js
@@ -139,6 +139,9 @@ function renderHadoopElapsedTime(data, type, full) {
   return data;  
 }
 
+//JSON array element is formatted like
+//"<a href="/proxy/application_1360183373897_0001>">application_1360183373897_0001</a>"
+//this function removes <a> tag and return a string value for sorting
 function parseHadoopID(data, type, full) {
   if (type === 'display') {
     return data;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
index 540f6e6..c7fe318 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
@@ -53,8 +53,8 @@ public class AppAttemptPage extends AHSView {
 
   protected String getContainersTableColumnDefs() {
     StringBuilder sb = new StringBuilder();
-    return sb.append("[\n").append("{'sType':'numeric', 'aTargets': [0]")
+    return sb.append("[\n").append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }]").toString();
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
index cf92c1d..50dcd96 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
@@ -55,7 +55,7 @@ public class AppPage extends AHSView {
 
   protected String getAttemptsTableColumnDefs() {
     StringBuilder sb = new StringBuilder();
-    return sb.append("[\n").append("{'sType':'numeric', 'aTargets': [0]")
+    return sb.append("[\n").append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }")
 
       .append("\n, {'sType':'numeric', 'aTargets': [1]")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
index 8695c6c..d320f4a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
@@ -158,8 +158,6 @@ public class AppAttemptBlock extends HtmlBlock {
     StringBuilder containersTableData = new StringBuilder("[\n");
     for (ContainerReport containerReport : containers) {
       ContainerInfo container = new ContainerInfo(containerReport);
-      // ConatinerID numerical value parsed by parseHadoopID in
-      // yarn.dt.plugins.js
       containersTableData
         .append("[\"<a href='")
         .append(url("container", container.getContainerId()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index 0d80339..dd5a4c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -285,8 +285,6 @@ public class AppBlock extends HtmlBlock {
         logsLink = containerReport.getLogUrl();
         nodeLink = containerReport.getNodeHttpAddress();
       }
-      // AppAttemptID numerical value parsed by parseHadoopID in
-      // yarn.dt.plugins.js
       attemptsTableData
         .append("[\"<a href='")
         .append(url("appattempt", appAttempt.getAppAttemptId()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
index 6a35349..93248ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
@@ -124,7 +124,6 @@ public class AppsBlock extends HtmlBlock {
       }
       AppInfo app = new AppInfo(appReport);
       String percent = String.format("%.1f", app.getProgress());
-      // AppID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
       appsTableData
         .append("[\"<a href='")
         .append(url("app", app.getAppId()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
index 8cfd246..8a5f4bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
@@ -116,7 +116,6 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
         // FairScheduler#applications don't have the entry. Skip it.
         continue;
       }
-      //AppID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
       appsTableData.append("[\"<a href='")
       .append(url("app", appInfo.getAppId())).append("'>")
       .append(appInfo.getAppId()).append("</a>\",\"")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
index 4942079..3779b91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
@@ -172,8 +172,6 @@ public class RMAppBlock extends AppBlock{
         blacklistedNodesCount = String.valueOf(nodes.size());
       }
 
-      // AppAttemptID numerical value parsed by parseHadoopID in
-      // yarn.dt.plugins.js
       attemptsTableData
           .append("[\"<a href='")
           .append(url("appattempt", appAttempt.getAppAttemptId()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e2ab614/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
index 94f3afa..29565e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
@@ -81,7 +81,6 @@ public class RMAppsBlock extends AppsBlock {
         blacklistedNodesCount = String.valueOf(nodes.size());
       }
       String percent = String.format("%.1f", app.getProgress());
-      // AppID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
       appsTableData
         .append("[\"<a href='")
         .append(url("app", app.getAppId()))


[48/50] [abbrv] hadoop git commit: HADOOP-11837. AuthenticationFilter should destroy SignerSecretProvider in Tomcat deployments. Contributed by Bowen Zhang.

Posted by zj...@apache.org.
HADOOP-11837. AuthenticationFilter should destroy SignerSecretProvider in Tomcat deployments. Contributed by Bowen Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f846cc5a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f846cc5a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f846cc5a

Branch: refs/heads/YARN-2928
Commit: f846cc5a06e8c66c2d198e870043e0ece81e5adf
Parents: 919f178
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Apr 17 10:59:47 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:48 2015 -0700

----------------------------------------------------------------------
 .../security/authentication/server/AuthenticationFilter.java   | 6 ++++++
 hadoop-common-project/hadoop-common/CHANGES.txt                | 3 +++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f846cc5a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
index 684e91c..203ee41 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
@@ -187,6 +187,7 @@ public class AuthenticationFilter implements Filter {
   private long validity;
   private String cookieDomain;
   private String cookiePath;
+  private boolean isInitializedByTomcat;
 
   /**
    * <p>Initializes the authentication filter and signer secret provider.</p>
@@ -252,6 +253,7 @@ public class AuthenticationFilter implements Filter {
         secretProvider = constructSecretProvider(
             filterConfig.getServletContext(),
             config, false);
+        isInitializedByTomcat = true;
       } catch (Exception ex) {
         throw new ServletException(ex);
       }
@@ -380,6 +382,10 @@ public class AuthenticationFilter implements Filter {
       authHandler.destroy();
       authHandler = null;
     }
+    if (secretProvider != null && isInitializedByTomcat) {
+      secretProvider.destroy();
+      secretProvider = null;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f846cc5a/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index bb2eca5..a48baf8 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1248,6 +1248,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11815. HttpServer2 should destroy SignerSecretProvider when it
     stops. (Rohith via wheat9)
 
+    HADOOP-11837. AuthenticationFilter should destroy SignerSecretProvider in
+    Tomcat deployments. (Bowen Zhang via wheat9)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[34/50] [abbrv] hadoop git commit: YARN-3354. Add node label expression in ContainerTokenIdentifier to support RM recovery. Contributed by Wangda Tan

Posted by zj...@apache.org.
YARN-3354. Add node label expression in ContainerTokenIdentifier to support RM recovery. Contributed by Wangda Tan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/25366bc7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/25366bc7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/25366bc7

Branch: refs/heads/YARN-2928
Commit: 25366bc7847899971f170e110ae5a7c83982d4e1
Parents: 1aa894e
Author: Jian He <ji...@apache.org>
Authored: Wed Apr 15 13:57:06 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:45 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/security/ContainerTokenIdentifier.java |  21 +-
 .../main/proto/server/yarn_security_token.proto |   1 +
 .../api/protocolrecords/NMContainerStatus.java  |  22 +-
 .../impl/pb/NMContainerStatusPBImpl.java        |  21 +-
 .../yarn_server_common_service_protos.proto     |   1 +
 .../container/ContainerImpl.java                |   7 +-
 .../containermanager/TestContainerManager.java  |   2 +-
 .../rmcontainer/RMContainer.java                |   2 +
 .../rmcontainer/RMContainerImpl.java            |  26 +-
 .../scheduler/AbstractYarnScheduler.java        |   2 +-
 .../scheduler/SchedulerApplicationAttempt.java  |  13 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |   7 +-
 .../security/RMContainerTokenSecretManager.java |   7 +-
 .../server/resourcemanager/TestRMRestart.java   |  13 +-
 .../capacity/TestContainerAllocation.java       |   5 +-
 ...TestWorkPreservingRMRestartForNodeLabel.java | 282 +++++++++++++++++++
 17 files changed, 408 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8b08f98..1a58988 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -140,6 +140,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3326. Support RESTful API for getLabelsToNodes. (Naganarasimha G R
     via ozawa)
 
+    YARN-3354. Add node label expression in ContainerTokenIdentifier to support
+    RM recovery. (Wangda Tan via jianhe)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
index 593bfc3..9a60d01 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.ContainerTokenIdentifierProto;
 
 import com.google.protobuf.TextFormat;
@@ -64,13 +65,14 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
       String hostName, String appSubmitter, Resource r, long expiryTimeStamp,
       int masterKeyId, long rmIdentifier, Priority priority, long creationTime) {
     this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
-        rmIdentifier, priority, creationTime, null);
+        rmIdentifier, priority, creationTime, null,
+        CommonNodeLabelsManager.NO_LABEL);
   }
 
   public ContainerTokenIdentifier(ContainerId containerID, String hostName,
       String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
       long rmIdentifier, Priority priority, long creationTime,
-      LogAggregationContext logAggregationContext) {
+      LogAggregationContext logAggregationContext, String nodeLabelExpression) {
     ContainerTokenIdentifierProto.Builder builder = 
         ContainerTokenIdentifierProto.newBuilder();
     if (containerID != null) {
@@ -93,6 +95,11 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
       builder.setLogAggregationContext(
           ((LogAggregationContextPBImpl)logAggregationContext).getProto());
     }
+    
+    if (nodeLabelExpression != null) {
+      builder.setNodeLabelExpression(nodeLabelExpression);
+    }
+    
     proto = builder.build();
   }
 
@@ -186,6 +193,16 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     return UserGroupInformation.createRemoteUser(
         containerId);
   }
+  
+  /**
+   * Get the node-label-expression in the original ResourceRequest
+   */
+  public String getNodeLabelExpression() {
+    if (proto.hasNodeLabelExpression()) {
+      return proto.getNodeLabelExpression();
+    }
+    return CommonNodeLabelsManager.NO_LABEL;
+  }
 
   // TODO: Needed?
   @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto
index 317032d..d1bef21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto
@@ -49,6 +49,7 @@ message ContainerTokenIdentifierProto {
   optional PriorityProto priority = 8;
   optional int64 creationTime = 9;
   optional LogAggregationContextProto logAggregationContext = 10;
+  optional string nodeLabelExpression = 11;
 }
 
 message ClientToAMTokenIdentifierProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
index 2f8f92d..4067c11 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -31,11 +32,21 @@ import org.apache.hadoop.yarn.util.Records;
  * inside YARN and by end-users.
  */
 public abstract class NMContainerStatus {
-
+  
+  // Used by tests only
   public static NMContainerStatus newInstance(ContainerId containerId,
       ContainerState containerState, Resource allocatedResource,
       String diagnostics, int containerExitStatus, Priority priority,
       long creationTime) {
+    return newInstance(containerId, containerState, allocatedResource,
+        diagnostics, containerExitStatus, priority, creationTime,
+        CommonNodeLabelsManager.NO_LABEL);
+  }
+
+  public static NMContainerStatus newInstance(ContainerId containerId,
+      ContainerState containerState, Resource allocatedResource,
+      String diagnostics, int containerExitStatus, Priority priority,
+      long creationTime, String nodeLabelExpression) {
     NMContainerStatus status =
         Records.newRecord(NMContainerStatus.class);
     status.setContainerId(containerId);
@@ -45,6 +56,7 @@ public abstract class NMContainerStatus {
     status.setContainerExitStatus(containerExitStatus);
     status.setPriority(priority);
     status.setCreationTime(creationTime);
+    status.setNodeLabelExpression(nodeLabelExpression);
     return status;
   }
 
@@ -105,4 +117,12 @@ public abstract class NMContainerStatus {
   public abstract long getCreationTime();
 
   public abstract void setCreationTime(long creationTime);
+  
+  /**
+   * Get the node-label-expression in the original ResourceRequest
+   */
+  public abstract String getNodeLabelExpression();
+
+  public abstract void setNodeLabelExpression(
+      String nodeLabelExpression);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
index 86e1d97..624b89b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
@@ -207,6 +208,25 @@ public class NMContainerStatusPBImpl extends NMContainerStatus {
     maybeInitBuilder();
     builder.setCreationTime(creationTime);
   }
+  
+  @Override
+  public String getNodeLabelExpression() {
+    NMContainerStatusProtoOrBuilder p = viaProto ? proto : builder;
+    if (p.hasNodeLabelExpression()) {
+      return p.getNodeLabelExpression();
+    }
+    return CommonNodeLabelsManager.NO_LABEL;
+  }
+
+  @Override
+  public void setNodeLabelExpression(String nodeLabelExpression) {
+    maybeInitBuilder();
+    if (nodeLabelExpression == null) {
+      builder.clearNodeLabelExpression();
+      return;
+    }
+    builder.setNodeLabelExpression(nodeLabelExpression);
+  }
 
   private void mergeLocalToBuilder() {
     if (this.containerId != null
@@ -274,5 +294,4 @@ public class NMContainerStatusPBImpl extends NMContainerStatus {
   private PriorityProto convertToProtoFormat(Priority t) {
     return ((PriorityPBImpl)t).getProto();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index caada23..7615c66 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -123,6 +123,7 @@ message NMContainerStatusProto {
   optional string diagnostics = 5 [default = "N/A"];
   optional int32 container_exit_status = 6;
   optional int64 creation_time = 7;
+  optional string nodeLabelExpression = 8;
 }
 
 message SCMUploaderNotifyRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 131d439..c9874a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -432,9 +432,10 @@ public class ContainerImpl implements Container {
     this.readLock.lock();
     try {
       return NMContainerStatus.newInstance(this.containerId, getCurrentState(),
-        getResource(), diagnostics.toString(), exitCode,
-        containerTokenIdentifier.getPriority(),
-        containerTokenIdentifier.getCreationTime());
+          getResource(), diagnostics.toString(), exitCode,
+          containerTokenIdentifier.getPriority(),
+          containerTokenIdentifier.getCreationTime(),
+          containerTokenIdentifier.getNodeLabelExpression());
     } finally {
       this.readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 86cc4dc..34495a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -809,7 +809,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerTokenIdentifier containerTokenIdentifier =
         new ContainerTokenIdentifier(cId, nodeId.toString(), user, r,
           System.currentTimeMillis() + 100000L, 123, rmIdentifier,
-          Priority.newInstance(0), 0, logAggregationContext);
+          Priority.newInstance(0), 0, logAggregationContext, null);
     Token containerToken =
         BuilderUtils
           .newContainerToken(nodeId, containerTokenSecretManager

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index 20087f5..21d79ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -80,4 +80,6 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
   List<ResourceRequest> getResourceRequests();
 
   String getNodeHttpAddress();
+  
+  String getNodeLabelExpression();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 38a03ae..2750d4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
@@ -153,6 +154,7 @@ public class RMContainerImpl implements RMContainer {
   private final EventHandler eventHandler;
   private final ContainerAllocationExpirer containerAllocationExpirer;
   private final String user;
+  private final String nodeLabelExpression;
 
   private Resource reservedResource;
   private NodeId reservedNode;
@@ -162,17 +164,24 @@ public class RMContainerImpl implements RMContainer {
   private ContainerStatus finishedStatus;
   private boolean isAMContainer;
   private List<ResourceRequest> resourceRequests;
-
+  
   public RMContainerImpl(Container container,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext) {
     this(container, appAttemptId, nodeId, user, rmContext, System
-      .currentTimeMillis());
+        .currentTimeMillis(), "");
+  }
+
+  public RMContainerImpl(Container container,
+      ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
+      RMContext rmContext, String nodeLabelExpression) {
+    this(container, appAttemptId, nodeId, user, rmContext, System
+      .currentTimeMillis(), nodeLabelExpression);
   }
 
   public RMContainerImpl(Container container,
-      ApplicationAttemptId appAttemptId, NodeId nodeId,
-      String user, RMContext rmContext, long creationTime) {
+      ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
+      RMContext rmContext, long creationTime, String nodeLabelExpression) {
     this.stateMachine = stateMachineFactory.make(this);
     this.containerId = container.getId();
     this.nodeId = nodeId;
@@ -185,6 +194,7 @@ public class RMContainerImpl implements RMContainer {
     this.containerAllocationExpirer = rmContext.getContainerAllocationExpirer();
     this.isAMContainer = false;
     this.resourceRequests = null;
+    this.nodeLabelExpression = nodeLabelExpression;
 
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     this.readLock = lock.readLock();
@@ -597,4 +607,12 @@ public class RMContainerImpl implements RMContainer {
       readLock.unlock();
     }
   }
+
+  @Override
+  public String getNodeLabelExpression() {
+    if (nodeLabelExpression == null) {
+      return RMNodeLabelsManager.NO_LABEL;
+    }
+    return nodeLabelExpression;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index e1f94cf..6699b05 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -408,7 +408,7 @@ public abstract class AbstractYarnScheduler
     RMContainer rmContainer =
         new RMContainerImpl(container, attemptId, node.getNodeID(),
           applications.get(attemptId.getApplicationId()).getUser(), rmContext,
-          status.getCreationTime());
+          status.getCreationTime(), status.getNodeLabelExpression());
     return rmContainer;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index fccf766..4823390 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Scheduli
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultiset;
 import com.google.common.collect.Multiset;
@@ -466,9 +467,10 @@ public class SchedulerApplicationAttempt {
       try {
         // create container token and NMToken altogether.
         container.setContainerToken(rmContext.getContainerTokenSecretManager()
-          .createContainerToken(container.getId(), container.getNodeId(),
-            getUser(), container.getResource(), container.getPriority(),
-            rmContainer.getCreationTime(), this.logAggregationContext));
+            .createContainerToken(container.getId(), container.getNodeId(),
+                getUser(), container.getResource(), container.getPriority(),
+                rmContainer.getCreationTime(), this.logAggregationContext,
+                rmContainer.getNodeLabelExpression()));
         NMToken nmToken =
             rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
               getApplicationAttemptId(), container);
@@ -703,4 +705,9 @@ public class SchedulerApplicationAttempt {
         this.attemptResourceUsage, nodePartition, cluster,
         schedulingMode);
   }
+  
+  @VisibleForTesting
+  public ResourceUsage getAppAttemptResourceUsage() {
+    return this.attemptResourceUsage;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index e041389..3085d93 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -146,9 +146,10 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
     
     // Create RMContainer
-    RMContainer rmContainer = new RMContainerImpl(container, this
-        .getApplicationAttemptId(), node.getNodeID(),
-        appSchedulingInfo.getUser(), this.rmContext);
+    RMContainer rmContainer =
+        new RMContainerImpl(container, this.getApplicationAttemptId(),
+            node.getNodeID(), appSchedulingInfo.getUser(), this.rmContext,
+            request.getNodeLabelExpression());
 
     // Add it to allContainers list.
     newlyAllocatedContainers.add(rmContainer);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
index 1595d17..1c0533d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
@@ -179,7 +179,7 @@ public class RMContainerTokenSecretManager extends
       String appSubmitter, Resource capability, Priority priority,
       long createTime) {
     return createContainerToken(containerId, nodeId, appSubmitter, capability,
-      priority, createTime, null);
+      priority, createTime, null, null);
   }
 
   /**
@@ -196,7 +196,8 @@ public class RMContainerTokenSecretManager extends
    */
   public Token createContainerToken(ContainerId containerId, NodeId nodeId,
       String appSubmitter, Resource capability, Priority priority,
-      long createTime, LogAggregationContext logAggregationContext) {
+      long createTime, LogAggregationContext logAggregationContext,
+      String nodeLabelExpression) {
     byte[] password;
     ContainerTokenIdentifier tokenIdentifier;
     long expiryTimeStamp =
@@ -210,7 +211,7 @@ public class RMContainerTokenSecretManager extends
             appSubmitter, capability, expiryTimeStamp, this.currentMasterKey
               .getMasterKey().getKeyId(),
             ResourceManager.getClusterTimeStamp(), priority, createTime,
-            logAggregationContext);
+            logAggregationContext, nodeLabelExpression);
       password = this.createPassword(tokenIdentifier);
 
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index a0b67f6..06a8830 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -1984,14 +1984,21 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       }
     }
   }
-
+  
   public static NMContainerStatus createNMContainerStatus(
       ApplicationAttemptId appAttemptId, int id, ContainerState containerState) {
+    return createNMContainerStatus(appAttemptId, id, containerState,
+        RMNodeLabelsManager.NO_LABEL);
+  }
+
+  public static NMContainerStatus createNMContainerStatus(
+      ApplicationAttemptId appAttemptId, int id, ContainerState containerState,
+      String nodeLabelExpression) {
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
     NMContainerStatus containerReport =
         NMContainerStatus.newInstance(containerId, containerState,
-          Resource.newInstance(1024, 1), "recover container", 0,
-          Priority.newInstance(0), 0);
+            Resource.newInstance(1024, 1), "recover container", 0,
+            Priority.newInstance(0), 0, nodeLabelExpression);
     return containerReport;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
index 54ba617..47398e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
@@ -293,10 +293,11 @@ public class TestContainerAllocation {
         public Token createContainerToken(ContainerId containerId,
             NodeId nodeId, String appSubmitter, Resource capability,
             Priority priority, long createTime,
-            LogAggregationContext logAggregationContext) {
+            LogAggregationContext logAggregationContext, String nodeLabelExp) {
           numRetries++;
           return super.createContainerToken(containerId, nodeId, appSubmitter,
-            capability, priority, createTime, logAggregationContext);
+              capability, priority, createTime, logAggregationContext,
+              nodeLabelExp);
         }
       };
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25366bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestWorkPreservingRMRestartForNodeLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestWorkPreservingRMRestartForNodeLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestWorkPreservingRMRestartForNodeLabel.java
new file mode 100644
index 0000000..fc9e14a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestWorkPreservingRMRestartForNodeLabel.java
@@ -0,0 +1,282 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+public class TestWorkPreservingRMRestartForNodeLabel {
+  private Configuration conf;
+  private static final int GB = 1024; // 1024 MB
+  
+  RMNodeLabelsManager mgr;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+      ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
+    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+  }
+  
+  @SuppressWarnings("unchecked")
+  private <E> Set<E> toSet(E... elements) {
+    Set<E> set = Sets.newHashSet(elements);
+    return set;
+  }
+  
+  private void checkRMContainerLabelExpression(ContainerId containerId,
+      MockRM rm, String labelExpression) {
+    RMContainer container =
+        rm.getRMContext().getScheduler().getRMContainer(containerId);
+    Assert.assertNotNull("Cannot find RMContainer=" + containerId, container);
+    Assert.assertEquals(labelExpression,
+        container.getNodeLabelExpression());
+  }
+  
+  @SuppressWarnings("rawtypes")
+  public static void waitForNumContainersToRecover(int num, MockRM rm,
+      ApplicationAttemptId attemptId) throws Exception {
+    AbstractYarnScheduler scheduler =
+        (AbstractYarnScheduler) rm.getResourceScheduler();
+    SchedulerApplicationAttempt attempt =
+        scheduler.getApplicationAttempt(attemptId);
+    while (attempt == null) {
+      System.out.println("Wait for scheduler attempt " + attemptId
+          + " to be created");
+      Thread.sleep(200);
+      attempt = scheduler.getApplicationAttempt(attemptId);
+    }
+    while (attempt.getLiveContainers().size() < num) {
+      System.out.println("Wait for " + num
+          + " containers to recover. currently: "
+          + attempt.getLiveContainers().size());
+      Thread.sleep(200);
+    }
+  }
+  
+  private void checkAppResourceUsage(String partition, ApplicationId appId,
+      MockRM rm, int expectedMemUsage) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    FiCaSchedulerApp app =
+        cs.getSchedulerApplications().get(appId).getCurrentAppAttempt();
+    Assert.assertEquals(expectedMemUsage, app.getAppAttemptResourceUsage()
+        .getUsed(partition).getMemory());
+  }
+  
+  private void checkQueueResourceUsage(String partition, String queueName, MockRM rm, int expectedMemUsage) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    CSQueue queue = cs.getQueue(queueName);
+    Assert.assertEquals(expectedMemUsage, queue.getQueueResourceUsage()
+        .getUsed(partition).getMemory());
+  }
+
+  @Test
+  public void testWorkPreservingRestartForNodeLabel() throws Exception {
+    // This test is pretty much similar to testContainerAllocateWithLabel.
+    // Difference is, this test doesn't specify label expression in ResourceRequest,
+    // instead, it uses default queue label expression
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
+        NodeId.newInstance("h2", 0), toSet("y")));
+
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    
+    conf = TestUtils.getConfigurationWithDefaultQueueLabels(conf);
+    
+    // inject node label manager
+    MockRM rm1 =
+        new MockRM(conf,
+            memStore) {
+          @Override
+          public RMNodeLabelsManager createNodeLabelManager() {
+            return mgr;
+          }
+        };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
+    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
+    
+    ContainerId containerId;
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // request a container.
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am1.getApplicationAttemptId(), 1), rm1, "x");
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am1.getApplicationAttemptId(), 2), rm1, "x");
+
+    // launch an app to queue b1 (label = y), and check all container will
+    // be allocated in h2
+    RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // request a container.
+    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am2.getApplicationAttemptId(), 1), rm1, "y");
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am2.getApplicationAttemptId(), 2), rm1, "y");
+    
+    // launch an app to queue c1 (label = ""), and check all container will
+    // be allocated in h3
+    RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
+
+    // request a container.
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am3.getApplicationAttemptId(), 1), rm1, "");
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am3.getApplicationAttemptId(), 2), rm1, "");
+    
+    // Re-start RM
+    mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
+        NodeId.newInstance("h2", 0), toSet("y")));
+    MockRM rm2 =
+        new MockRM(conf,
+            memStore) {
+          @Override
+          public RMNodeLabelsManager createNodeLabelManager() {
+            return mgr;
+          }
+        };
+    rm2.start();
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+    nm2.setResourceTrackerService(rm2.getResourceTrackerService());
+    nm3.setResourceTrackerService(rm2.getResourceTrackerService());
+    
+    // recover app
+    NMContainerStatus app1c1 =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 1,
+          ContainerState.RUNNING, "x");
+    NMContainerStatus app1c2 =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 2,
+          ContainerState.RUNNING, "x");
+    nm1.registerNode(Arrays.asList(app1c1, app1c2), null);
+    waitForNumContainersToRecover(2, rm2, am1.getApplicationAttemptId());
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am1.getApplicationAttemptId(), 1), rm1, "x");
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am1.getApplicationAttemptId(), 2), rm1, "x");
+    
+    NMContainerStatus app2c1 =
+        TestRMRestart.createNMContainerStatus(am2.getApplicationAttemptId(), 1,
+          ContainerState.RUNNING, "y");
+    NMContainerStatus app2c2 =
+        TestRMRestart.createNMContainerStatus(am2.getApplicationAttemptId(), 2,
+          ContainerState.RUNNING, "y");
+    nm2.registerNode(Arrays.asList(app2c1, app2c2), null);
+    waitForNumContainersToRecover(2, rm2, am2.getApplicationAttemptId());
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am2.getApplicationAttemptId(), 1), rm1, "y");
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am2.getApplicationAttemptId(), 2), rm1, "y");
+    
+    NMContainerStatus app3c1 =
+        TestRMRestart.createNMContainerStatus(am3.getApplicationAttemptId(), 1,
+          ContainerState.RUNNING, "");
+    NMContainerStatus app3c2 =
+        TestRMRestart.createNMContainerStatus(am3.getApplicationAttemptId(), 2,
+          ContainerState.RUNNING, "");
+    nm3.registerNode(Arrays.asList(app3c1, app3c2), null);
+    waitForNumContainersToRecover(2, rm2, am3.getApplicationAttemptId());
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am3.getApplicationAttemptId(), 1), rm1, "");
+    checkRMContainerLabelExpression(ContainerId.newContainerId(
+        am3.getApplicationAttemptId(), 2), rm1, "");
+    
+    // Check recovered resource usage
+    checkAppResourceUsage("x", app1.getApplicationId(), rm1, 2 * GB);
+    checkAppResourceUsage("y", app2.getApplicationId(), rm1, 2 * GB);
+    checkAppResourceUsage("", app3.getApplicationId(), rm1, 2 * GB);
+    checkQueueResourceUsage("x", "a1", rm1, 2 * GB);
+    checkQueueResourceUsage("y", "b1", rm1, 2 * GB);
+    checkQueueResourceUsage("", "c1", rm1, 2 * GB);
+    checkQueueResourceUsage("x", "a", rm1, 2 * GB);
+    checkQueueResourceUsage("y", "b", rm1, 2 * GB);
+    checkQueueResourceUsage("", "c", rm1, 2 * GB);
+    checkQueueResourceUsage("x", "root", rm1, 2 * GB);
+    checkQueueResourceUsage("y", "root", rm1, 2 * GB);
+    checkQueueResourceUsage("", "root", rm1, 2 * GB);
+
+
+    rm1.close();
+    rm2.close();
+  }
+}
\ No newline at end of file


[30/50] [abbrv] hadoop git commit: HDFS-8127. NameNode Failover during HA upgrade can cause DataNode to finalize upgrade. Contributed by Jing Zhao.

Posted by zj...@apache.org.
HDFS-8127. NameNode Failover during HA upgrade can cause DataNode to finalize upgrade. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3980eecd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3980eecd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3980eecd

Branch: refs/heads/YARN-2928
Commit: 3980eecd231062a9592564397cea1d041cf08d58
Parents: 2239ba0
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Apr 14 15:05:23 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:44 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../NamenodeProtocolServerSideTranslatorPB.java |  17 +++
 .../NamenodeProtocolTranslatorPB.java           |  16 +++
 .../hadoop/hdfs/server/namenode/FSImage.java    |  15 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   4 +-
 .../hadoop/hdfs/server/namenode/NNStorage.java  |   2 +-
 .../hdfs/server/namenode/NNUpgradeUtil.java     |  52 ++++---
 .../hdfs/server/namenode/NameNodeRpcServer.java |   7 +
 .../server/namenode/ha/BootstrapStandby.java    | 139 ++++++++++++++++---
 .../hdfs/server/protocol/NamenodeProtocol.java  |   7 +
 .../src/main/proto/NamenodeProtocol.proto       |  16 +++
 .../ha/TestBootstrapStandbyWithQJM.java         |  89 +++++++++++-
 .../namenode/ha/TestDFSUpgradeWithHA.java       |   5 +-
 13 files changed, 321 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 274e9cb..690056d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -517,6 +517,9 @@ Release 2.7.1 - UNRELEASED
 
   BUG FIXES
 
+    HDFS-8127. NameNode Failover during HA upgrade can cause DataNode to
+    finalize upgrade. (jing9)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
index 284808e..dbacded 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
@@ -36,6 +37,8 @@ import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecen
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto;
@@ -222,4 +225,18 @@ public class NamenodeProtocolServerSideTranslatorPB implements
     return VersionResponseProto.newBuilder()
         .setInfo(PBHelper.convert(info)).build();
   }
+
+  @Override
+  public IsUpgradeFinalizedResponseProto isUpgradeFinalized(
+      RpcController controller, IsUpgradeFinalizedRequestProto request)
+      throws ServiceException {
+    boolean isUpgradeFinalized;
+    try {
+      isUpgradeFinalized = impl.isUpgradeFinalized();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return IsUpgradeFinalizedResponseProto.newBuilder()
+        .setIsUpgradeFinalized(isUpgradeFinalized).build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
index 98e99f0..82c5c4c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto;
@@ -34,6 +35,8 @@ import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksReq
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto;
@@ -232,4 +235,17 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
         RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(NamenodeProtocolPB.class), methodName);
   }
+
+  @Override
+  public boolean isUpgradeFinalized() throws IOException {
+    IsUpgradeFinalizedRequestProto req = IsUpgradeFinalizedRequestProto
+        .newBuilder().build();
+    try {
+      IsUpgradeFinalizedResponseProto response = rpcProxy.isUpgradeFinalized(
+          NULL_CONTROLLER, req);
+      return response.getIsUpgradeFinalized();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 5665084..433b05a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -212,7 +212,7 @@ public class FSImage implements Closeable {
     // check whether all is consistent before transitioning.
     Map<StorageDirectory, StorageState> dataDirStates = 
              new HashMap<StorageDirectory, StorageState>();
-    boolean isFormatted = recoverStorageDirs(startOpt, dataDirStates);
+    boolean isFormatted = recoverStorageDirs(startOpt, storage, dataDirStates);
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("Data dir states:\n  " +
@@ -301,8 +301,9 @@ public class FSImage implements Closeable {
    * @param dataDirStates output of storage directory states
    * @return true if there is at least one valid formatted storage directory
    */
-  private boolean recoverStorageDirs(StartupOption startOpt,
-      Map<StorageDirectory, StorageState> dataDirStates) throws IOException {
+  public static boolean recoverStorageDirs(StartupOption startOpt,
+      NNStorage storage, Map<StorageDirectory, StorageState> dataDirStates)
+      throws IOException {
     boolean isFormatted = false;
     // This loop needs to be over all storage dirs, even shared dirs, to make
     // sure that we properly examine their state, but we make sure we don't
@@ -352,7 +353,7 @@ public class FSImage implements Closeable {
   }
 
   /** Check if upgrade is in progress. */
-  void checkUpgrade(FSNamesystem target) throws IOException {
+  public static void checkUpgrade(NNStorage storage) throws IOException {
     // Upgrade or rolling upgrade is allowed only if there are 
     // no previous fs states in any of the local directories
     for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
@@ -364,6 +365,10 @@ public class FSImage implements Closeable {
     }
   }
 
+  void checkUpgrade() throws IOException {
+    checkUpgrade(storage);
+  }
+
   /**
    * @return true if there is rollback fsimage (for rolling upgrade) in NameNode
    * directory.
@@ -381,7 +386,7 @@ public class FSImage implements Closeable {
   }
 
   void doUpgrade(FSNamesystem target) throws IOException {
-    checkUpgrade(target);
+    checkUpgrade();
 
     // load the latest image
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 35abb5a..a587443 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7504,7 +7504,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void startRollingUpgradeInternal(long startTime)
       throws IOException {
     checkRollingUpgrade("start rolling upgrade");
-    getFSImage().checkUpgrade(this);
+    getFSImage().checkUpgrade();
     setRollingUpgradeInfo(false, startTime);
   }
 
@@ -7521,7 +7521,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           + "in order to create namespace image.");
     }
     checkRollingUpgrade("start rolling upgrade");
-    getFSImage().checkUpgrade(this);
+    getFSImage().checkUpgrade();
     // in non-HA setup, we do an extra checkpoint to generate a rollback image
     getFSImage().saveNamespace(this, NameNodeFile.IMAGE_ROLLBACK, null);
     LOG.info("Successfully saved namespace for preparing rolling upgrade.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index cb5252e..dbb2c50 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -975,7 +975,7 @@ public class NNStorage extends Storage implements Closeable,
   }
 
   /** Validate and set block pool ID */
-  void setBlockPoolID(String bpid) {
+  public void setBlockPoolID(String bpid) {
     blockpoolID = bpid;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
index 8e105bd..ee651fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.io.IOUtils;
 
-abstract class NNUpgradeUtil {
+public abstract class NNUpgradeUtil {
   
   private static final Log LOG = LogFactory.getLog(NNUpgradeUtil.class);
   
@@ -111,25 +111,12 @@ abstract class NNUpgradeUtil {
   static void doPreUpgrade(Configuration conf, StorageDirectory sd)
       throws IOException {
     LOG.info("Starting upgrade of storage directory " + sd.getRoot());
-    File curDir = sd.getCurrentDir();
-    File prevDir = sd.getPreviousDir();
-    final File tmpDir = sd.getPreviousTmp();
-
-    Preconditions.checkState(curDir.exists(),
-        "Current directory must exist for preupgrade.");
-    Preconditions.checkState(!prevDir.exists(),
-        "Previous directory must not exist for preupgrade.");
-    Preconditions.checkState(!tmpDir.exists(),
-        "Previous.tmp directory must not exist for preupgrade."
-            + "Consider restarting for recovery.");
 
     // rename current to tmp
-    NNStorage.rename(curDir, tmpDir);
-    
-    if (!curDir.mkdir()) {
-      throw new IOException("Cannot create directory " + curDir);
-    }
+    renameCurToTmp(sd);
 
+    final File curDir = sd.getCurrentDir();
+    final File tmpDir = sd.getPreviousTmp();
     List<String> fileNameList = IOUtils.listDirectory(tmpDir, new FilenameFilter() {
       @Override
       public boolean accept(File dir, String name) {
@@ -159,6 +146,31 @@ abstract class NNUpgradeUtil {
       in.close();
     }
   }
+
+  /**
+   * Rename the existing current dir to previous.tmp, and create a new empty
+   * current dir.
+   */
+  public static void renameCurToTmp(StorageDirectory sd) throws IOException {
+    File curDir = sd.getCurrentDir();
+    File prevDir = sd.getPreviousDir();
+    final File tmpDir = sd.getPreviousTmp();
+
+    Preconditions.checkState(curDir.exists(),
+        "Current directory must exist for preupgrade.");
+    Preconditions.checkState(!prevDir.exists(),
+        "Previous directory must not exist for preupgrade.");
+    Preconditions.checkState(!tmpDir.exists(),
+        "Previous.tmp directory must not exist for preupgrade."
+            + "Consider restarting for recovery.");
+
+    // rename current to tmp
+    NNStorage.rename(curDir, tmpDir);
+
+    if (!curDir.mkdir()) {
+      throw new IOException("Cannot create directory " + curDir);
+    }
+  }
   
   /**
    * Perform the upgrade of the storage dir to the given storage info. The new
@@ -169,14 +181,14 @@ abstract class NNUpgradeUtil {
    * @param storage info about the new upgraded versions.
    * @throws IOException in the event of error
    */
-  static void doUpgrade(StorageDirectory sd, Storage storage) throws
-      IOException {
+  public static void doUpgrade(StorageDirectory sd, Storage storage)
+      throws IOException {
     LOG.info("Performing upgrade of storage directory " + sd.getRoot());
     try {
       // Write the version file, since saveFsImage only makes the
       // fsimage_<txid>, and the directory is otherwise empty.
       storage.writeProperties(sd);
-      
+
       File prevDir = sd.getPreviousDir();
       File tmpDir = sd.getPreviousTmp();
       Preconditions.checkState(!prevDir.exists(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 7ab8b86..af19858 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1130,6 +1130,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.checkSuperuserPrivilege();
     return namesystem.getEditLog().getEditLogManifest(sinceTxId);
   }
+
+  @Override // NamenodeProtocol
+  public boolean isUpgradeFinalized() throws IOException {
+    checkNNStartup();
+    namesystem.checkSuperuserPrivilege();
+    return namesystem.isUpgradeFinalized();
+  }
     
   @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
index deda30f..9d8f2f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
@@ -26,7 +26,10 @@ import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedAction;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -38,12 +41,16 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.NNUpgradeUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@@ -142,11 +149,12 @@ public class BootstrapStandby implements Tool, Configurable {
   }
   
   private int doRun() throws IOException {
-
     NamenodeProtocol proxy = createNNProtocolProxy();
     NamespaceInfo nsInfo;
+    boolean isUpgradeFinalized;
     try {
       nsInfo = proxy.versionRequest();
+      isUpgradeFinalized = proxy.isUpgradeFinalized();
     } catch (IOException ioe) {
       LOG.fatal("Unable to fetch namespace information from active NN at " +
           otherIpcAddr + ": " + ioe.getMessage());
@@ -163,7 +171,6 @@ public class BootstrapStandby implements Tool, Configurable {
       return ERR_CODE_INVALID_VERSION;
     }
 
-    
     System.out.println(
         "=====================================================\n" +
         "About to bootstrap Standby ID " + nnId + " from:\n" +
@@ -175,35 +182,133 @@ public class BootstrapStandby implements Tool, Configurable {
         "            Block pool ID: " + nsInfo.getBlockPoolID() + "\n" +
         "               Cluster ID: " + nsInfo.getClusterID() + "\n" +
         "           Layout version: " + nsInfo.getLayoutVersion() + "\n" +
+        "       isUpgradeFinalized: " + isUpgradeFinalized + "\n" +
         "=====================================================");
-
-    long imageTxId = proxy.getMostRecentCheckpointTxId();
-    long curTxId = proxy.getTransactionID();
     
     NNStorage storage = new NNStorage(conf, dirsToFormat, editUrisToFormat);
-    
+
+    if (!isUpgradeFinalized) {
+      // the remote NameNode is in upgrade state, this NameNode should also
+      // create the previous directory. First prepare the upgrade and rename
+      // the current dir to previous.tmp.
+      LOG.info("The active NameNode is in Upgrade. " +
+          "Prepare the upgrade for the standby NameNode as well.");
+      if (!doPreUpgrade(storage, nsInfo)) {
+        return ERR_CODE_ALREADY_FORMATTED;
+      }
+    } else if (!format(storage, nsInfo)) { // prompt the user to format storage
+      return ERR_CODE_ALREADY_FORMATTED;
+    }
+
+    // download the fsimage from active namenode
+    int download = downloadImage(storage, proxy);
+    if (download != 0) {
+      return download;
+    }
+
+    // finish the upgrade: rename previous.tmp to previous
+    if (!isUpgradeFinalized) {
+      doUpgrade(storage);
+    }
+    return 0;
+  }
+
+  /**
+   * Iterate over all the storage directories, checking if it should be
+   * formatted. Format the storage if necessary and allowed by the user.
+   * @return True if formatting is processed
+   */
+  private boolean format(NNStorage storage, NamespaceInfo nsInfo)
+      throws IOException {
     // Check with the user before blowing away data.
-    if (!Storage.confirmFormat(storage.dirIterable(null),
-            force, interactive)) {
+    if (!Storage.confirmFormat(storage.dirIterable(null), force, interactive)) {
       storage.close();
-      return ERR_CODE_ALREADY_FORMATTED;
+      return false;
+    } else {
+      // Format the storage (writes VERSION file)
+      storage.format(nsInfo);
+      return true;
     }
-    
-    // Format the storage (writes VERSION file)
-    storage.format(nsInfo);
+  }
+
+  /**
+   * This is called when using bootstrapStandby for HA upgrade. The SBN should
+   * also create previous directory so that later when it starts, it understands
+   * that the cluster is in the upgrade state. This function renames the old
+   * current directory to previous.tmp.
+   */
+  private boolean doPreUpgrade(NNStorage storage, NamespaceInfo nsInfo)
+      throws IOException {
+    boolean isFormatted = false;
+    Map<StorageDirectory, StorageState> dataDirStates =
+        new HashMap<>();
+    try {
+      isFormatted = FSImage.recoverStorageDirs(StartupOption.UPGRADE, storage,
+          dataDirStates);
+      if (dataDirStates.values().contains(StorageState.NOT_FORMATTED)) {
+        // recoverStorageDirs returns true if there is a formatted directory
+        isFormatted = false;
+        System.err.println("The original storage directory is not formatted.");
+      }
+    } catch (InconsistentFSStateException e) {
+      // if the storage is in a bad state,
+      LOG.warn("The storage directory is in an inconsistent state", e);
+    } finally {
+      storage.unlockAll();
+    }
+
+    // if there is InconsistentFSStateException or the storage is not formatted,
+    // format the storage. Although this format is done through the new
+    // software, since in HA setup the SBN is rolled back through
+    // "-bootstrapStandby", we should still be fine.
+    if (!isFormatted && !format(storage, nsInfo)) {
+      return false;
+    }
+
+    // make sure there is no previous directory
+    FSImage.checkUpgrade(storage);
+    // Do preUpgrade for each directory
+    for (Iterator<StorageDirectory> it = storage.dirIterator(false);
+         it.hasNext();) {
+      StorageDirectory sd = it.next();
+      try {
+        NNUpgradeUtil.renameCurToTmp(sd);
+      } catch (IOException e) {
+        LOG.error("Failed to move aside pre-upgrade storage " +
+            "in image directory " + sd.getRoot(), e);
+        throw e;
+      }
+    }
+    storage.setStorageInfo(nsInfo);
+    storage.setBlockPoolID(nsInfo.getBlockPoolID());
+    return true;
+  }
 
-    // Load the newly formatted image, using all of the directories (including shared
-    // edits)
+  private void doUpgrade(NNStorage storage) throws IOException {
+    for (Iterator<StorageDirectory> it = storage.dirIterator(false);
+         it.hasNext();) {
+      StorageDirectory sd = it.next();
+      NNUpgradeUtil.doUpgrade(sd, storage);
+    }
+  }
+
+  private int downloadImage(NNStorage storage, NamenodeProtocol proxy)
+      throws IOException {
+    // Load the newly formatted image, using all of the directories
+    // (including shared edits)
+    final long imageTxId = proxy.getMostRecentCheckpointTxId();
+    final long curTxId = proxy.getTransactionID();
     FSImage image = new FSImage(conf);
     try {
       image.getStorage().setStorageInfo(storage);
       image.initEditLog(StartupOption.REGULAR);
       assert image.getEditLog().isOpenForRead() :
-        "Expected edit log to be open for read";
+          "Expected edit log to be open for read";
 
       // Ensure that we have enough edits already in the shared directory to
       // start up from the last checkpoint on the active.
-      if (!skipSharedEditsCheck && !checkLogsAvailableForRead(image, imageTxId, curTxId)) {
+      if (!skipSharedEditsCheck &&
+          !checkLogsAvailableForRead(image, imageTxId, curTxId)) {
         return ERR_CODE_LOGS_UNAVAILABLE;
       }
 
@@ -211,7 +316,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
       // Download that checkpoint into our storage directories.
       MD5Hash hash = TransferFsImage.downloadImageToStorage(
-        otherHttpAddr, imageTxId, storage, true);
+          otherHttpAddr, imageTxId, storage, true);
       image.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, imageTxId,
           hash);
     } catch (IOException ioe) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
index 91f1e77..4048372 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
@@ -186,5 +186,12 @@ public interface NamenodeProtocol {
   @Idempotent
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
     throws IOException;
+
+  /**
+   * @return Whether the NameNode is in upgrade state (false) or not (true)
+   */
+  @Idempotent
+  public boolean isUpgradeFinalized() throws IOException;
+
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
index 4bcbeed..f7c1312 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
@@ -182,6 +182,16 @@ message GetEditLogManifestResponseProto {
 }
 
 /**
+ * void request
+ */
+message IsUpgradeFinalizedRequestProto {
+}
+
+message IsUpgradeFinalizedResponseProto {
+  required bool isUpgradeFinalized = 1;
+}
+
+/**
  * Protocol used by the sub-ordinate namenode to send requests
  * the active/primary namenode.
  *
@@ -250,4 +260,10 @@ service NamenodeProtocolService {
    */
   rpc getEditLogManifest(GetEditLogManifestRequestProto) 
       returns(GetEditLogManifestResponseProto);
+
+  /**
+   * Return whether the NameNode is in upgrade state (false) or not (true)
+   */
+  rpc isUpgradeFinalized(IsUpgradeFinalizedRequestProto)
+      returns (IsUpgradeFinalizedResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
index c7756ca..ca8f563 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
@@ -18,7 +18,9 @@
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
+import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -28,18 +30,26 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableList;
+import org.mockito.internal.util.reflection.Whitebox;
 
 /**
  * Test BootstrapStandby when QJM is used for shared edits. 
  */
-public class TestBootstrapStandbyWithQJM {  
-  private MiniQJMHACluster miniQjmHaCluster;
+public class TestBootstrapStandbyWithQJM {
+  enum UpgradeState {
+    NORMAL,
+    RECOVER,
+    FORMAT
+  }
+
   private MiniDFSCluster cluster;
   private MiniJournalCluster jCluster;
   
@@ -52,7 +62,7 @@ public class TestBootstrapStandbyWithQJM {
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
 
-    miniQjmHaCluster = new MiniQJMHACluster.Builder(conf).build();
+    MiniQJMHACluster miniQjmHaCluster = new MiniQJMHACluster.Builder(conf).build();
     cluster = miniQjmHaCluster.getDfsCluster();
     jCluster = miniQjmHaCluster.getJournalCluster();
     
@@ -112,4 +122,77 @@ public class TestBootstrapStandbyWithQJM {
         ImmutableList.of(0));
     FSImageTestUtil.assertNNFilesMatch(cluster);
   }
+
+  /**
+   * Test the bootstrapstandby while the other namenode is in upgrade state.
+   * Make sure a previous directory can be created.
+   */
+  @Test
+  public void testUpgrade() throws Exception {
+    testUpgrade(UpgradeState.NORMAL);
+  }
+
+  /**
+   * Similar with testUpgrade, but rename nn1's current directory to
+   * previous.tmp before bootstrapStandby, and make sure the nn1 is recovered
+   * first then converted into upgrade state.
+   */
+  @Test
+  public void testUpgradeWithRecover() throws Exception {
+    testUpgrade(UpgradeState.RECOVER);
+  }
+
+  /**
+   * Similar with testUpgrade, but rename nn1's current directory to a random
+   * name so that it's not formatted. Make sure the nn1 is formatted and then
+   * converted into upgrade state.
+   */
+  @Test
+  public void testUpgradeWithFormat() throws Exception {
+    testUpgrade(UpgradeState.FORMAT);
+  }
+
+  private void testUpgrade(UpgradeState state) throws Exception {
+    cluster.transitionToActive(0);
+    final Configuration confNN1 = cluster.getConfiguration(1);
+
+    final File current = cluster.getNameNode(1).getFSImage().getStorage()
+        .getStorageDir(0).getCurrentDir();
+    final File tmp = cluster.getNameNode(1).getFSImage().getStorage()
+        .getStorageDir(0).getPreviousTmp();
+    // shut down nn1
+    cluster.shutdownNameNode(1);
+
+    // make NN0 in upgrade state
+    FSImage fsImage0 = cluster.getNameNode(0).getNamesystem().getFSImage();
+    Whitebox.setInternalState(fsImage0, "isUpgradeFinalized", false);
+
+    switch (state) {
+      case RECOVER:
+        // rename the current directory to previous.tmp in nn1
+        NNStorage.rename(current, tmp);
+        break;
+      case FORMAT:
+        // rename the current directory to a random name so it's not formatted
+        final File wrongPath = new File(current.getParentFile(), "wrong");
+        NNStorage.rename(current, wrongPath);
+        break;
+      default:
+        break;
+    }
+
+    int rc = BootstrapStandby.run(new String[] { "-force" }, confNN1);
+    assertEquals(0, rc);
+
+    // Should have copied over the namespace from the standby
+    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
+        ImmutableList.of(0));
+    FSImageTestUtil.assertNNFilesMatch(cluster);
+
+    // make sure the NN1 is in upgrade state, i.e., the previous directory has
+    // been successfully created
+    cluster.restartNameNode(1);
+    assertFalse(cluster.getNameNode(1).getNamesystem().isUpgradeFinalized());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3980eecd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
index c749431..a77b435 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster.Builder;
 import org.apache.hadoop.hdfs.qjournal.server.Journal;
-import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -594,7 +593,7 @@ public class TestDFSUpgradeWithHA {
       cluster.restartNameNode(1);
       
       checkNnPreviousDirExistence(cluster, 0, true);
-      checkNnPreviousDirExistence(cluster, 1, false);
+      checkNnPreviousDirExistence(cluster, 1, true);
       checkPreviousDirExistence(sharedDir, true);
       assertCTimesEqual(cluster);
       
@@ -671,7 +670,7 @@ public class TestDFSUpgradeWithHA {
       cluster.restartNameNode(1);
       
       checkNnPreviousDirExistence(cluster, 0, true);
-      checkNnPreviousDirExistence(cluster, 1, false);
+      checkNnPreviousDirExistence(cluster, 1, true);
       checkJnPreviousDirExistence(qjCluster, true);
       assertCTimesEqual(cluster);
       


[14/50] [abbrv] hadoop git commit: YARN-3382. Some of UserMetricsInfo metrics are incorrectly set to root queue metrics. Contributed by Rohit Agarwal

Posted by zj...@apache.org.
YARN-3382. Some of UserMetricsInfo metrics are incorrectly set to root queue metrics. Contributed by Rohit Agarwal


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/00346bac
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/00346bac
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/00346bac

Branch: refs/heads/YARN-2928
Commit: 00346bac2ca10ecc9162dae6f8bfb6fe40a5d44e
Parents: 0e60995
Author: Jian He <ji...@apache.org>
Authored: Mon Apr 13 10:53:34 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:41 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                           |  3 +++
 .../resourcemanager/webapp/dao/UserMetricsInfo.java       | 10 +++++-----
 2 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/00346bac/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3925f1c..d90d79a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -236,6 +236,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3465. Use LinkedHashMap to preserve order of resource requests. 
     (Zhihai Xu via kasha)
 
+    YARN-3382. Some of UserMetricsInfo metrics are incorrectly set to root
+    queue metrics. (Rohit Agarwal via jianhe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00346bac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java
index 73a83d7..bfa5bd2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java
@@ -63,11 +63,11 @@ public class UserMetricsInfo {
       this.userMetricsAvailable = true;
 
       this.appsSubmitted = userMetrics.getAppsSubmitted();
-      this.appsCompleted = metrics.getAppsCompleted();
-      this.appsPending = metrics.getAppsPending();
-      this.appsRunning = metrics.getAppsRunning();
-      this.appsFailed = metrics.getAppsFailed();
-      this.appsKilled = metrics.getAppsKilled();
+      this.appsCompleted = userMetrics.getAppsCompleted();
+      this.appsPending = userMetrics.getAppsPending();
+      this.appsRunning = userMetrics.getAppsRunning();
+      this.appsFailed = userMetrics.getAppsFailed();
+      this.appsKilled = userMetrics.getAppsKilled();
 
       this.runningContainers = userMetrics.getAllocatedContainers();
       this.pendingContainers = userMetrics.getPendingContainers();


[19/50] [abbrv] hadoop git commit: HADOOP-9642. Configuration to resolve environment variables via ${env.VARIABLE} references (Kengo Seki via aw)

Posted by zj...@apache.org.
HADOOP-9642. Configuration to resolve environment variables via ${env.VARIABLE} references (Kengo Seki via aw)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/737b437e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/737b437e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/737b437e

Branch: refs/heads/YARN-2928
Commit: 737b437e9104cf59395260cc06701ad29a410411
Parents: 8bb1209
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Apr 14 08:20:13 2015 +0200
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:42 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  6 +-
 .../org/apache/hadoop/conf/Configuration.java   | 55 +++++++++++++-
 .../apache/hadoop/conf/TestConfiguration.java   | 77 ++++++++++++++------
 3 files changed, 110 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/737b437e/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 0fb6e92..68913bc 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -25,7 +25,8 @@ Trunk (Unreleased)
 
   NEW FEATURES
 
-    HADOOP-6590. Add a username check for hadoop sub-commands (John Smith via aw)
+    HADOOP-6590. Add a username check for hadoop sub-commands (John Smith via
+    aw)
 
     HADOOP-11353. Add support for .hadooprc (aw)
 
@@ -41,6 +42,9 @@ Trunk (Unreleased)
 
     HADOOP-11565. Add --slaves shell option (aw)
 
+    HADOOP-9642. Configuration to resolve environment variables via
+    ${env.VARIABLE} references (Kengo Seki via aw)
+
   IMPROVEMENTS
 
     HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution

http://git-wip-us.apache.org/repos/asf/hadoop/blob/737b437e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index 8a312ff..7c25e6c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -146,6 +146,8 @@ import com.google.common.base.Preconditions;
  * available properties are:<ol>
  * <li>Other properties defined in this Configuration; and, if a name is
  * undefined here,</li>
+ * <li>Environment variables in {@link System#getenv()} if a name starts with
+ * "env.", or</li>
  * <li>Properties in {@link System#getProperties()}.</li>
  * </ol>
  *
@@ -160,13 +162,25 @@ import com.google.common.base.Preconditions;
  *  &lt;property&gt;
  *    &lt;name&gt;tempdir&lt;/name&gt;
  *    &lt;value&gt;${<i>basedir</i>}/tmp&lt;/value&gt;
- *  &lt;/property&gt;</pre></tt>
+ *  &lt;/property&gt;
+ *
+ *  &lt;property&gt;
+ *    &lt;name&gt;otherdir&lt;/name&gt;
+ *    &lt;value&gt;${<i>env.BASE_DIR</i>}/other&lt;/value&gt;
+ *  &lt;/property&gt;
+ *  </pre></tt>
  *
- * When <tt>conf.get("tempdir")</tt> is called, then <tt>${<i>basedir</i>}</tt>
+ * <p>When <tt>conf.get("tempdir")</tt> is called, then <tt>${<i>basedir</i>}</tt>
  * will be resolved to another property in this Configuration, while
  * <tt>${<i>user.name</i>}</tt> would then ordinarily be resolved to the value
  * of the System property with that name.
- * By default, warnings will be given to any deprecated configuration 
+ * <p>When <tt>conf.get("otherdir")</tt> is called, then <tt>${<i>env.BASE_DIR</i>}</tt>
+ * will be resolved to the value of the <tt>${<i>BASE_DIR</i>}</tt> environment variable.
+ * It supports <tt>${<i>env.NAME:-default</i>}</tt> and <tt>${<i>env.NAME-default</i>}</tt> notations.
+ * The former is resolved to “default” if <tt>${<i>NAME</i>}</tt> environment variable is undefined
+ * or its value is empty.
+ * The latter behaves the same way only if <tt>${<i>NAME</i>}</tt> is undefined.
+ * <p>By default, warnings will be given to any deprecated configuration 
  * parameters and these are suppressible by configuring
  * <tt>log4j.logger.org.apache.hadoop.conf.Configuration.deprecation</tt> in
  * log4j.properties file.
@@ -915,6 +929,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * Attempts to repeatedly expand the value {@code expr} by replacing the
    * left-most substring of the form "${var}" in the following precedence order
    * <ol>
+   *   <li>by the value of the environment variable "var" if defined</li>
    *   <li>by the value of the Java system property "var" if defined</li>
    *   <li>by the value of the configuration key "var" if defined</li>
    * </ol>
@@ -947,7 +962,31 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           varBounds[SUB_END_IDX]);
       String val = null;
       try {
-        val = System.getProperty(var);
+        if (var.startsWith("env.") && 4 < var.length()) {
+          String v = var.substring(4);
+          int i = 0;
+          for (; i < v.length(); i++) {
+            char c = v.charAt(i);
+            if (c == ':' && i < v.length() - 1 && v.charAt(i + 1) == '-') {
+              val = getenv(v.substring(0, i));
+              if (val == null || val.length() == 0) {
+                val = v.substring(i + 2);
+              }
+              break;
+            } else if (c == '-') {
+              val = getenv(v.substring(0, i));
+              if (val == null) {
+                val = v.substring(i + 1);
+              }
+              break;
+            }
+          }
+          if (i == v.length()) {
+            val = getenv(v);
+          }
+        } else {
+          val = getProperty(var);
+        }
       } catch(SecurityException se) {
         LOG.warn("Unexpected SecurityException in Configuration", se);
       }
@@ -979,6 +1018,14 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
                                     + MAX_SUBST + " " + expr);
   }
   
+  String getenv(String name) {
+    return System.getenv(name);
+  }
+
+  String getProperty(String key) {
+    return System.getProperty(key);
+  }
+
   /**
    * Get the value of the <code>name</code> property, <code>null</code> if
    * no such property exists. If the key is deprecated, it returns the value of

http://git-wip-us.apache.org/repos/asf/hadoop/blob/737b437e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
index a367553..ec6c964 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.mockito.Mockito;
 
 public class TestConfiguration extends TestCase {
 
@@ -148,46 +149,77 @@ public class TestConfiguration extends TestCase {
   }
 
   public void testVariableSubstitution() throws IOException {
+    // stubbing only environment dependent functions
+    Configuration mock = Mockito.spy(conf);
+    Mockito.when(mock.getProperty("user.name")).thenReturn("hadoop_user");
+    Mockito.when(mock.getenv("FILE_NAME")).thenReturn("hello");
+
     out=new BufferedWriter(new FileWriter(CONFIG));
     startConfig();
     declareProperty("my.int", "${intvar}", "42");
     declareProperty("intvar", "42", "42");
-    declareProperty("my.base", "/tmp/${user.name}", UNSPEC);
-    declareProperty("my.file", "hello", "hello");
+    declareProperty("my.base", "/tmp/${user.name}", "/tmp/hadoop_user");
+    declareProperty("my.file", "${env.FILE_NAME}", "hello");
     declareProperty("my.suffix", ".txt", ".txt");
     declareProperty("my.relfile", "${my.file}${my.suffix}", "hello.txt");
-    declareProperty("my.fullfile", "${my.base}/${my.file}${my.suffix}", UNSPEC);
+    declareProperty("my.fullfile", "${my.base}/${my.file}${my.suffix}", "/tmp/hadoop_user/hello.txt");
     // check that undefined variables are returned as-is
     declareProperty("my.failsexpand", "a${my.undefvar}b", "a${my.undefvar}b");
     endConfig();
     Path fileResource = new Path(CONFIG);
-    conf.addResource(fileResource);
+    mock.addResource(fileResource);
 
     for (Prop p : props) {
       System.out.println("p=" + p.name);
-      String gotVal = conf.get(p.name);
-      String gotRawVal = conf.getRaw(p.name);
+      String gotVal = mock.get(p.name);
+      String gotRawVal = mock.getRaw(p.name);
       assertEq(p.val, gotRawVal);
-      if (p.expectEval == UNSPEC) {
-        // expansion is system-dependent (uses System properties)
-        // can't do exact match so just check that all variables got expanded
-        assertTrue(gotVal != null && -1 == gotVal.indexOf("${"));
-      } else {
-        assertEq(p.expectEval, gotVal);
-      }
+      assertEq(p.expectEval, gotVal);
     }
       
     // check that expansion also occurs for getInt()
-    assertTrue(conf.getInt("intvar", -1) == 42);
-    assertTrue(conf.getInt("my.int", -1) == 42);
+    assertTrue(mock.getInt("intvar", -1) == 42);
+    assertTrue(mock.getInt("my.int", -1) == 42);
+  }
+
+  public void testEnvDefault() throws IOException {
+    Configuration mock = Mockito.spy(conf);
+    Mockito.when(mock.getenv("NULL_VALUE")).thenReturn(null);
+    Mockito.when(mock.getenv("EMPTY_VALUE")).thenReturn("");
+    Mockito.when(mock.getenv("SOME_VALUE")).thenReturn("some value");
+
+    out=new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
 
-    Map<String, String> results = conf.getValByRegex("^my.*file$");
-    assertTrue(results.keySet().contains("my.relfile"));
-    assertTrue(results.keySet().contains("my.fullfile"));
-    assertTrue(results.keySet().contains("my.file"));
-    assertEquals(-1, results.get("my.relfile").indexOf("${"));
-    assertEquals(-1, results.get("my.fullfile").indexOf("${"));
-    assertEquals(-1, results.get("my.file").indexOf("${"));
+    // if var is unbound, literal ${var} is returned
+    declareProperty("null1", "${env.NULL_VALUE}", "${env.NULL_VALUE}");
+    declareProperty("null2", "${env.NULL_VALUE-a}", "a");
+    declareProperty("null3", "${env.NULL_VALUE:-b}", "b");
+    declareProperty("empty1", "${env.EMPTY_VALUE}", "");
+    declareProperty("empty2", "${env.EMPTY_VALUE-c}", "");
+    declareProperty("empty3", "${env.EMPTY_VALUE:-d}", "d");
+    declareProperty("some1", "${env.SOME_VALUE}", "some value");
+    declareProperty("some2", "${env.SOME_VALUE-e}", "some value");
+    declareProperty("some3", "${env.SOME_VALUE:-f}", "some value");
+
+    // some edge cases
+    declareProperty("edge1", "${env.NULL_VALUE-g-h}", "g-h");
+    declareProperty("edge2", "${env.NULL_VALUE:-i:-j}", "i:-j");
+    declareProperty("edge3", "${env.NULL_VALUE-}", "");
+    declareProperty("edge4", "${env.NULL_VALUE:-}", "");
+    declareProperty("edge5", "${env.NULL_VALUE:}", "${env.NULL_VALUE:}");
+
+    endConfig();
+    Path fileResource = new Path(CONFIG);
+    mock.addResource(fileResource);
+
+    for (Prop p : props) {
+      System.out.println("p=" + p.name);
+      String gotVal = mock.get(p.name);
+      String gotRawVal = mock.getRaw(p.name);
+      assertEq(p.val, gotRawVal);
+      assertEq(p.expectEval, gotVal);
+    }
   }
 
   public void testFinalParam() throws IOException {
@@ -247,7 +279,6 @@ public class TestConfiguration extends TestCase {
     String expectEval;
   }
 
-  final String UNSPEC = null;
   ArrayList<Prop> props = new ArrayList<Prop>();
 
   void declareProperty(String name, String val, String expectEval)


[15/50] [abbrv] hadoop git commit: HDFS-8083. Move dfs.client.write.* conf from DFSConfigKeys to HdfsClientConfigKeys.Write.

Posted by zj...@apache.org.
HDFS-8083. Move dfs.client.write.* conf from DFSConfigKeys to HdfsClientConfigKeys.Write.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ad69744d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ad69744d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ad69744d

Branch: refs/heads/YARN-2928
Commit: ad69744d62de2f95b31f4d8c801ef895b2378182
Parents: 00346ba
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Apr 13 11:43:35 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:41 2015 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |  50 ++++++++-
 .../hadoop/hdfs/web/resources/EnumSetParam.java |   2 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   | 102 +++++++++++++------
 .../org/apache/hadoop/hdfs/DataStreamer.java    |   5 +-
 .../hadoop/hdfs/client/impl/DfsClientConf.java  |  52 ++++------
 .../datatransfer/ReplaceDatanodeOnFailure.java  |  24 ++---
 .../TestClientProtocolForPipelineRecovery.java  |   8 +-
 .../hadoop/hdfs/TestDFSClientExcludedNodes.java |   6 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |  11 +-
 .../namenode/TestNamenodeCapacityReport.java    |   7 +-
 .../hadoop/hdfs/util/TestByteArrayManager.java  |   6 +-
 12 files changed, 180 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index f8f615e..44d9657 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -19,6 +19,9 @@ package org.apache.hadoop.hdfs.client;
 
 /** Client configuration properties */
 public interface HdfsClientConfigKeys {
+  long SECOND = 1000L;
+  long MINUTE = 60 * SECOND;
+
   String  DFS_BLOCK_SIZE_KEY = "dfs.blocksize";
   long    DFS_BLOCK_SIZE_DEFAULT = 128*1024*1024;
   String  DFS_REPLICATION_KEY = "dfs.replication";
@@ -65,7 +68,50 @@ public interface HdfsClientConfigKeys {
     int     CONNECTION_RETRIES_DEFAULT = 0;
     String  CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = PREFIX + "connection.retries.on.timeouts";
     int     CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
-    
+  }
+  
+  interface Write {
+    String PREFIX = HdfsClientConfigKeys.PREFIX + "write.";
+
+    String  MAX_PACKETS_IN_FLIGHT_KEY = PREFIX + "max-packets-in-flight";
+    int     MAX_PACKETS_IN_FLIGHT_DEFAULT = 80;
+    String  EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY = PREFIX + "exclude.nodes.cache.expiry.interval.millis";
+    long    EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10*MINUTE;
+
+    interface ByteArrayManager {
+      String PREFIX = Write.PREFIX + "byte-array-manager.";
+
+      String  ENABLED_KEY = PREFIX + "enabled";
+      boolean ENABLED_DEFAULT = false;
+      String  COUNT_THRESHOLD_KEY = PREFIX + "count-threshold";
+      int     COUNT_THRESHOLD_DEFAULT = 128;
+      String  COUNT_LIMIT_KEY = PREFIX + "count-limit";
+      int     COUNT_LIMIT_DEFAULT = 2048;
+      String  COUNT_RESET_TIME_PERIOD_MS_KEY = PREFIX + "count-reset-time-period-ms";
+      long    COUNT_RESET_TIME_PERIOD_MS_DEFAULT = 10*SECOND;
+    }
+  }
+
+  interface BlockWrite {
+    String PREFIX = HdfsClientConfigKeys.PREFIX + "block.write.";
+
+    String  RETRIES_KEY = PREFIX + "retries";
+    int     RETRIES_DEFAULT = 3;
+    String  LOCATEFOLLOWINGBLOCK_RETRIES_KEY = PREFIX + "locateFollowingBlock.retries";
+    int     LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT = 5;
+    String  LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY = PREFIX + "locateFollowingBlock.initial.delay.ms";
+    int     LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT = 400;
+
+    interface ReplaceDatanodeOnFailure {
+      String PREFIX = BlockWrite.PREFIX + "replace-datanode-on-failure.";
+
+      String  ENABLE_KEY = PREFIX + "enable";
+      boolean ENABLE_DEFAULT = true;
+      String  POLICY_KEY = PREFIX + "policy";
+      String  POLICY_DEFAULT = "DEFAULT";
+      String  BEST_EFFORT_KEY = PREFIX + "best-effort";
+      boolean BEST_EFFORT_DEFAULT = false;
+    }
   }
 
   /** HTTP client configuration properties */
@@ -87,5 +133,5 @@ public interface HdfsClientConfigKeys {
     int     FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
     String  FAILOVER_SLEEPTIME_MAX_KEY = PREFIX + "failover.sleep.max.millis";
     int     FAILOVER_SLEEPTIME_MAX_DEFAULT =  15000;
-  }  
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java
index 06b7bc6..030abe7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java
@@ -39,7 +39,7 @@ abstract class EnumSetParam<E extends Enum<E>> extends Param<EnumSet<E>, EnumSet
   }
 
   static <E extends Enum<E>> EnumSet<E> toEnumSet(final Class<E> clazz,
-      final E... values) {
+      final E[] values) {
     final EnumSet<E> set = EnumSet.noneOf(clazz);
     set.addAll(Arrays.asList(values));
     return set;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index abbfe6a..2a26544 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -428,6 +428,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7933. fsck should also report decommissioning replicas.
     (Xiaoyu Yao via cnauroth)
 
+    HDFS-8083. Move dfs.client.write.* conf from DFSConfigKeys to 
+    HdfsClientConfigKeys.Write.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index eb3bd4f..11ca7f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -760,38 +760,88 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   @Deprecated
   public static final int     DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT
       = HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT;
-
-  
-  
   
-  public static final String  DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_KEY = "dfs.client.write.max-packets-in-flight";
-  public static final int     DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_DEFAULT = 80;
-  public static final String  DFS_CLIENT_WRITE_PACKET_SIZE_KEY = "dfs.client-write-packet-size";
-  public static final int     DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
+  // client write confs are moved to HdfsClientConfigKeys.Write 
+  @Deprecated
+  public static final String  DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_KEY
+      = HdfsClientConfigKeys.Write.MAX_PACKETS_IN_FLIGHT_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_DEFAULT
+      = HdfsClientConfigKeys.Write.MAX_PACKETS_IN_FLIGHT_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL
+      = HdfsClientConfigKeys.Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY;
+  @Deprecated
+  public static final long    DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT
+      = HdfsClientConfigKeys.Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT; // 10 minutes, in ms
+  @Deprecated
   public static final String  DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_ENABLED_KEY
-      = "dfs.client.write.byte-array-manager.enabled";
+      = HdfsClientConfigKeys.Write.ByteArrayManager.ENABLED_KEY;
+  @Deprecated
   public static final boolean DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_ENABLED_DEFAULT
-      = false;
+      = HdfsClientConfigKeys.Write.ByteArrayManager.ENABLED_DEFAULT;
+  @Deprecated
   public static final String  DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_KEY
-      = "dfs.client.write.byte-array-manager.count-threshold";
+      = HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_THRESHOLD_KEY;
+  @Deprecated
   public static final int     DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_DEFAULT
-      = 128;
+      = HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_THRESHOLD_DEFAULT;
+  @Deprecated
   public static final String  DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_LIMIT_KEY
-      = "dfs.client.write.byte-array-manager.count-limit";
+      = HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_LIMIT_KEY;
+  @Deprecated
   public static final int     DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_LIMIT_DEFAULT
-      = 2048;
+      = HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_LIMIT_DEFAULT;
+  @Deprecated
   public static final String  DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_KEY
-      = "dfs.client.write.byte-array-manager.count-reset-time-period-ms";
+      = HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_RESET_TIME_PERIOD_MS_KEY;
+  @Deprecated
   public static final long    DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_DEFAULT
-      = 10L * 1000;
-  public static final String  DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY = "dfs.client.block.write.replace-datanode-on-failure.enable";
-  public static final boolean DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_DEFAULT = true;
-  public static final String  DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY = "dfs.client.block.write.replace-datanode-on-failure.policy";
-  public static final String  DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_DEFAULT = "DEFAULT";
-  public static final String  DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_KEY = "dfs.client.block.write.replace-datanode-on-failure.best-effort";
-  public static final boolean DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_DEFAULT = false;
-  public static final String  DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL = "dfs.client.write.exclude.nodes.cache.expiry.interval.millis";
-  public static final long    DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10 * 60 * 1000; // 10 minutes, in ms
+      = HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_RESET_TIME_PERIOD_MS_DEFAULT;
+
+  // client block.write confs are moved to HdfsClientConfigKeys.BlockWrite 
+  @Deprecated
+  public static final String  DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY
+      = HdfsClientConfigKeys.BlockWrite.RETRIES_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT
+      = HdfsClientConfigKeys.BlockWrite.RETRIES_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY
+      = HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT
+      = HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY
+      = HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_DEFAULT
+      = HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY
+      = HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY;
+  @Deprecated
+  public static final boolean DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_DEFAULT
+      = HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY
+      = HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY;
+  @Deprecated
+  public static final String  DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_DEFAULT
+      = HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_KEY
+      = HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY;
+  @Deprecated
+  public static final boolean DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_DEFAULT
+      = HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT;
+
+
+  
+  
+  public static final String  DFS_CLIENT_WRITE_PACKET_SIZE_KEY = "dfs.client-write-packet-size";
+  public static final int     DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
 
   public static final String  DFS_CLIENT_SOCKET_TIMEOUT_KEY = "dfs.client.socket-timeout";
   public static final String  DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY = "dfs.client.socketcache.capacity";
@@ -822,13 +872,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_CLIENT_HTTPS_NEED_AUTH_KEY = "dfs.client.https.need-auth";
   public static final boolean DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT = false;
   // Much code in hdfs is not yet updated to use these keys.
-  public static final String  DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY = "dfs.client.block.write.locateFollowingBlock.retries";
-  public static final int     DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT = 5;
   // the initial delay (unit is ms) for locateFollowingBlock, the delay time will increase exponentially(double) for each retry.
-  public static final String  DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY = "dfs.client.block.write.locateFollowingBlock.initial.delay.ms";
-  public static final int     DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_DEFAULT = 400;
-  public static final String  DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY = "dfs.client.block.write.retries";
-  public static final int     DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT = 3;
   public static final String  DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY = "dfs.client.max.block.acquire.failures";
   public static final int     DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT = 3;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 405f775..d53e73f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -43,6 +43,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@@ -1029,7 +1030,7 @@ class DataStreamer extends Daemon {
               .append("The current failed datanode replacement policy is ")
               .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
               .append("a client may configure this via '")
-              .append(DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY)
+              .append(HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
               .append("' in its configuration.")
               .toString());
     }
@@ -1240,7 +1241,7 @@ class DataStreamer extends Daemon {
           }
           DFSClient.LOG.warn("Failed to replace datanode."
               + " Continue with the remaining datanodes since "
-              + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_KEY
+              + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY
               + " is set to true.", ioe);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
index 8ba43d6..bc1831a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
@@ -21,12 +21,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT;
@@ -41,10 +35,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIR
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
@@ -148,27 +138,28 @@ public class DfsClientConf {
     socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
         HdfsServerConstants.READ_TIMEOUT);
     /** dfs.write.packet.size is an internal config variable */
-    writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
-        DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
+    writePacketSize = conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
+        DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
     writeMaxPackets = conf.getInt(
-        DFSConfigKeys.DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_KEY,
-        DFSConfigKeys.DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_DEFAULT);
+        HdfsClientConfigKeys.Write.MAX_PACKETS_IN_FLIGHT_KEY,
+        HdfsClientConfigKeys.Write.MAX_PACKETS_IN_FLIGHT_DEFAULT);
     
     final boolean byteArrayManagerEnabled = conf.getBoolean(
-        DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_ENABLED_KEY,
-        DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_ENABLED_DEFAULT);
+        HdfsClientConfigKeys.Write.ByteArrayManager.ENABLED_KEY,
+        HdfsClientConfigKeys.Write.ByteArrayManager.ENABLED_DEFAULT);
     if (!byteArrayManagerEnabled) {
       writeByteArrayManagerConf = null;
     } else {
       final int countThreshold = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_KEY,
-          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_DEFAULT);
+          HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_THRESHOLD_KEY,
+          HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_THRESHOLD_DEFAULT);
       final int countLimit = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_LIMIT_KEY,
-          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_LIMIT_DEFAULT);
+          HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_LIMIT_KEY,
+          HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_LIMIT_DEFAULT);
       final long countResetTimePeriodMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_KEY,
-          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_DEFAULT);
+          HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_RESET_TIME_PERIOD_MS_KEY,
+          HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_RESET_TIME_PERIOD_MS_DEFAULT);
       writeByteArrayManagerConf = new ByteArrayManager.Conf(
           countThreshold, countLimit, countResetTimePeriodMs); 
     }
@@ -179,20 +170,21 @@ public class DfsClientConf {
         DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT);
     taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
     excludedNodesCacheExpiry = conf.getLong(
-        DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
-        DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
+        HdfsClientConfigKeys.Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY,
+        HdfsClientConfigKeys.Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
     prefetchSize = conf.getLong(DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
         10 * defaultBlockSize);
     numCachedConnRetry = conf.getInt(DFS_CLIENT_CACHED_CONN_RETRY_KEY,
         DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
-    numBlockWriteRetry = conf.getInt(DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
-        DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT);
+    numBlockWriteRetry = conf.getInt(
+        HdfsClientConfigKeys.BlockWrite.RETRIES_KEY,
+        HdfsClientConfigKeys.BlockWrite.RETRIES_DEFAULT);
     numBlockWriteLocateFollowingRetry = conf.getInt(
-        DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
-        DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
+        HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
+        HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
     blockWriteLocateFollowingInitialDelayMs = conf.getInt(
-        DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY,
-        DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_DEFAULT);
+        HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY,
+        HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT);
     uMask = FsPermission.getUMask(conf);
     connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
         DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
index 0f2c1ab..c69986a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 
 /**
@@ -115,7 +115,7 @@ public class ReplaceDatanodeOnFailure {
     if (policy == Policy.DISABLE) {
       throw new UnsupportedOperationException(
           "This feature is disabled.  Please refer to "
-          + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY
+          + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY
           + " configuration property.");
     }
   }
@@ -156,23 +156,23 @@ public class ReplaceDatanodeOnFailure {
   public static ReplaceDatanodeOnFailure get(final Configuration conf) {
     final Policy policy = getPolicy(conf);
     final boolean bestEffort = conf.getBoolean(
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_KEY,
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_DEFAULT);
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
     
     return new ReplaceDatanodeOnFailure(policy, bestEffort);
   }
 
   private static Policy getPolicy(final Configuration conf) {
     final boolean enabled = conf.getBoolean(
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY,
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_DEFAULT);
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_DEFAULT);
     if (!enabled) {
       return Policy.DISABLE;
     }
 
     final String policy = conf.get(
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY,
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_DEFAULT);
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_DEFAULT);
     for(int i = 1; i < Policy.values().length; i++) {
       final Policy p = Policy.values()[i];
       if (p.name().equalsIgnoreCase(policy)) {
@@ -180,7 +180,7 @@ public class ReplaceDatanodeOnFailure {
       }
     }
     throw new HadoopIllegalArgumentException("Illegal configuration value for "
-        + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY
+        + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
         + ": " + policy);
   }
 
@@ -188,13 +188,13 @@ public class ReplaceDatanodeOnFailure {
   public static void write(final Policy policy,
       final boolean bestEffort, final Configuration conf) {
     conf.setBoolean(
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
         policy != Policy.DISABLE);
     conf.set(
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
         policy.name());
     conf.setBoolean(
-        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
         bestEffort);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
index 04853bd..91de2b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
@@ -23,18 +23,16 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
-
 import org.mockito.Mockito;
-import org.mockito.stubbing.Answer;
 
 /**
  * This tests pipeline recovery related client protocol works correct or not.
@@ -130,7 +128,7 @@ public class TestClientProtocolForPipelineRecovery {
     DFSClientFaultInjector.instance = faultInjector;
     Configuration conf = new HdfsConfiguration();
 
-    conf.setInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 3);
+    conf.setInt(HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 3);
     MiniDFSCluster cluster = null;
 
     try {
@@ -147,7 +145,7 @@ public class TestClientProtocolForPipelineRecovery {
       // Read should succeed.
       FSDataInputStream in = fileSys.open(file);
       try {
-        int c = in.read();
+        in.read();
         // Test will fail with BlockMissingException if NN does not update the
         // replica state based on the latest report.
       } catch (org.apache.hadoop.hdfs.BlockMissingException bme) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
index 08842b5..c253f47 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
@@ -21,15 +21,14 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.util.ThreadUtil;
-
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -87,13 +86,12 @@ public class TestDFSClientExcludedNodes {
   public void testExcludedNodesForgiveness() throws IOException {
     // Forgive nodes in under 2.5s for this test case.
     conf.setLong(
-        DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
+        HdfsClientConfigKeys.Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY,
         2500);
     // We'll be using a 512 bytes block size just for tests
     // so making sure the checksum bytes too match it.
     conf.setInt("io.bytes.per.checksum", 512);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
-    List<DataNodeProperties> props = cluster.dataNodes;
     FileSystem fs = cluster.getFileSystem();
     Path filePath = new Path("/testForgivingExcludedNodes");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 51add1c..d16ce1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -228,7 +228,7 @@ public class TestDFSClientRetries {
   { 
     final String exceptionMsg = "Nope, not replicated yet...";
     final int maxRetries = 1; // Allow one retry (total of two calls)
-    conf.setInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, maxRetries);
+    conf.setInt(HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY, maxRetries);
     
     NamenodeProtocols mockNN = mock(NamenodeProtocols.class);
     Answer<Object> answer = new ThrowsException(new IOException()) {
@@ -1133,7 +1133,7 @@ public class TestDFSClientRetries {
   @Test
   public void testDFSClientConfigurationLocateFollowingBlockInitialDelay()
       throws Exception {
-    // test if DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY
+    // test if HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY
     // is not configured, verify DFSClient uses the default value 400.
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     try {
@@ -1143,10 +1143,11 @@ public class TestDFSClientRetries {
       assertEquals(client.getConf().
           getBlockWriteLocateFollowingInitialDelayMs(), 400);
 
-      // change DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY,
+      // change HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY,
       // verify DFSClient uses the configured value 1000.
-      conf.setInt(DFSConfigKeys.
-          DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY, 1000);
+      conf.setInt(
+          HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY,
+          1000);
       client = new DFSClient(null, nn, conf, null);
       assertEquals(client.getConf().
           getBlockWriteLocateFollowingInitialDelayMs(), 1000);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
index 6f54722..53ddbf7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -169,7 +170,7 @@ public class TestNamenodeCapacityReport {
   public void testXceiverCount() throws Exception {
     Configuration conf = new HdfsConfiguration();
     // retry one time, if close fails
-    conf.setInt(DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 1);
+    conf.setInt(HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 1);
     MiniDFSCluster cluster = null;
 
     final int nodes = 8;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad69744d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestByteArrayManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestByteArrayManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestByteArrayManager.java
index 77a68c6..1348033 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestByteArrayManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestByteArrayManager.java
@@ -34,8 +34,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.util.ByteArrayManager.Counter;
 import org.apache.hadoop.hdfs.util.ByteArrayManager.CounterMap;
 import org.apache.hadoop.hdfs.util.ByteArrayManager.FixedLengthManager;
@@ -578,9 +578,9 @@ public class TestByteArrayManager {
         new ByteArrayManager.NewByteArrayWithoutLimit(),
         new NewByteArrayWithLimit(maxArrays),
         new ByteArrayManager.Impl(new ByteArrayManager.Conf(
-            DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_DEFAULT,
+            HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_THRESHOLD_DEFAULT,
             maxArrays,
-            DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_DEFAULT))
+            HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_RESET_TIME_PERIOD_MS_DEFAULT))
     };
     final double[] avg = new double[impls.length];
 


[07/50] [abbrv] hadoop git commit: HDFS-7933. fsck should also report decommissioning replicas. Contributed by Xiaoyu Yao.

Posted by zj...@apache.org.
HDFS-7933. fsck should also report decommissioning replicas. Contributed by Xiaoyu Yao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8f7c5de4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8f7c5de4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8f7c5de4

Branch: refs/heads/YARN-2928
Commit: 8f7c5de4c1fc0095054466f0b664310621c95626
Parents: 72df815
Author: cnauroth <cn...@apache.org>
Authored: Sat Apr 11 13:23:18 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:40 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../server/blockmanagement/BlockManager.java    | 39 +++++----
 .../blockmanagement/DecommissionManager.java    |  7 +-
 .../server/blockmanagement/NumberReplicas.java  | 57 ++++++++++++--
 .../hdfs/server/namenode/NamenodeFsck.java      | 40 +++++++---
 .../hadoop/hdfs/TestClientReportBadBlock.java   |  2 +-
 .../datanode/TestReadOnlySharedStorage.java     |  2 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   | 83 +++++++++++++++++++-
 8 files changed, 193 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f7c5de4/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6e30990..134bba0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -425,6 +425,9 @@ Release 2.8.0 - UNRELEASED
     HdfsClientConfigKeys.Failover and fix typos in the dfs.http.client.*
     configuration keys.  (szetszwo)
 
+    HDFS-7933. fsck should also report decommissioning replicas.
+    (Xiaoyu Yao via cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f7c5de4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index e2c9b89..8540dc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -539,7 +539,7 @@ public class BlockManager {
     // not included in the numReplicas.liveReplicas() count
     assert containingLiveReplicasNodes.size() >= numReplicas.liveReplicas();
     int usableReplicas = numReplicas.liveReplicas() +
-                         numReplicas.decommissionedReplicas();
+                         numReplicas.decommissionedAndDecommissioning();
     
     if (block instanceof BlockInfoContiguous) {
       BlockCollection bc = ((BlockInfoContiguous) block).getBlockCollection();
@@ -550,7 +550,7 @@ public class BlockManager {
     out.print(block + ((usableReplicas > 0)? "" : " MISSING") + 
               " (replicas:" +
               " l: " + numReplicas.liveReplicas() +
-              " d: " + numReplicas.decommissionedReplicas() +
+              " d: " + numReplicas.decommissionedAndDecommissioning() +
               " c: " + numReplicas.corruptReplicas() +
               " e: " + numReplicas.excessReplicas() + ") "); 
 
@@ -730,7 +730,7 @@ public class BlockManager {
     // Remove block from replication queue.
     NumberReplicas replicas = countNodes(ucBlock);
     neededReplications.remove(ucBlock, replicas.liveReplicas(),
-        replicas.decommissionedReplicas(), getReplication(ucBlock));
+        replicas.decommissionedAndDecommissioning(), getReplication(ucBlock));
     pendingReplications.remove(ucBlock);
 
     // remove this block from the list of pending blocks to be deleted. 
@@ -1614,6 +1614,7 @@ public class BlockManager {
     DatanodeDescriptor srcNode = null;
     int live = 0;
     int decommissioned = 0;
+    int decommissioning = 0;
     int corrupt = 0;
     int excess = 0;
     
@@ -1625,9 +1626,11 @@ public class BlockManager {
       int countableReplica = storage.getState() == State.NORMAL ? 1 : 0; 
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
         corrupt += countableReplica;
-      else if (node.isDecommissionInProgress() || node.isDecommissioned())
+      else if (node.isDecommissionInProgress()) {
+        decommissioning += countableReplica;
+      } else if (node.isDecommissioned()) {
         decommissioned += countableReplica;
-      else if (excessBlocks != null && excessBlocks.contains(block)) {
+      } else if (excessBlocks != null && excessBlocks.contains(block)) {
         excess += countableReplica;
       } else {
         nodesContainingLiveReplicas.add(storage);
@@ -1667,7 +1670,8 @@ public class BlockManager {
         srcNode = node;
     }
     if(numReplicas != null)
-      numReplicas.initialize(live, decommissioned, corrupt, excess, 0);
+      numReplicas.initialize(live, decommissioned, decommissioning, corrupt,
+          excess, 0);
     return srcNode;
   }
 
@@ -1686,7 +1690,7 @@ public class BlockManager {
                                  num.liveReplicas())) {
             neededReplications.add(timedOutItems[i],
                                    num.liveReplicas(),
-                                   num.decommissionedReplicas(),
+                                   num.decommissionedAndDecommissioning(),
                                    getReplication(timedOutItems[i]));
           }
         }
@@ -2573,7 +2577,7 @@ public class BlockManager {
     short fileReplication = bc.getBlockReplication();
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
-          num.decommissionedReplicas(), fileReplication);
+          num.decommissionedAndDecommissioning(), fileReplication);
     } else {
       updateNeededReplications(storedBlock, curReplicaDelta, 0);
     }
@@ -2807,7 +2811,7 @@ public class BlockManager {
     // add to under-replicated queue if need to be
     if (isNeededReplication(block, expectedReplication, numCurrentReplica)) {
       if (neededReplications.add(block, numCurrentReplica, num
-          .decommissionedReplicas(), expectedReplication)) {
+          .decommissionedAndDecommissioning(), expectedReplication)) {
         return MisReplicationResult.UNDER_REPLICATED;
       }
     }
@@ -3221,6 +3225,7 @@ public class BlockManager {
    */
   public NumberReplicas countNodes(Block b) {
     int decommissioned = 0;
+    int decommissioning = 0;
     int live = 0;
     int corrupt = 0;
     int excess = 0;
@@ -3230,7 +3235,9 @@ public class BlockManager {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
         corrupt++;
-      } else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
+      } else if (node.isDecommissionInProgress()) {
+        decommissioning++;
+      } else if (node.isDecommissioned()) {
         decommissioned++;
       } else {
         LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
@@ -3245,7 +3252,7 @@ public class BlockManager {
         stale++;
       }
     }
-    return new NumberReplicas(live, decommissioned, corrupt, excess, stale);
+    return new NumberReplicas(live, decommissioned, decommissioning, corrupt, excess, stale);
   }
 
   /** 
@@ -3382,13 +3389,13 @@ public class BlockManager {
       int curExpectedReplicas = getReplication(block);
       if (isNeededReplication(block, curExpectedReplicas, repl.liveReplicas())) {
         neededReplications.update(block, repl.liveReplicas(), repl
-            .decommissionedReplicas(), curExpectedReplicas, curReplicasDelta,
-            expectedReplicasDelta);
+            .decommissionedAndDecommissioning(), curExpectedReplicas,
+            curReplicasDelta, expectedReplicasDelta);
       } else {
         int oldReplicas = repl.liveReplicas()-curReplicasDelta;
         int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-        neededReplications.remove(block, oldReplicas, repl.decommissionedReplicas(),
-                                  oldExpectedReplicas);
+        neededReplications.remove(block, oldReplicas,
+            repl.decommissionedAndDecommissioning(), oldExpectedReplicas);
       }
     } finally {
       namesystem.writeUnlock();
@@ -3407,7 +3414,7 @@ public class BlockManager {
       final NumberReplicas n = countNodes(block);
       if (isNeededReplication(block, expected, n.liveReplicas())) { 
         neededReplications.add(block, n.liveReplicas(),
-            n.decommissionedReplicas(), expected);
+            n.decommissionedAndDecommissioning(), expected);
       } else if (n.liveReplicas() > expected) {
         processOverReplicatedBlock(block, expected, null, null);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f7c5de4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 7f3d778..5c9aec7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -298,7 +298,8 @@ public class DecommissionManager {
     LOG.info("Block: " + block + ", Expected Replicas: "
         + curExpectedReplicas + ", live replicas: " + curReplicas
         + ", corrupt replicas: " + num.corruptReplicas()
-        + ", decommissioned replicas: " + num.decommissionedReplicas()
+        + ", decommissioned replicas: " + num.decommissioned()
+        + ", decommissioning replicas: " + num.decommissioning()
         + ", excess replicas: " + num.excessReplicas()
         + ", Is Open File: " + bc.isUnderConstruction()
         + ", Datanodes having this block: " + nodeList + ", Current Datanode: "
@@ -571,7 +572,7 @@ public class DecommissionManager {
             // Process these blocks only when active NN is out of safe mode.
             blockManager.neededReplications.add(block,
                 curReplicas,
-                num.decommissionedReplicas(),
+                num.decommissionedAndDecommissioning(),
                 bc.getBlockReplication());
           }
         }
@@ -600,7 +601,7 @@ public class DecommissionManager {
         if (bc.isUnderConstruction()) {
           underReplicatedInOpenFiles++;
         }
-        if ((curReplicas == 0) && (num.decommissionedReplicas() > 0)) {
+        if ((curReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) {
           decommissionOnlyReplicas++;
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f7c5de4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
index 9e5c8df..e567bbf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
@@ -19,26 +19,33 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 /**
  * A immutable object that stores the number of live replicas and
- * the number of decommissined Replicas.
+ * the number of decommissioned Replicas.
  */
 public class NumberReplicas {
   private int liveReplicas;
-  private int decommissionedReplicas;
+
+  // Tracks only the decommissioning replicas
+  private int decommissioning;
+  // Tracks only the decommissioned replicas
+  private int decommissioned;
   private int corruptReplicas;
   private int excessReplicas;
   private int replicasOnStaleNodes;
 
   NumberReplicas() {
-    initialize(0, 0, 0, 0, 0);
+    initialize(0, 0, 0, 0, 0, 0);
   }
 
-  NumberReplicas(int live, int decommissioned, int corrupt, int excess, int stale) {
-    initialize(live, decommissioned, corrupt, excess, stale);
+  NumberReplicas(int live, int decommissioned, int decommissioning, int corrupt,
+                 int excess, int stale) {
+    initialize(live, decommissioned, decommissioning, corrupt, excess, stale);
   }
 
-  void initialize(int live, int decommissioned, int corrupt, int excess, int stale) {
+  void initialize(int live, int decommissioned, int decommissioning,
+                  int corrupt, int excess, int stale) {
     liveReplicas = live;
-    decommissionedReplicas = decommissioned;
+    this.decommissioning = decommissioning;
+    this.decommissioned = decommissioned;
     corruptReplicas = corrupt;
     excessReplicas = excess;
     replicasOnStaleNodes = stale;
@@ -47,12 +54,46 @@ public class NumberReplicas {
   public int liveReplicas() {
     return liveReplicas;
   }
+
+  /**
+   *
+   * @return decommissioned replicas + decommissioning replicas
+   * It is deprecated by decommissionedAndDecommissioning
+   * due to its misleading name.
+   */
+  @Deprecated
   public int decommissionedReplicas() {
-    return decommissionedReplicas;
+    return decommissionedAndDecommissioning();
+  }
+
+  /**
+   *
+   * @return decommissioned and decommissioning replicas
+   */
+  public int decommissionedAndDecommissioning() {
+    return decommissioned + decommissioning;
+  }
+
+  /**
+   *
+   * @return decommissioned replicas only
+   */
+  public int decommissioned() {
+    return decommissioned;
   }
+
+  /**
+   *
+   * @return decommissioning replicas only
+   */
+  public int decommissioning() {
+    return decommissioning;
+  }
+
   public int corruptReplicas() {
     return corruptReplicas;
   }
+
   public int excessReplicas() {
     return excessReplicas;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f7c5de4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 669f68a..a8586dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -250,8 +250,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       out.println("No. of live Replica: " + numberReplicas.liveReplicas());
       out.println("No. of excess Replica: " + numberReplicas.excessReplicas());
       out.println("No. of stale Replica: " + numberReplicas.replicasOnStaleNodes());
-      out.println("No. of decommission Replica: "
-          + numberReplicas.decommissionedReplicas());
+      out.println("No. of decommissioned Replica: "
+          + numberReplicas.decommissioned());
+      out.println("No. of decommissioning Replica: "
+          + numberReplicas.decommissioning());
       out.println("No. of corrupted Replica: " + numberReplicas.corruptReplicas());
       //record datanodes that have corrupted block replica
       Collection<DatanodeDescriptor> corruptionRecord = null;
@@ -509,10 +511,16 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       NumberReplicas numberReplicas =
           namenode.getNamesystem().getBlockManager().countNodes(block.getLocalBlock());
       int liveReplicas = numberReplicas.liveReplicas();
-      res.totalReplicas += liveReplicas;
+      int decommissionedReplicas = numberReplicas.decommissioned();;
+      int decommissioningReplicas = numberReplicas.decommissioning();
+      res.decommissionedReplicas +=  decommissionedReplicas;
+      res.decommissioningReplicas += decommissioningReplicas;
+      int totalReplicas = liveReplicas + decommissionedReplicas +
+          decommissioningReplicas;
+      res.totalReplicas += totalReplicas;
       short targetFileReplication = file.getReplication();
       res.numExpectedReplicas += targetFileReplication;
-      if(liveReplicas<minReplication){
+      if(totalReplicas < minReplication){
         res.numUnderMinReplicatedBlocks++;
       }
       if (liveReplicas > targetFileReplication) {
@@ -532,10 +540,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         out.print("\n" + path + ": CORRUPT blockpool " + block.getBlockPoolId() + 
             " block " + block.getBlockName()+"\n");
       }
-      if (liveReplicas >= minReplication)
+      if (totalReplicas >= minReplication)
         res.numMinReplicatedBlocks++;
-      if (liveReplicas < targetFileReplication && liveReplicas > 0) {
-        res.missingReplicas += (targetFileReplication - liveReplicas);
+      if (totalReplicas < targetFileReplication && totalReplicas > 0) {
+        res.missingReplicas += (targetFileReplication - totalReplicas);
         res.numUnderReplicatedBlocks += 1;
         underReplicatedPerFile++;
         if (!showFiles) {
@@ -544,7 +552,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         out.println(" Under replicated " + block +
                     ". Target Replicas is " +
                     targetFileReplication + " but found " +
-                    liveReplicas + " replica(s).");
+                    liveReplicas + " live replica(s), " +
+                    decommissionedReplicas + " decommissioned replica(s) and " +
+                    decommissioningReplicas + " decommissioning replica(s).");
       }
       // verify block placement policy
       BlockPlacementStatus blockPlacementStatus = bpPolicy
@@ -561,7 +571,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                     block + ". " + blockPlacementStatus.getErrorDescription());
       }
       report.append(i + ". " + blkName + " len=" + block.getNumBytes());
-      if (liveReplicas == 0) {
+      if (totalReplicas == 0) {
         report.append(" MISSING!");
         res.addMissing(block.toString(), block.getNumBytes());
         missing++;
@@ -861,6 +871,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     long corruptBlocks = 0L;
     long excessiveReplicas = 0L;
     long missingReplicas = 0L;
+    long decommissionedReplicas = 0L;
+    long decommissioningReplicas = 0L;
     long numUnderMinReplicatedBlocks=0L;
     long numOverReplicatedBlocks = 0L;
     long numUnderReplicatedBlocks = 0L;
@@ -932,7 +944,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         res.append(" (Total open file blocks (not validated): ").append(
             totalOpenFilesBlocks).append(")");
       }
-      if (corruptFiles > 0 || numUnderMinReplicatedBlocks>0) {
+      if (corruptFiles > 0 || numUnderMinReplicatedBlocks > 0) {
         res.append("\n  ********************************");
         if(numUnderMinReplicatedBlocks>0){
           res.append("\n  UNDER MIN REPL'D BLOCKS:\t").append(numUnderMinReplicatedBlocks);
@@ -995,6 +1007,14 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
             ((float) (missingReplicas * 100) / (float) numExpectedReplicas)).append(
             " %)");
       }
+      if (decommissionedReplicas > 0) {
+        res.append("\n DecommissionedReplicas:\t").append(
+            decommissionedReplicas);
+      }
+      if (decommissioningReplicas > 0) {
+        res.append("\n DecommissioningReplicas:\t").append(
+            decommissioningReplicas);
+      }
       return res.toString();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f7c5de4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
index 0c9660e..b3580b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
@@ -192,7 +192,7 @@ public class TestClientReportBadBlock {
       verifyFirstBlockCorrupted(filePath, false);
       int expectedReplicaCount = repl-corruptBlocReplicas;
       verifyCorruptedBlockCount(filePath, expectedReplicaCount);
-      verifyFsckHealth("Target Replicas is 3 but found 1 replica");
+      verifyFsckHealth("Target Replicas is 3 but found 1 live replica");
       testFsckListCorruptFilesBlocks(filePath, 0);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f7c5de4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
index e6bf067..8f99afb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
@@ -192,7 +192,7 @@ public class TestReadOnlySharedStorage {
     assertThat(numberReplicas.liveReplicas(), is(expectedReplicas));
     assertThat(numberReplicas.excessReplicas(), is(0));
     assertThat(numberReplicas.corruptReplicas(), is(0));
-    assertThat(numberReplicas.decommissionedReplicas(), is(0));
+    assertThat(numberReplicas.decommissionedAndDecommissioning(), is(0));
     assertThat(numberReplicas.replicasOnStaleNodes(), is(0));
     
     BlockManagerTestUtil.updateState(blockManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f7c5de4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index f6bab7d..68b7e38 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -1464,4 +1464,85 @@ public class TestFsck {
       }
     }
   }
-}
+
+  /**
+   * Test for blocks on decommissioning hosts are not shown as missing
+   */
+  @Test
+  public void testFsckWithDecommissionedReplicas() throws Exception {
+
+    final short REPL_FACTOR = 1;
+    short NUM_DN = 2;
+    final long blockSize = 512;
+    final long fileSize = 1024;
+    boolean checkDecommissionInProgress = false;
+    String [] racks = {"/rack1", "/rack2"};
+    String [] hosts = {"host1", "host2"};
+
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
+
+    MiniDFSCluster cluster;
+    DistributedFileSystem dfs ;
+    cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DN).hosts(hosts)
+            .racks(racks).build();
+
+    assertNotNull("Failed Cluster Creation", cluster);
+    cluster.waitClusterUp();
+    dfs = cluster.getFileSystem();
+    assertNotNull("Failed to get FileSystem", dfs);
+
+    DFSTestUtil util = new DFSTestUtil.Builder().
+        setName(getClass().getSimpleName()).setNumFiles(1).build();
+
+    //create files
+    final String testFile = new String("/testfile");
+    final Path path = new Path(testFile);
+    util.createFile(dfs, path, fileSize, REPL_FACTOR, 1000L);
+    util.waitReplication(dfs, path, REPL_FACTOR);
+    try {
+      // make sure datanode that has replica is fine before decommission
+      String outStr = runFsck(conf, 0, true, testFile);
+      System.out.println(outStr);
+      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
+
+      // decommission datanode
+      ExtendedBlock eb = util.getFirstBlock(dfs, path);
+      DatanodeDescriptor dn = cluster.getNameNode().getNamesystem()
+          .getBlockManager().getBlockCollection(eb.getLocalBlock())
+          .getBlocks()[0].getDatanode(0);
+      cluster.getNameNode().getNamesystem().getBlockManager()
+          .getDatanodeManager().getDecomManager().startDecommission(dn);
+      String dnName = dn.getXferAddr();
+
+      // wait for decommission start
+      DatanodeInfo datanodeInfo = null;
+      int count = 0;
+      do {
+        Thread.sleep(2000);
+        for (DatanodeInfo info : dfs.getDataNodeStats()) {
+          if (dnName.equals(info.getXferAddr())) {
+            datanodeInfo = info;
+          }
+        }
+        // check the replica status should be healthy(0)
+        // instead of corruption (1) during decommissioning
+        if(!checkDecommissionInProgress && datanodeInfo != null
+            && datanodeInfo.isDecommissionInProgress()) {
+          String fsckOut = runFsck(conf, 0, true, testFile);
+          checkDecommissionInProgress =  true;
+        }
+      } while (datanodeInfo != null && !datanodeInfo.isDecommissioned());
+
+      // check the replica status should be healthy(0) after decommission
+      // is done
+      String fsckOut = runFsck(conf, 0, true, testFile);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}
\ No newline at end of file


[25/50] [abbrv] hadoop git commit: HDFS-8144. Split TestLazyPersistFiles into multiple tests. (Arpit Agarwal)

Posted by zj...@apache.org.
HDFS-8144. Split TestLazyPersistFiles into multiple tests. (Arpit Agarwal)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/89ed0e72
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/89ed0e72
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/89ed0e72

Branch: refs/heads/YARN-2928
Commit: 89ed0e728f05b15844a060a2faaca4a75a5107a2
Parents: e2cd8ed
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Apr 15 10:25:04 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:44 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../fsdataset/impl/LazyPersistTestCase.java     |   1 +
 .../fsdataset/impl/TestLazyPersistFiles.java    | 459 +------------------
 .../fsdataset/impl/TestLazyPersistPolicy.java   |  91 ++++
 .../impl/TestLazyPersistReplicaPlacement.java   | 148 ++++++
 .../impl/TestLazyPersistReplicaRecovery.java    |  75 +++
 .../datanode/fsdataset/impl/TestLazyWriter.java | 276 +++++++++++
 7 files changed, 594 insertions(+), 458 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/89ed0e72/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 690056d..5a0f6f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -434,6 +434,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8117. More accurate verification in SimulatedFSDataset: replace
     DEFAULT_DATABYTE with patterned data. (Zhe Zhang via wang)
 
+    HDFS-8144. Split TestLazyPersistFiles into multiple tests. (Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89ed0e72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index 89a70c9..baa540d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -64,6 +64,7 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
 public abstract class LazyPersistTestCase {
+  static final byte LAZY_PERSIST_POLICY_ID = (byte) 15;
 
   static {
     DFSTestUtil.setNameNodeLogLevel(Level.ALL);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89ed0e72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
index bd64cbe..30e5d26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
@@ -48,182 +48,8 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class TestLazyPersistFiles extends LazyPersistTestCase {
-  private static final byte LAZY_PERSIST_POLICY_ID = (byte) 15;
-
   private static final int THREADPOOL_SIZE = 10;
 
-  @Test
-  public void testPolicyNotSetByDefault() throws IOException {
-    startUpCluster(false, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, 0, false);
-    // Stat the file and check that the LAZY_PERSIST policy is not
-    // returned back.
-    HdfsFileStatus status = client.getFileInfo(path.toString());
-    assertThat(status.getStoragePolicy(), not(LAZY_PERSIST_POLICY_ID));
-  }
-
-  @Test
-  public void testPolicyPropagation() throws IOException {
-    startUpCluster(false, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, 0, true);
-    // Stat the file and check that the lazyPersist flag is returned back.
-    HdfsFileStatus status = client.getFileInfo(path.toString());
-    assertThat(status.getStoragePolicy(), is(LAZY_PERSIST_POLICY_ID));
-  }
-
-  @Test
-  public void testPolicyPersistenceInEditLog() throws IOException {
-    startUpCluster(false, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, 0, true);
-    cluster.restartNameNode(true);
-
-    // Stat the file and check that the lazyPersist flag is returned back.
-    HdfsFileStatus status = client.getFileInfo(path.toString());
-    assertThat(status.getStoragePolicy(), is(LAZY_PERSIST_POLICY_ID));
-  }
-
-  @Test
-  public void testPolicyPersistenceInFsImage() throws IOException {
-    startUpCluster(false, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, 0, true);
-    // checkpoint
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
-    fs.saveNamespace();
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
-    cluster.restartNameNode(true);
-
-    // Stat the file and check that the lazyPersist flag is returned back.
-    HdfsFileStatus status = client.getFileInfo(path.toString());
-    assertThat(status.getStoragePolicy(), is(LAZY_PERSIST_POLICY_ID));
-  }
-
-  @Test
-  public void testPlacementOnRamDisk() throws IOException {
-    startUpCluster(true, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, BLOCK_SIZE, true);
-    ensureFileReplicasOnStorageType(path, RAM_DISK);
-  }
-
-  @Test
-  public void testPlacementOnSizeLimitedRamDisk() throws IOException {
-    startUpCluster(true, 3);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
-
-    makeTestFile(path1, BLOCK_SIZE, true);
-    makeTestFile(path2, BLOCK_SIZE, true);
-
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-    ensureFileReplicasOnStorageType(path2, RAM_DISK);
-  }
-
-  /**
-   * Client tries to write LAZY_PERSIST to same DN with no RamDisk configured
-   * Write should default to disk. No error.
-   * @throws IOException
-   */
-  @Test
-  public void testFallbackToDisk() throws IOException {
-    startUpCluster(false, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, BLOCK_SIZE, true);
-    ensureFileReplicasOnStorageType(path, DEFAULT);
-  }
-
-  /**
-   * File can not fit in RamDisk even with eviction
-   * @throws IOException
-   */
-  @Test
-  public void testFallbackToDiskFull() throws Exception {
-    startUpCluster(false, 0);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, BLOCK_SIZE, true);
-    ensureFileReplicasOnStorageType(path, DEFAULT);
-
-    verifyRamDiskJMXMetric("RamDiskBlocksWriteFallback", 1);
-  }
-
-  /**
-   * File partially fit in RamDisk after eviction.
-   * RamDisk can fit 2 blocks. Write a file with 5 blocks.
-   * Expect 2 or less blocks are on RamDisk and 3 or more on disk.
-   * @throws IOException
-   */
-  @Test
-  public void testFallbackToDiskPartial()
-    throws IOException, InterruptedException {
-    startUpCluster(true, 2);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, BLOCK_SIZE * 5, true);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job
-    Thread.sleep(6 * LAZY_WRITER_INTERVAL_SEC * 1000);
-
-    triggerBlockReport();
-
-    int numBlocksOnRamDisk = 0;
-    int numBlocksOnDisk = 0;
-
-    long fileLength = client.getFileInfo(path.toString()).getLen();
-    LocatedBlocks locatedBlocks =
-      client.getLocatedBlocks(path.toString(), 0, fileLength);
-    for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
-      if (locatedBlock.getStorageTypes()[0] == RAM_DISK) {
-        numBlocksOnRamDisk++;
-      } else if (locatedBlock.getStorageTypes()[0] == DEFAULT) {
-        numBlocksOnDisk++;
-      }
-    }
-
-    // Since eviction is asynchronous, depending on the timing of eviction
-    // wrt writes, we may get 2 or less blocks on RAM disk.
-    assert(numBlocksOnRamDisk <= 2);
-    assert(numBlocksOnDisk >= 3);
-  }
-
-  /**
-   * If the only available storage is RAM_DISK and the LAZY_PERSIST flag is not
-   * specified, then block placement should fail.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testRamDiskNotChosenByDefault() throws IOException {
-    startUpCluster(true, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    try {
-      makeTestFile(path, BLOCK_SIZE, false);
-      fail("Block placement to RAM_DISK should have failed without lazyPersist flag");
-    } catch (Throwable t) {
-      LOG.info("Got expected exception ", t);
-    }
-  }
-
   /**
    * Append to lazy persist file is denied.
    * @throws IOException
@@ -271,7 +97,7 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
    * 'corrupt' file.
    */
   @Test
-  public void testLazyPersistFilesAreDiscarded()
+  public void testCorruptFilesAreDiscarded()
       throws IOException, InterruptedException {
     startUpCluster(true, 2);
     final String METHOD_NAME = GenericTestUtils.getMethodName();
@@ -304,244 +130,6 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
                is(0L));
   }
 
-  @Test
-  public void testLazyPersistBlocksAreSaved()
-      throws IOException, InterruptedException {
-    startUpCluster(true, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    // Create a test file
-    makeTestFile(path, BLOCK_SIZE * 10, true);
-    LocatedBlocks locatedBlocks = ensureFileReplicasOnStorageType(path, RAM_DISK);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job
-    Thread.sleep(6 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    
-    LOG.info("Verifying copy was saved to lazyPersist/");
-
-    // Make sure that there is a saved copy of the replica on persistent
-    // storage.
-    ensureLazyPersistBlocksAreSaved(locatedBlocks);
-  }
-
-  /**
-   * RamDisk eviction after lazy persist to disk.
-   * @throws Exception
-   */
-  @Test
-  public void testRamDiskEviction() throws Exception {
-    startUpCluster(true, 1 + EVICTION_LOW_WATERMARK);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
-
-    final int SEED = 0xFADED;
-    makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-
-    // Create another file with a replica on RAM_DISK.
-    makeTestFile(path2, BLOCK_SIZE, true);
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    triggerBlockReport();
-
-    // Ensure the first file was evicted to disk, the second is still on
-    // RAM_DISK.
-    ensureFileReplicasOnStorageType(path2, RAM_DISK);
-    ensureFileReplicasOnStorageType(path1, DEFAULT);
-
-    verifyRamDiskJMXMetric("RamDiskBlocksEvicted", 1);
-    verifyRamDiskJMXMetric("RamDiskBlocksEvictedWithoutRead", 1);
-  }
-
-  /**
-   * RamDisk eviction should not happen on blocks that are not yet
-   * persisted on disk.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  @Test
-  public void testRamDiskEvictionBeforePersist()
-    throws IOException, InterruptedException {
-    startUpCluster(true, 1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
-    final int SEED = 0XFADED;
-
-    // Stop lazy writer to ensure block for path1 is not persisted to disk.
-    FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
-
-    makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-
-    // Create second file with a replica on RAM_DISK.
-    makeTestFile(path2, BLOCK_SIZE, true);
-
-    // Eviction should not happen for block of the first file that is not
-    // persisted yet.
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-    ensureFileReplicasOnStorageType(path2, DEFAULT);
-
-    assert(fs.exists(path1));
-    assert(fs.exists(path2));
-    assertTrue(verifyReadRandomFile(path1, BLOCK_SIZE, SEED));
-  }
-
-  /**
-   * Validates lazy persisted blocks are evicted from RAM_DISK based on LRU.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  @Test
-  public void testRamDiskEvictionIsLru()
-    throws Exception {
-    final int NUM_PATHS = 5;
-    startUpCluster(true, NUM_PATHS + EVICTION_LOW_WATERMARK);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path paths[] = new Path[NUM_PATHS * 2];
-
-    for (int i = 0; i < paths.length; i++) {
-      paths[i] = new Path("/" + METHOD_NAME + "." + i +".dat");
-    }
-
-    for (int i = 0; i < NUM_PATHS; i++) {
-      makeTestFile(paths[i], BLOCK_SIZE, true);
-    }
-
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-
-    for (int i = 0; i < NUM_PATHS; ++i) {
-      ensureFileReplicasOnStorageType(paths[i], RAM_DISK);
-    }
-
-    // Open the files for read in a random order.
-    ArrayList<Integer> indexes = new ArrayList<Integer>(NUM_PATHS);
-    for (int i = 0; i < NUM_PATHS; ++i) {
-      indexes.add(i);
-    }
-    Collections.shuffle(indexes);
-
-    for (int i = 0; i < NUM_PATHS; ++i) {
-      LOG.info("Touching file " + paths[indexes.get(i)]);
-      DFSTestUtil.readFile(fs, paths[indexes.get(i)]);
-    }
-
-    // Create an equal number of new files ensuring that the previous
-    // files are evicted in the same order they were read.
-    for (int i = 0; i < NUM_PATHS; ++i) {
-      makeTestFile(paths[i + NUM_PATHS], BLOCK_SIZE, true);
-      triggerBlockReport();
-      Thread.sleep(3000);
-      ensureFileReplicasOnStorageType(paths[i + NUM_PATHS], RAM_DISK);
-      ensureFileReplicasOnStorageType(paths[indexes.get(i)], DEFAULT);
-      for (int j = i + 1; j < NUM_PATHS; ++j) {
-        ensureFileReplicasOnStorageType(paths[indexes.get(j)], RAM_DISK);
-      }
-    }
-
-    verifyRamDiskJMXMetric("RamDiskBlocksWrite", NUM_PATHS * 2);
-    verifyRamDiskJMXMetric("RamDiskBlocksWriteFallback", 0);
-    verifyRamDiskJMXMetric("RamDiskBytesWrite", BLOCK_SIZE * NUM_PATHS * 2);
-    verifyRamDiskJMXMetric("RamDiskBlocksReadHits", NUM_PATHS);
-    verifyRamDiskJMXMetric("RamDiskBlocksEvicted", NUM_PATHS);
-    verifyRamDiskJMXMetric("RamDiskBlocksEvictedWithoutRead", 0);
-    verifyRamDiskJMXMetric("RamDiskBlocksDeletedBeforeLazyPersisted", 0);
-  }
-
-  /**
-   * Delete lazy-persist file that has not been persisted to disk.
-   * Memory is freed up and file is gone.
-   * @throws IOException
-   */
-  @Test
-  public void testDeleteBeforePersist()
-    throws Exception {
-    startUpCluster(true, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
-
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-    makeTestFile(path, BLOCK_SIZE, true);
-    LocatedBlocks locatedBlocks =
-      ensureFileReplicasOnStorageType(path, RAM_DISK);
-
-    // Delete before persist
-    client.delete(path.toString(), false);
-    Assert.assertFalse(fs.exists(path));
-
-    assertThat(verifyDeletedBlocks(locatedBlocks), is(true));
-
-    verifyRamDiskJMXMetric("RamDiskBlocksDeletedBeforeLazyPersisted", 1);
-  }
-
-  /**
-   * Delete lazy-persist file that has been persisted to disk
-   * Both memory blocks and disk blocks are deleted.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  @Test
-  public void testDeleteAfterPersist()
-    throws Exception {
-    startUpCluster(true, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    makeTestFile(path, BLOCK_SIZE, true);
-    LocatedBlocks locatedBlocks = ensureFileReplicasOnStorageType(path, RAM_DISK);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job
-    Thread.sleep(6 * LAZY_WRITER_INTERVAL_SEC * 1000);
-
-    // Delete after persist
-    client.delete(path.toString(), false);
-    Assert.assertFalse(fs.exists(path));
-
-    assertThat(verifyDeletedBlocks(locatedBlocks), is(true));
-
-    verifyRamDiskJMXMetric("RamDiskBlocksLazyPersisted", 1);
-    verifyRamDiskJMXMetric("RamDiskBytesLazyPersisted", BLOCK_SIZE);
-  }
-
-  /**
-   * RAM_DISK used/free space
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  @Test
-  public void testDfsUsageCreateDelete()
-    throws IOException, InterruptedException {
-    startUpCluster(true, 4);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-
-    // Get the usage before write BLOCK_SIZE
-    long usedBeforeCreate = fs.getUsed();
-
-    makeTestFile(path, BLOCK_SIZE, true);
-    long usedAfterCreate = fs.getUsed();
-
-    assertThat(usedAfterCreate, is((long) BLOCK_SIZE));
-
-    // Sleep for a short time to allow the lazy writer thread to do its job
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-
-    long usedAfterPersist = fs.getUsed();
-    assertThat(usedAfterPersist, is((long) BLOCK_SIZE));
-
-    // Delete after persist
-    client.delete(path.toString(), false);
-    long usedAfterDelete = fs.getUsed();
-
-    assertThat(usedBeforeCreate, is(usedAfterDelete));
-  }
-
   /**
    * Concurrent read from the same node and verify the contents.
    */
@@ -632,51 +220,6 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
     assertThat(testFailed.get(), is(false));
   }
 
-  @Test
-  public void testDnRestartWithSavedReplicas()
-      throws IOException, InterruptedException {
-
-    startUpCluster(true, -1);
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-
-    makeTestFile(path1, BLOCK_SIZE, true);
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    // However the block replica should not be evicted from RAM_DISK yet.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-
-    LOG.info("Restarting the DataNode");
-    cluster.restartDataNode(0, true);
-    cluster.waitActive();
-    triggerBlockReport();
-
-    // Ensure that the replica is now on persistent storage.
-    ensureFileReplicasOnStorageType(path1, DEFAULT);
-  }
-
-  @Test
-  public void testDnRestartWithUnsavedReplicas()
-      throws IOException, InterruptedException {
-
-    startUpCluster(true, 1);
-    FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
-
-    final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    makeTestFile(path1, BLOCK_SIZE, true);
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-
-    LOG.info("Restarting the DataNode");
-    cluster.restartDataNode(0, true);
-    cluster.waitActive();
-
-    // Ensure that the replica is still on transient storage.
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-  }
-
   class WriterRunnable implements Runnable {
     private final int id;
     private final Path paths[];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89ed0e72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistPolicy.java
new file mode 100644
index 0000000..873e2b0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistPolicy.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.hamcrest.core.Is.is;
+import static org.hamcrest.core.IsNot.not;
+import static org.junit.Assert.assertThat;
+
+
+public class TestLazyPersistPolicy extends LazyPersistTestCase {
+  @Test
+  public void testPolicyNotSetByDefault() throws IOException {
+    startUpCluster(false, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, 0, false);
+    // Stat the file and check that the LAZY_PERSIST policy is not
+    // returned back.
+    HdfsFileStatus status = client.getFileInfo(path.toString());
+    assertThat(status.getStoragePolicy(), not(LAZY_PERSIST_POLICY_ID));
+  }
+
+  @Test
+  public void testPolicyPropagation() throws IOException {
+    startUpCluster(false, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, 0, true);
+    // Stat the file and check that the lazyPersist flag is returned back.
+    HdfsFileStatus status = client.getFileInfo(path.toString());
+    assertThat(status.getStoragePolicy(), is(LAZY_PERSIST_POLICY_ID));
+  }
+
+  @Test
+  public void testPolicyPersistenceInEditLog() throws IOException {
+    startUpCluster(false, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, 0, true);
+    cluster.restartNameNode(true);
+
+    // Stat the file and check that the lazyPersist flag is returned back.
+    HdfsFileStatus status = client.getFileInfo(path.toString());
+    assertThat(status.getStoragePolicy(), is(LAZY_PERSIST_POLICY_ID));
+  }
+
+  @Test
+  public void testPolicyPersistenceInFsImage() throws IOException {
+    startUpCluster(false, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, 0, true);
+    // checkpoint
+    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    fs.saveNamespace();
+    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    cluster.restartNameNode(true);
+
+    // Stat the file and check that the lazyPersist flag is returned back.
+    HdfsFileStatus status = client.getFileInfo(path.toString());
+    assertThat(status.getStoragePolicy(), is(LAZY_PERSIST_POLICY_ID));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89ed0e72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java
new file mode 100644
index 0000000..4e1a893
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.StorageType.DEFAULT;
+import static org.apache.hadoop.fs.StorageType.RAM_DISK;
+import static org.junit.Assert.fail;
+
+public class TestLazyPersistReplicaPlacement extends LazyPersistTestCase {
+  @Test
+  public void testPlacementOnRamDisk() throws IOException {
+    startUpCluster(true, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path, RAM_DISK);
+  }
+
+  @Test
+  public void testPlacementOnSizeLimitedRamDisk() throws IOException {
+    startUpCluster(true, 3);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
+
+    makeTestFile(path1, BLOCK_SIZE, true);
+    makeTestFile(path2, BLOCK_SIZE, true);
+
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+    ensureFileReplicasOnStorageType(path2, RAM_DISK);
+  }
+
+  /**
+   * Client tries to write LAZY_PERSIST to same DN with no RamDisk configured
+   * Write should default to disk. No error.
+   * @throws IOException
+   */
+  @Test
+  public void testFallbackToDisk() throws IOException {
+    startUpCluster(false, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path, DEFAULT);
+  }
+
+  /**
+   * File can not fit in RamDisk even with eviction
+   * @throws IOException
+   */
+  @Test
+  public void testFallbackToDiskFull() throws Exception {
+    startUpCluster(false, 0);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path, DEFAULT);
+
+    verifyRamDiskJMXMetric("RamDiskBlocksWriteFallback", 1);
+  }
+
+  /**
+   * File partially fit in RamDisk after eviction.
+   * RamDisk can fit 2 blocks. Write a file with 5 blocks.
+   * Expect 2 or less blocks are on RamDisk and 3 or more on disk.
+   * @throws IOException
+   */
+  @Test
+  public void testFallbackToDiskPartial()
+      throws IOException, InterruptedException {
+    startUpCluster(true, 2);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, BLOCK_SIZE * 5, true);
+
+    // Sleep for a short time to allow the lazy writer thread to do its job
+    Thread.sleep(6 * LAZY_WRITER_INTERVAL_SEC * 1000);
+
+    triggerBlockReport();
+
+    int numBlocksOnRamDisk = 0;
+    int numBlocksOnDisk = 0;
+
+    long fileLength = client.getFileInfo(path.toString()).getLen();
+    LocatedBlocks locatedBlocks =
+        client.getLocatedBlocks(path.toString(), 0, fileLength);
+    for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
+      if (locatedBlock.getStorageTypes()[0] == RAM_DISK) {
+        numBlocksOnRamDisk++;
+      } else if (locatedBlock.getStorageTypes()[0] == DEFAULT) {
+        numBlocksOnDisk++;
+      }
+    }
+
+    // Since eviction is asynchronous, depending on the timing of eviction
+    // wrt writes, we may get 2 or less blocks on RAM disk.
+    assert(numBlocksOnRamDisk <= 2);
+    assert(numBlocksOnDisk >= 3);
+  }
+
+  /**
+   * If the only available storage is RAM_DISK and the LAZY_PERSIST flag is not
+   * specified, then block placement should fail.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testRamDiskNotChosenByDefault() throws IOException {
+    startUpCluster(true, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    try {
+      makeTestFile(path, BLOCK_SIZE, false);
+      fail("Block placement to RAM_DISK should have failed without lazyPersist flag");
+    } catch (Throwable t) {
+      LOG.info("Got expected exception ", t);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89ed0e72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaRecovery.java
new file mode 100644
index 0000000..b8b2653
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaRecovery.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.StorageType.DEFAULT;
+import static org.apache.hadoop.fs.StorageType.RAM_DISK;
+
+public class TestLazyPersistReplicaRecovery extends LazyPersistTestCase {
+  @Test
+  public void testDnRestartWithSavedReplicas()
+      throws IOException, InterruptedException {
+
+    startUpCluster(true, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+
+    makeTestFile(path1, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    // Sleep for a short time to allow the lazy writer thread to do its job.
+    // However the block replica should not be evicted from RAM_DISK yet.
+    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    LOG.info("Restarting the DataNode");
+    cluster.restartDataNode(0, true);
+    cluster.waitActive();
+    triggerBlockReport();
+
+    // Ensure that the replica is now on persistent storage.
+    ensureFileReplicasOnStorageType(path1, DEFAULT);
+  }
+
+  @Test
+  public void testDnRestartWithUnsavedReplicas()
+      throws IOException, InterruptedException {
+
+    startUpCluster(true, 1);
+    FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
+
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    makeTestFile(path1, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    LOG.info("Restarting the DataNode");
+    cluster.restartDataNode(0, true);
+    cluster.waitActive();
+
+    // Ensure that the replica is still on transient storage.
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89ed0e72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java
new file mode 100644
index 0000000..23203c7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java
@@ -0,0 +1,276 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static org.apache.hadoop.fs.StorageType.DEFAULT;
+import static org.apache.hadoop.fs.StorageType.RAM_DISK;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public class TestLazyWriter extends LazyPersistTestCase {
+  @Test
+  public void testLazyPersistBlocksAreSaved()
+      throws IOException, InterruptedException {
+    startUpCluster(true, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    // Create a test file
+    makeTestFile(path, BLOCK_SIZE * 10, true);
+    LocatedBlocks locatedBlocks = ensureFileReplicasOnStorageType(path, RAM_DISK);
+
+    // Sleep for a short time to allow the lazy writer thread to do its job
+    Thread.sleep(6 * LAZY_WRITER_INTERVAL_SEC * 1000);
+
+    LOG.info("Verifying copy was saved to lazyPersist/");
+
+    // Make sure that there is a saved copy of the replica on persistent
+    // storage.
+    ensureLazyPersistBlocksAreSaved(locatedBlocks);
+  }
+
+  /**
+   * RamDisk eviction after lazy persist to disk.
+   * @throws Exception
+   */
+  @Test
+  public void testRamDiskEviction() throws Exception {
+    startUpCluster(true, 1 + EVICTION_LOW_WATERMARK);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
+
+    final int SEED = 0xFADED;
+    makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    // Sleep for a short time to allow the lazy writer thread to do its job.
+    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    // Create another file with a replica on RAM_DISK.
+    makeTestFile(path2, BLOCK_SIZE, true);
+    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
+    triggerBlockReport();
+
+    // Ensure the first file was evicted to disk, the second is still on
+    // RAM_DISK.
+    ensureFileReplicasOnStorageType(path2, RAM_DISK);
+    ensureFileReplicasOnStorageType(path1, DEFAULT);
+
+    verifyRamDiskJMXMetric("RamDiskBlocksEvicted", 1);
+    verifyRamDiskJMXMetric("RamDiskBlocksEvictedWithoutRead", 1);
+  }
+
+  /**
+   * RamDisk eviction should not happen on blocks that are not yet
+   * persisted on disk.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test
+  public void testRamDiskEvictionBeforePersist()
+      throws IOException, InterruptedException {
+    startUpCluster(true, 1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
+    final int SEED = 0XFADED;
+
+    // Stop lazy writer to ensure block for path1 is not persisted to disk.
+    FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
+
+    makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    // Create second file with a replica on RAM_DISK.
+    makeTestFile(path2, BLOCK_SIZE, true);
+
+    // Eviction should not happen for block of the first file that is not
+    // persisted yet.
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+    ensureFileReplicasOnStorageType(path2, DEFAULT);
+
+    assert(fs.exists(path1));
+    assert(fs.exists(path2));
+    assertTrue(verifyReadRandomFile(path1, BLOCK_SIZE, SEED));
+  }
+
+  /**
+   * Validates lazy persisted blocks are evicted from RAM_DISK based on LRU.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test
+  public void testRamDiskEvictionIsLru()
+      throws Exception {
+    final int NUM_PATHS = 5;
+    startUpCluster(true, NUM_PATHS + EVICTION_LOW_WATERMARK);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path paths[] = new Path[NUM_PATHS * 2];
+
+    for (int i = 0; i < paths.length; i++) {
+      paths[i] = new Path("/" + METHOD_NAME + "." + i +".dat");
+    }
+
+    for (int i = 0; i < NUM_PATHS; i++) {
+      makeTestFile(paths[i], BLOCK_SIZE, true);
+    }
+
+    // Sleep for a short time to allow the lazy writer thread to do its job.
+    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
+
+    for (int i = 0; i < NUM_PATHS; ++i) {
+      ensureFileReplicasOnStorageType(paths[i], RAM_DISK);
+    }
+
+    // Open the files for read in a random order.
+    ArrayList<Integer> indexes = new ArrayList<Integer>(NUM_PATHS);
+    for (int i = 0; i < NUM_PATHS; ++i) {
+      indexes.add(i);
+    }
+    Collections.shuffle(indexes);
+
+    for (int i = 0; i < NUM_PATHS; ++i) {
+      LOG.info("Touching file " + paths[indexes.get(i)]);
+      DFSTestUtil.readFile(fs, paths[indexes.get(i)]);
+    }
+
+    // Create an equal number of new files ensuring that the previous
+    // files are evicted in the same order they were read.
+    for (int i = 0; i < NUM_PATHS; ++i) {
+      makeTestFile(paths[i + NUM_PATHS], BLOCK_SIZE, true);
+      triggerBlockReport();
+      Thread.sleep(3000);
+      ensureFileReplicasOnStorageType(paths[i + NUM_PATHS], RAM_DISK);
+      ensureFileReplicasOnStorageType(paths[indexes.get(i)], DEFAULT);
+      for (int j = i + 1; j < NUM_PATHS; ++j) {
+        ensureFileReplicasOnStorageType(paths[indexes.get(j)], RAM_DISK);
+      }
+    }
+
+    verifyRamDiskJMXMetric("RamDiskBlocksWrite", NUM_PATHS * 2);
+    verifyRamDiskJMXMetric("RamDiskBlocksWriteFallback", 0);
+    verifyRamDiskJMXMetric("RamDiskBytesWrite", BLOCK_SIZE * NUM_PATHS * 2);
+    verifyRamDiskJMXMetric("RamDiskBlocksReadHits", NUM_PATHS);
+    verifyRamDiskJMXMetric("RamDiskBlocksEvicted", NUM_PATHS);
+    verifyRamDiskJMXMetric("RamDiskBlocksEvictedWithoutRead", 0);
+    verifyRamDiskJMXMetric("RamDiskBlocksDeletedBeforeLazyPersisted", 0);
+  }
+
+  /**
+   * Delete lazy-persist file that has not been persisted to disk.
+   * Memory is freed up and file is gone.
+   * @throws IOException
+   */
+  @Test
+  public void testDeleteBeforePersist()
+      throws Exception {
+    startUpCluster(true, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
+
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+    makeTestFile(path, BLOCK_SIZE, true);
+    LocatedBlocks locatedBlocks =
+        ensureFileReplicasOnStorageType(path, RAM_DISK);
+
+    // Delete before persist
+    client.delete(path.toString(), false);
+    Assert.assertFalse(fs.exists(path));
+
+    assertThat(verifyDeletedBlocks(locatedBlocks), is(true));
+
+    verifyRamDiskJMXMetric("RamDiskBlocksDeletedBeforeLazyPersisted", 1);
+  }
+
+  /**
+   * Delete lazy-persist file that has been persisted to disk
+   * Both memory blocks and disk blocks are deleted.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test
+  public void testDeleteAfterPersist()
+      throws Exception {
+    startUpCluster(true, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, BLOCK_SIZE, true);
+    LocatedBlocks locatedBlocks = ensureFileReplicasOnStorageType(path, RAM_DISK);
+
+    // Sleep for a short time to allow the lazy writer thread to do its job
+    Thread.sleep(6 * LAZY_WRITER_INTERVAL_SEC * 1000);
+
+    // Delete after persist
+    client.delete(path.toString(), false);
+    Assert.assertFalse(fs.exists(path));
+
+    assertThat(verifyDeletedBlocks(locatedBlocks), is(true));
+
+    verifyRamDiskJMXMetric("RamDiskBlocksLazyPersisted", 1);
+    verifyRamDiskJMXMetric("RamDiskBytesLazyPersisted", BLOCK_SIZE);
+  }
+
+  /**
+   * RAM_DISK used/free space
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test
+  public void testDfsUsageCreateDelete()
+      throws IOException, InterruptedException {
+    startUpCluster(true, 4);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    // Get the usage before write BLOCK_SIZE
+    long usedBeforeCreate = fs.getUsed();
+
+    makeTestFile(path, BLOCK_SIZE, true);
+    long usedAfterCreate = fs.getUsed();
+
+    assertThat(usedAfterCreate, is((long) BLOCK_SIZE));
+
+    // Sleep for a short time to allow the lazy writer thread to do its job
+    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
+
+    long usedAfterPersist = fs.getUsed();
+    assertThat(usedAfterPersist, is((long) BLOCK_SIZE));
+
+    // Delete after persist
+    client.delete(path.toString(), false);
+    long usedAfterDelete = fs.getUsed();
+
+    assertThat(usedBeforeCreate, is(usedAfterDelete));
+  }
+}


[46/50] [abbrv] hadoop git commit: HDFS-8153. Error Message points to wrong parent directory in case of path component name length error. Contributed by Anu Engineer.

Posted by zj...@apache.org.
HDFS-8153. Error Message points to wrong parent directory in case of path component name length error. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a9e116ef
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a9e116ef
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a9e116ef

Branch: refs/heads/YARN-2928
Commit: a9e116ef2e13b6dd6bf1b24e297ef77d9319e678
Parents: 35c7c5c
Author: Jitendra Pandey <ji...@apache.org>
Authored: Thu Apr 16 22:13:09 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:47 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/namenode/FSDirectory.java       |  2 +-
 .../hdfs/server/namenode/TestFsLimits.java      | 83 +++++++++++++++++++-
 3 files changed, 85 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9e116ef/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e977e6a..b980f93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -543,6 +543,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8149. The footer of the Web UI "Hadoop, 2014" is old.
     (Brahma Reddy Battula via aajisaka)
 
+    HDFS-8153. Error Message points to wrong parent directory in case of
+    path component name length error (Anu Engineer via jitendra)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9e116ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 966cf3a..f74c42a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -972,7 +972,7 @@ public class FSDirectory implements Closeable {
     // original location because a quota violation would cause the the item
     // to go "poof".  The fs limits must be bypassed for the same reason.
     if (checkQuota) {
-      final String parentPath = existing.getPath(pos - 1);
+      final String parentPath = existing.getPath();
       verifyMaxComponentLength(inode.getLocalNameBytes(), parentPath);
       verifyMaxDirItems(parent, parentPath);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9e116ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
index 945972d..d6c5183 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 
@@ -173,29 +174,107 @@ public class TestFsLimits {
         HadoopIllegalArgumentException.class);
   }
 
-  private void mkdirs(String name, Class<?> expected)
+  @Test
+  /**
+   * This test verifies that error string contains the
+   * right parent directory name if the operation fails with
+   * PathComponentTooLongException
+   */
+  public void testParentDirectoryNameIsCorrect() throws Exception {
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY, 20);
+    mkdirs("/user", null);
+    mkdirs("/user/testHome", null);
+    mkdirs("/user/testHome/FileNameLength", null);
+
+    mkdirCheckParentDirectory(
+      "/user/testHome/FileNameLength/really_big_name_0003_fail",
+      "/user/testHome/FileNameLength", PathComponentTooLongException.class);
+
+    renameCheckParentDirectory("/user/testHome/FileNameLength",
+      "/user/testHome/really_big_name_0003_fail", "/user/testHome/",
+      PathComponentTooLongException.class);
+
+  }
+
+
+  /**
+   * Verifies that Parent Directory is correct after a failed call to mkdir
+   * @param name Directory Name
+   * @param ParentDirName Expected Parent Directory
+   * @param expected Exception that is expected
+   * @throws Exception
+   */
+  private void mkdirCheckParentDirectory(String name, String ParentDirName,
+                                         Class<?> expected)
+    throws Exception {
+    verify(mkdirs(name, expected), ParentDirName);
+  }
+
+  /**
+   *
+   /**
+   * Verifies that Parent Directory is correct after a failed call to mkdir
+   * @param name Directory Name
+   * @param dst Destination Name
+   * @param ParentDirName Expected Parent Directory
+   * @param expected Exception that is expected
+   * @throws Exception
+   */
+  private void renameCheckParentDirectory(String name, String dst,
+                                          String ParentDirName,
+                                          Class<?> expected)
+    throws Exception {
+    verify(rename(name, dst, expected), ParentDirName);
+  }
+
+  /**
+   * verifies the ParentDirectory Name is present in the message given.
+   * @param message - Expection Message
+   * @param ParentDirName - Parent Directory Name to look for.
+   */
+  private void verify(String message, String ParentDirName) {
+    boolean found = false;
+    if (message != null) {
+      String[] tokens = message.split("\\s+");
+      for (String token : tokens) {
+        if (token != null && token.equals(ParentDirName)) {
+          found = true;
+          break;
+        }
+      }
+    }
+    assertTrue(found);
+  }
+
+  private String mkdirs(String name, Class<?> expected)
   throws Exception {
     lazyInitFSDirectory();
     Class<?> generated = null;
+    String errorString = null;
     try {
       fs.mkdirs(name, perms, false);
     } catch (Throwable e) {
       generated = e.getClass();
       e.printStackTrace();
+      errorString = e.getMessage();
     }
     assertEquals(expected, generated);
+    return errorString;
   }
 
-  private void rename(String src, String dst, Class<?> expected)
+  private String rename(String src, String dst, Class<?> expected)
       throws Exception {
     lazyInitFSDirectory();
     Class<?> generated = null;
+    String errorString = null;
     try {
       fs.renameTo(src, dst, false, new Rename[] { });
     } catch (Throwable e) {
       generated = e.getClass();
+      errorString = e.getMessage();
     }
     assertEquals(expected, generated);
+    return errorString;
   }
 
   @SuppressWarnings("deprecation")


[37/50] [abbrv] hadoop git commit: HDFS-7934. Update RollingUpgrade rollback documentation: should use bootstrapstandby for standby NN. Contributed by J. Andreina.

Posted by zj...@apache.org.
HDFS-7934. Update RollingUpgrade rollback documentation: should use bootstrapstandby for standby NN. Contributed by J. Andreina.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/504fab95
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/504fab95
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/504fab95

Branch: refs/heads/YARN-2928
Commit: 504fab95e8f28e004cf4cac36153d4d9827da2ea
Parents: 83ac8e0
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Apr 15 13:10:38 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml             | 4 +++-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/504fab95/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 574faa2..60fff16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -525,6 +525,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8151. Always use snapshot path as source when invalid snapshot names
     are used for diff based distcp. (jing9)
 
+    HDFS-7934. Update RollingUpgrade rollback documentation: should use
+    bootstrapstandby for standby NN. (J. Andreina via jing9)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/504fab95/hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml b/hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml
index 2ad28e1..1c3dc60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml
@@ -263,14 +263,16 @@
   <p>
     Rollback from a newer release to the pre-upgrade release is always supported.
     However, it cannot be done in a rolling fashion.  It requires cluster downtime.
+    Suppose <em>NN1</em> and <em>NN2</em> are respectively in active and standby states.
     Below are the steps for rollback:
   </p>
   <ul>
     <li>Rollback HDFS<ol>
       <li>Shutdown all <em>NNs</em> and <em>DNs</em>.</li>
       <li>Restore the pre-upgrade release in all machines.</li>
-      <li>Start <em>NNs</em> with the
+      <li>Start <em>NN1</em> as Active with the
         "<a href="#namenode_-rollingUpgrade"><code>-rollingUpgrade rollback</code></a>" option.</li>
+      <li>Run `-bootstrapStandby' on NN2 and start it normally as standby.</li>
       <li>Start <em>DNs</em> with the "<code>-rollback</code>" option.</li>
     </ol></li>
   </ul>


[36/50] [abbrv] hadoop git commit: HDFS-8151. Always use snapshot path as source when invalid snapshot names are used for diff based distcp. Contributed by Jing Zhao.

Posted by zj...@apache.org.
HDFS-8151. Always use snapshot path as source when invalid snapshot names are used for diff based distcp. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/83ac8e02
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/83ac8e02
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/83ac8e02

Branch: refs/heads/YARN-2928
Commit: 83ac8e02952fa624c2bbca14d8c7c012b5d6c9f3
Parents: fb79463
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Apr 15 12:37:20 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt          |  3 +++
 .../java/org/apache/hadoop/tools/DistCpSync.java     | 12 +++++++-----
 .../java/org/apache/hadoop/tools/TestDistCpSync.java | 15 ++++++++++++++-
 3 files changed, 24 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/83ac8e02/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 5a0f6f2..574faa2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -522,6 +522,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8127. NameNode Failover during HA upgrade can cause DataNode to
     finalize upgrade. (jing9)
 
+    HDFS-8151. Always use snapshot path as source when invalid snapshot names
+    are used for diff based distcp. (jing9)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83ac8e02/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
index 8e71b6f..5bf638d 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
@@ -47,8 +47,8 @@ class DistCpSync {
     List<Path> sourcePaths = inputOptions.getSourcePaths();
     if (sourcePaths.size() != 1) {
       // we only support one source dir which must be a snapshottable directory
-      DistCp.LOG.warn(sourcePaths.size() + " source paths are provided");
-      return false;
+      throw new IllegalArgumentException(sourcePaths.size()
+          + " source paths are provided");
     }
     final Path sourceDir = sourcePaths.get(0);
     final Path targetDir = inputOptions.getTargetPath();
@@ -59,15 +59,17 @@ class DistCpSync {
     // DistributedFileSystem.
     if (!(sfs instanceof DistributedFileSystem) ||
         !(tfs instanceof DistributedFileSystem)) {
-      DistCp.LOG.warn("To use diff-based distcp, the FileSystems needs to" +
-          " be DistributedFileSystem");
-      return false;
+      throw new IllegalArgumentException("The FileSystems needs to" +
+          " be DistributedFileSystem for using snapshot-diff-based distcp");
     }
     final DistributedFileSystem sourceFs = (DistributedFileSystem) sfs;
     final DistributedFileSystem targetFs= (DistributedFileSystem) tfs;
 
     // make sure targetFS has no change between from and the current states
     if (!checkNoChange(inputOptions, targetFs, targetDir)) {
+      // set the source path using the snapshot path
+      inputOptions.setSourcePaths(Arrays.asList(getSourceSnapshotPath(sourceDir,
+          inputOptions.getToSnapshot())));
       return false;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83ac8e02/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
index 75d1de5..0a9a11f 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
@@ -88,24 +88,37 @@ public class TestDistCpSync {
   public void testFallback() throws Exception {
     // the source/target dir are not snapshottable dir
     Assert.assertFalse(DistCpSync.sync(options, conf));
+    // make sure the source path has been updated to the snapshot path
+    final Path spath = new Path(source,
+        HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s2");
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
 
+    // reset source path in options
+    options.setSourcePaths(Arrays.asList(source));
     // the source/target does not have the given snapshots
     dfs.allowSnapshot(source);
     dfs.allowSnapshot(target);
     Assert.assertFalse(DistCpSync.sync(options, conf));
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
 
+    // reset source path in options
+    options.setSourcePaths(Arrays.asList(source));
     dfs.createSnapshot(source, "s1");
     dfs.createSnapshot(source, "s2");
     dfs.createSnapshot(target, "s1");
     Assert.assertTrue(DistCpSync.sync(options, conf));
+
     // reset source paths in options
     options.setSourcePaths(Arrays.asList(source));
-
     // changes have been made in target
     final Path subTarget = new Path(target, "sub");
     dfs.mkdirs(subTarget);
     Assert.assertFalse(DistCpSync.sync(options, conf));
+    // make sure the source path has been updated to the snapshot path
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
 
+    // reset source paths in options
+    options.setSourcePaths(Arrays.asList(source));
     dfs.delete(subTarget, true);
     Assert.assertTrue(DistCpSync.sync(options, conf));
   }


[05/50] [abbrv] hadoop git commit: HADOOP-11811. Fix typos in hadoop-project/pom.xml. Contributed by Brahma Reddy Battula.

Posted by zj...@apache.org.
HADOOP-11811. Fix typos in hadoop-project/pom.xml. Contributed by Brahma Reddy Battula.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0e60995c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0e60995c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0e60995c

Branch: refs/heads/YARN-2928
Commit: 0e60995c14fd9e6ac4056f5e0a36a8ce98d55101
Parents: 668d158
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon Apr 13 06:59:51 2015 -0500
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:40 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../apache/hadoop/security/authorize/TestAccessControlList.java   | 2 +-
 hadoop-project/pom.xml                                            | 2 +-
 3 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e60995c/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index f181a96..b80d5cb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -509,6 +509,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11800. Clean up some test methods in TestCodec.java.
     (Brahma Reddy Battula via aajisaka)
 
+    HADOOP-11811. Fix typos in hadoop-project/pom.xml and TestAccessControlList.
+    (Brahma Reddy Battula via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e60995c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java
index ca7bec4..75b944d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java
@@ -143,7 +143,7 @@ public class TestAccessControlList {
     List<String> jerryLeeLewisGroups = groups.getGroups("jerryLeeLewis");
     assertTrue(jerryLeeLewisGroups.contains("@memphis"));
 
-    // allowed becuase his netgroup is in ACL
+    // allowed because his netgroup is in ACL
     UserGroupInformation elvis = 
       UserGroupInformation.createRemoteUser("elvis");
     assertUserAllowed(elvis, acl);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e60995c/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 2c8837c..d205749 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -39,7 +39,7 @@
     <!-- number of threads/forks to use when running tests in parallel, see parallel-tests profile -->
     <testsThreadCount>4</testsThreadCount>
 
-    <!-- These 2 versions are defined here becuase they are used -->
+    <!-- These 2 versions are defined here because they are used -->
     <!-- JDIFF generation from embedded ant in the antrun plugin -->
     <jdiff.version>1.0.9</jdiff.version>
     <!-- Version number for xerces used by JDiff -->


[45/50] [abbrv] hadoop git commit: YARN-3021. YARN's delegation-token handling disallows certain trust setups to operate properly over DistCp. Contributed by Yongjun Zhang

Posted by zj...@apache.org.
YARN-3021. YARN's delegation-token handling disallows certain trust setups to operate properly over DistCp. Contributed by Yongjun Zhang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/35c7c5ca
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/35c7c5ca
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/35c7c5ca

Branch: refs/heads/YARN-2928
Commit: 35c7c5ca51150cbad97b80acf111d2d63c8d0706
Parents: ee5bcb1
Author: Jian He <ji...@apache.org>
Authored: Thu Apr 16 19:33:15 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:47 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  2 ++
 .../hadoop/mapreduce/security/TokenCache.java   | 27 ++++++++++++++---
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../security/DelegationTokenRenewer.java        | 31 +++++++++++++++-----
 .../security/TestDelegationTokenRenewer.java    | 30 +++++++++++++++----
 5 files changed, 77 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35c7c5ca/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 947c814..e64b9b6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -367,6 +367,8 @@ public interface MRJobConfig {
 
   public static final String JOB_NAMENODES = "mapreduce.job.hdfs-servers";
 
+  public static final String JOB_NAMENODES_TOKEN_RENEWAL_EXCLUDE = "mapreduce.job.hdfs-servers.token-renewal.exclude";
+
   public static final String JOB_JOBTRACKER_ID = "mapreduce.job.kerberos.jtprinicipal";
 
   public static final String JOB_CANCEL_DELEGATION_TOKEN = "mapreduce.job.complete.cancel.delegation.tokens";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35c7c5ca/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
index 6c0de1b..e66f770 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
@@ -101,6 +101,20 @@ public class TokenCache {
     }
   }
 
+  static boolean isTokenRenewalExcluded(FileSystem fs, Configuration conf) {
+    String [] nns =
+        conf.getStrings(MRJobConfig.JOB_NAMENODES_TOKEN_RENEWAL_EXCLUDE);
+    if (nns != null) {
+      String host = fs.getUri().getHost();
+      for(int i=0; i< nns.length; i++) {
+        if (nns[i].equals(host)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
   /**
    * get delegation token for a specific FS
    * @param fs
@@ -110,11 +124,16 @@ public class TokenCache {
    */
   static void obtainTokensForNamenodesInternal(FileSystem fs, 
       Credentials credentials, Configuration conf) throws IOException {
-    String delegTokenRenewer = Master.getMasterPrincipal(conf);
-    if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
-      throw new IOException(
-          "Can't get Master Kerberos principal for use as renewer");
+    // RM skips renewing token with empty renewer
+    String delegTokenRenewer = "";
+    if (!isTokenRenewalExcluded(fs, conf)) {
+      delegTokenRenewer = Master.getMasterPrincipal(conf);
+      if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+        throw new IOException(
+            "Can't get Master Kerberos principal for use as renewer");
+      }
     }
+
     mergeBinaryTokens(credentials, conf);
 
     final Token<?> tokens[] = fs.addDelegationTokens(delegTokenRenewer,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35c7c5ca/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1a58988..cf64609 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -271,6 +271,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3436. Fix URIs in documantion of YARN web service REST APIs.
     (Bibin A Chundatt via ozawa)
 
+    YARN-3021. YARN's delegation-token handling disallows certain trust setups
+    to operate properly over DistCp. (Yongjun Zhang via jianhe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35c7c5ca/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index e307645..2cb092f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -79,7 +79,9 @@ public class DelegationTokenRenewer extends AbstractService {
   
   private static final Log LOG = 
       LogFactory.getLog(DelegationTokenRenewer.class);
-  
+  @VisibleForTesting
+  public static final Text HDFS_DELEGATION_KIND =
+      new Text("HDFS_DELEGATION_TOKEN");
   public static final String SCHEME = "hdfs";
 
   // global single timer (daemon)
@@ -244,7 +246,7 @@ public class DelegationTokenRenewer extends AbstractService {
         String user) {
       this.token = token;
       this.user = user;
-      if (token.getKind().equals(new Text("HDFS_DELEGATION_TOKEN"))) {
+      if (token.getKind().equals(HDFS_DELEGATION_KIND)) {
         try {
           AbstractDelegationTokenIdentifier identifier =
               (AbstractDelegationTokenIdentifier) token.decodeIdentifier();
@@ -424,10 +426,13 @@ public class DelegationTokenRenewer extends AbstractService {
     boolean hasHdfsToken = false;
     for (Token<?> token : tokens) {
       if (token.isManaged()) {
-        if (token.getKind().equals(new Text("HDFS_DELEGATION_TOKEN"))) {
+        if (token.getKind().equals(HDFS_DELEGATION_KIND)) {
           LOG.info(applicationId + " found existing hdfs token " + token);
           hasHdfsToken = true;
         }
+        if (skipTokenRenewal(token)) {
+          continue;
+        }
 
         DelegationTokenToRenew dttr = allTokens.get(token);
         if (dttr == null) {
@@ -508,14 +513,26 @@ public class DelegationTokenRenewer extends AbstractService {
       return super.cancel();
     }
   }
-  
+
+  /*
+   * Skip renewing token if the renewer of the token is set to ""
+   * Caller is expected to have examined that token.isManaged() returns
+   * true before calling this method.
+   */
+  private boolean skipTokenRenewal(Token<?> token)
+      throws IOException {
+    @SuppressWarnings("unchecked")
+    Text renewer = ((Token<AbstractDelegationTokenIdentifier>)token).
+        decodeIdentifier().getRenewer();
+    return (renewer != null && renewer.toString().equals(""));
+  }
+
   /**
    * set task to renew the token
    */
   @VisibleForTesting
   protected void setTimerForTokenRenewal(DelegationTokenToRenew token)
       throws IOException {
-      
     // calculate timer time
     long expiresIn = token.expirationDate - System.currentTimeMillis();
     long renewIn = token.expirationDate - expiresIn/10; // little bit before the expiration
@@ -558,7 +575,7 @@ public class DelegationTokenRenewer extends AbstractService {
 
     if (hasProxyUserPrivileges
         && dttr.maxDate - dttr.expirationDate < credentialsValidTimeRemaining
-        && dttr.token.getKind().equals(new Text("HDFS_DELEGATION_TOKEN"))) {
+        && dttr.token.getKind().equals(HDFS_DELEGATION_KIND)) {
 
       final Collection<ApplicationId> applicationIds;
       synchronized (dttr.referringAppIds) {
@@ -575,7 +592,7 @@ public class DelegationTokenRenewer extends AbstractService {
         synchronized (tokenSet) {
           while (iter.hasNext()) {
             DelegationTokenToRenew t = iter.next();
-            if (t.token.getKind().equals(new Text("HDFS_DELEGATION_TOKEN"))) {
+            if (t.token.getKind().equals(HDFS_DELEGATION_KIND)) {
               iter.remove();
               allTokens.remove(t.token);
               t.cancelTimer();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35c7c5ca/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
index f2c0a30..10e840b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
@@ -110,7 +110,8 @@ import com.google.common.base.Supplier;
 public class TestDelegationTokenRenewer {
   private static final Log LOG = 
       LogFactory.getLog(TestDelegationTokenRenewer.class);
-  private static final Text KIND = new Text("HDFS_DELEGATION_TOKEN");
+  private static final Text KIND =
+      DelegationTokenRenewer.HDFS_DELEGATION_KIND;
   
   private static BlockingQueue<Event> eventQueue;
   private static volatile AtomicInteger counter;
@@ -480,7 +481,26 @@ public class TestDelegationTokenRenewer {
     }
     fail("App submission with a cancelled token should have failed");
   }
-  
+
+  // Testcase for YARN-3021, let RM skip renewing token if the renewer string
+  // is empty
+  @Test(timeout=60000)
+  public void testAppTokenWithNonRenewer() throws Exception {
+    MyFS dfs = (MyFS)FileSystem.get(conf);
+    LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
+
+    // Test would fail if using non-empty renewer string here
+    MyToken token = dfs.getDelegationToken("");
+    token.cancelToken();
+
+    Credentials ts = new Credentials();
+    ts.addToken(token.getKind(), token);
+    
+    // register the tokens for renewal
+    ApplicationId appId =  BuilderUtils.newApplicationId(0, 0);
+    delegationTokenRenewer.addApplicationSync(appId, ts, true, "user");
+  }
+
   /**
    * Basic idea of the test:
    * 1. register a token for 2 seconds with no cancel at the end
@@ -721,7 +741,7 @@ public class TestDelegationTokenRenewer {
       throws IOException, InterruptedException, BrokenBarrierException {
     final Credentials credsx = new Credentials();
     final Token<DelegationTokenIdentifier> tokenx = mock(Token.class);
-    when(tokenx.getKind()).thenReturn(new Text("HDFS_DELEGATION_TOKEN"));
+    when(tokenx.getKind()).thenReturn(KIND);
     DelegationTokenIdentifier dtId1 = 
         new DelegationTokenIdentifier(new Text("user1"), new Text("renewer"),
           new Text("user1"));
@@ -765,7 +785,7 @@ public class TestDelegationTokenRenewer {
     // this token uses barriers to block during renew                          
     final Credentials creds1 = new Credentials();                              
     final Token<DelegationTokenIdentifier> token1 = mock(Token.class);    
-    when(token1.getKind()).thenReturn(new Text("HDFS_DELEGATION_TOKEN"));
+    when(token1.getKind()).thenReturn(KIND);
     DelegationTokenIdentifier dtId1 = 
         new DelegationTokenIdentifier(new Text("user1"), new Text("renewer"),
           new Text("user1"));
@@ -783,7 +803,7 @@ public class TestDelegationTokenRenewer {
     // this dummy token fakes renewing                                         
     final Credentials creds2 = new Credentials();                              
     final Token<DelegationTokenIdentifier> token2 = mock(Token.class);           
-    when(token2.getKind()).thenReturn(new Text("HDFS_DELEGATION_TOKEN"));
+    when(token2.getKind()).thenReturn(KIND);
     when(token2.decodeIdentifier()).thenReturn(dtId1);
     creds2.addToken(new Text("token"), token2);                                
     doReturn(true).when(token2).isManaged();                                   


[47/50] [abbrv] hadoop git commit: HDFS-8165. Move GRANDFATHER_GENERATION_STAMP and GRANDFATER_INODE_ID to hdfs-client. Contributed by Haohui Mai.

Posted by zj...@apache.org.
HDFS-8165. Move GRANDFATHER_GENERATION_STAMP and GRANDFATER_INODE_ID to hdfs-client. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/919f178d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/919f178d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/919f178d

Branch: refs/heads/YARN-2928
Commit: 919f178d10e202f2bbdc238c92fffaec8c073fee
Parents: a9e116e
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Apr 16 13:46:58 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:47 2015 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/HdfsConstantsClient.java      | 34 ++++++++++++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/protocol/Block.java  |  5 ++-
 ...tNamenodeProtocolServerSideTranslatorPB.java |  4 +--
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  5 ++-
 .../server/blockmanagement/BlockIdManager.java  |  7 ++--
 .../hdfs/server/common/GenerationStamp.java     |  6 ----
 .../hdfs/server/datanode/DirectoryScanner.java  |  4 +--
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  6 ++--
 .../datanode/fsdataset/impl/FsDatasetUtil.java  |  4 +--
 .../hdfs/server/namenode/FSEditLogLoader.java   |  5 +--
 .../hdfs/server/namenode/FSEditLogOp.java       |  7 ++--
 .../hdfs/server/namenode/FSNamesystem.java      | 11 ++++---
 .../hadoop/hdfs/server/namenode/INodeId.java    |  9 ++----
 .../offlineImageViewer/ImageLoaderCurrent.java  |  4 +--
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  4 +--
 .../apache/hadoop/hdfs/TestFileCreation.java    |  6 ++--
 .../org/apache/hadoop/hdfs/TestGetBlocks.java   |  5 ++-
 .../server/datanode/TestDirectoryScanner.java   |  6 ++--
 .../server/namenode/NNThroughputBenchmark.java  |  7 ++--
 .../hdfs/server/namenode/TestAddBlockRetry.java | 11 ++++---
 .../namenode/TestFSPermissionChecker.java       |  5 +--
 .../hdfs/server/namenode/TestINodeFile.java     | 15 +++++----
 .../snapshot/TestOpenFilesWithSnapshot.java     |  4 +--
 .../apache/hadoop/hdfs/web/TestJsonUtil.java    |  4 +--
 25 files changed, 106 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
new file mode 100644
index 0000000..aa5f1f9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
@@ -0,0 +1,34 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface HdfsConstantsClient {
+  /**
+   * Generation stamp of blocks that pre-date the introduction
+   * of a generation stamp.
+   */
+  long GRANDFATHER_GENERATION_STAMP = 0;
+  /**
+   * The inode id validation of lease check will be skipped when the request
+   * uses GRANDFATHER_INODE_ID for backward compatibility.
+   */
+  long GRANDFATHER_INODE_ID = 0;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b980f93..062b2e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -445,6 +445,9 @@ Release 2.8.0 - UNRELEASED
     dfs.client.mmap.* and dfs.client.hedged.read.* conf from DFSConfigKeys
     to HdfsClientConfigKeys.  (szetszwo)
 
+    HDFS-8165. Move GRANDFATHER_GENERATION_STAMP and GRANDFATER_INODE_ID to
+    hdfs-client. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
index b35365a..2dc1d04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
@@ -23,7 +23,6 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.io.*;
 
 /**************************************************
@@ -79,7 +78,7 @@ public class Block implements Writable, Comparable<Block> {
   public static long getGenerationStamp(String metaFile) {
     Matcher m = metaFilePattern.matcher(metaFile);
     return m.matches() ? Long.parseLong(m.group(2))
-        : GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+        : HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
   }
 
   /**
@@ -101,7 +100,7 @@ public class Block implements Writable, Comparable<Block> {
   }
 
   public Block(final long blkid) {
-    this(blkid, 0, GenerationStamp.GRANDFATHER_GENERATION_STAMP);
+    this(blkid, 0, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
   }
 
   public Block(Block blk) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index e26158b..563d10b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -210,7 +211,6 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
@@ -533,7 +533,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       boolean result = 
           server.complete(req.getSrc(), req.getClientName(),
           req.hasLast() ? PBHelper.convert(req.getLast()) : null,
-          req.hasFileId() ? req.getFileId() : INodeId.GRANDFATHER_INODE_ID);
+          req.hasFileId() ? req.getFileId() : HdfsConstantsClient.GRANDFATHER_INODE_ID);
       return CompleteResponseProto.newBuilder().setResult(result).build();
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 1942ea9..a7872d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -111,7 +112,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rollin
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@@ -190,7 +190,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
@@ -1438,7 +1437,7 @@ public class PBHelper {
         fs.getFileType().equals(FileType.IS_SYMLINK) ? 
             fs.getSymlink().toByteArray() : null,
         fs.getPath().toByteArray(),
-        fs.hasFileId()? fs.getFileId(): INodeId.GRANDFATHER_INODE_ID,
+        fs.hasFileId()? fs.getFileId(): HdfsConstantsClient.GRANDFATHER_INODE_ID,
         fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 1c69203..eac6ed2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 
 import java.io.IOException;
@@ -55,7 +56,7 @@ public class BlockIdManager {
   private final SequentialBlockIdGenerator blockIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
-    this.generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+    this.generationStampV1Limit = HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
   }
 
@@ -82,7 +83,7 @@ public class BlockIdManager {
    * @param stamp set generation stamp limit to this value
    */
   public void setGenerationStampV1Limit(long stamp) {
-    Preconditions.checkState(generationStampV1Limit == GenerationStamp
+    Preconditions.checkState(generationStampV1Limit == HdfsConstantsClient
       .GRANDFATHER_GENERATION_STAMP);
     generationStampV1Limit = stamp;
   }
@@ -203,6 +204,6 @@ public class BlockIdManager {
     generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+    generationStampV1Limit = HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java
index d09c446..e38c87f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java
@@ -31,12 +31,6 @@ public class GenerationStamp extends SequentialNumber {
   public static final long LAST_RESERVED_STAMP = 1000L;
 
   /**
-   * Generation stamp of blocks that pre-date the introduction
-   * of a generation stamp.
-   */
-  public static final long GRANDFATHER_GENERATION_STAMP = 0;
-
-  /**
    * Create a new instance, initialized to {@link #LAST_RESERVED_STAMP}.
    */
   public GenerationStamp() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 61dfb14..f6b058b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.util.Daemon;
@@ -305,7 +305,7 @@ public class DirectoryScanner implements Runnable {
     public long getGenStamp() {
       return metaSuffix != null ? Block.getGenerationStamp(
           getMetaFile().getName()) : 
-            GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+            HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 6800984..8869f5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -65,8 +65,8 @@ import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
@@ -2101,7 +2101,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
       final long diskGS = diskMetaFile != null && diskMetaFile.exists() ?
           Block.getGenerationStamp(diskMetaFile.getName()) :
-            GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+            HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
 
       if (diskFile == null || !diskFile.exists()) {
         if (memBlockInfo == null) {
@@ -2202,7 +2202,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           // as the block file, then use the generation stamp from it
           long gs = diskMetaFile != null && diskMetaFile.exists()
               && diskMetaFile.getParent().equals(memFile.getParent()) ? diskGS
-              : GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+              : HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
 
           LOG.warn("Updating generation stamp for block " + blockId
               + " from " + memBlockInfo.getGenerationStamp() + " to " + gs);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
index adefbdb..1932706 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
@@ -24,7 +24,7 @@ import java.util.Arrays;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 
 /** Utility methods. */
@@ -88,7 +88,7 @@ public class FsDatasetUtil {
       return Block.getGenerationStamp(listdir[j].getName());
     }
     FsDatasetImpl.LOG.warn("Block " + blockFile + " does not have a metafile!");
-    return GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+    return HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
   }
 
   /** Find the corresponding meta data file from a given block file */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index f50dc4d..d38ef67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -34,6 +34,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -301,7 +302,7 @@ public class FSEditLogLoader {
       long lastInodeId) throws IOException {
     long inodeId = inodeIdFromOp;
 
-    if (inodeId == INodeId.GRANDFATHER_INODE_ID) {
+    if (inodeId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
       if (NameNodeLayoutVersion.supports(
           LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
         throw new IOException("The layout version " + logVersion
@@ -321,7 +322,7 @@ public class FSEditLogLoader {
   @SuppressWarnings("deprecation")
   private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
       StartupOption startOpt, int logVersion, long lastInodeId) throws IOException {
-    long inodeId = INodeId.GRANDFATHER_INODE_ID;
+    long inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
     if (LOG.isTraceEnabled()) {
       LOG.trace("replaying edit log: " + op);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index c768690..2b9bbd2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -101,6 +101,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEditLogProto;
@@ -577,7 +578,7 @@ public abstract class FSEditLogOp {
         this.inodeId = in.readLong();
       } else {
         // The inodeId should be updated when this editLogOp is applied
-        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
       }
       if ((-17 < logVersion && length != 4) ||
           (logVersion <= -17 && length != 5 && !NameNodeLayoutVersion.supports(
@@ -1652,7 +1653,7 @@ public abstract class FSEditLogOp {
         this.inodeId = FSImageSerialization.readLong(in);
       } else {
         // This id should be updated when this editLogOp is applied
-        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
       }
       this.path = FSImageSerialization.readString(in);
       if (NameNodeLayoutVersion.supports(
@@ -2545,7 +2546,7 @@ public abstract class FSEditLogOp {
         this.inodeId = FSImageSerialization.readLong(in);
       } else {
         // This id should be updated when the editLogOp is applied
-        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
       }
       this.path = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index a587443..02a66de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -183,6 +183,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@@ -3200,7 +3201,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     Block previousBlock = ExtendedBlock.getLocalBlock(previous);
     final INode inode;
     final INodesInPath iip;
-    if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+    if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
       // Older clients may not have given us an inode ID to work with.
       // In this case, we have to try to resolve the path and hope it
       // hasn't changed or been deleted since the file was opened for write.
@@ -3320,7 +3321,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       //check lease
       final INode inode;
-      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
         // In this case, we have to try to resolve the path and hope it
         // hasn't changed or been deleted since the file was opened for write.
@@ -3376,7 +3377,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       final INode inode;
       final INodesInPath iip;
-      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
         // In this case, we have to try to resolve the path and hope it
         // hasn't changed or been deleted since the file was opened for write.
@@ -3495,7 +3496,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final INodesInPath iip;
     INode inode = null;
     try {
-      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
         // In this case, we have to try to resolve the path and hope it
         // hasn't changed or been deleted since the file was opened for write.
@@ -3945,7 +3946,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkNameNodeSafeMode("Cannot fsync file " + src);
       src = dir.resolvePath(pc, src, pathComponents);
       final INode inode;
-      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
         // In this case, we have to try to resolve the path and hope it
         // hasn't changed or been deleted since the file was opened for write.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
index 312bb38..5344ca7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.FileNotFoundException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.util.SequentialNumber;
 
 /**
@@ -38,18 +39,12 @@ public class INodeId extends SequentialNumber {
   public static final long ROOT_INODE_ID = LAST_RESERVED_ID + 1;
 
   /**
-   * The inode id validation of lease check will be skipped when the request
-   * uses GRANDFATHER_INODE_ID for backward compatibility.
-   */
-  public static final long GRANDFATHER_INODE_ID = 0;
-
-  /**
    * To check if the request id is the same as saved id. Don't check fileId
    * with GRANDFATHER_INODE_ID for backward compatibility.
    */
   public static void checkId(long requestId, INode inode)
       throws FileNotFoundException {
-    if (requestId != GRANDFATHER_INODE_ID && requestId != inode.getId()) {
+    if (requestId != HdfsConstantsClient.GRANDFATHER_INODE_ID && requestId != inode.getId()) {
       throw new FileNotFoundException(
           "ID mismatch. Request id and saved id: " + requestId + " , "
               + inode.getId() + " for file " + inode.getFullPathName());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
index d8b7e36..3693239 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
@@ -28,11 +28,11 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
 import org.apache.hadoop.io.Text;
@@ -687,7 +687,7 @@ class ImageLoaderCurrent implements ImageLoader {
     final String pathName = readINodePath(in, parentName);
     v.visit(ImageElement.INODE_PATH, pathName);
 
-    long inodeId = INodeId.GRANDFATHER_INODE_ID;
+    long inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
     if (supportInodeId) {
       inodeId = in.readLong();
       v.visit(ImageElement.INODE_ID, inodeId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index 6fa50fe..ad17a7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -33,13 +33,13 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 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.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -124,7 +124,7 @@ class JsonUtilClient {
     final long blockSize = ((Number) m.get("blockSize")).longValue();
     final short replication = ((Number) m.get("replication")).shortValue();
     final long fileId = m.containsKey("fileId") ?
-        ((Number) m.get("fileId")).longValue() : INodeId.GRANDFATHER_INODE_ID;
+        ((Number) m.get("fileId")).longValue() : HdfsConstantsClient.GRANDFATHER_INODE_ID;
     final int childrenNum = getInt(m, "childrenNum", -1);
     final byte storagePolicy = m.containsKey("storagePolicy") ?
         (byte) ((Number) m.get("storagePolicy")).longValue() :

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index fd916a9..0a7b712 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -78,7 +79,6 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -537,7 +537,7 @@ public class TestFileCreation {
 
       // add one block to the file
       LocatedBlock location = client.getNamenode().addBlock(file1.toString(),
-          client.clientName, null, null, INodeId.GRANDFATHER_INODE_ID, null);
+          client.clientName, null, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
       System.out.println("testFileCreationError2: "
           + "Added block " + location.getBlock());
 
@@ -588,7 +588,7 @@ public class TestFileCreation {
       createFile(dfs, f, 3);
       try {
         cluster.getNameNodeRpc().addBlock(f.toString(), client.clientName,
-            null, null, INodeId.GRANDFATHER_INODE_ID, null);
+            null, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
         fail();
       } catch(IOException ioe) {
         FileSystem.LOG.info("GOOD!", ioe);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
index fcc8e35..8ff8655 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
@@ -35,17 +35,16 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 /**
@@ -279,7 +278,7 @@ public class TestGetBlocks {
 
     for (int i = 0; i < blkids.length; i++) {
       Block b = new Block(blkids[i], 0,
-          GenerationStamp.GRANDFATHER_GENERATION_STAMP);
+          HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
       Long v = map.get(b);
       System.out.println(b + " => " + v);
       assertEquals(blkids[i], v.longValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index 0e0e959..fa7a838 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -47,8 +47,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -405,7 +405,7 @@ public class TestDirectoryScanner {
       // Test2: block metafile is missing
       long blockId = deleteMetaFile();
       scan(totalBlocks, 1, 1, 0, 0, 1);
-      verifyGenStamp(blockId, GenerationStamp.GRANDFATHER_GENERATION_STAMP);
+      verifyGenStamp(blockId, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
       scan(totalBlocks, 0, 0, 0, 0, 0);
 
       // Test3: block file is missing
@@ -420,7 +420,7 @@ public class TestDirectoryScanner {
       blockId = createBlockFile();
       totalBlocks++;
       scan(totalBlocks, 1, 1, 0, 1, 0);
-      verifyAddition(blockId, GenerationStamp.GRANDFATHER_GENERATION_STAMP, 0);
+      verifyAddition(blockId, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP, 0);
       scan(totalBlocks, 0, 0, 0, 0, 0);
 
       // Test5: A metafile exists for which there is no block file and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index 9e24f72..360261d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -595,7 +596,7 @@ public class NNThroughputBenchmark implements Tool {
       long end = Time.now();
       for(boolean written = !closeUponCreate; !written; 
         written = nameNodeProto.complete(fileNames[daemonId][inputIdx],
-                                    clientName, null, INodeId.GRANDFATHER_INODE_ID));
+                                    clientName, null, HdfsConstantsClient.GRANDFATHER_INODE_ID));
       return end-start;
     }
 
@@ -1141,7 +1142,7 @@ public class NNThroughputBenchmark implements Tool {
             new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication,
             BLOCK_SIZE, null);
         ExtendedBlock lastBlock = addBlocks(fileName, clientName);
-        nameNodeProto.complete(fileName, clientName, lastBlock, INodeId.GRANDFATHER_INODE_ID);
+        nameNodeProto.complete(fileName, clientName, lastBlock, HdfsConstantsClient.GRANDFATHER_INODE_ID);
       }
       // prepare block reports
       for(int idx=0; idx < nrDatanodes; idx++) {
@@ -1154,7 +1155,7 @@ public class NNThroughputBenchmark implements Tool {
       ExtendedBlock prevBlock = null;
       for(int jdx = 0; jdx < blocksPerFile; jdx++) {
         LocatedBlock loc = nameNodeProto.addBlock(fileName, clientName,
-            prevBlock, null, INodeId.GRANDFATHER_INODE_ID, null);
+            prevBlock, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
         prevBlock = loc.getBlock();
         for(DatanodeInfo dnInfo : loc.getLocations()) {
           int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getXferAddr());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
index d6d2b5e..2d29a68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -88,14 +89,14 @@ public class TestAddBlockRetry {
     LOG.info("Starting first addBlock for " + src);
     LocatedBlock[] onRetryBlock = new LocatedBlock[1];
     DatanodeStorageInfo targets[] = ns.getNewBlockTargets(
-        src, INodeId.GRANDFATHER_INODE_ID, "clientName",
+        src, HdfsConstantsClient.GRANDFATHER_INODE_ID, "clientName",
         null, null, null, onRetryBlock);
     assertNotNull("Targets must be generated", targets);
 
     // run second addBlock()
     LOG.info("Starting second addBlock for " + src);
     nn.addBlock(src, "clientName", null, null,
-        INodeId.GRANDFATHER_INODE_ID, null);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
     assertTrue("Penultimate block must be complete",
         checkFileProgress(src, false));
     LocatedBlocks lbs = nn.getBlockLocations(src, 0, Long.MAX_VALUE);
@@ -105,7 +106,7 @@ public class TestAddBlockRetry {
 
     // continue first addBlock()
     LocatedBlock newBlock = ns.storeAllocatedBlock(
-        src, INodeId.GRANDFATHER_INODE_ID, "clientName", null, targets);
+        src, HdfsConstantsClient.GRANDFATHER_INODE_ID, "clientName", null, targets);
     assertEquals("Blocks are not equal", lb2.getBlock(), newBlock.getBlock());
 
     // check locations
@@ -143,14 +144,14 @@ public class TestAddBlockRetry {
     // start first addBlock()
     LOG.info("Starting first addBlock for " + src);
     LocatedBlock lb1 = nameNodeRpc.addBlock(src, "clientName", null, null,
-        INodeId.GRANDFATHER_INODE_ID, null);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
     assertTrue("Block locations should be present",
         lb1.getLocations().length > 0);
 
     cluster.restartNameNode();
     nameNodeRpc = cluster.getNameNodeRpc();
     LocatedBlock lb2 = nameNodeRpc.addBlock(src, "clientName", null, null,
-        INodeId.GRANDFATHER_INODE_ID, null);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
     assertEquals("Blocks are not equal", lb1.getBlock(), lb2.getBlock());
     assertTrue("Wrong locations with retry", lb2.getLocations().length > 0);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
index 0154a03..d11b783 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -429,7 +430,7 @@ public class TestFSPermissionChecker {
     PermissionStatus permStatus = PermissionStatus.createImmutable(owner, group,
       FsPermission.createImmutable(perm));
     INodeDirectory inodeDirectory = new INodeDirectory(
-      INodeId.GRANDFATHER_INODE_ID, name.getBytes("UTF-8"), permStatus, 0L);
+      HdfsConstantsClient.GRANDFATHER_INODE_ID, name.getBytes("UTF-8"), permStatus, 0L);
     parent.addChild(inodeDirectory);
     return inodeDirectory;
   }
@@ -438,7 +439,7 @@ public class TestFSPermissionChecker {
       String owner, String group, short perm) throws IOException {
     PermissionStatus permStatus = PermissionStatus.createImmutable(owner, group,
       FsPermission.createImmutable(perm));
-    INodeFile inodeFile = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
+    INodeFile inodeFile = new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID,
       name.getBytes("UTF-8"), permStatus, 0L, 0L, null, REPLICATION,
       PREFERRED_BLOCK_SIZE);
     parent.addChild(inodeFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index daac442..8e1f1ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -81,12 +82,12 @@ public class TestINodeFile {
   private long preferredBlockSize = 1024;
 
   INodeFile createINodeFile(short replication, long preferredBlockSize) {
-    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, replication, preferredBlockSize);
   }
 
   private static INodeFile createINodeFile(byte storagePolicyID) {
-    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)3, 1024L, storagePolicyID);
   }
 
@@ -189,9 +190,9 @@ public class TestINodeFile {
     INodeFile inf = createINodeFile(replication, preferredBlockSize);
     inf.setLocalName(DFSUtil.string2Bytes("f"));
 
-    INodeDirectory root = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
+    INodeDirectory root = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID,
         INodeDirectory.ROOT_NAME, perm, 0L);
-    INodeDirectory dir = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
+    INodeDirectory dir = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID,
         DFSUtil.string2Bytes("d"), perm, 0L);
 
     assertEquals("f", inf.getFullPathName());
@@ -340,7 +341,7 @@ public class TestINodeFile {
 
     {//cast from INodeFileUnderConstruction
       final INode from = new INodeFile(
-          INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
+          HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
           1024L);
       from.asFile().toUnderConstruction("client", "machine");
     
@@ -358,7 +359,7 @@ public class TestINodeFile {
     }
 
     {//cast from INodeDirectory
-      final INode from = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, null,
+      final INode from = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID, null,
           perm, 0L);
 
       //cast to INodeFile, should fail
@@ -1103,7 +1104,7 @@ public class TestINodeFile {
   @Test
   public void testFileUnderConstruction() {
     replication = 3;
-    final INodeFile file = new INodeFile(INodeId.GRANDFATHER_INODE_ID, null,
+    final INodeFile file = new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null,
         perm, 0L, 0L, null, replication, 1024L);
     assertFalse(file.isUnderConstruction());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
index ba318de..29c6b10 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -196,7 +196,7 @@ public class TestOpenFilesWithSnapshot {
     String clientName = fs.getClient().getClientName();
     // create one empty block
     nameNodeRpc.addBlock(fileWithEmptyBlock.toString(), clientName, null, null,
-        INodeId.GRANDFATHER_INODE_ID, null);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
     fs.createSnapshot(path, "s2");
 
     fs.rename(new Path("/test/test"), new Path("/test/test-renamed"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/919f178d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
index dfca023..a5d5087 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.util.Time;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectReader;
@@ -65,7 +65,7 @@ public class TestJsonUtil {
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
         now, now + 10, new FsPermission((short) 0644), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
-        INodeId.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);


[28/50] [abbrv] hadoop git commit: YARN-3361. CapacityScheduler side changes to support non-exclusive node labels. Contributed by Wangda Tan

Posted by zj...@apache.org.
YARN-3361. CapacityScheduler side changes to support non-exclusive node labels. Contributed by Wangda Tan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/025787b3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/025787b3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/025787b3

Branch: refs/heads/YARN-2928
Commit: 025787b3d5b9bcefcb8f5ed422e3b1759599fd92
Parents: 77714c8
Author: Jian He <ji...@apache.org>
Authored: Tue Apr 14 11:36:37 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:44 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |    3 +
 .../hadoop/yarn/server/utils/BuilderUtils.java  |    1 +
 .../rmapp/attempt/RMAppAttemptImpl.java         |   10 +-
 .../scheduler/AppSchedulingInfo.java            |   25 +-
 .../scheduler/ResourceUsage.java                |    8 +
 .../scheduler/SchedulerApplicationAttempt.java  |   44 +-
 .../scheduler/SchedulerUtils.java               |   87 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  243 +++--
 .../scheduler/capacity/CSQueue.java             |    5 +-
 .../scheduler/capacity/CapacityScheduler.java   |   91 +-
 .../CapacitySchedulerConfiguration.java         |    5 +
 .../scheduler/capacity/LeafQueue.java           |  368 ++++---
 .../scheduler/capacity/ParentQueue.java         |   59 +-
 .../scheduler/capacity/SchedulingMode.java      |   44 +
 .../server/resourcemanager/Application.java     |    4 +
 .../yarn/server/resourcemanager/MockAM.java     |    8 +-
 .../yarn/server/resourcemanager/MockRM.java     |   35 +-
 .../capacity/TestApplicationLimits.java         |    8 +-
 .../scheduler/capacity/TestChildQueueOrder.java |   41 +-
 .../capacity/TestContainerAllocation.java       |  390 +------
 .../scheduler/capacity/TestLeafQueue.java       |  148 +--
 .../TestNodeLabelContainerAllocation.java       | 1027 ++++++++++++++++++
 .../scheduler/capacity/TestParentQueue.java     |  111 +-
 .../scheduler/capacity/TestReservations.java    |  101 +-
 .../scheduler/capacity/TestUtils.java           |    2 +
 25 files changed, 1914 insertions(+), 954 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6ca51c9..c0bc974 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -131,6 +131,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3443. Create a 'ResourceHandler' subsystem to ease addition of support 
     for new resource types on the NM. (Sidharta Seethana via junping_du)
 
+    YARN-3361. CapacityScheduler side changes to support non-exclusive node
+    labels. (Wangda Tan via jianhe)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
index 68d4ef9..f2146c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
@@ -313,6 +313,7 @@ public class BuilderUtils {
     request.setResourceName(r.getResourceName());
     request.setCapability(r.getCapability());
     request.setNumContainers(r.getNumContainers());
+    request.setNodeLabelExpression(r.getNodeLabelExpression());
     return request;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 1be1727..1071831 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -146,7 +146,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private ConcurrentMap<NodeId, List<ContainerStatus>>
       finishedContainersSentToAM =
       new ConcurrentHashMap<NodeId, List<ContainerStatus>>();
-  private Container masterContainer;
+  private volatile Container masterContainer;
 
   private float progress = 0;
   private String host = "N/A";
@@ -762,13 +762,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
   @Override
   public Container getMasterContainer() {
-    this.readLock.lock();
-
-    try {
-      return this.masterContainer;
-    } finally {
-      this.readLock.unlock();
-    }
+    return this.masterContainer;
   }
 
   @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 5521d47..5604f0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -73,10 +73,11 @@ public class AppSchedulingInfo {
   /* Allocated by scheduler */
   boolean pending = true; // for app metrics
   
+  private ResourceUsage appResourceUsage;
  
   public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
       String user, Queue queue, ActiveUsersManager activeUsersManager,
-      long epoch) {
+      long epoch, ResourceUsage appResourceUsage) {
     this.applicationAttemptId = appAttemptId;
     this.applicationId = appAttemptId.getApplicationId();
     this.queue = queue;
@@ -84,6 +85,7 @@ public class AppSchedulingInfo {
     this.user = user;
     this.activeUsersManager = activeUsersManager;
     this.containerIdCounter = new AtomicLong(epoch << EPOCH_BIT_SHIFT);
+    this.appResourceUsage = appResourceUsage;
   }
 
   public ApplicationId getApplicationId() {
@@ -191,13 +193,19 @@ public class AppSchedulingInfo {
             lastRequestCapability);
         
         // update queue:
+        Resource increasedResource = Resources.multiply(request.getCapability(),
+            request.getNumContainers());
         queue.incPendingResource(
             request.getNodeLabelExpression(),
-            Resources.multiply(request.getCapability(),
-                request.getNumContainers()));
+            increasedResource);
+        appResourceUsage.incPending(request.getNodeLabelExpression(), increasedResource);
         if (lastRequest != null) {
+          Resource decreasedResource =
+              Resources.multiply(lastRequestCapability, lastRequestContainers);
           queue.decPendingResource(lastRequest.getNodeLabelExpression(),
-              Resources.multiply(lastRequestCapability, lastRequestContainers));
+              decreasedResource);
+          appResourceUsage.decPending(lastRequest.getNodeLabelExpression(),
+              decreasedResource);
         }
       }
     }
@@ -385,6 +393,8 @@ public class AppSchedulingInfo {
       checkForDeactivation();
     }
     
+    appResourceUsage.decPending(offSwitchRequest.getNodeLabelExpression(),
+        offSwitchRequest.getCapability());
     queue.decPendingResource(offSwitchRequest.getNodeLabelExpression(),
         offSwitchRequest.getCapability());
   }
@@ -492,9 +502,10 @@ public class AppSchedulingInfo {
   }
   
   public ResourceRequest cloneResourceRequest(ResourceRequest request) {
-    ResourceRequest newRequest = ResourceRequest.newInstance(
-        request.getPriority(), request.getResourceName(),
-        request.getCapability(), 1, request.getRelaxLocality());
+    ResourceRequest newRequest =
+        ResourceRequest.newInstance(request.getPriority(),
+            request.getResourceName(), request.getCapability(), 1,
+            request.getRelaxLocality(), request.getNodeLabelExpression());
     return newRequest;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
index 36ee4da..5169b78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
@@ -27,6 +27,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
@@ -250,6 +251,10 @@ public class ResourceUsage {
   }
 
   private Resource _get(String label, ResourceType type) {
+    if (label == null) {
+      label = RMNodeLabelsManager.NO_LABEL;
+    }
+    
     try {
       readLock.lock();
       UsageByLabel usage = usages.get(label);
@@ -263,6 +268,9 @@ public class ResourceUsage {
   }
 
   private UsageByLabel getAndAddIfMissing(String label) {
+    if (label == null) {
+      label = RMNodeLabelsManager.NO_LABEL;
+    }
     if (!usages.containsKey(label)) {
       UsageByLabel u = new UsageByLabel(label);
       usages.put(label, u);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 5e0bbc7..fccf766 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.base.Preconditions;
@@ -108,14 +110,24 @@ public class SchedulerApplicationAttempt {
   private Set<ContainerId> pendingRelease = null;
 
   /**
-   * Count how many times the application has been given an opportunity
-   * to schedule a task at each priority. Each time the scheduler
-   * asks the application for a task at this priority, it is incremented,
-   * and each time the application successfully schedules a task, it
+   * Count how many times the application has been given an opportunity to
+   * schedule a task at each priority. Each time the scheduler asks the
+   * application for a task at this priority, it is incremented, and each time
+   * the application successfully schedules a task (at rack or node local), it
    * is reset to 0.
    */
   Multiset<Priority> schedulingOpportunities = HashMultiset.create();
   
+  /**
+   * Count how many times the application has been given an opportunity to
+   * schedule a non-partitioned resource request at each priority. Each time the
+   * scheduler asks the application for a task at this priority, it is
+   * incremented, and each time the application successfully schedules a task,
+   * it is reset to 0 when schedule any task at corresponding priority.
+   */
+  Multiset<Priority> missedNonPartitionedRequestSchedulingOpportunity =
+      HashMultiset.create();
+  
   // Time of the last container scheduled at the current allowed level
   protected Map<Priority, Long> lastScheduledContainer =
       new HashMap<Priority, Long>();
@@ -132,7 +144,7 @@ public class SchedulerApplicationAttempt {
     this.rmContext = rmContext;
     this.appSchedulingInfo = 
         new AppSchedulingInfo(applicationAttemptId, user, queue,  
-            activeUsersManager, rmContext.getEpoch());
+            activeUsersManager, rmContext.getEpoch(), attemptResourceUsage);
     this.queue = queue;
     this.pendingRelease = new HashSet<ContainerId>();
     this.attemptId = applicationAttemptId;
@@ -489,6 +501,18 @@ public class SchedulerApplicationAttempt {
     return this.appSchedulingInfo.isBlacklisted(resourceName);
   }
 
+  public synchronized int addMissedNonPartitionedRequestSchedulingOpportunity(
+      Priority priority) {
+    missedNonPartitionedRequestSchedulingOpportunity.add(priority);
+    return missedNonPartitionedRequestSchedulingOpportunity.count(priority);
+  }
+
+  public synchronized void
+      resetMissedNonPartitionedRequestSchedulingOpportunity(Priority priority) {
+    missedNonPartitionedRequestSchedulingOpportunity.setCount(priority, 0);
+  }
+
+  
   public synchronized void addSchedulingOpportunity(Priority priority) {
     schedulingOpportunities.setCount(priority,
         schedulingOpportunities.count(priority) + 1);
@@ -518,6 +542,7 @@ public class SchedulerApplicationAttempt {
   public synchronized void resetSchedulingOpportunities(Priority priority) {
     resetSchedulingOpportunities(priority, System.currentTimeMillis());
   }
+
   // used for continuous scheduling
   public synchronized void resetSchedulingOpportunities(Priority priority,
       long currentTimeMs) {
@@ -669,4 +694,13 @@ public class SchedulerApplicationAttempt {
   public Set<String> getBlacklistedNodes() {
     return this.appSchedulingInfo.getBlackListCopy();
   }
+  
+  @Private
+  public boolean hasPendingResourceRequest(ResourceCalculator rc,
+      String nodePartition, Resource cluster,
+      SchedulingMode schedulingMode) {
+    return SchedulerUtils.hasPendingResourceRequest(rc,
+        this.attemptResourceUsage, nodePartition, cluster,
+        schedulingMode);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
index 248cc08..7a1a528 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
@@ -37,11 +37,10 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import com.google.common.collect.Sets;
-
 /**
  * Utilities shared by schedulers. 
  */
@@ -235,9 +234,13 @@ public class SchedulerUtils {
     if (labelExp == null && queueInfo != null
         && ResourceRequest.ANY.equals(resReq.getResourceName())) {
       labelExp = queueInfo.getDefaultNodeLabelExpression();
-      resReq.setNodeLabelExpression(labelExp);
     }
     
+    // If labelExp still equals to null, set it to be NO_LABEL
+    resReq
+        .setNodeLabelExpression(labelExp == null ? RMNodeLabelsManager.NO_LABEL
+            : labelExp);
+    
     // we don't allow specify label expression other than resourceName=ANY now
     if (!ResourceRequest.ANY.equals(resReq.getResourceName())
         && labelExp != null && !labelExp.trim().isEmpty()) {
@@ -273,25 +276,6 @@ public class SchedulerUtils {
     }
   }
   
-  public static boolean checkQueueAccessToNode(Set<String> queueLabels,
-      Set<String> nodeLabels) {
-    // if queue's label is *, it can access any node
-    if (queueLabels != null && queueLabels.contains(RMNodeLabelsManager.ANY)) {
-      return true;
-    }
-    // any queue can access to a node without label
-    if (nodeLabels == null || nodeLabels.isEmpty()) {
-      return true;
-    }
-    // a queue can access to a node only if it contains any label of the node
-    if (queueLabels != null
-        && Sets.intersection(queueLabels, nodeLabels).size() > 0) {
-      return true;
-    }
-    // sorry, you cannot access
-    return false;
-  }
-  
   public static void checkIfLabelInClusterNodeLabels(RMNodeLabelsManager mgr,
       Set<String> labels) throws IOException {
     if (mgr == null) {
@@ -311,26 +295,6 @@ public class SchedulerUtils {
       }
     }
   }
-  
-  public static boolean checkNodeLabelExpression(Set<String> nodeLabels,
-      String labelExpression) {
-    // empty label expression can only allocate on node with empty labels
-    if (labelExpression == null || labelExpression.trim().isEmpty()) {
-      if (!nodeLabels.isEmpty()) {
-        return false;
-      }
-    }
-
-    if (labelExpression != null) {
-      for (String str : labelExpression.split("&&")) {
-        if (!str.trim().isEmpty()
-            && (nodeLabels == null || !nodeLabels.contains(str.trim()))) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
 
   public static boolean checkQueueLabelExpression(Set<String> queueLabels,
       String labelExpression) {
@@ -360,4 +324,43 @@ public class SchedulerUtils {
     }
     return null;
   }
+  
+  public static boolean checkResourceRequestMatchingNodePartition(
+      ResourceRequest offswitchResourceRequest, String nodePartition,
+      SchedulingMode schedulingMode) {
+    // We will only look at node label = nodeLabelToLookAt according to
+    // schedulingMode and partition of node.
+    String nodePartitionToLookAt = null;
+    if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
+      nodePartitionToLookAt = nodePartition;
+    } else {
+      nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL;
+    }
+    
+    String askedNodePartition = offswitchResourceRequest.getNodeLabelExpression();
+    if (null == askedNodePartition) {
+      askedNodePartition = RMNodeLabelsManager.NO_LABEL;
+    }
+    return askedNodePartition.equals(nodePartitionToLookAt);
+  }
+  
+  private static boolean hasPendingResourceRequest(ResourceCalculator rc,
+      ResourceUsage usage, String partitionToLookAt, Resource cluster) {
+    if (Resources.greaterThan(rc, cluster,
+        usage.getPending(partitionToLookAt), Resources.none())) {
+      return true;
+    }
+    return false;
+  }
+
+  @Private
+  public static boolean hasPendingResourceRequest(ResourceCalculator rc,
+      ResourceUsage usage, String nodePartition, Resource cluster,
+      SchedulingMode schedulingMode) {
+    String partitionToLookAt = nodePartition;
+    if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
+      partitionToLookAt = RMNodeLabelsManager.NO_LABEL;
+    }
+    return hasPendingResourceRequest(rc, usage, partitionToLookAt, cluster);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index 42ea089..d95c45c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
@@ -38,12 +37,12 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
@@ -56,6 +55,11 @@ import com.google.common.collect.Sets;
 public abstract class AbstractCSQueue implements CSQueue {
   private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);
   
+  static final CSAssignment NULL_ASSIGNMENT =
+      new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
+  
+  static final CSAssignment SKIP_ASSIGNMENT = new CSAssignment(true);
+  
   CSQueue parent;
   final String queueName;
   volatile int numContainers;
@@ -343,16 +347,8 @@ public abstract class AbstractCSQueue implements CSQueue {
   }
   
   synchronized void allocateResource(Resource clusterResource, 
-      Resource resource, Set<String> nodeLabels) {
-    
-    // Update usedResources by labels
-    if (nodeLabels == null || nodeLabels.isEmpty()) {
-      queueUsage.incUsed(resource);
-    } else {
-      for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
-        queueUsage.incUsed(label, resource);
-      }
-    }
+      Resource resource, String nodePartition) {
+    queueUsage.incUsed(nodePartition, resource);
 
     ++numContainers;
     CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
@@ -360,15 +356,8 @@ public abstract class AbstractCSQueue implements CSQueue {
   }
   
   protected synchronized void releaseResource(Resource clusterResource,
-      Resource resource, Set<String> nodeLabels) {
-    // Update usedResources by labels
-    if (null == nodeLabels || nodeLabels.isEmpty()) {
-      queueUsage.decUsed(resource);
-    } else {
-      for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
-        queueUsage.decUsed(label, resource);
-      }
-    }
+      Resource resource, String nodePartition) {
+    queueUsage.decUsed(nodePartition, resource);
 
     CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
         clusterResource, minimumAllocation);
@@ -434,103 +423,108 @@ public abstract class AbstractCSQueue implements CSQueue {
                                         parentQ.getPreemptionDisabled());
   }
   
-  private Resource getCurrentLimitResource(String nodeLabel,
-      Resource clusterResource, ResourceLimits currentResourceLimits) {
-    /*
-     * Current limit resource: For labeled resource: limit = queue-max-resource
-     * (TODO, this part need update when we support labeled-limit) For
-     * non-labeled resource: limit = min(queue-max-resource,
-     * limit-set-by-parent)
-     */
-    Resource queueMaxResource =
-        Resources.multiplyAndNormalizeDown(resourceCalculator,
-            labelManager.getResourceByLabel(nodeLabel, clusterResource),
-            queueCapacities.getAbsoluteMaximumCapacity(nodeLabel), minimumAllocation);
-    if (nodeLabel.equals(RMNodeLabelsManager.NO_LABEL)) {
-      return Resources.min(resourceCalculator, clusterResource,
-          queueMaxResource, currentResourceLimits.getLimit());
+  private Resource getCurrentLimitResource(String nodePartition,
+      Resource clusterResource, ResourceLimits currentResourceLimits,
+      SchedulingMode schedulingMode) {
+    if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
+      /*
+       * Current limit resource: For labeled resource: limit = queue-max-resource
+       * (TODO, this part need update when we support labeled-limit) For
+       * non-labeled resource: limit = min(queue-max-resource,
+       * limit-set-by-parent)
+       */
+      Resource queueMaxResource =
+          Resources.multiplyAndNormalizeDown(resourceCalculator,
+              labelManager.getResourceByLabel(nodePartition, clusterResource),
+              queueCapacities.getAbsoluteMaximumCapacity(nodePartition), minimumAllocation);
+      if (nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
+        return Resources.min(resourceCalculator, clusterResource,
+            queueMaxResource, currentResourceLimits.getLimit());
+      }
+      return queueMaxResource;  
+    } else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
+      // When we doing non-exclusive resource allocation, maximum capacity of
+      // all queues on this label equals to total resource with the label.
+      return labelManager.getResourceByLabel(nodePartition, clusterResource);
     }
-    return queueMaxResource;
+    
+    return Resources.none();
   }
   
   synchronized boolean canAssignToThisQueue(Resource clusterResource,
-      Set<String> nodeLabels, ResourceLimits currentResourceLimits,
-      Resource nowRequired, Resource resourceCouldBeUnreserved) {
-    // Get label of this queue can access, it's (nodeLabel AND queueLabel)
-    Set<String> labelCanAccess;
-    if (null == nodeLabels || nodeLabels.isEmpty()) {
-      labelCanAccess = new HashSet<String>();
-      // Any queue can always access any node without label
-      labelCanAccess.add(RMNodeLabelsManager.NO_LABEL);
-    } else {
-      labelCanAccess = new HashSet<String>(
-          accessibleLabels.contains(CommonNodeLabelsManager.ANY) ? nodeLabels
-              : Sets.intersection(accessibleLabels, nodeLabels));
-    }
-    
-    for (String label : labelCanAccess) {
-      // New total resource = used + required
-      Resource newTotalResource =
-          Resources.add(queueUsage.getUsed(label), nowRequired);
-
-      Resource currentLimitResource =
-          getCurrentLimitResource(label, clusterResource, currentResourceLimits);
-
-      // if reservation continous looking enabled, check to see if could we
-      // potentially use this node instead of a reserved node if the application
-      // has reserved containers.
-      // TODO, now only consider reservation cases when the node has no label
-      if (this.reservationsContinueLooking
-          && label.equals(RMNodeLabelsManager.NO_LABEL)
-          && Resources.greaterThan(resourceCalculator, clusterResource,
-              resourceCouldBeUnreserved, Resources.none())) {
-        // resource-without-reserved = used - reserved
-        Resource newTotalWithoutReservedResource =
-            Resources.subtract(newTotalResource, resourceCouldBeUnreserved);
-        
-        // when total-used-without-reserved-resource < currentLimit, we still
-        // have chance to allocate on this node by unreserving some containers
-        if (Resources.lessThan(resourceCalculator, clusterResource,
-            newTotalWithoutReservedResource, currentLimitResource)) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("try to use reserved: " + getQueueName()
-                + " usedResources: " + queueUsage.getUsed()
-                + ", clusterResources: " + clusterResource
-                + ", reservedResources: " + resourceCouldBeUnreserved
-                + ", capacity-without-reserved: "
-                + newTotalWithoutReservedResource + ", maxLimitCapacity: "
-                + currentLimitResource); 
-          }
-          return true;
+      String nodePartition, ResourceLimits currentResourceLimits,
+      Resource nowRequired, Resource resourceCouldBeUnreserved,
+      SchedulingMode schedulingMode) {
+    // New total resource = used + required
+    Resource newTotalResource =
+        Resources.add(queueUsage.getUsed(nodePartition), nowRequired);
+
+    // Get current limited resource: 
+    // - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect
+    // queues' max capacity.
+    // - When doing IGNORE_PARTITION_EXCLUSIVITY allocation, we will not respect
+    // queue's max capacity, queue's max capacity on the partition will be
+    // considered to be 100%. Which is a queue can use all resource in the
+    // partition. 
+    // Doing this because: for non-exclusive allocation, we make sure there's
+    // idle resource on the partition, to avoid wastage, such resource will be
+    // leveraged as much as we can, and preemption policy will reclaim it back
+    // when partitoned-resource-request comes back.  
+    Resource currentLimitResource =
+        getCurrentLimitResource(nodePartition, clusterResource,
+            currentResourceLimits, schedulingMode);
+
+    // if reservation continous looking enabled, check to see if could we
+    // potentially use this node instead of a reserved node if the application
+    // has reserved containers.
+    // TODO, now only consider reservation cases when the node has no label
+    if (this.reservationsContinueLooking
+        && nodePartition.equals(RMNodeLabelsManager.NO_LABEL)
+        && Resources.greaterThan(resourceCalculator, clusterResource,
+            resourceCouldBeUnreserved, Resources.none())) {
+      // resource-without-reserved = used - reserved
+      Resource newTotalWithoutReservedResource =
+          Resources.subtract(newTotalResource, resourceCouldBeUnreserved);
+
+      // when total-used-without-reserved-resource < currentLimit, we still
+      // have chance to allocate on this node by unreserving some containers
+      if (Resources.lessThan(resourceCalculator, clusterResource,
+          newTotalWithoutReservedResource, currentLimitResource)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("try to use reserved: " + getQueueName()
+              + " usedResources: " + queueUsage.getUsed()
+              + ", clusterResources: " + clusterResource
+              + ", reservedResources: " + resourceCouldBeUnreserved
+              + ", capacity-without-reserved: "
+              + newTotalWithoutReservedResource + ", maxLimitCapacity: "
+              + currentLimitResource);
         }
+        return true;
       }
-      
-      // Otherwise, if any of the label of this node beyond queue limit, we
-      // cannot allocate on this node. Consider a small epsilon here.
-      if (Resources.greaterThan(resourceCalculator, clusterResource,
-          newTotalResource, currentLimitResource)) {
-        return false;
-      }
+    }
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(getQueueName()
-            + "Check assign to queue, label=" + label
-            + " usedResources: " + queueUsage.getUsed(label)
-            + " clusterResources: " + clusterResource
-            + " currentUsedCapacity "
-            + Resources.divide(resourceCalculator, clusterResource,
-                queueUsage.getUsed(label),
-                labelManager.getResourceByLabel(label, clusterResource))
-            + " max-capacity: "
-            + queueCapacities.getAbsoluteMaximumCapacity(label)
-            + ")");
-      }
-      return true;
+    // Check if we over current-resource-limit computed.
+    if (Resources.greaterThan(resourceCalculator, clusterResource,
+        newTotalResource, currentLimitResource)) {
+      return false;
     }
-    
-    // Actually, this will not happen, since labelCanAccess will be always
-    // non-empty
-    return false;
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(getQueueName()
+          + "Check assign to queue, nodePartition="
+          + nodePartition
+          + " usedResources: "
+          + queueUsage.getUsed(nodePartition)
+          + " clusterResources: "
+          + clusterResource
+          + " currentUsedCapacity "
+          + Resources.divide(resourceCalculator, clusterResource,
+              queueUsage.getUsed(nodePartition),
+              labelManager.getResourceByLabel(nodePartition, clusterResource))
+          + " max-capacity: "
+          + queueCapacities.getAbsoluteMaximumCapacity(nodePartition) + ")");
+    }
+    return true;
   }
   
   @Override
@@ -556,4 +550,33 @@ public abstract class AbstractCSQueue implements CSQueue {
       parent.decPendingResource(nodeLabel, resourceToDec);
     }
   }
+  
+  /**
+   * Return if the queue has pending resource on given nodePartition and
+   * schedulingMode. 
+   */
+  boolean hasPendingResourceRequest(String nodePartition, 
+      Resource cluster, SchedulingMode schedulingMode) {
+    return SchedulerUtils.hasPendingResourceRequest(resourceCalculator,
+        queueUsage, nodePartition, cluster, schedulingMode);
+  }
+  
+  boolean accessibleToPartition(String nodePartition) {
+    // if queue's label is *, it can access any node
+    if (accessibleLabels != null
+        && accessibleLabels.contains(RMNodeLabelsManager.ANY)) {
+      return true;
+    }
+    // any queue can access to a node without label
+    if (nodePartition == null
+        || nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
+      return true;
+    }
+    // a queue can access to a node only if it contains any label of the node
+    if (accessibleLabels != null && accessibleLabels.contains(nodePartition)) {
+      return true;
+    }
+    // sorry, you cannot access
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
index 1a9448a..b06a646 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
@@ -190,10 +190,13 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    * @param clusterResource the resource of the cluster.
    * @param node node on which resources are available
    * @param resourceLimits how much overall resource of this queue can use. 
+   * @param schedulingMode Type of exclusive check when assign container on a 
+   * NodeManager, see {@link SchedulingMode}.
    * @return the assignment
    */
   public CSAssignment assignContainers(Resource clusterResource,
-      FiCaSchedulerNode node, ResourceLimits resourceLimits);
+      FiCaSchedulerNode node, ResourceLimits resourceLimits,
+      SchedulingMode schedulingMode);
   
   /**
    * A container assigned to the queue has completed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index e93c529..cfeee37 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -35,6 +35,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
@@ -1114,28 +1115,30 @@ public class CapacityScheduler extends
     if (reservedContainer != null) {
       FiCaSchedulerApp reservedApplication =
           getCurrentAttemptForContainer(reservedContainer.getContainerId());
-      
+
       // Try to fulfill the reservation
-      LOG.info("Trying to fulfill reservation for application " + 
-          reservedApplication.getApplicationId() + " on node: " + 
-          node.getNodeID());
-      
-      LeafQueue queue = ((LeafQueue)reservedApplication.getQueue());
-      assignment = queue.assignContainers(
+      LOG.info("Trying to fulfill reservation for application "
+          + reservedApplication.getApplicationId() + " on node: "
+          + node.getNodeID());
+
+      LeafQueue queue = ((LeafQueue) reservedApplication.getQueue());
+      assignment =
+          queue.assignContainers(
               clusterResource,
               node,
               // TODO, now we only consider limits for parent for non-labeled
               // resources, should consider labeled resources as well.
               new ResourceLimits(labelManager.getResourceByLabel(
-                  RMNodeLabelsManager.NO_LABEL, clusterResource)));
+                  RMNodeLabelsManager.NO_LABEL, clusterResource)),
+              SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
       if (assignment.isFulfilledReservation()) {
         CSAssignment tmp =
             new CSAssignment(reservedContainer.getReservedResource(),
-              assignment.getType());
+                assignment.getType());
         Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
-          reservedContainer.getReservedResource());
+            reservedContainer.getReservedResource());
         tmp.getAssignmentInformation().addAllocationDetails(
-          reservedContainer.getContainerId(), queue.getQueuePath());
+            reservedContainer.getContainerId(), queue.getQueuePath());
         tmp.getAssignmentInformation().incrAllocations();
         updateSchedulerHealth(lastNodeUpdateTime, node, tmp);
         schedulerHealth.updateSchedulerFulfilledReservationCounts(1);
@@ -1143,16 +1146,13 @@ public class CapacityScheduler extends
 
       RMContainer excessReservation = assignment.getExcessReservation();
       if (excessReservation != null) {
-      Container container = excessReservation.getContainer();
-      queue.completedContainer(
-          clusterResource, assignment.getApplication(), node, 
-          excessReservation, 
-          SchedulerUtils.createAbnormalContainerStatus(
-              container.getId(), 
-              SchedulerUtils.UNRESERVED_CONTAINER), 
-          RMContainerEventType.RELEASED, null, true);
+        Container container = excessReservation.getContainer();
+        queue.completedContainer(clusterResource, assignment.getApplication(),
+            node, excessReservation, SchedulerUtils
+                .createAbnormalContainerStatus(container.getId(),
+                    SchedulerUtils.UNRESERVED_CONTAINER),
+            RMContainerEventType.RELEASED, null, true);
       }
-
     }
 
     // Try to schedule more if there are no reservations to fulfill
@@ -1163,22 +1163,61 @@ public class CapacityScheduler extends
           LOG.debug("Trying to schedule on node: " + node.getNodeName() +
               ", available: " + node.getAvailableResource());
         }
+
         assignment = root.assignContainers(
             clusterResource,
             node,
             // TODO, now we only consider limits for parent for non-labeled
             // resources, should consider labeled resources as well.
             new ResourceLimits(labelManager.getResourceByLabel(
-                RMNodeLabelsManager.NO_LABEL, clusterResource)));
+                RMNodeLabelsManager.NO_LABEL, clusterResource)),
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+        if (Resources.greaterThan(calculator, clusterResource,
+            assignment.getResource(), Resources.none())) {
+          updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
+          return;
+        }
+        
+        // Only do non-exclusive allocation when node has node-labels.
+        if (StringUtils.equals(node.getPartition(),
+            RMNodeLabelsManager.NO_LABEL)) {
+          return;
+        }
+        
+        // Only do non-exclusive allocation when the node-label supports that
+        try {
+          if (rmContext.getNodeLabelManager().isExclusiveNodeLabel(
+              node.getPartition())) {
+            return;
+          }
+        } catch (IOException e) {
+          LOG.warn("Exception when trying to get exclusivity of node label="
+              + node.getPartition(), e);
+          return;
+        }
+        
+        // Try to use NON_EXCLUSIVE
+        assignment = root.assignContainers(
+            clusterResource,
+            node,
+            // TODO, now we only consider limits for parent for non-labeled
+            // resources, should consider labeled resources as well.
+            new ResourceLimits(labelManager.getResourceByLabel(
+                RMNodeLabelsManager.NO_LABEL, clusterResource)),
+            SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY);
         updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
+        if (Resources.greaterThan(calculator, clusterResource,
+            assignment.getResource(), Resources.none())) {
+          return;
+        }
       }
     } else {
-      LOG.info("Skipping scheduling since node " + node.getNodeID() + 
-          " is reserved by application " + 
-          node.getReservedContainer().getContainerId().getApplicationAttemptId()
-          );
+      LOG.info("Skipping scheduling since node "
+          + node.getNodeID()
+          + " is reserved by application "
+          + node.getReservedContainer().getContainerId()
+              .getApplicationAttemptId());
     }
-  
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
index 102e553..4e8d617 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
@@ -319,6 +319,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     		getMaximumApplicationMasterResourcePercent());
   }
   
+  public void setMaximumApplicationMasterResourcePerQueuePercent(String queue,
+      float percent) {
+    setFloat(getQueuePrefix(queue) + MAXIMUM_AM_RESOURCE_SUFFIX, percent);
+  }
+  
   public float getNonLabeledQueueCapacity(String queue) {
     float capacity = queue.equals("root") ? 100.0f : getFloat(
         getQueuePrefix(queue) + CAPACITY, UNDEFINED);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 59a016f..8a6a601 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -24,7 +24,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -58,6 +57,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
@@ -718,39 +718,11 @@ public class LeafQueue extends AbstractCSQueue {
       ApplicationAttemptId applicationAttemptId) {
     return applicationAttemptMap.get(applicationAttemptId);
   }
-
-  private static final CSAssignment NULL_ASSIGNMENT =
-      new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
-  
-  private static final CSAssignment SKIP_ASSIGNMENT = new CSAssignment(true);
-  
-  private static Set<String> getRequestLabelSetByExpression(
-      String labelExpression) {
-    Set<String> labels = new HashSet<String>();
-    if (null == labelExpression) {
-      return labels;
-    }
-    for (String l : labelExpression.split("&&")) {
-      if (l.trim().isEmpty()) {
-        continue;
-      }
-      labels.add(l.trim());
-    }
-    return labels;
-  }
-  
-  private boolean checkResourceRequestMatchingNodeLabel(ResourceRequest offswitchResourceRequest,
-      FiCaSchedulerNode node) {
-    String askedNodeLabel = offswitchResourceRequest.getNodeLabelExpression();
-    if (null == askedNodeLabel) {
-      askedNodeLabel = RMNodeLabelsManager.NO_LABEL;
-    }
-    return askedNodeLabel.equals(node.getPartition());
-  }
   
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
-      FiCaSchedulerNode node, ResourceLimits currentResourceLimits) {
+      FiCaSchedulerNode node, ResourceLimits currentResourceLimits,
+      SchedulingMode schedulingMode) {
     updateCurrentResourceLimits(currentResourceLimits, clusterResource);
     
     if(LOG.isDebugEnabled()) {
@@ -758,12 +730,6 @@ public class LeafQueue extends AbstractCSQueue {
         + " #applications=" + activeApplications.size());
     }
     
-    // if our queue cannot access this node, just return
-    if (!SchedulerUtils.checkQueueAccessToNode(accessibleLabels,
-        node.getLabels())) {
-      return NULL_ASSIGNMENT;
-    }
-    
     // Check for reserved resources
     RMContainer reservedContainer = node.getReservedContainer();
     if (reservedContainer != null) {
@@ -771,8 +737,26 @@ public class LeafQueue extends AbstractCSQueue {
           getApplication(reservedContainer.getApplicationAttemptId());
       synchronized (application) {
         return assignReservedContainer(application, node, reservedContainer,
-            clusterResource);
+            clusterResource, schedulingMode);
+      }
+    }
+    
+    // if our queue cannot access this node, just return
+    if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
+        && !accessibleToPartition(node.getPartition())) {
+      return NULL_ASSIGNMENT;
+    }
+    
+    // Check if this queue need more resource, simply skip allocation if this
+    // queue doesn't need more resources.
+    if (!hasPendingResourceRequest(node.getPartition(),
+        clusterResource, schedulingMode)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skip this queue=" + getQueuePath()
+            + ", because it doesn't need more resource, schedulingMode="
+            + schedulingMode.name() + " node-partition=" + node.getPartition());
       }
+      return NULL_ASSIGNMENT;
     }
     
     // Try to assign containers to applications in order
@@ -783,6 +767,17 @@ public class LeafQueue extends AbstractCSQueue {
         + application.getApplicationId());
         application.showRequests();
       }
+      
+      // Check if application needs more resource, skip if it doesn't need more.
+      if (!application.hasPendingResourceRequest(resourceCalculator,
+          node.getPartition(), clusterResource, schedulingMode)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId()
+              + ", because it doesn't need more resource, schedulingMode="
+              + schedulingMode.name() + " node-label=" + node.getPartition());
+        }
+        continue;
+      }
 
       synchronized (application) {
         // Check if this resource is on the blacklist
@@ -806,10 +801,27 @@ public class LeafQueue extends AbstractCSQueue {
             continue;
           }
           
+          // AM container allocation doesn't support non-exclusive allocation to
+          // avoid painful of preempt an AM container
+          if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
+            RMAppAttempt rmAppAttempt =
+                csContext.getRMContext().getRMApps()
+                    .get(application.getApplicationId()).getCurrentAppAttempt();
+            if (null == rmAppAttempt.getMasterContainer()) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Skip allocating AM container to app_attempt="
+                    + application.getApplicationAttemptId()
+                    + ", don't allow to allocate AM container in non-exclusive mode");
+              }
+              break;
+            }
+          }
+          
           // Is the node-label-expression of this offswitch resource request
           // matches the node's label?
           // If not match, jump to next priority.
-          if (!checkResourceRequestMatchingNodeLabel(anyRequest, node)) {
+          if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
+              anyRequest, node.getPartition(), schedulingMode)) {
             continue;
           }
           
@@ -822,10 +834,6 @@ public class LeafQueue extends AbstractCSQueue {
             }
           }
           
-          Set<String> requestedNodeLabels =
-              getRequestLabelSetByExpression(anyRequest
-                  .getNodeLabelExpression());
-
           // Compute user-limit & set headroom
           // Note: We compute both user-limit & headroom with the highest 
           //       priority request as the target. 
@@ -833,27 +841,61 @@ public class LeafQueue extends AbstractCSQueue {
           //       before all higher priority ones are serviced.
           Resource userLimit = 
               computeUserLimitAndSetHeadroom(application, clusterResource, 
-                  required, requestedNodeLabels);          
+                  required, node.getPartition(), schedulingMode);          
           
           // Check queue max-capacity limit
-          if (!super.canAssignToThisQueue(clusterResource, node.getLabels(),
-              this.currentResourceLimits, required, application.getCurrentReservation())) {
+          if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
+              this.currentResourceLimits, required,
+              application.getCurrentReservation(), schedulingMode)) {
             return NULL_ASSIGNMENT;
           }
 
           // Check user limit
           if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
-              application, true, requestedNodeLabels)) {
+              application, true, node.getPartition())) {
             break;
           }
 
           // Inform the application it is about to get a scheduling opportunity
           application.addSchedulingOpportunity(priority);
           
+          // Increase missed-non-partitioned-resource-request-opportunity.
+          // This is to make sure non-partitioned-resource-request will prefer
+          // to be allocated to non-partitioned nodes
+          int missedNonPartitionedRequestSchedulingOpportunity = 0;
+          if (anyRequest.getNodeLabelExpression().equals(
+              RMNodeLabelsManager.NO_LABEL)) {
+            missedNonPartitionedRequestSchedulingOpportunity =
+                application
+                    .addMissedNonPartitionedRequestSchedulingOpportunity(priority);
+          }
+          
+          if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
+            // Before doing allocation, we need to check scheduling opportunity to
+            // make sure : non-partitioned resource request should be scheduled to
+            // non-partitioned partition first.
+            if (missedNonPartitionedRequestSchedulingOpportunity < scheduler
+                .getNumClusterNodes()) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Skip app_attempt="
+                    + application.getApplicationAttemptId()
+                    + " priority="
+                    + priority
+                    + " because missed-non-partitioned-resource-request"
+                    + " opportunity under requred:"
+                    + " Now=" + missedNonPartitionedRequestSchedulingOpportunity
+                    + " required="
+                    + scheduler.getNumClusterNodes());
+              }
+
+              break;
+            }
+          }
+          
           // Try to schedule
           CSAssignment assignment =  
             assignContainersOnNode(clusterResource, node, application, priority, 
-                null);
+                null, schedulingMode);
 
           // Did the application skip this node?
           if (assignment.getSkipped()) {
@@ -870,9 +912,9 @@ public class LeafQueue extends AbstractCSQueue {
             // Book-keeping 
             // Note: Update headroom to account for current allocation too...
             allocateResource(clusterResource, application, assigned,
-                node.getLabels());
+                node.getPartition());
             
-            // Don't reset scheduling opportunities for non-local assignments
+            // Don't reset scheduling opportunities for offswitch assignments
             // otherwise the app will be delayed for each non-local assignment.
             // This helps apps with many off-cluster requests schedule faster.
             if (assignment.getType() != NodeType.OFF_SWITCH) {
@@ -881,6 +923,10 @@ public class LeafQueue extends AbstractCSQueue {
               }
               application.resetSchedulingOpportunities(priority);
             }
+            // Non-exclusive scheduling opportunity is different: we need reset
+            // it every time to make sure non-labeled resource request will be
+            // most likely allocated on non-labeled nodes first. 
+            application.resetMissedNonPartitionedRequestSchedulingOpportunity(priority);
             
             // Done
             return assignment;
@@ -904,7 +950,8 @@ public class LeafQueue extends AbstractCSQueue {
 
   private synchronized CSAssignment assignReservedContainer(
       FiCaSchedulerApp application, FiCaSchedulerNode node,
-      RMContainer rmContainer, Resource clusterResource) {
+      RMContainer rmContainer, Resource clusterResource,
+      SchedulingMode schedulingMode) {
     // Do we still need this reservation?
     Priority priority = rmContainer.getReservedPriority();
     if (application.getTotalRequiredResources(priority) == 0) {
@@ -915,7 +962,7 @@ public class LeafQueue extends AbstractCSQueue {
     // Try to assign if we have sufficient resources
     CSAssignment tmp =
         assignContainersOnNode(clusterResource, node, application, priority,
-          rmContainer);
+          rmContainer, schedulingMode);
     
     // Doesn't matter... since it's already charged for at time of reservation
     // "re-reservation" is *free*
@@ -929,7 +976,8 @@ public class LeafQueue extends AbstractCSQueue {
   protected Resource getHeadroom(User user, Resource queueCurrentLimit,
       Resource clusterResource, FiCaSchedulerApp application, Resource required) {
     return getHeadroom(user, queueCurrentLimit, clusterResource,
-	  computeUserLimit(application, clusterResource, required, user, null));
+        computeUserLimit(application, clusterResource, required, user,
+            RMNodeLabelsManager.NO_LABEL, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
   }
   
   private Resource getHeadroom(User user, Resource currentResourceLimit,
@@ -973,7 +1021,8 @@ public class LeafQueue extends AbstractCSQueue {
 
   @Lock({LeafQueue.class, FiCaSchedulerApp.class})
   Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application,
-      Resource clusterResource, Resource required, Set<String> requestedLabels) {
+      Resource clusterResource, Resource required, String nodePartition,
+      SchedulingMode schedulingMode) {
     String user = application.getUser();
     User queueUser = getUser(user);
 
@@ -981,7 +1030,7 @@ public class LeafQueue extends AbstractCSQueue {
     // TODO, need consider headroom respect labels also
     Resource userLimit =
         computeUserLimit(application, clusterResource, required,
-            queueUser, requestedLabels);
+            queueUser, nodePartition, schedulingMode);
 
     setQueueResourceLimitsInfo(clusterResource);
     
@@ -1010,34 +1059,18 @@ public class LeafQueue extends AbstractCSQueue {
   @Lock(NoLock.class)
   private Resource computeUserLimit(FiCaSchedulerApp application,
       Resource clusterResource, Resource required, User user,
-      Set<String> requestedLabels) {
+      String nodePartition, SchedulingMode schedulingMode) {
     // What is our current capacity? 
     // * It is equal to the max(required, queue-capacity) if
     //   we're running below capacity. The 'max' ensures that jobs in queues
     //   with miniscule capacity (< 1 slot) make progress
     // * If we're running over capacity, then its
     //   (usedResources + required) (which extra resources we are allocating)
-    Resource queueCapacity = Resource.newInstance(0, 0);
-    if (requestedLabels != null && !requestedLabels.isEmpty()) {
-      // if we have multiple labels to request, we will choose to use the first
-      // label
-      String firstLabel = requestedLabels.iterator().next();
-      queueCapacity =
-          Resources
-              .max(resourceCalculator, clusterResource, queueCapacity,
-                  Resources.multiplyAndNormalizeUp(resourceCalculator,
-                      labelManager.getResourceByLabel(firstLabel,
-                          clusterResource),
-                      queueCapacities.getAbsoluteCapacity(firstLabel),
-                      minimumAllocation));
-    } else {
-      // else there's no label on request, just to use absolute capacity as
-      // capacity for nodes without label
-      queueCapacity =
-          Resources.multiplyAndNormalizeUp(resourceCalculator, labelManager
-                .getResourceByLabel(CommonNodeLabelsManager.NO_LABEL, clusterResource),
-              queueCapacities.getAbsoluteCapacity(), minimumAllocation);
-    }
+    Resource queueCapacity =
+        Resources.multiplyAndNormalizeUp(resourceCalculator,
+            labelManager.getResourceByLabel(nodePartition, clusterResource),
+            queueCapacities.getAbsoluteCapacity(nodePartition),
+            minimumAllocation);
 
     // Allow progress for queues with miniscule capacity
     queueCapacity =
@@ -1047,33 +1080,56 @@ public class LeafQueue extends AbstractCSQueue {
             required);
 
     Resource currentCapacity =
-        Resources.lessThan(resourceCalculator, clusterResource, 
-            queueUsage.getUsed(), queueCapacity) ?
-            queueCapacity : Resources.add(queueUsage.getUsed(), required);
+        Resources.lessThan(resourceCalculator, clusterResource,
+            queueUsage.getUsed(nodePartition), queueCapacity) ? queueCapacity
+            : Resources.add(queueUsage.getUsed(nodePartition), required);
     
     // Never allow a single user to take more than the 
     // queue's configured capacity * user-limit-factor.
     // Also, the queue's configured capacity should be higher than 
     // queue-hard-limit * ulMin
     
-    final int activeUsers = activeUsersManager.getNumActiveUsers();  
-    		
-    Resource limit =
+    final int activeUsers = activeUsersManager.getNumActiveUsers();
+    
+    // User limit resource is determined by:
+    // max{currentCapacity / #activeUsers, currentCapacity * user-limit-percentage%)
+    Resource userLimitResource = Resources.max(
+        resourceCalculator, clusterResource, 
+        Resources.divideAndCeil(
+            resourceCalculator, currentCapacity, activeUsers),
+        Resources.divideAndCeil(
+            resourceCalculator, 
+            Resources.multiplyAndRoundDown(
+                currentCapacity, userLimit), 
+            100)
+        );
+    
+    // User limit is capped by maxUserLimit
+    // - maxUserLimit = queueCapacity * user-limit-factor (RESPECT_PARTITION_EXCLUSIVITY)
+    // - maxUserLimit = total-partition-resource (IGNORE_PARTITION_EXCLUSIVITY)
+    //
+    // In IGNORE_PARTITION_EXCLUSIVITY mode, if a queue cannot access a
+    // partition, its guaranteed resource on that partition is 0. And
+    // user-limit-factor computation is based on queue's guaranteed capacity. So
+    // we will not cap user-limit as well as used resource when doing
+    // IGNORE_PARTITION_EXCLUSIVITY allocation.
+    Resource maxUserLimit = Resources.none();
+    if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
+      maxUserLimit =
+          Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor);
+    } else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
+      maxUserLimit =
+          labelManager.getResourceByLabel(nodePartition, clusterResource);
+    }
+    
+    // Cap final user limit with maxUserLimit
+    userLimitResource =
         Resources.roundUp(
             resourceCalculator, 
             Resources.min(
                 resourceCalculator, clusterResource,   
-                Resources.max(
-                    resourceCalculator, clusterResource, 
-                    Resources.divideAndCeil(
-                        resourceCalculator, currentCapacity, activeUsers),
-                    Resources.divideAndCeil(
-                        resourceCalculator, 
-                        Resources.multiplyAndRoundDown(
-                            currentCapacity, userLimit), 
-                        100)
-                    ), 
-                Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor)
+                  userLimitResource,
+                  maxUserLimit
                 ), 
             minimumAllocation);
 
@@ -1081,11 +1137,11 @@ public class LeafQueue extends AbstractCSQueue {
       String userName = application.getUser();
       LOG.debug("User limit computation for " + userName + 
           " in queue " + getQueueName() +
-          " userLimit=" + userLimit +
+          " userLimitPercent=" + userLimit +
           " userLimitFactor=" + userLimitFactor +
           " required: " + required + 
           " consumed: " + user.getUsed() + 
-          " limit: " + limit +
+          " user-limit-resource: " + userLimitResource +
           " queueCapacity: " + queueCapacity + 
           " qconsumed: " + queueUsage.getUsed() +
           " currentCapacity: " + currentCapacity +
@@ -1093,31 +1149,26 @@ public class LeafQueue extends AbstractCSQueue {
           " clusterCapacity: " + clusterResource
       );
     }
-    user.setUserResourceLimit(limit);
-    return limit;
+    user.setUserResourceLimit(userLimitResource);
+    return userLimitResource;
   }
   
   @Private
   protected synchronized boolean canAssignToUser(Resource clusterResource,
       String userName, Resource limit, FiCaSchedulerApp application,
-      boolean checkReservations, Set<String> requestLabels) {
+      boolean checkReservations, String nodePartition) {
     User user = getUser(userName);
-    
-    String label = CommonNodeLabelsManager.NO_LABEL;
-    if (requestLabels != null && !requestLabels.isEmpty()) {
-      label = requestLabels.iterator().next();
-    }
 
     // Note: We aren't considering the current request since there is a fixed
     // overhead of the AM, but it's a > check, not a >= check, so...
     if (Resources
         .greaterThan(resourceCalculator, clusterResource,
-            user.getUsed(label),
+            user.getUsed(nodePartition),
             limit)) {
       // if enabled, check to see if could we potentially use this node instead
       // of a reserved node if the application has reserved containers
       if (this.reservationsContinueLooking && checkReservations
-          && label.equals(CommonNodeLabelsManager.NO_LABEL)) {
+          && nodePartition.equals(CommonNodeLabelsManager.NO_LABEL)) {
         if (Resources.lessThanOrEqual(
             resourceCalculator,
             clusterResource,
@@ -1136,7 +1187,7 @@ public class LeafQueue extends AbstractCSQueue {
       if (LOG.isDebugEnabled()) {
         LOG.debug("User " + userName + " in queue " + getQueueName()
             + " will exceed limit - " + " consumed: "
-            + user.getUsed() + " limit: " + limit);
+            + user.getUsed(nodePartition) + " limit: " + limit);
       }
       return false;
     }
@@ -1176,7 +1227,7 @@ public class LeafQueue extends AbstractCSQueue {
 
   private CSAssignment assignContainersOnNode(Resource clusterResource,
       FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority,
-      RMContainer reservedContainer) {
+      RMContainer reservedContainer, SchedulingMode schedulingMode) {
 
     CSAssignment assigned;
 
@@ -1190,7 +1241,7 @@ public class LeafQueue extends AbstractCSQueue {
       assigned =
           assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest, 
             node, application, priority, reservedContainer,
-            allocatedContainer);
+            allocatedContainer, schedulingMode);
       if (Resources.greaterThan(resourceCalculator, clusterResource,
         assigned.getResource(), Resources.none())) {
 
@@ -1219,7 +1270,7 @@ public class LeafQueue extends AbstractCSQueue {
       assigned = 
           assignRackLocalContainers(clusterResource, rackLocalResourceRequest, 
             node, application, priority, reservedContainer,
-            allocatedContainer);
+            allocatedContainer, schedulingMode);
       if (Resources.greaterThan(resourceCalculator, clusterResource,
         assigned.getResource(), Resources.none())) {
 
@@ -1248,7 +1299,7 @@ public class LeafQueue extends AbstractCSQueue {
       assigned =
           assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
             node, application, priority, reservedContainer,
-            allocatedContainer);
+            allocatedContainer, schedulingMode);
 
       // update locality statistics
       if (allocatedContainer.getValue() != null) {
@@ -1314,16 +1365,17 @@ public class LeafQueue extends AbstractCSQueue {
 
   @Private
   protected boolean checkLimitsToReserve(Resource clusterResource,
-      FiCaSchedulerApp application, Resource capability) {
+      FiCaSchedulerApp application, Resource capability, String nodePartition,
+      SchedulingMode schedulingMode) {
     // we can't reserve if we got here based on the limit
     // checks assuming we could unreserve!!!
     Resource userLimit = computeUserLimitAndSetHeadroom(application,
-        clusterResource, capability, null);
+        clusterResource, capability, nodePartition, schedulingMode);
 
     // Check queue max-capacity limit,
     // TODO: Consider reservation on labels
-    if (!canAssignToThisQueue(clusterResource, null,
-        this.currentResourceLimits, capability, Resources.none())) {
+    if (!canAssignToThisQueue(clusterResource, RMNodeLabelsManager.NO_LABEL,
+        this.currentResourceLimits, capability, Resources.none(), schedulingMode)) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("was going to reserve but hit queue limit");
       }
@@ -1332,7 +1384,7 @@ public class LeafQueue extends AbstractCSQueue {
 
     // Check user limit
     if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
-        application, false, null)) {
+        application, false, nodePartition)) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("was going to reserve but hit user limit");
       }
@@ -1345,12 +1397,13 @@ public class LeafQueue extends AbstractCSQueue {
   private CSAssignment assignNodeLocalContainers(Resource clusterResource,
       ResourceRequest nodeLocalResourceRequest, FiCaSchedulerNode node,
       FiCaSchedulerApp application, Priority priority,
-      RMContainer reservedContainer, MutableObject allocatedContainer) {
+      RMContainer reservedContainer, MutableObject allocatedContainer,
+      SchedulingMode schedulingMode) {
     if (canAssign(application, priority, node, NodeType.NODE_LOCAL, 
         reservedContainer)) {
       return assignContainer(clusterResource, node, application, priority,
           nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer,
-          allocatedContainer);
+          allocatedContainer, schedulingMode);
     }
 
     return new CSAssignment(Resources.none(), NodeType.NODE_LOCAL);
@@ -1359,12 +1412,13 @@ public class LeafQueue extends AbstractCSQueue {
   private CSAssignment assignRackLocalContainers(Resource clusterResource,
       ResourceRequest rackLocalResourceRequest, FiCaSchedulerNode node,
       FiCaSchedulerApp application, Priority priority,
-      RMContainer reservedContainer, MutableObject allocatedContainer) {
+      RMContainer reservedContainer, MutableObject allocatedContainer,
+      SchedulingMode schedulingMode) {
     if (canAssign(application, priority, node, NodeType.RACK_LOCAL,
         reservedContainer)) {
       return assignContainer(clusterResource, node, application, priority,
           rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer,
-          allocatedContainer);
+          allocatedContainer, schedulingMode);
     }
 
     return new CSAssignment(Resources.none(), NodeType.RACK_LOCAL);
@@ -1373,16 +1427,21 @@ public class LeafQueue extends AbstractCSQueue {
   private CSAssignment assignOffSwitchContainers(Resource clusterResource,
       ResourceRequest offSwitchResourceRequest, FiCaSchedulerNode node,
       FiCaSchedulerApp application, Priority priority,
-      RMContainer reservedContainer, MutableObject allocatedContainer) {
+      RMContainer reservedContainer, MutableObject allocatedContainer,
+      SchedulingMode schedulingMode) {
     if (canAssign(application, priority, node, NodeType.OFF_SWITCH,
         reservedContainer)) {
       return assignContainer(clusterResource, node, application, priority,
           offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer,
-          allocatedContainer);
+          allocatedContainer, schedulingMode);
     }
     
     return new CSAssignment(Resources.none(), NodeType.OFF_SWITCH);
   }
+  
+  private int getActualNodeLocalityDelay() {
+    return Math.min(scheduler.getNumClusterNodes(), getNodeLocalityDelay());
+  }
 
   boolean canAssign(FiCaSchedulerApp application, Priority priority, 
       FiCaSchedulerNode node, NodeType type, RMContainer reservedContainer) {
@@ -1417,10 +1476,7 @@ public class LeafQueue extends AbstractCSQueue {
     if (type == NodeType.RACK_LOCAL) {
       // 'Delay' rack-local just a little bit...
       long missedOpportunities = application.getSchedulingOpportunities(priority);
-      return (
-          Math.min(scheduler.getNumClusterNodes(), getNodeLocalityDelay()) < 
-          missedOpportunities
-          );
+      return getActualNodeLocalityDelay() < missedOpportunities;
     }
 
     // Check if we need containers on this host
@@ -1460,7 +1516,7 @@ public class LeafQueue extends AbstractCSQueue {
   private CSAssignment assignContainer(Resource clusterResource, FiCaSchedulerNode node,
       FiCaSchedulerApp application, Priority priority, 
       ResourceRequest request, NodeType type, RMContainer rmContainer,
-      MutableObject createdContainer) {
+      MutableObject createdContainer, SchedulingMode schedulingMode) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("assignContainers: node=" + node.getNodeName()
         + " application=" + application.getApplicationId()
@@ -1469,9 +1525,8 @@ public class LeafQueue extends AbstractCSQueue {
     }
     
     // check if the resource request can access the label
-    if (!SchedulerUtils.checkNodeLabelExpression(
-        node.getLabels(),
-        request.getNodeLabelExpression())) {
+    if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(request,
+        node.getPartition(), schedulingMode)) {
       // this is a reserved container, but we cannot allocate it now according
       // to label not match. This can be caused by node label changed
       // We should un-reserve this container.
@@ -1576,8 +1631,8 @@ public class LeafQueue extends AbstractCSQueue {
           // If we're trying to reserve a container here, not container will be
           // unreserved for reserving the new one. Check limits again before
           // reserve the new container
-          if (!checkLimitsToReserve(clusterResource, 
-              application, capability)) {
+          if (!checkLimitsToReserve(clusterResource,
+              application, capability, node.getPartition(), schedulingMode)) {
             return new CSAssignment(Resources.none(), type);
           }
         }
@@ -1666,7 +1721,7 @@ public class LeafQueue extends AbstractCSQueue {
         // Book-keeping
         if (removed) {
           releaseResource(clusterResource, application,
-              container.getResource(), node.getLabels());
+              container.getResource(), node.getPartition());
           LOG.info("completedContainer" +
               " container=" + container +
               " queue=" + this +
@@ -1684,13 +1739,13 @@ public class LeafQueue extends AbstractCSQueue {
 
   synchronized void allocateResource(Resource clusterResource,
       SchedulerApplicationAttempt application, Resource resource,
-      Set<String> nodeLabels) {
-    super.allocateResource(clusterResource, resource, nodeLabels);
+      String nodePartition) {
+    super.allocateResource(clusterResource, resource, nodePartition);
     
     // Update user metrics
     String userName = application.getUser();
     User user = getUser(userName);
-    user.assignContainer(resource, nodeLabels);
+    user.assignContainer(resource, nodePartition);
     // Note this is a bit unconventional since it gets the object and modifies
     // it here, rather then using set routine
     Resources.subtractFrom(application.getHeadroom(), resource); // headroom
@@ -1707,13 +1762,13 @@ public class LeafQueue extends AbstractCSQueue {
   }
 
   synchronized void releaseResource(Resource clusterResource, 
-      FiCaSchedulerApp application, Resource resource, Set<String> nodeLabels) {
-    super.releaseResource(clusterResource, resource, nodeLabels);
+      FiCaSchedulerApp application, Resource resource, String nodePartition) {
+    super.releaseResource(clusterResource, resource, nodePartition);
     
     // Update user metrics
     String userName = application.getUser();
     User user = getUser(userName);
-    user.releaseContainer(resource, nodeLabels);
+    user.releaseContainer(resource, nodePartition);
     metrics.setAvailableResourcesToUser(userName, application.getHeadroom());
       
     LOG.info(getQueueName() + 
@@ -1723,7 +1778,8 @@ public class LeafQueue extends AbstractCSQueue {
   
   private void updateAbsoluteCapacityResource(Resource clusterResource) {
     absoluteCapacityResource =
-        Resources.multiplyAndNormalizeUp(resourceCalculator, clusterResource,
+        Resources.multiplyAndNormalizeUp(resourceCalculator, labelManager
+            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
             queueCapacities.getAbsoluteCapacity(), minimumAllocation);
   }
   
@@ -1769,8 +1825,9 @@ public class LeafQueue extends AbstractCSQueue {
     // Update application properties
     for (FiCaSchedulerApp application : activeApplications) {
       synchronized (application) {
-        computeUserLimitAndSetHeadroom(application, clusterResource, 
-            Resources.none(), null);
+        computeUserLimitAndSetHeadroom(application, clusterResource,
+            Resources.none(), RMNodeLabelsManager.NO_LABEL,
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
       }
     }
   }
@@ -1828,25 +1885,12 @@ public class LeafQueue extends AbstractCSQueue {
       }
     }
 
-    public void assignContainer(Resource resource,
-        Set<String> nodeLabels) {
-      if (nodeLabels == null || nodeLabels.isEmpty()) {
-        userResourceUsage.incUsed(resource);
-      } else {
-        for (String label : nodeLabels) {
-          userResourceUsage.incUsed(label, resource);
-        }
-      }
+    public void assignContainer(Resource resource, String nodePartition) {
+      userResourceUsage.incUsed(nodePartition, resource);
     }
 
-    public void releaseContainer(Resource resource, Set<String> nodeLabels) {
-      if (nodeLabels == null || nodeLabels.isEmpty()) {
-        userResourceUsage.decUsed(resource);
-      } else {
-        for (String label : nodeLabels) {
-          userResourceUsage.decUsed(label, resource);
-        }
-      }
+    public void releaseContainer(Resource resource, String nodePartition) {
+      userResourceUsage.decUsed(nodePartition, resource);
     }
 
     public Resource getUserResourceLimit() {
@@ -1869,7 +1913,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       allocateResource(clusterResource, attempt, rmContainer.getContainer()
-          .getResource(), node.getLabels());
+          .getResource(), node.getPartition());
     }
     getParent().recoverContainer(clusterResource, attempt, rmContainer);
   }
@@ -1909,7 +1953,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       allocateResource(clusterResource, application, rmContainer.getContainer()
-          .getResource(), node.getLabels());
+          .getResource(), node.getPartition());
       LOG.info("movedContainer" + " container=" + rmContainer.getContainer()
           + " resource=" + rmContainer.getContainer().getResource()
           + " queueMoveIn=" + this + " usedCapacity=" + getUsedCapacity()
@@ -1927,7 +1971,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       releaseResource(clusterResource, application, rmContainer.getContainer()
-          .getResource(), node.getLabels());
+          .getResource(), node.getPartition());
       LOG.info("movedContainer" + " container=" + rmContainer.getContainer()
           + " resource=" + rmContainer.getContainer().getResource()
           + " queueMoveOut=" + this + " usedCapacity=" + getUsedCapacity()


[42/50] [abbrv] hadoop git commit: HDFS-8082. Move dfs.client.read.*, dfs.client.short.circuit.*, dfs.client.mmap.* and dfs.client.hedged.read.* conf from DFSConfigKeys to HdfsClientConfigKeys.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java
index 6e381c1..c24eb33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.TestBlockReaderLocal;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -247,8 +248,8 @@ public class TestShortCircuitLocalRead {
       int readOffset, String shortCircuitUser, String readingUser,
       boolean legacyShortCircuitFails) throws IOException, InterruptedException {
     Configuration conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
         ignoreChecksum);
     // Set a random client context name so that we don't share a cache with
     // other invocations of this function.
@@ -384,8 +385,8 @@ public class TestShortCircuitLocalRead {
   public void testSkipWithVerifyChecksum() throws IOException {
     int size = blockSize;
     Configuration conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
         "/tmp/testSkipWithVerifyChecksum._PORT");
     DomainSocket.disableBindPathValidation();
@@ -431,8 +432,8 @@ public class TestShortCircuitLocalRead {
   public void testHandleTruncatedBlockFile() throws IOException {
     MiniDFSCluster cluster = null;
     HdfsConfiguration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
         "/tmp/testHandleTruncatedBlockFile._PORT");
     conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
@@ -529,10 +530,10 @@ public class TestShortCircuitLocalRead {
 
     // Setup create a file
     final Configuration conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, shortcircuit);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, shortcircuit);
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
         "/tmp/TestShortCircuitLocalRead._PORT");
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
         checksum);
     
     //Override fileSize and DATA_TO_WRITE to much larger values for benchmark test
@@ -592,7 +593,7 @@ public class TestShortCircuitLocalRead {
                                                           int readOffset, boolean shortCircuitFails) throws IOException, InterruptedException {
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER, true);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
              .format(true).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
index 2981558..5d6db16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.tracing;
 
 import static org.junit.Assume.assumeTrue;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
@@ -26,19 +28,17 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.htrace.Sampler;
-import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import java.io.IOException;
 
 public class TestTracingShortCircuitLocalRead {
   private static Configuration conf;
@@ -67,8 +67,8 @@ public class TestTracingShortCircuitLocalRead {
     conf.set(SpanReceiverHost.SPAN_RECEIVERS_CONF_KEY,
         TestTracing.SetSpanReceiver.class.getName());
     conf.setLong("dfs.blocksize", 100 * 1024);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
         "testShortCircuitTraceHooks._PORT");
     conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");


[50/50] [abbrv] hadoop git commit: YARN-2696. Queue sorting in CapacityScheduler should consider node label. Contributed by Wangda Tan

Posted by zj...@apache.org.
YARN-2696. Queue sorting in CapacityScheduler should consider node label. Contributed by Wangda Tan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4459349c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4459349c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4459349c

Branch: refs/heads/YARN-2928
Commit: 4459349c8905062d137e86ad6fcad108d335fa76
Parents: 2353676
Author: Jian He <ji...@apache.org>
Authored: Fri Apr 17 13:36:46 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:31:30 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../dev-support/findbugs-exclude.xml            |   6 +-
 .../nodelabels/RMNodeLabelsManager.java         |   2 +-
 .../scheduler/AbstractYarnScheduler.java        |   4 +
 .../scheduler/ResourceUsage.java                |  10 +
 .../scheduler/capacity/AbstractCSQueue.java     |  12 +-
 .../scheduler/capacity/CSQueueUtils.java        | 129 ++++--
 .../scheduler/capacity/CapacityScheduler.java   |  15 +-
 .../capacity/CapacitySchedulerContext.java      |   4 +-
 .../scheduler/capacity/LeafQueue.java           |   5 +-
 .../scheduler/capacity/ParentQueue.java         |  46 +-
 .../capacity/PartitionedQueueComparator.java    |  68 +++
 .../scheduler/capacity/QueueCapacities.java     |  11 +-
 .../scheduler/capacity/ReservationQueue.java    |   7 +-
 .../capacity/TestApplicationLimits.java         |  12 +-
 .../scheduler/capacity/TestChildQueueOrder.java |   4 +-
 .../scheduler/capacity/TestLeafQueue.java       |   4 +-
 .../TestNodeLabelContainerAllocation.java       | 451 +++++++++++++++++++
 .../scheduler/capacity/TestParentQueue.java     |   4 +-
 .../scheduler/capacity/TestReservations.java    |   4 +-
 .../scheduler/fifo/TestFifoScheduler.java       |   4 +
 21 files changed, 722 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6749b4e..415cf9c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -198,6 +198,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3404. Display queue name on application page. (Ryu Kobayashi via jianhe)
 
+    YARN-2696. Queue sorting in CapacityScheduler should consider node label.
+    (Wangda Tan via jianhe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 4b01a4d..8aae152 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -141,10 +141,14 @@
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.resource.Priority$Comparator" />
     <Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE" />
   </Match>
-    <Match>
+  <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoComparator" />
     <Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE" />
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PartitionedQueueComparator" />
+    <Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE" />
+  </Match>
   <!-- Ignore some irrelevant class name warning -->
   <Match>
     <Class name="org.apache.hadoop.yarn.api.records.SerializedException" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
index 574e24c..25e5bc09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
@@ -254,7 +254,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
     }
   }
 
-  public void updateNodeResource(NodeId node, Resource newResource) throws IOException {
+  public void updateNodeResource(NodeId node, Resource newResource) {
     deactivateNode(node);
     activateNode(node, newResource);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 6699b05..1a8c653 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -548,6 +548,10 @@ public abstract class AbstractYarnScheduler
     Resource newResource = resourceOption.getResource();
     Resource oldResource = node.getTotalResource();
     if(!oldResource.equals(newResource)) {
+      // Notify NodeLabelsManager about this change
+      rmContext.getNodeLabelManager().updateNodeResource(nm.getNodeID(),
+          newResource);
+      
       // Log resource change
       LOG.info("Update resource on node: " + node.getNodeName()
           + " from: " + oldResource + ", to: "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
index 2f7e19d..88e93c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -372,4 +373,13 @@ public class ResourceUsage {
       readLock.unlock();
     }
   }
+  
+  public Set<String> getNodePartitionsSet() {
+    try {
+      readLock.lock();
+      return usages.keySet();
+    } finally {
+      readLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index d95c45c..550c6aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -271,8 +271,8 @@ public abstract class AbstractCSQueue implements CSQueue {
     this.acls = csContext.getConfiguration().getAcls(getQueuePath());
 
     // Update metrics
-    CSQueueUtils.updateQueueStatistics(
-        resourceCalculator, this, parent, clusterResource, minimumAllocation);
+    CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
+        minimumAllocation, this, labelManager, null);
     
     // Check if labels of this queue is a subset of parent queue, only do this
     // when we not root
@@ -351,16 +351,16 @@ public abstract class AbstractCSQueue implements CSQueue {
     queueUsage.incUsed(nodePartition, resource);
 
     ++numContainers;
-    CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
-        clusterResource, minimumAllocation);
+    CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
+        minimumAllocation, this, labelManager, nodePartition);
   }
   
   protected synchronized void releaseResource(Resource clusterResource,
       Resource resource, String nodePartition) {
     queueUsage.decUsed(nodePartition, resource);
 
-    CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
-        clusterResource, minimumAllocation);
+    CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
+        minimumAllocation, this, labelManager, nodePartition);
     --numContainers;
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
index 1921195..8f9362e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
@@ -20,18 +20,17 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import com.google.common.collect.Sets;
+
 class CSQueueUtils {
-  
-  private static final Log LOG = LogFactory.getLog(CSQueueUtils.class);
 
   final static float EPSILON = 0.0001f;
   
@@ -188,41 +187,103 @@ class CSQueueUtils {
     }
   }
   
-  @Lock(CSQueue.class)
-  public static void updateQueueStatistics(
-      final ResourceCalculator calculator,
-      final CSQueue childQueue, final CSQueue parentQueue, 
-      final Resource clusterResource, final Resource minimumAllocation) {
-    Resource queueLimit = Resources.none();
-    Resource usedResources = childQueue.getUsedResources();
-    
+  /**
+   * Update partitioned resource usage, if nodePartition == null, will update
+   * used resource for all partitions of this queue.
+   */
+  private static void updateUsedCapacity(final ResourceCalculator rc,
+      final Resource totalPartitionResource, final Resource minimumAllocation,
+      ResourceUsage queueResourceUsage, QueueCapacities queueCapacities,
+      String nodePartition) {
     float absoluteUsedCapacity = 0.0f;
     float usedCapacity = 0.0f;
 
-    if (Resources.greaterThan(
-        calculator, clusterResource, clusterResource, Resources.none())) {
-      queueLimit = 
-          Resources.multiply(clusterResource, childQueue.getAbsoluteCapacity());
-      absoluteUsedCapacity = 
-          Resources.divide(calculator, clusterResource, 
-              usedResources, clusterResource);
-      usedCapacity = 
-          Resources.equals(queueLimit, Resources.none()) ? 0 :
-          Resources.divide(calculator, clusterResource, 
-              usedResources, queueLimit);
+    if (Resources.greaterThan(rc, totalPartitionResource,
+        totalPartitionResource, Resources.none())) {
+      // queueGuaranteed = totalPartitionedResource *
+      // absolute_capacity(partition)
+      Resource queueGuranteedResource =
+          Resources.multiply(totalPartitionResource,
+              queueCapacities.getAbsoluteCapacity(nodePartition));
+
+      // make queueGuranteed >= minimum_allocation to avoid divided by 0.
+      queueGuranteedResource =
+          Resources.max(rc, totalPartitionResource, queueGuranteedResource,
+              minimumAllocation);
+
+      Resource usedResource = queueResourceUsage.getUsed(nodePartition);
+      absoluteUsedCapacity =
+          Resources.divide(rc, totalPartitionResource, usedResource,
+              totalPartitionResource);
+      usedCapacity =
+          Resources.divide(rc, totalPartitionResource, usedResource,
+              queueGuranteedResource);
+    }
+
+    queueCapacities
+        .setAbsoluteUsedCapacity(nodePartition, absoluteUsedCapacity);
+    queueCapacities.setUsedCapacity(nodePartition, usedCapacity);
+  }
+  
+  private static Resource getNonPartitionedMaxAvailableResourceToQueue(
+      final ResourceCalculator rc, Resource totalNonPartitionedResource,
+      CSQueue queue) {
+    Resource queueLimit = Resources.none();
+    Resource usedResources = queue.getUsedResources();
+
+    if (Resources.greaterThan(rc, totalNonPartitionedResource,
+        totalNonPartitionedResource, Resources.none())) {
+      queueLimit =
+          Resources.multiply(totalNonPartitionedResource,
+              queue.getAbsoluteCapacity());
     }
 
-    childQueue.setUsedCapacity(usedCapacity);
-    childQueue.setAbsoluteUsedCapacity(absoluteUsedCapacity);
-    
     Resource available = Resources.subtract(queueLimit, usedResources);
-    childQueue.getMetrics().setAvailableResourcesToQueue(
-        Resources.max(
-            calculator, 
-            clusterResource, 
-            available, 
-            Resources.none()
-            )
-        );
+    return Resources.max(rc, totalNonPartitionedResource, available,
+        Resources.none());
+  }
+  
+  /**
+   * <p>
+   * Update Queue Statistics:
+   * </p>
+   *  
+   * <li>used-capacity/absolute-used-capacity by partition</li> 
+   * <li>non-partitioned max-avail-resource to queue</li>
+   * 
+   * <p>
+   * When nodePartition is null, all partition of
+   * used-capacity/absolute-used-capacity will be updated.
+   * </p>
+   */
+  @Lock(CSQueue.class)
+  public static void updateQueueStatistics(
+      final ResourceCalculator rc, final Resource cluster, final Resource minimumAllocation,
+      final CSQueue childQueue, final RMNodeLabelsManager nlm, 
+      final String nodePartition) {
+    QueueCapacities queueCapacities = childQueue.getQueueCapacities();
+    ResourceUsage queueResourceUsage = childQueue.getQueueResourceUsage();
+    
+    if (nodePartition == null) {
+      for (String partition : Sets.union(
+          queueCapacities.getNodePartitionsSet(),
+          queueResourceUsage.getNodePartitionsSet())) {
+        updateUsedCapacity(rc, nlm.getResourceByLabel(partition, cluster),
+            minimumAllocation, queueResourceUsage, queueCapacities, partition);
+      }
+    } else {
+      updateUsedCapacity(rc, nlm.getResourceByLabel(nodePartition, cluster),
+          minimumAllocation, queueResourceUsage, queueCapacities, nodePartition);
+    }
+    
+    // Now in QueueMetrics, we only store available-resource-to-queue for
+    // default partition.
+    if (nodePartition == null
+        || nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
+      childQueue.getMetrics().setAvailableResourcesToQueue(
+          getNonPartitionedMaxAvailableResourceToQueue(rc,
+              nlm.getResourceByLabel(RMNodeLabelsManager.NO_LABEL, cluster),
+              childQueue));
+    }
    }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index cfeee37..5d58b15 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -136,7 +136,8 @@ public class CapacityScheduler extends
   // timeout to join when we stop this service
   protected final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
-  static final Comparator<CSQueue> queueComparator = new Comparator<CSQueue>() {
+  static final Comparator<CSQueue> nonPartitionedQueueComparator =
+      new Comparator<CSQueue>() {
     @Override
     public int compare(CSQueue q1, CSQueue q2) {
       if (q1.getUsedCapacity() < q2.getUsedCapacity()) {
@@ -148,6 +149,9 @@ public class CapacityScheduler extends
       return q1.getQueuePath().compareTo(q2.getQueuePath());
     }
   };
+  
+  static final PartitionedQueueComparator partitionedQueueComparator =
+      new PartitionedQueueComparator();
 
   static final Comparator<FiCaSchedulerApp> applicationComparator = 
     new Comparator<FiCaSchedulerApp>() {
@@ -274,8 +278,13 @@ public class CapacityScheduler extends
   }
 
   @Override
-  public Comparator<CSQueue> getQueueComparator() {
-    return queueComparator;
+  public Comparator<CSQueue> getNonPartitionedQueueComparator() {
+    return nonPartitionedQueueComparator;
+  }
+  
+  @Override
+  public PartitionedQueueComparator getPartitionedQueueComparator() {
+    return partitionedQueueComparator;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java
index 28dc988..707c463 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java
@@ -58,7 +58,9 @@ public interface CapacitySchedulerContext {
 
   ResourceCalculator getResourceCalculator();
 
-  Comparator<CSQueue> getQueueComparator();
+  Comparator<CSQueue> getNonPartitionedQueueComparator();
+  
+  PartitionedQueueComparator getPartitionedQueueComparator();
   
   FiCaSchedulerNode getNode(NodeId nodeId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 8a6a601..f860574 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -1814,9 +1814,8 @@ public class LeafQueue extends AbstractCSQueue {
     setQueueResourceLimitsInfo(clusterResource);
     
     // Update metrics
-    CSQueueUtils.updateQueueStatistics(
-        resourceCalculator, this, getParent(), clusterResource, 
-        minimumAllocation);
+    CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
+        minimumAllocation, this, labelManager, null);
 
     // queue metrics are updated, more resource may be available
     // activate the pending applications if possible

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
index eb64d43..53142b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -68,7 +69,8 @@ public class ParentQueue extends AbstractCSQueue {
 
   protected final Set<CSQueue> childQueues;  
   private final boolean rootQueue;
-  final Comparator<CSQueue> queueComparator;
+  final Comparator<CSQueue> nonPartitionedQueueComparator;
+  final PartitionedQueueComparator partitionQueueComparator;
   volatile int numApplications;
   private final CapacitySchedulerContext scheduler;
 
@@ -79,7 +81,8 @@ public class ParentQueue extends AbstractCSQueue {
       String queueName, CSQueue parent, CSQueue old) throws IOException {
     super(cs, queueName, parent, old);
     this.scheduler = cs;
-    this.queueComparator = cs.getQueueComparator();
+    this.nonPartitionedQueueComparator = cs.getNonPartitionedQueueComparator();
+    this.partitionQueueComparator = cs.getPartitionedQueueComparator();
 
     this.rootQueue = (parent == null);
 
@@ -92,7 +95,7 @@ public class ParentQueue extends AbstractCSQueue {
           ". Must be " + CapacitySchedulerConfiguration.MAXIMUM_CAPACITY_VALUE);
     }
     
-    this.childQueues = new TreeSet<CSQueue>(queueComparator);
+    this.childQueues = new TreeSet<CSQueue>(nonPartitionedQueueComparator);
     
     setupQueueConfigs(cs.getClusterResource());
 
@@ -522,6 +525,17 @@ public class ParentQueue extends AbstractCSQueue {
     return new ResourceLimits(childLimit);
   }
   
+  private Iterator<CSQueue> sortAndGetChildrenAllocationIterator(FiCaSchedulerNode node) {
+    if (node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) {
+      return childQueues.iterator();
+    }
+
+    partitionQueueComparator.setPartitionToLookAt(node.getPartition());
+    List<CSQueue> childrenList = new ArrayList<>(childQueues);
+    Collections.sort(childrenList, partitionQueueComparator);
+    return childrenList.iterator();
+  }
+  
   private synchronized CSAssignment assignContainersToChildQueues(
       Resource cluster, FiCaSchedulerNode node, ResourceLimits limits,
       SchedulingMode schedulingMode) {
@@ -531,7 +545,8 @@ public class ParentQueue extends AbstractCSQueue {
     printChildQueues();
 
     // Try to assign to most 'under-served' sub-queue
-    for (Iterator<CSQueue> iter = childQueues.iterator(); iter.hasNext();) {
+    for (Iterator<CSQueue> iter = sortAndGetChildrenAllocationIterator(node); iter
+        .hasNext();) {
       CSQueue childQueue = iter.next();
       if(LOG.isDebugEnabled()) {
         LOG.debug("Trying to assign to queue: " + childQueue.getQueuePath()
@@ -554,13 +569,17 @@ public class ParentQueue extends AbstractCSQueue {
       if (Resources.greaterThan(
               resourceCalculator, cluster, 
               assignment.getResource(), Resources.none())) {
-        // Remove and re-insert to sort
-        iter.remove();
-        LOG.info("Re-sorting assigned queue: " + childQueue.getQueuePath() + 
-            " stats: " + childQueue);
-        childQueues.add(childQueue);
-        if (LOG.isDebugEnabled()) {
-          printChildQueues();
+        // Only update childQueues when we doing non-partitioned node
+        // allocation.
+        if (RMNodeLabelsManager.NO_LABEL.equals(node.getPartition())) {
+          // Remove and re-insert to sort
+          iter.remove();
+          LOG.info("Re-sorting assigned queue: " + childQueue.getQueuePath()
+              + " stats: " + childQueue);
+          childQueues.add(childQueue);
+          if (LOG.isDebugEnabled()) {
+            printChildQueues();
+          }
         }
         break;
       }
@@ -647,9 +666,8 @@ public class ParentQueue extends AbstractCSQueue {
       childQueue.updateClusterResource(clusterResource, childLimits);
     }
     
-    // Update metrics
-    CSQueueUtils.updateQueueStatistics(
-        resourceCalculator, this, parent, clusterResource, minimumAllocation);
+    CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
+        minimumAllocation, this, labelManager, null);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PartitionedQueueComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PartitionedQueueComparator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PartitionedQueueComparator.java
new file mode 100644
index 0000000..ddcc719
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PartitionedQueueComparator.java
@@ -0,0 +1,68 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.util.Comparator;
+
+public class PartitionedQueueComparator implements Comparator<CSQueue> {
+  private String partitionToLookAt = null;
+  
+  public void setPartitionToLookAt(String partitionToLookAt) {
+    this.partitionToLookAt = partitionToLookAt;
+  }
+  
+
+  @Override
+  public int compare(CSQueue q1, CSQueue q2) {
+    /*
+     * 1. Check accessible to given partition, if one queue accessible and
+     * the other not, accessible queue goes first.
+     */
+    boolean q1Accessible =
+        q1.getAccessibleNodeLabels().contains(partitionToLookAt);
+    boolean q2Accessible =
+        q2.getAccessibleNodeLabels().contains(partitionToLookAt);
+    if (q1Accessible && !q2Accessible) {
+      return -1;
+    } else if (!q1Accessible && q2Accessible) {
+      return 1;
+    }
+    
+    /*
+     * 
+     * 2. When two queue has same accessibility, check who will go first:
+     * Now we simply compare their used resource on the partition to lookAt
+     */
+    float used1 = q1.getQueueCapacities().getUsedCapacity(partitionToLookAt);
+    float used2 = q2.getQueueCapacities().getUsedCapacity(partitionToLookAt);
+    if (Math.abs(used1 - used2) < 1e-6) {
+      // When used capacity is same, compare their guaranteed-capacity
+      float cap1 = q1.getQueueCapacities().getCapacity(partitionToLookAt);
+      float cap2 = q2.getQueueCapacities().getCapacity(partitionToLookAt);
+      
+      // when cap1 == cap2, we will compare queue's name
+      if (Math.abs(cap1 - cap2) < 1e-6) {
+        return q1.getQueueName().compareTo(q2.getQueueName());
+      }
+      return Float.compare(cap2, cap1);
+    }
+    
+    return Float.compare(used1, used2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java
index 962a636..d0a26d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java
@@ -30,8 +30,6 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 
-import com.google.common.collect.Sets;
-
 public class QueueCapacities {
   private static final String NL = CommonNodeLabelsManager.NO_LABEL;
   private static final float LABEL_DOESNT_EXIST_CAP = 0f;
@@ -254,4 +252,13 @@ public class QueueCapacities {
       readLock.unlock();
     }
   }
+  
+  public Set<String> getNodePartitionsSet() {
+    try {
+      readLock.lock();
+      return capacitiesMap.keySet();
+    } finally {
+      readLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
index a8d17cf..4790cc7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
@@ -63,10 +63,9 @@ public class ReservationQueue extends LeafQueue {
           + " from " + newlyParsedQueue.getQueuePath());
     }
     super.reinitialize(newlyParsedQueue, clusterResource);
-    CSQueueUtils.updateQueueStatistics(
-        parent.schedulerContext.getResourceCalculator(), newlyParsedQueue,
-        parent, parent.schedulerContext.getClusterResource(),
-        parent.schedulerContext.getMinimumResourceCapability());
+    CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
+        minimumAllocation, this, labelManager, null);
+
     updateQuotas(parent.getUserLimitForReservation(),
         parent.getUserLimitFactor(),
         parent.getMaxApplicationsForReservations(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
index 46167ca..a41fdfa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
@@ -95,8 +95,8 @@ public class TestApplicationLimits {
         thenReturn(Resources.createResource(10 * 16 * GB, 10 * 32));
     when(csContext.getApplicationComparator()).
         thenReturn(CapacityScheduler.applicationComparator);
-    when(csContext.getQueueComparator()).
-        thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getNonPartitionedQueueComparator()).
+        thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
@@ -255,8 +255,8 @@ public class TestApplicationLimits {
         thenReturn(Resources.createResource(16*GB, 16));
     when(csContext.getApplicationComparator()).
         thenReturn(CapacityScheduler.applicationComparator);
-    when(csContext.getQueueComparator()).
-        thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getNonPartitionedQueueComparator()).
+        thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
     
@@ -554,8 +554,8 @@ public class TestApplicationLimits {
         thenReturn(Resources.createResource(16*GB));
     when(csContext.getApplicationComparator()).
         thenReturn(CapacityScheduler.applicationComparator);
-    when(csContext.getQueueComparator()).
-        thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getNonPartitionedQueueComparator()).
+        thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
index 970a98a..2608dcb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
@@ -96,8 +96,8 @@ public class TestChildQueueOrder {
     thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getApplicationComparator()).
     thenReturn(CapacityScheduler.applicationComparator);
-    when(csContext.getQueueComparator()).
-    thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getNonPartitionedQueueComparator()).
+    thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 0b5250b..0a19604 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -152,8 +152,8 @@ public class TestLeafQueue {
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getApplicationComparator()).
     thenReturn(CapacityScheduler.applicationComparator);
-    when(csContext.getQueueComparator()).
-        thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getNonPartitionedQueueComparator()).
+        thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
index cf1b26f..5155db5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
@@ -1024,4 +1024,455 @@ public class TestNodeLabelContainerAllocation {
 
     rm1.close();
   }
+  
+  private void checkQueueUsedCapacity(String queueName, CapacityScheduler cs,
+      String nodePartition, float usedCapacity, float absoluteUsedCapacity) {
+    float epsilon = 1e-6f;
+    CSQueue queue = cs.getQueue(queueName);
+    Assert.assertNotNull("Failed to get queue=" + queueName, queue);
+
+    Assert.assertEquals(usedCapacity, queue.getQueueCapacities()
+        .getUsedCapacity(nodePartition), epsilon);
+    Assert.assertEquals(absoluteUsedCapacity, queue.getQueueCapacities()
+        .getAbsoluteUsedCapacity(nodePartition), epsilon);
+  }
+  
+  private void doNMHeartbeat(MockRM rm, NodeId nodeId, int nHeartbeat) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nodeId);
+    for (int i = 0; i < nHeartbeat; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+  }
+  
+  private void waitSchedulerNodeJoined(MockRM rm, int expectedNodeNum)
+      throws InterruptedException {
+    int totalWaitTick = 100; // wait 10 sec at most.
+    while (expectedNodeNum > rm.getResourceScheduler().getNumClusterNodes()
+        && totalWaitTick > 0) {
+      Thread.sleep(100);
+      totalWaitTick--;
+    }
+  }
+  
+  @Test
+  public void testQueueUsedCapacitiesUpdate()
+          throws Exception {
+    /**
+     * Test case: have a following queue structure:
+     * 
+     * <pre>
+     *            root
+     *         /      \
+     *        a        b
+     *       / \      (x)
+     *      a1  a2
+     *     (x)  (x)
+     * </pre>
+     * 
+     * Both a/b can access x, we need to verify when
+     * <pre>
+     * 1) container allocated/released in both partitioned/non-partitioned node, 
+     * 2) clusterResource updates
+     * 3) queue guaranteed resource changed
+     * </pre>
+     * 
+     * used capacity / absolute used capacity of queues are correctly updated.
+     */
+
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(this.conf);
+
+    // Define top-level queues
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a",
+        "b" });
+    csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+
+    /**
+     * Initially, we set A/B's resource 50:50
+     */
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    csConf.setCapacity(A, 50);
+    csConf.setAccessibleNodeLabels(A, toSet("x"));
+    csConf.setCapacityByLabel(A, "x", 50);
+    
+    csConf.setQueues(A, new String[] { "a1", "a2" });
+    
+    final String A1 = A + ".a1";
+    csConf.setCapacity(A1, 50);
+    csConf.setAccessibleNodeLabels(A1, toSet("x"));
+    csConf.setCapacityByLabel(A1, "x", 50);
+    
+    final String A2 = A + ".a2";
+    csConf.setCapacity(A2, 50);
+    csConf.setAccessibleNodeLabels(A2, toSet("x"));
+    csConf.setCapacityByLabel(A2, "x", 50);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(B, 50);
+    csConf.setAccessibleNodeLabels(B, toSet("x"));
+    csConf.setCapacityByLabel(B, "x", 50);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x"));
+    // Makes x to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+    
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    /*
+     * Before we adding any node to the cluster, used-capacity/abs-used-capacity
+     * should be 0
+     */
+    checkQueueUsedCapacity("a", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("a1", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a1", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("a2", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a2", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "", 0f, 0f);
+    
+    MockNM nm1 = rm.registerNode("h1:1234", 10 * GB); // label = x
+    MockNM nm2 = rm.registerNode("h2:1234", 10 * GB); // label = <empty>
+    
+    /*
+     * After we adding nodes to the cluster, and before starting to use them,
+     * used-capacity/abs-used-capacity should be 0
+     */
+    checkQueueUsedCapacity("a", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("a1", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a1", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("a2", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a2", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "", 0f, 0f);
+
+    // app1 -> a1
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+
+    // app1 asks for 1 partition= containers
+    am1.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>());
+    
+    doNMHeartbeat(rm, nm2.getNodeId(), 10);
+    
+    // Now check usage, app1 uses:
+    //   a1: used(no-label) = 80%
+    //       abs-used(no-label) = 20%
+    //   a: used(no-label) = 40%
+    //       abs-used(no-label) = 20%
+    //   root: used(no-label) = 20%
+    //       abs-used(no-label) = 20%
+    checkQueueUsedCapacity("a", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a", cs, "", 0.4f, 0.2f);
+    checkQueueUsedCapacity("a1", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a1", cs, "", 0.8f, 0.2f);
+    checkQueueUsedCapacity("a2", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a2", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "", 0.2f, 0.2f);
+    
+    // app1 asks for 2 partition=x containers
+    am1.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>(), "x");
+    doNMHeartbeat(rm, nm1.getNodeId(), 10);
+    
+    // Now check usage, app1 uses:
+    //   a1: used(x) = 80%
+    //       abs-used(x) = 20%
+    //   a: used(x) = 40%
+    //       abs-used(x) = 20%
+    //   root: used(x) = 20%
+    //       abs-used(x) = 20%
+    checkQueueUsedCapacity("a", cs, "x", 0.4f, 0.2f);
+    checkQueueUsedCapacity("a", cs, "", 0.4f, 0.2f);
+    checkQueueUsedCapacity("a1", cs, "x", 0.8f, 0.2f);
+    checkQueueUsedCapacity("a1", cs, "", 0.8f, 0.2f);
+    checkQueueUsedCapacity("a2", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("a2", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "x", 0.2f, 0.2f);
+    checkQueueUsedCapacity("root", cs, "", 0.2f, 0.2f);
+    
+    // submit an app to a2, uses 1 NON_PARTITIONED container and 1 PARTITIONED
+    // container
+    // app2 -> a2
+    RMApp app2 = rm.submitApp(1 * GB, "app", "user", null, "a2");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+
+    // app1 asks for 1 partition= containers
+    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "x");
+    doNMHeartbeat(rm, nm1.getNodeId(), 10);
+    
+    // Now check usage, app1 uses:
+    //   a2: used(x) = 40%
+    //       abs-used(x) = 10%
+    //   a: used(x) = 20%
+    //       abs-used(x) = 10%
+    //   root: used(x) = 10%
+    //       abs-used(x) = 10%
+    checkQueueUsedCapacity("a", cs, "x", 0.6f, 0.3f);
+    checkQueueUsedCapacity("a", cs, "", 0.6f, 0.3f);
+    checkQueueUsedCapacity("a1", cs, "x", 0.8f, 0.2f);
+    checkQueueUsedCapacity("a1", cs, "", 0.8f, 0.2f);
+    checkQueueUsedCapacity("a2", cs, "x", 0.4f, 0.1f);
+    checkQueueUsedCapacity("a2", cs, "", 0.4f, 0.1f);
+    checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "x", 0.3f, 0.3f);
+    checkQueueUsedCapacity("root", cs, "", 0.3f, 0.3f);
+    
+    // Add nm3/nm4, double resource for both partitioned/non-partitioned
+    // resource, used capacity should be 1/2 of before
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h3", 0), toSet("x")));
+    rm.registerNode("h3:1234", 10 * GB); // label = x
+    rm.registerNode("h4:1234", 10 * GB); // label = <empty>
+    
+    waitSchedulerNodeJoined(rm, 4);
+    
+    checkQueueUsedCapacity("a", cs, "x", 0.3f, 0.15f);
+    checkQueueUsedCapacity("a", cs, "", 0.3f, 0.15f);
+    checkQueueUsedCapacity("a1", cs, "x", 0.4f, 0.1f);
+    checkQueueUsedCapacity("a1", cs, "", 0.4f, 0.1f);
+    checkQueueUsedCapacity("a2", cs, "x", 0.2f, 0.05f);
+    checkQueueUsedCapacity("a2", cs, "", 0.2f, 0.05f);
+    checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "x", 0.15f, 0.15f);
+    checkQueueUsedCapacity("root", cs, "", 0.15f, 0.15f);
+    
+    // Reinitialize queue, makes A's capacity double, and B's capacity to be 0
+    csConf.setCapacity(A, 100); // was 50
+    csConf.setCapacityByLabel(A, "x", 100); // was 50
+    csConf.setCapacity(B, 0); // was 50
+    csConf.setCapacityByLabel(B, "x", 0); // was 50
+    cs.reinitialize(csConf, rm.getRMContext());
+    
+    checkQueueUsedCapacity("a", cs, "x", 0.15f, 0.15f);
+    checkQueueUsedCapacity("a", cs, "", 0.15f, 0.15f);
+    checkQueueUsedCapacity("a1", cs, "x", 0.2f, 0.1f);
+    checkQueueUsedCapacity("a1", cs, "", 0.2f, 0.1f);
+    checkQueueUsedCapacity("a2", cs, "x", 0.1f, 0.05f);
+    checkQueueUsedCapacity("a2", cs, "", 0.1f, 0.05f);
+    checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "x", 0.15f, 0.15f);
+    checkQueueUsedCapacity("root", cs, "", 0.15f, 0.15f);
+    
+    // Release all task containers from a1, check usage
+    am1.allocate(null, Arrays.asList(
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2),
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3),
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 4)));
+    checkQueueUsedCapacity("a", cs, "x", 0.05f, 0.05f);
+    checkQueueUsedCapacity("a", cs, "", 0.10f, 0.10f);
+    checkQueueUsedCapacity("a1", cs, "x", 0.0f, 0.0f);
+    checkQueueUsedCapacity("a1", cs, "", 0.1f, 0.05f);
+    checkQueueUsedCapacity("a2", cs, "x", 0.1f, 0.05f);
+    checkQueueUsedCapacity("a2", cs, "", 0.1f, 0.05f);
+    checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
+    checkQueueUsedCapacity("b", cs, "", 0f, 0f);
+    checkQueueUsedCapacity("root", cs, "x", 0.05f, 0.05f);
+    checkQueueUsedCapacity("root", cs, "", 0.10f, 0.10f);
+
+    rm.close();
+  }
+  
+  @Test
+  public void testOrderOfAllocationOnPartitions()
+          throws Exception {
+    /**
+     * Test case: have a following queue structure:
+     * 
+     * <pre>
+     *                root
+     *          ________________
+     *         /     |     \    \
+     *        a (x)  b (x)  c    d
+     * </pre>
+     * 
+     * Both a/b can access x, we need to verify when
+     * <pre>
+     * When doing allocation on partitioned nodes,
+     *    - Queue has accessibility to the node will go first
+     *    - When accessibility is same
+     *      - Queue has less used_capacity on given partition will go first
+     *      - When used_capacity is same
+     *        - Queue has more abs_capacity will go first
+     * </pre>
+     * 
+     * used capacity / absolute used capacity of queues are correctly updated.
+     */
+
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(this.conf);
+
+    // Define top-level queues
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a",
+        "b", "c", "d" });
+    csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    csConf.setCapacity(A, 25);
+    csConf.setAccessibleNodeLabels(A, toSet("x"));
+    csConf.setCapacityByLabel(A, "x", 30);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(B, 25);
+    csConf.setAccessibleNodeLabels(B, toSet("x"));
+    csConf.setCapacityByLabel(B, "x", 70);
+    
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    csConf.setCapacity(C, 25);
+    
+    final String D = CapacitySchedulerConfiguration.ROOT + ".d";
+    csConf.setCapacity(D, 25);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x"));
+    // Makes x to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+    
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    MockNM nm1 = rm.registerNode("h1:1234", 10 * GB); // label = x
+    MockNM nm2 = rm.registerNode("h2:1234", 10 * GB); // label = <empty>
+    
+    // app1 -> a
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+    
+    // app2 -> b
+    RMApp app2 = rm.submitApp(1 * GB, "app", "user", null, "b");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+    
+    // app3 -> c
+    RMApp app3 = rm.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm, nm2);
+    
+    // app4 -> d
+    RMApp app4 = rm.submitApp(1 * GB, "app", "user", null, "d");
+    MockAM am4 = MockRM.launchAndRegisterAM(app4, rm, nm2);
+
+    // Test case 1
+    // Both a/b has used_capacity(x) = 0, when doing exclusive allocation, b
+    // will go first since b has more capacity(x)
+    am1.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "x");
+    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "x");
+    doNMHeartbeat(rm, nm1.getNodeId(), 1);
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+    
+    // Test case 2
+    // Do another allocation, a will go first since it has 0 use_capacity(x) and
+    // b has 1/7 used_capacity(x)
+    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "x");
+    doNMHeartbeat(rm, nm1.getNodeId(), 1);
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+    
+    // Test case 3
+    // Just like above, when doing non-exclusive allocation, b will go first as well.
+    am1.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "");
+    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "");
+    doNMHeartbeat(rm, nm1.getNodeId(), 2);
+    checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+    
+    // Test case 4
+    // After b allocated, we should be able to allocate non-exlusive container in a
+    doNMHeartbeat(rm, nm1.getNodeId(), 2);
+    checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+    
+    // Test case 5
+    // b/c/d asks non-exclusive container together, b will go first irrelated to
+    // used_capacity(x)
+    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "");
+    am3.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>(), "");
+    am4.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>(), "");
+    doNMHeartbeat(rm, nm1.getNodeId(), 2);
+    checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(0, nm1.getNodeId(),
+        cs.getApplicationAttempt(am3.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(0, nm1.getNodeId(),
+        cs.getApplicationAttempt(am4.getApplicationAttemptId()));
+    
+    // Test case 6
+    // After b allocated, c will go first by lexicographic order
+    doNMHeartbeat(rm, nm1.getNodeId(), 1);
+    checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am3.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(0, nm1.getNodeId(),
+        cs.getApplicationAttempt(am4.getApplicationAttemptId()));
+    
+    // Test case 7
+    // After c allocated, d will go first because it has less used_capacity(x)
+    // than c
+    doNMHeartbeat(rm, nm1.getNodeId(), 2);
+    checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am3.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am4.getApplicationAttemptId()));
+    
+    // Test case 8
+    // After d allocated, c will go first, c/d has same use_capacity(x), so compare c/d's lexicographic order
+    doNMHeartbeat(rm, nm1.getNodeId(), 1);
+    checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
+        cs.getApplicationAttempt(am3.getApplicationAttemptId()));
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am4.getApplicationAttemptId()));
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
index 52d0bc1..bdbd168 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
@@ -92,8 +92,8 @@ public class TestParentQueue {
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getApplicationComparator()).
     thenReturn(CapacityScheduler.applicationComparator);
-    when(csContext.getQueueComparator()).
-    thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getNonPartitionedQueueComparator()).
+    thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
index 47be618..fc546ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
@@ -122,8 +122,8 @@ public class TestReservations {
         Resources.createResource(100 * 16 * GB, 100 * 12));
     when(csContext.getApplicationComparator()).thenReturn(
         CapacityScheduler.applicationComparator);
-    when(csContext.getQueueComparator()).thenReturn(
-        CapacityScheduler.queueComparator);
+    when(csContext.getNonPartitionedQueueComparator()).thenReturn(
+        CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4459349c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 77eebdf..e4583d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.Task;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -301,6 +302,9 @@ public class TestFifoScheduler {
         scheduler);
     ((RMContextImpl) rmContext).setSystemMetricsPublisher(
         mock(SystemMetricsPublisher.class));
+    NullRMNodeLabelsManager nlm = new NullRMNodeLabelsManager();
+    nlm.init(new Configuration());
+    rmContext.setNodeLabelManager(nlm);
 
     scheduler.setRMContext(rmContext);
     scheduler.init(conf);


[13/50] [abbrv] hadoop git commit: HDFS-8117. More accurate verification in SimulatedFSDataset: replace DEFAULT_DATABYTE with patterned data. Contributed by Zhe Zhang.

Posted by zj...@apache.org.
HDFS-8117. More accurate verification in SimulatedFSDataset: replace DEFAULT_DATABYTE with patterned data. Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/45fa3321
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/45fa3321
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/45fa3321

Branch: refs/heads/YARN-2928
Commit: 45fa3321cb0b389c781be11c30c3fb2a280f7a5b
Parents: ad69744
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Apr 13 13:01:10 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:41 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 25 ++++++++++++++++++++
 .../org/apache/hadoop/hdfs/TestFileAppend.java  | 12 +++++-----
 .../java/org/apache/hadoop/hdfs/TestPread.java  | 21 +++++++++-------
 .../org/apache/hadoop/hdfs/TestSmallBlock.java  | 16 ++++++-------
 .../server/datanode/SimulatedFSDataset.java     | 25 ++++++++++----------
 .../server/datanode/TestSimulatedFSDataset.java |  3 ++-
 7 files changed, 70 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/45fa3321/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2a26544..1aaf42c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -431,6 +431,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8083. Move dfs.client.write.* conf from DFSConfigKeys to 
     HdfsClientConfigKeys.Write.  (szetszwo)
 
+    HDFS-8117. More accurate verification in SimulatedFSDataset: replace
+    DEFAULT_DATABYTE with patterned data. (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45fa3321/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 1b3b62d..ae2d403 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -91,6 +91,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -117,6 +118,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
@@ -1769,4 +1771,27 @@ public class DFSTestUtil {
     dn.setLastUpdateMonotonic(Time.monotonicNow() + offset);
   }
 
+  /**
+   * This method takes a set of block locations and fills the provided buffer
+   * with expected bytes based on simulated content from
+   * {@link SimulatedFSDataset}.
+   *
+   * @param lbs The block locations of a file
+   * @param expected The buffer to be filled with expected bytes on the above
+   *                 locations.
+   */
+  public static void fillExpectedBuf(LocatedBlocks lbs, byte[] expected) {
+    Block[] blks = new Block[lbs.getLocatedBlocks().size()];
+    for (int i = 0; i < lbs.getLocatedBlocks().size(); i++) {
+      blks[i] = lbs.getLocatedBlocks().get(i).getBlock().getLocalBlock();
+    }
+    int bufPos = 0;
+    for (Block b : blks) {
+      for (long blkPos = 0; blkPos < b.getNumBytes(); blkPos++) {
+        assert bufPos < expected.length;
+        expected[bufPos++] = SimulatedFSDataset.simulatedByte(b, blkPos);
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45fa3321/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
index af404cd..6a7c3ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
@@ -68,7 +68,7 @@ public class TestFileAppend{
   //
   // verify that the data written to the full blocks are sane
   // 
-  private void checkFile(FileSystem fileSys, Path name, int repl)
+  private void checkFile(DistributedFileSystem fileSys, Path name, int repl)
     throws IOException {
     boolean done = false;
 
@@ -96,9 +96,9 @@ public class TestFileAppend{
     byte[] expected = 
         new byte[AppendTestUtil.NUM_BLOCKS * AppendTestUtil.BLOCK_SIZE];
     if (simulatedStorage) {
-      for (int i= 0; i < expected.length; i++) {  
-        expected[i] = SimulatedFSDataset.DEFAULT_DATABYTE;
-      }
+      LocatedBlocks lbs = fileSys.getClient().getLocatedBlocks(name.toString(),
+          0, AppendTestUtil.FILE_SIZE);
+      DFSTestUtil.fillExpectedBuf(lbs, expected);
     } else {
       System.arraycopy(fileContents, 0, expected, 0, expected.length);
     }
@@ -193,7 +193,7 @@ public class TestFileAppend{
     }
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    FileSystem fs = cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     try {
 
       // create a new file.
@@ -249,7 +249,7 @@ public class TestFileAppend{
     }
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    FileSystem fs = cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     try {
 
       // create a new file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45fa3321/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index 62f6c06..1a28404 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.io.IOUtils;
@@ -54,6 +55,8 @@ import org.mockito.stubbing.Answer;
 public class TestPread {
   static final long seed = 0xDEADBEEFL;
   static final int blockSize = 4096;
+  static final int numBlocksPerFile = 12;
+  static final int fileSize = numBlocksPerFile * blockSize;
   boolean simulatedStorage;
   boolean isHedgedRead;
 
@@ -66,10 +69,10 @@ public class TestPread {
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
     int replication = 3;// We need > 1 blocks to test out the hedged reads.
     // test empty file open and read
-    DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 0,
+    DFSTestUtil.createFile(fileSys, name, fileSize, 0,
       blockSize, (short)replication, seed);
     FSDataInputStream in = fileSys.open(name);
-    byte[] buffer = new byte[12 * blockSize];
+    byte[] buffer = new byte[fileSize];
     in.readFully(0, buffer, 0, 0);
     IOException res = null;
     try { // read beyond the end of the file
@@ -84,7 +87,7 @@ public class TestPread {
       assertTrue("Cannot delete file", false);
     
     // now create the real file
-    DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 12 * blockSize,
+    DFSTestUtil.createFile(fileSys, name, fileSize, fileSize,
         blockSize, (short) replication, seed);
   }
   
@@ -128,11 +131,13 @@ public class TestPread {
   
   private void pReadFile(FileSystem fileSys, Path name) throws IOException {
     FSDataInputStream stm = fileSys.open(name);
-    byte[] expected = new byte[12 * blockSize];
+    byte[] expected = new byte[fileSize];
     if (simulatedStorage) {
-      for (int i= 0; i < expected.length; i++) {  
-        expected[i] = SimulatedFSDataset.DEFAULT_DATABYTE;
-      }
+      assert fileSys instanceof DistributedFileSystem;
+      DistributedFileSystem dfs = (DistributedFileSystem) fileSys;
+      LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(name.toString(),
+          0, fileSize);
+      DFSTestUtil.fillExpectedBuf(lbs, expected);
     } else {
       Random rand = new Random(seed);
       rand.nextBytes(expected);
@@ -447,7 +452,7 @@ public class TestPread {
     FileSystem fileSys = cluster.getFileSystem();
     fileSys.setVerifyChecksum(verifyChecksum);
     try {
-      Path file1 = new Path("preadtest.dat");
+      Path file1 = new Path("/preadtest.dat");
       writeFile(fileSys, file1);
       pReadFile(fileSys, file1);
       datanodeRestartTest(cluster, fileSys, file1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45fa3321/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
index 90f47e5..6983cde 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
@@ -25,11 +25,10 @@ import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.junit.Test;
 
@@ -52,16 +51,17 @@ public class TestSmallBlock {
     }
   }
   
-  private void checkFile(FileSystem fileSys, Path name) throws IOException {
+  private void checkFile(DistributedFileSystem fileSys, Path name)
+      throws IOException {
     BlockLocation[] locations = fileSys.getFileBlockLocations(
         fileSys.getFileStatus(name), 0, fileSize);
     assertEquals("Number of blocks", fileSize, locations.length);
     FSDataInputStream stm = fileSys.open(name);
     byte[] expected = new byte[fileSize];
     if (simulatedStorage) {
-      for (int i = 0; i < expected.length; ++i) {  
-        expected[i] = SimulatedFSDataset.DEFAULT_DATABYTE;
-      }
+      LocatedBlocks lbs = fileSys.getClient().getLocatedBlocks(name.toString(),
+          0, fileSize);
+      DFSTestUtil.fillExpectedBuf(lbs, expected);
     } else {
       Random rand = new Random(seed);
       rand.nextBytes(expected);
@@ -90,9 +90,9 @@ public class TestSmallBlock {
     }
     conf.set(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, "1");
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    FileSystem fileSys = cluster.getFileSystem();
+    DistributedFileSystem fileSys = cluster.getFileSystem();
     try {
-      Path file1 = new Path("smallblocktest.dat");
+      Path file1 = new Path("/smallblocktest.dat");
       DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
           (short) 1, seed);
       checkFile(fileSys, file1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45fa3321/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index a358e22..344d1fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -97,12 +97,16 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
         Factory.class.getName());
   }
+
+  public static byte simulatedByte(Block b, long offsetInBlk) {
+    byte firstByte = (byte) (b.getBlockId() % Byte.MAX_VALUE);
+    return (byte) ((firstByte + offsetInBlk) % Byte.MAX_VALUE);
+  }
   
   public static final String CONFIG_PROPERTY_CAPACITY =
       "dfs.datanode.simulateddatastorage.capacity";
   
   public static final long DEFAULT_CAPACITY = 2L<<40; // 1 terabyte
-  public static final byte DEFAULT_DATABYTE = 9;
   
   public static final String CONFIG_PROPERTY_STATE =
       "dfs.datanode.simulateddatastorage.state";
@@ -182,9 +186,9 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     synchronized SimulatedInputStream getIStream() {
       if (!finalized) {
         // throw new IOException("Trying to read an unfinalized block");
-         return new SimulatedInputStream(oStream.getLength(), DEFAULT_DATABYTE);
+         return new SimulatedInputStream(oStream.getLength(), theBlock);
       } else {
-        return new SimulatedInputStream(theBlock.getNumBytes(), DEFAULT_DATABYTE);
+        return new SimulatedInputStream(theBlock.getNumBytes(), theBlock);
       }
     }
     
@@ -991,21 +995,19 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
    *
    */
   static private class SimulatedInputStream extends java.io.InputStream {
-    
-
-    byte theRepeatedData = 7;
     final long length; // bytes
     int currentPos = 0;
     byte[] data = null;
+    Block theBlock = null;
     
     /**
      * An input stream of size l with repeated bytes
      * @param l size of the stream
      * @param iRepeatedData byte that is repeated in the stream
      */
-    SimulatedInputStream(long l, byte iRepeatedData) {
+    SimulatedInputStream(long l, Block b) {
       length = l;
-      theRepeatedData = iRepeatedData;
+      theBlock = b;
     }
     
     /**
@@ -1031,8 +1033,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       if (data !=null) {
         return data[currentPos++];
       } else {
-        currentPos++;
-        return theRepeatedData;
+        return simulatedByte(theBlock, currentPos++);
       }
     }
     
@@ -1052,8 +1053,8 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       if (data != null) {
         System.arraycopy(data, currentPos, b, 0, bytesRead);
       } else { // all data is zero
-        for (int i : b) {  
-          b[i] = theRepeatedData;
+        for (int i = 0; i < bytesRead; i++) {
+          b[i] = simulatedByte(theBlock, currentPos + i);
         }
       }
       currentPos += bytesRead;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45fa3321/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
index dd24685..f76781d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
@@ -144,7 +144,8 @@ public class TestSimulatedFSDataset {
     long lengthRead = 0;
     int data;
     while ((data = input.read()) != -1) {
-      assertEquals(SimulatedFSDataset.DEFAULT_DATABYTE, data);
+      assertEquals(SimulatedFSDataset.simulatedByte(b.getLocalBlock(),
+          lengthRead), data);
       lengthRead++;
     }
     assertEquals(expectedLen, lengthRead);


[29/50] [abbrv] hadoop git commit: YARN-3436. Fix URIs in documantion of YARN web service REST APIs. Contributed by Bibin A Chundatt.

Posted by zj...@apache.org.
YARN-3436. Fix URIs in documantion of YARN web service REST APIs. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2239ba0a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2239ba0a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2239ba0a

Branch: refs/heads/YARN-2928
Commit: 2239ba0a2158b17aaa78d79060c0b91779f80e78
Parents: 025787b
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Apr 14 14:13:55 2015 -0500
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:44 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 3 +++
 .../hadoop-yarn-site/src/site/markdown/WebServicesIntro.md     | 6 +++---
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2239ba0a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c0bc974..f416ab2 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -257,6 +257,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3266. RMContext#inactiveNodes should have NodeId as map key.
     (Chengbing Liu via jianhe)
 
+    YARN-3436. Fix URIs in documantion of YARN web service REST APIs.
+    (Bibin A Chundatt via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2239ba0a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
index 0e89a50..0a396d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
@@ -97,7 +97,7 @@ After calling an HTTP request, an application should check the response status c
 
 #### JSON response with single resource
 
-HTTP Request: GET http://rmhost.domain:8088/ws/v1/cluster/app/application\_1324057493980\_0001
+HTTP Request: GET http://rmhost.domain:8088/ws/v1/cluster/apps/application\_1324057493980\_0001
 
 Response Status Line: HTTP/1.1 200 OK
 
@@ -137,7 +137,7 @@ Response Body:
 
 Here we request information about an application that doesn't exist yet.
 
-HTTP Request: GET http://rmhost.domain:8088/ws/v1/cluster/app/application\_1324057493980\_9999
+HTTP Request: GET http://rmhost.domain:8088/ws/v1/cluster/apps/application\_1324057493980\_9999
 
 Response Status Line: HTTP/1.1 404 Not Found
 
@@ -566,4 +566,4 @@ Output:
       "queue" : "a1"
    }
 }
-```
\ No newline at end of file
+```


[43/50] [abbrv] hadoop git commit: HDFS-8082. Move dfs.client.read.*, dfs.client.short.circuit.*, dfs.client.mmap.* and dfs.client.hedged.read.* conf from DFSConfigKeys to HdfsClientConfigKeys.

Posted by zj...@apache.org.
HDFS-8082. Move dfs.client.read.*, dfs.client.short.circuit.*, dfs.client.mmap.* and dfs.client.hedged.read.* conf from DFSConfigKeys to HdfsClientConfigKeys.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/05eac262
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/05eac262
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/05eac262

Branch: refs/heads/YARN-2928
Commit: 05eac262522fafd90fc6453c9c0c4d55f0ff684f
Parents: b9788ae
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Apr 16 13:22:31 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:46 2015 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |  63 +++++++++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   4 +
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |   3 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  22 +---
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   | 125 +++++++++++++------
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |   5 +-
 .../apache/hadoop/hdfs/HdfsConfiguration.java   |   6 +-
 .../hadoop/hdfs/client/impl/DfsClientConf.java  |  69 ++++++----
 .../hadoop/hdfs/server/datanode/DataNode.java   |  37 +++---
 .../hdfs/shortcircuit/ShortCircuitCache.java    |  23 ----
 .../hadoop/fs/TestEnhancedByteBufferAccess.java |  29 +++--
 .../java/org/apache/hadoop/fs/TestUnbuffer.java |   6 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   2 +-
 .../hadoop/hdfs/TestBlockReaderFactory.java     |   9 +-
 .../hadoop/hdfs/TestBlockReaderLocal.java       |   9 +-
 .../hadoop/hdfs/TestBlockReaderLocalLegacy.java |   4 +-
 .../apache/hadoop/hdfs/TestParallelRead.java    |   6 +-
 .../TestParallelShortCircuitLegacyRead.java     |   7 +-
 .../hdfs/TestParallelShortCircuitRead.java      |  10 +-
 .../TestParallelShortCircuitReadNoChecksum.java |  10 +-
 .../TestParallelShortCircuitReadUnCached.java   |  15 ++-
 .../hadoop/hdfs/TestParallelUnixDomainRead.java |   6 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |  23 ++--
 .../datanode/TestFsDatasetCacheRevocation.java  |   5 +-
 .../fsdataset/impl/LazyPersistTestCase.java     |  54 ++++----
 .../shortcircuit/TestShortCircuitCache.java     |  18 ++-
 .../shortcircuit/TestShortCircuitLocalRead.java |  19 +--
 .../TestTracingShortCircuitLocalRead.java       |  10 +-
 28 files changed, 360 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 44d9657..f9965b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -32,8 +32,8 @@ public interface HdfsClientConfigKeys {
 
   static final String PREFIX = "dfs.client.";
 
-  /** Client retry configuration properties */
-  public interface Retry {
+  /** dfs.client.retry configuration properties */
+  interface Retry {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
 
     String  POLICY_ENABLED_KEY = PREFIX + "policy.enabled";
@@ -53,7 +53,7 @@ public interface HdfsClientConfigKeys {
     int     WINDOW_BASE_DEFAULT = 3000;
   }
 
-  /** Client failover configuration properties */
+  /** dfs.client.failover configuration properties */
   interface Failover {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "failover.";
 
@@ -70,6 +70,7 @@ public interface HdfsClientConfigKeys {
     int     CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
   }
   
+  /** dfs.client.write configuration properties */
   interface Write {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "write.";
 
@@ -92,6 +93,7 @@ public interface HdfsClientConfigKeys {
     }
   }
 
+  /** dfs.client.block.write configuration properties */
   interface BlockWrite {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "block.write.";
 
@@ -114,7 +116,60 @@ public interface HdfsClientConfigKeys {
     }
   }
 
-  /** HTTP client configuration properties */
+  /** dfs.client.read configuration properties */
+  interface Read {
+    String PREFIX = HdfsClientConfigKeys.PREFIX + "read.";
+    
+    String  PREFETCH_SIZE_KEY = PREFIX + "prefetch.size"; 
+
+    interface ShortCircuit {
+      String PREFIX = Read.PREFIX + "shortcircuit.";
+
+      String  KEY = PREFIX.substring(0, PREFIX.length()-1);
+      boolean DEFAULT = false;
+      String  SKIP_CHECKSUM_KEY = PREFIX + "skip.checksum";
+      boolean SKIP_CHECKSUM_DEFAULT = false;
+      String  BUFFER_SIZE_KEY = PREFIX + "buffer.size";
+      int     BUFFER_SIZE_DEFAULT = 1024 * 1024;
+
+      String  STREAMS_CACHE_SIZE_KEY = PREFIX + "streams.cache.size";
+      int     STREAMS_CACHE_SIZE_DEFAULT = 256;
+      String  STREAMS_CACHE_EXPIRY_MS_KEY = PREFIX + "streams.cache.expiry.ms";
+      long    STREAMS_CACHE_EXPIRY_MS_DEFAULT = 5*MINUTE;
+    }
+  }
+
+  /** dfs.client.short.circuit configuration properties */
+  interface ShortCircuit {
+    String PREFIX = Read.PREFIX + "short.circuit.";
+
+    String  REPLICA_STALE_THRESHOLD_MS_KEY = PREFIX + "replica.stale.threshold.ms";
+    long    REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30*MINUTE;
+  }
+
+  /** dfs.client.mmap configuration properties */
+  interface Mmap {
+    String PREFIX = HdfsClientConfigKeys.PREFIX + "mmap.";
+
+    String  ENABLED_KEY = PREFIX + "enabled";
+    boolean ENABLED_DEFAULT = true;
+    String  CACHE_SIZE_KEY = PREFIX + "cache.size";
+    int     CACHE_SIZE_DEFAULT = 256;
+    String  CACHE_TIMEOUT_MS_KEY = PREFIX + "cache.timeout.ms";
+    long    CACHE_TIMEOUT_MS_DEFAULT  = 60*MINUTE;
+    String  RETRY_TIMEOUT_MS_KEY = PREFIX + "retry.timeout.ms";
+    long    RETRY_TIMEOUT_MS_DEFAULT = 5*MINUTE;
+  }
+
+  /** dfs.client.hedged.read configuration properties */
+  interface HedgedRead {
+    String  THRESHOLD_MILLIS_KEY = PREFIX + "threshold.millis";
+    long    THRESHOLD_MILLIS_DEFAULT = 500;
+    String  THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
+    int     THREADPOOL_SIZE_DEFAULT = 0;
+  }
+
+  /** dfs.http.client configuration properties */
   interface HttpClient {
     String  PREFIX = "dfs.http.client.";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 488eefc..c0472aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -438,6 +438,10 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8144. Split TestLazyPersistFiles into multiple tests. (Arpit Agarwal)
 
+    HDFS-8082. Move dfs.client.read.*, dfs.client.short.circuit.*,
+    dfs.client.mmap.* and dfs.client.hedged.read.* conf from DFSConfigKeys
+    to HdfsClientConfigKeys.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index 8df44f8..c16ffdf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
@@ -307,7 +308,7 @@ class BlockReaderLocalLegacy implements BlockReader {
       throw new IllegalArgumentException("Configured BlockReaderLocalLegacy " +
           "buffer size (" + bufferSizeBytes + ") is not large enough to hold " +
           "a single chunk (" + bytesPerChecksum +  "). Please configure " +
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY +
+          HdfsClientConfigKeys.Read.ShortCircuit.BUFFER_SIZE_KEY +
           " appropriately");
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index f79d160..cc5727f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -232,7 +232,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   private final CachingStrategy defaultReadCachingStrategy;
   private final CachingStrategy defaultWriteCachingStrategy;
   private final ClientContext clientContext;
-  private volatile long hedgedReadThresholdMillis;
+
   private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
       new DFSHedgedReadMetrics();
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
@@ -367,14 +367,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     this.clientContext = ClientContext.get(
         conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
         dfsClientConf);
-    this.hedgedReadThresholdMillis = conf.getLong(
-        DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS,
-        DFSConfigKeys.DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS);
-    int numThreads = conf.getInt(
-        DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE,
-        DFSConfigKeys.DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE);
-    if (numThreads > 0) {
-      this.initThreadsNumForHedgedReads(numThreads);
+
+    if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
+      this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
     }
     this.saslClient = new SaslDataTransferClient(
       conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
@@ -3133,15 +3128,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  long getHedgedReadTimeout() {
-    return this.hedgedReadThresholdMillis;
-  }
-
-  @VisibleForTesting
-  void setHedgedReadTimeout(long timeoutMillis) {
-    this.hedgedReadThresholdMillis = timeoutMillis;
-  }
-
   ThreadPoolExecutor getHedgedReadsThreadPool() {
     return HEDGED_READ_THREAD_POOL;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 11ca7f6..9cb00a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -688,7 +688,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
   
   
-  // client retry confs are moved to HdfsClientConfigKeys.Retry 
+  // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY
       = HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY;
@@ -726,7 +726,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_CLIENT_RETRY_WINDOW_BASE_DEFAULT
       = HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT;
 
-  // client failover confs are moved to HdfsClientConfigKeys.Failover 
+  // dfs.client.failover confs are moved to HdfsClientConfigKeys.Failover 
   @Deprecated
   public static final String  DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX
       = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX;
@@ -761,7 +761,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT
       = HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT;
   
-  // client write confs are moved to HdfsClientConfigKeys.Write 
+  // dfs.client.write confs are moved to HdfsClientConfigKeys.Write 
   @Deprecated
   public static final String  DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_KEY
       = HdfsClientConfigKeys.Write.MAX_PACKETS_IN_FLIGHT_KEY;
@@ -799,7 +799,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long    DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_DEFAULT
       = HdfsClientConfigKeys.Write.ByteArrayManager.COUNT_RESET_TIME_PERIOD_MS_DEFAULT;
 
-  // client block.write confs are moved to HdfsClientConfigKeys.BlockWrite 
+  // dfs.client.block.write confs are moved to HdfsClientConfigKeys.BlockWrite 
   @Deprecated
   public static final String  DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY
       = HdfsClientConfigKeys.BlockWrite.RETRIES_KEY;
@@ -837,9 +837,91 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_DEFAULT
       = HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT;
 
+  // dfs.client.read confs are moved to HdfsClientConfigKeys.Read 
+  @Deprecated
+  public static final String  DFS_CLIENT_READ_PREFETCH_SIZE_KEY
+      = HdfsClientConfigKeys.Read.PREFETCH_SIZE_KEY; 
+  @Deprecated
+  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_KEY
+      = HdfsClientConfigKeys.Read.ShortCircuit.KEY; 
+  @Deprecated
+  public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT
+      = HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY
+      = HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY;
+  @Deprecated
+  public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT
+      = HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY
+      = HdfsClientConfigKeys.Read.ShortCircuit.BUFFER_SIZE_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT
+      = HdfsClientConfigKeys.Read.ShortCircuit.BUFFER_SIZE_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY
+      = HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_SIZE_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT
+      = HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_SIZE_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY
+      = HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_EXPIRY_MS_KEY;
+  @Deprecated
+  public static final long    DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT
+      = HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_EXPIRY_MS_DEFAULT;
+
+  // dfs.client.mmap confs are moved to HdfsClientConfigKeys.Mmap 
+  @Deprecated
+  public static final String  DFS_CLIENT_MMAP_ENABLED
+      = HdfsClientConfigKeys.Mmap.ENABLED_KEY;
+  @Deprecated
+  public static final boolean DFS_CLIENT_MMAP_ENABLED_DEFAULT
+      = HdfsClientConfigKeys.Mmap.ENABLED_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_MMAP_CACHE_SIZE
+      = HdfsClientConfigKeys.Mmap.CACHE_SIZE_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT
+      = HdfsClientConfigKeys.Mmap.CACHE_SIZE_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS
+      = HdfsClientConfigKeys.Mmap.CACHE_TIMEOUT_MS_KEY;
+  @Deprecated
+  public static final long    DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT
+      = HdfsClientConfigKeys.Mmap.CACHE_TIMEOUT_MS_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS
+      = HdfsClientConfigKeys.Mmap.RETRY_TIMEOUT_MS_KEY;
+  @Deprecated
+  public static final long    DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT
+      = HdfsClientConfigKeys.Mmap.RETRY_TIMEOUT_MS_DEFAULT;
+
+  // dfs.client.short.circuit confs are moved to HdfsClientConfigKeys.ShortCircuit 
+  @Deprecated
+  public static final String  DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS
+      = HdfsClientConfigKeys.ShortCircuit.REPLICA_STALE_THRESHOLD_MS_KEY;
+  @Deprecated
+  public static final long    DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT
+      = HdfsClientConfigKeys.ShortCircuit.REPLICA_STALE_THRESHOLD_MS_DEFAULT;
+
+  // dfs.client.hedged.read confs are moved to HdfsClientConfigKeys.HedgedRead 
+  @Deprecated
+  public static final String  DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS
+      = HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY;
+  @Deprecated
+  public static final long    DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS
+      = HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_DEFAULT;
+  @Deprecated
+  public static final String  DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE
+      = HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY;
+  @Deprecated
+  public static final int     DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE
+      = HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_DEFAULT;
+
+
 
-  
-  
   public static final String  DFS_CLIENT_WRITE_PACKET_SIZE_KEY = "dfs.client-write-packet-size";
   public static final int     DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
 
@@ -883,30 +965,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
   public static final String  DFS_CLIENT_LOCAL_INTERFACES = "dfs.client.local.interfaces";
 
-  public static final String  DFS_CLIENT_READ_PREFETCH_SIZE_KEY = "dfs.client.read.prefetch.size"; 
-  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_KEY = "dfs.client.read.shortcircuit";
-  public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT = false;
-  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY = "dfs.client.read.shortcircuit.skip.checksum";
-  public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT = false;
-  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY = "dfs.client.read.shortcircuit.buffer.size";
-  public static final int     DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT = 1024 * 1024;
-  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY = "dfs.client.read.shortcircuit.streams.cache.size";
-  public static final int     DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT = 256;
-  public static final String  DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY = "dfs.client.read.shortcircuit.streams.cache.expiry.ms";
-  public static final long    DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT = 5 * 60 * 1000;
 
   public static final String  DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC = "dfs.client.domain.socket.data.traffic";
   public static final boolean DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT = false;
-  public static final String  DFS_CLIENT_MMAP_ENABLED= "dfs.client.mmap.enabled";
-  public static final boolean DFS_CLIENT_MMAP_ENABLED_DEFAULT = true;
-  public static final String  DFS_CLIENT_MMAP_CACHE_SIZE = "dfs.client.mmap.cache.size";
-  public static final int     DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 256;
-  public static final String  DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS = "dfs.client.mmap.cache.timeout.ms";
-  public static final long    DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT  = 60 * 60 * 1000;
-  public static final String  DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS = "dfs.client.mmap.retry.timeout.ms";
-  public static final long    DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT = 5 * 60 * 1000;
-  public static final String  DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS = "dfs.client.short.circuit.replica.stale.threshold.ms";
-  public static final long    DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30 * 60 * 1000;
 
   // The number of NN response dropped by client proactively in each RPC call.
   // For testing NN retry cache, we can set this property with positive value.
@@ -919,14 +980,4 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.client.key.provider.cache.expiry";
   public static final long    DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
       TimeUnit.DAYS.toMillis(10); // 10 days
-
-  // hedged read properties
-  public static final String  DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS =
-      "dfs.client.hedged.read.threshold.millis";
-  public static final long    DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS =
-      500;
-
-  public static final String  DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE =
-      "dfs.client.hedged.read.threadpool.size";
-  public static final int     DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE = 0;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index dd0f6fe..3f90397 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1196,6 +1196,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       long end, byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
+    final DfsClientConf conf = dfsClient.getConf();
     ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
     CompletionService<ByteBuffer> hedgedService =
         new ExecutorCompletionService<ByteBuffer>(
@@ -1223,13 +1224,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         futures.add(firstRequest);
         try {
           Future<ByteBuffer> future = hedgedService.poll(
-              dfsClient.getHedgedReadTimeout(), TimeUnit.MILLISECONDS);
+              conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
           if (future != null) {
             future.get();
             return;
           }
           if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Waited " + dfsClient.getHedgedReadTimeout()
+            DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
                 + "ms to read from " + chosenNode.info
                 + "; spawning hedged read");
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
index 29a2667..cdb9371 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.hdfs;
 
-import org.apache.hadoop.conf.Configuration;
-
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 
 /**
  * Adds deprecated keys into the configuration.
@@ -85,7 +85,7 @@ public class HdfsConfiguration extends Configuration {
       new DeprecationDelta("dfs.name.edits.dir",
         DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY),
       new DeprecationDelta("dfs.read.prefetch.size",
-        DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY),
+        HdfsClientConfigKeys.Read.PREFETCH_SIZE_KEY),
       new DeprecationDelta("dfs.safemode.extension",
         DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY),
       new DeprecationDelta("dfs.safemode.threshold.pct",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
index bc1831a..a257e32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
@@ -27,7 +27,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_T
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
@@ -98,6 +97,9 @@ public class DfsClientConf {
   private final long slowIoWarningThresholdMs;
 
   private final ShortCircuitConf shortCircuitConf;
+  
+  private final long hedgedReadThresholdMillis;
+  private final int hedgedReadThreadpoolSize;
 
   public DfsClientConf(Configuration conf) {
     // The hdfsTimeout is currently the same as the ipc timeout 
@@ -172,7 +174,7 @@ public class DfsClientConf {
     excludedNodesCacheExpiry = conf.getLong(
         HdfsClientConfigKeys.Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY,
         HdfsClientConfigKeys.Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
-    prefetchSize = conf.getLong(DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
+    prefetchSize = conf.getLong(HdfsClientConfigKeys.Read.PREFETCH_SIZE_KEY,
         10 * defaultBlockSize);
     numCachedConnRetry = conf.getInt(DFS_CLIENT_CACHED_CONN_RETRY_KEY,
         DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
@@ -206,6 +208,13 @@ public class DfsClientConf {
         DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
     
     shortCircuitConf = new ShortCircuitConf(conf);
+
+    hedgedReadThresholdMillis = conf.getLong(
+        HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY,
+        HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_DEFAULT);
+    hedgedReadThreadpoolSize = conf.getInt(
+        HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY,
+        HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_DEFAULT);
   }
 
   private DataChecksum.Type getChecksumType(Configuration conf) {
@@ -469,6 +478,20 @@ public class DfsClientConf {
   }
 
   /**
+   * @return the hedgedReadThresholdMillis
+   */
+  public long getHedgedReadThresholdMillis() {
+    return hedgedReadThresholdMillis;
+  }
+
+  /**
+   * @return the hedgedReadThreadpoolSize
+   */
+  public int getHedgedReadThreadpoolSize() {
+    return hedgedReadThreadpoolSize;
+  }
+
+  /**
    * @return the shortCircuitConf
    */
   public ShortCircuitConf getShortCircuitConf() {
@@ -520,8 +543,8 @@ public class DfsClientConf {
           DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
           DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT);
       shortCircuitLocalReads = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
+          HdfsClientConfigKeys.Read.ShortCircuit.KEY,
+          HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT);
       domainSocketDataTraffic = conf.getBoolean(
           DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
           DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT);
@@ -532,7 +555,7 @@ public class DfsClientConf {
       if (LOG.isDebugEnabled()) {
         LOG.debug(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL
             + " = " + useLegacyBlockReaderLocal);
-        LOG.debug(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY
+        LOG.debug(HdfsClientConfigKeys.Read.ShortCircuit.KEY
             + " = " + shortCircuitLocalReads);
         LOG.debug(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC
             + " = " + domainSocketDataTraffic);
@@ -541,32 +564,32 @@ public class DfsClientConf {
       }
 
       skipShortCircuitChecksums = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
+          HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+          HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_DEFAULT);
       shortCircuitBufferSize = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
+          HdfsClientConfigKeys.Read.ShortCircuit.BUFFER_SIZE_KEY,
+          HdfsClientConfigKeys.Read.ShortCircuit.BUFFER_SIZE_DEFAULT);
       shortCircuitStreamsCacheSize = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT);
+          HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_SIZE_KEY,
+          HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_SIZE_DEFAULT);
       shortCircuitStreamsCacheExpiryMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT);
+          HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_EXPIRY_MS_KEY,
+          HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_EXPIRY_MS_DEFAULT);
       shortCircuitMmapEnabled = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED,
-          DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED_DEFAULT);
+          HdfsClientConfigKeys.Mmap.ENABLED_KEY,
+          HdfsClientConfigKeys.Mmap.ENABLED_DEFAULT);
       shortCircuitMmapCacheSize = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE,
-          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT);
+          HdfsClientConfigKeys.Mmap.CACHE_SIZE_KEY,
+          HdfsClientConfigKeys.Mmap.CACHE_SIZE_DEFAULT);
       shortCircuitMmapCacheExpiryMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
-          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT);
+          HdfsClientConfigKeys.Mmap.CACHE_TIMEOUT_MS_KEY,
+          HdfsClientConfigKeys.Mmap.CACHE_TIMEOUT_MS_DEFAULT);
       shortCircuitMmapCacheRetryTimeout = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
-          DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT);
+          HdfsClientConfigKeys.Mmap.RETRY_TIMEOUT_MS_KEY,
+          HdfsClientConfigKeys.Mmap.RETRY_TIMEOUT_MS_DEFAULT);
       shortCircuitCacheStaleThresholdMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
-          DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT);
+          HdfsClientConfigKeys.ShortCircuit.REPLICA_STALE_THRESHOLD_MS_KEY,
+          HdfsClientConfigKeys.ShortCircuit.REPLICA_STALE_THRESHOLD_MS_DEFAULT);
       shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
           DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
           DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index cb22b73..0ddb99c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -38,8 +38,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT;
@@ -87,10 +87,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.management.ObjectName;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -102,13 +98,14 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.BlockReportOptions;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.client.BlockReportOptions;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.net.DomainPeerServer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -142,8 +139,8 @@ import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier.AccessMode;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
@@ -186,7 +183,10 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.tracing.SpanReceiverHost;
+import org.apache.hadoop.tracing.SpanReceiverInfo;
 import org.apache.hadoop.tracing.TraceAdminPB.TraceAdminService;
+import org.apache.hadoop.tracing.TraceAdminProtocol;
 import org.apache.hadoop.tracing.TraceAdminProtocolPB;
 import org.apache.hadoop.tracing.TraceAdminProtocolServerSideTranslatorPB;
 import org.apache.hadoop.util.Daemon;
@@ -198,14 +198,15 @@ import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
-import org.apache.hadoop.tracing.SpanReceiverHost;
-import org.apache.hadoop.tracing.SpanReceiverInfo;
-import org.apache.hadoop.tracing.TraceAdminProtocol;
 import org.mortbay.util.ajax.JSON;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.Lists;
 import com.google.protobuf.BlockingService;
 
 /**********************************************************
@@ -410,8 +411,8 @@ public class DataNode extends ReconfigurableBase
         conf.get("hadoop.hdfs.configuration.version", "UNSPECIFIED");
 
     // Determine whether we should try to pass file descriptors to clients.
-    if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
-              DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT)) {
+    if (conf.getBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY,
+              HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT)) {
       String reason = DomainSocket.getLoadingFailureReason();
       if (reason != null) {
         LOG.warn("File descriptor passing is disabled because " + reason);
@@ -927,8 +928,8 @@ public class DataNode extends ReconfigurableBase
     this.dataXceiverServer = new Daemon(threadGroup, xserver);
     this.threadGroup.setDaemon(true); // auto destroy when empty
 
-    if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
-              DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT) ||
+    if (conf.getBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY,
+              HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT) ||
         conf.getBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
               DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
       DomainPeerServer domainPeerServer =
@@ -943,14 +944,14 @@ public class DataNode extends ReconfigurableBase
     this.shortCircuitRegistry = new ShortCircuitRegistry(conf);
   }
 
-  static DomainPeerServer getDomainPeerServer(Configuration conf,
+  private static DomainPeerServer getDomainPeerServer(Configuration conf,
       int port) throws IOException {
     String domainSocketPath =
         conf.getTrimmed(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
             DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT);
     if (domainSocketPath.isEmpty()) {
-      if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
-            DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT) &&
+      if (conf.getBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY,
+            HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT) &&
          (!conf.getBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
           DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT))) {
         LOG.warn("Although short-circuit local reads are configured, " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
index 27a9ef2..db4cbe2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
@@ -37,8 +37,6 @@ import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.net.DomainPeer;
@@ -339,27 +337,6 @@ public class ShortCircuitCache implements Closeable {
    */
   private final DfsClientShmManager shmManager;
 
-  /**
-   * Create a {@link ShortCircuitCache} object from a {@link Configuration}
-   */
-  public static ShortCircuitCache fromConf(Configuration conf) {
-    return new ShortCircuitCache(
-        conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY,
-            DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT),
-        conf.getLong(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
-            DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT),
-        conf.getInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE,
-            DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT),
-        conf.getLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
-            DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT),
-        conf.getLong(DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
-            DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT),
-        conf.getLong(DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
-            DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT),
-        conf.getInt(DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
-            DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT));
-  }
-
   public static ShortCircuitCache fromConf(ShortCircuitConf conf) {
     return new ShortCircuitCache(
         conf.getShortCircuitStreamsCacheSize(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
index 6d644ae..e4380c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
@@ -18,8 +18,6 @@
 package org.apache.hadoop.fs;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
@@ -48,6 +46,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -120,15 +119,15 @@ public class TestEnhancedByteBufferAccess {
     Assume.assumeTrue(NativeIO.isAvailable());
     Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
     HdfsConfiguration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE, 3);
-    conf.setLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS, 100);
+    conf.setInt(HdfsClientConfigKeys.Mmap.CACHE_SIZE_KEY, 3);
+    conf.setLong(HdfsClientConfigKeys.Mmap.CACHE_TIMEOUT_MS_KEY, 100);
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
         new File(sockDir.getDir(),
           "TestRequestMmapAccess._PORT.sock").getAbsolutePath());
-    conf.setBoolean(DFSConfigKeys.
-        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+        true);
     conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
     conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000);
@@ -597,8 +596,8 @@ public class TestEnhancedByteBufferAccess {
     final Path TEST_PATH = new Path("/a");
     final int RANDOM_SEED = 23453;
     HdfsConfiguration conf = initZeroCopyTest();
-    conf.setBoolean(DFSConfigKeys.
-        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+        false);
     final String CONTEXT = "testZeroCopyReadOfCachedData";
     conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT);
     conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
@@ -715,7 +714,7 @@ public class TestEnhancedByteBufferAccess {
   @Test
   public void testClientMmapDisable() throws Exception {
     HdfsConfiguration conf = initZeroCopyTest();
-    conf.setBoolean(DFS_CLIENT_MMAP_ENABLED, false);
+    conf.setBoolean(HdfsClientConfigKeys.Mmap.ENABLED_KEY, false);
     MiniDFSCluster cluster = null;
     final Path TEST_PATH = new Path("/a");
     final int TEST_FILE_LENGTH = 16385;
@@ -726,8 +725,8 @@ public class TestEnhancedByteBufferAccess {
     conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT);
 
     try {
-      // With DFS_CLIENT_MMAP_ENABLED set to false, we should not do memory
-      // mapped reads.
+      // With HdfsClientConfigKeys.Mmap.ENABLED_KEY set to false,
+      // we should not do memory mapped reads.
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
@@ -751,9 +750,9 @@ public class TestEnhancedByteBufferAccess {
     fs = null;
     cluster = null;
     try {
-      // Now try again with DFS_CLIENT_MMAP_CACHE_SIZE == 0.  It should work.
-      conf.setBoolean(DFS_CLIENT_MMAP_ENABLED, true);
-      conf.setInt(DFS_CLIENT_MMAP_CACHE_SIZE, 0);
+      // Now try again with HdfsClientConfigKeys.Mmap.CACHE_SIZE_KEY == 0.
+      conf.setBoolean(HdfsClientConfigKeys.Mmap.ENABLED_KEY, true);
+      conf.setInt(HdfsClientConfigKeys.Mmap.CACHE_SIZE_KEY, 0);
       conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT + ".1");
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster.waitActive();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
index 52c33e9..7193fe2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
@@ -20,12 +20,12 @@ package org.apache.hadoop.fs;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.PeerCache;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -47,7 +47,7 @@ public class TestUnbuffer {
 
     // Disable short-circuit reads.  With short-circuit, we wouldn't hold open a
     // TCP socket.
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, false);
 
     // Set a really long socket timeout to avoid test timing issues.
     conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
@@ -100,7 +100,7 @@ public class TestUnbuffer {
   public void testOpenManyFilesViaTcp() throws Exception {
     final int NUM_OPENS = 500;
     Configuration conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, false);
     MiniDFSCluster cluster = null;
     FSDataInputStream[] streams = new FSDataInputStream[NUM_OPENS];
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index ae2d403..5ef2f73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1462,7 +1462,7 @@ public class DFSTestUtil {
     
     public Configuration newConfiguration() {
       Configuration conf = new Configuration();
-      conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+      conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
       conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
           new File(sockDir.getDir(),
             testName + "._PORT.sock").getAbsolutePath());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
index 5c330cd..d8aceff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hdfs;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS;
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -43,6 +41,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.PerDatanodeVisitorInfo;
@@ -81,8 +80,8 @@ public class TestBlockReaderFactory {
     conf.setLong(DFS_BLOCK_SIZE_KEY, 4096);
     conf.set(DFS_DOMAIN_SOCKET_PATH_KEY, new File(sockDir.getDir(),
         testName + "._PORT").getAbsolutePath());
-    conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
         false);
     conf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
     return conf;
@@ -108,7 +107,7 @@ public class TestBlockReaderFactory {
         "testFallbackFromShortCircuitToUnixDomainTraffic_clientContext");
     clientConf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
     Configuration serverConf = new Configuration(clientConf);
-    serverConf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    serverConf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, false);
 
     MiniDFSCluster cluster =
         new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
index ab3515e..c69774c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -123,8 +124,8 @@ public class TestBlockReaderLocal {
     Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
     MiniDFSCluster cluster = null;
     HdfsConfiguration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.
-        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, !checksum);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+        !checksum);
     conf.setLong(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
         BlockReaderLocalTest.BYTES_PER_CHECKSUM);
     conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
@@ -721,10 +722,10 @@ public class TestBlockReaderLocal {
       conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
         new File(sockDir.getDir(), "TestStatisticsForLocalRead.%d.sock").
           getAbsolutePath());
-      conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+      conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
       DomainSocket.disableBindPathValidation();
     } else {
-      conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+      conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, false);
     }
     MiniDFSCluster cluster = null;
     final Path TEST_PATH = new Path("/a");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
index 6a8f9db..760a61f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
@@ -63,9 +63,9 @@ public class TestBlockReaderLocalLegacy {
         new File(socketDir.getDir(), "TestBlockReaderLocalLegacy.%d.sock").
           getAbsolutePath());
     }
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
         false);
     conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
         UserGroupInformation.getCurrentUser().getShortUserName());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java
index c06fab5..2c187b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java
@@ -17,9 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
-import org.apache.log4j.Level;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -29,7 +27,7 @@ public class TestParallelRead extends TestParallelReadUtil {
     // This is a test of the normal (TCP) read path.  For this reason, we turn
     // off both short-circuit local reads and UNIX domain socket data traffic.
     HdfsConfiguration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, false);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
                     false);
     // dfs.domain.socket.path should be ignored because the previous two keys

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitLegacyRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitLegacyRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitLegacyRead.java
index 4ba31bd..c3a3fb6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitLegacyRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitLegacyRead.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
@@ -30,9 +31,9 @@ public class TestParallelShortCircuitLegacyRead extends TestParallelReadUtil {
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, "");
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.
-        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+        false);
     conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
         UserGroupInformation.getCurrentUser().getShortUserName());
     DomainSocket.disableBindPathValidation();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitRead.java
index a0450fa..3f352b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitRead.java
@@ -17,15 +17,17 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.hamcrest.CoreMatchers.equalTo;
+
 import java.io.File;
 
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.AfterClass;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import static org.hamcrest.CoreMatchers.*;
 
 public class TestParallelShortCircuitRead extends TestParallelReadUtil {
   private static TemporarySocketDirectory sockDir;
@@ -38,9 +40,9 @@ public class TestParallelShortCircuitRead extends TestParallelReadUtil {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
       new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.
-        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+        false);
     DomainSocket.disableBindPathValidation();
     setupCluster(1, conf);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadNoChecksum.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadNoChecksum.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadNoChecksum.java
index 172af31..df110b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadNoChecksum.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadNoChecksum.java
@@ -17,15 +17,17 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.hamcrest.CoreMatchers.equalTo;
+
 import java.io.File;
 
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.AfterClass;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import static org.hamcrest.CoreMatchers.*;
 
 public class TestParallelShortCircuitReadNoChecksum extends TestParallelReadUtil {
   private static TemporarySocketDirectory sockDir;
@@ -38,9 +40,9 @@ public class TestParallelShortCircuitReadNoChecksum extends TestParallelReadUtil
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
       new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFSConfigKeys.
-        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+        true);
     DomainSocket.disableBindPathValidation();
     setupCluster(1, conf);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java
index 43ebde1..7485fa7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java
@@ -17,14 +17,17 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.hamcrest.CoreMatchers.equalTo;
+
 import java.io.File;
+
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.AfterClass;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import static org.hamcrest.CoreMatchers.*;
 
 /**
  * This class tests short-circuit local reads without any FileInputStream or
@@ -41,13 +44,13 @@ public class TestParallelShortCircuitReadUnCached extends TestParallelReadUtil {
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
       new File(sockDir.getDir(), 
         "TestParallelShortCircuitReadUnCached._PORT.sock").getAbsolutePath());
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
     // Enabling data transfer encryption should have no effect when using
     // short-circuit local reads.  This is a regression test for HDFS-5353.
     conf.setBoolean(DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
-    conf.setBoolean(DFSConfigKeys.
-        DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+        false);
     conf.setBoolean(DFSConfigKeys.
         DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
     // We want to test reading from stale sockets.
@@ -56,8 +59,8 @@ public class TestParallelShortCircuitReadUnCached extends TestParallelReadUtil {
         5 * 60 * 1000);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 32);
     // Avoid using the FileInputStreamCache.
-    conf.setInt(DFSConfigKeys.
-        DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY, 0);
+    conf.setInt(HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_SIZE_KEY,
+        0);
     DomainSocket.disableBindPathValidation();
     DFSInputStream.tcpReadsDisabledForTesting = true;
     setupCluster(1, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelUnixDomainRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelUnixDomainRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelUnixDomainRead.java
index 00c65f9..9db8503 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelUnixDomainRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelUnixDomainRead.java
@@ -17,15 +17,17 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.hamcrest.CoreMatchers.equalTo;
+
 import java.io.File;
 
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.AfterClass;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import static org.hamcrest.CoreMatchers.*;
 
 public class TestParallelUnixDomainRead extends TestParallelReadUtil {
   private static TemporarySocketDirectory sockDir;
@@ -38,7 +40,7 @@ public class TestParallelUnixDomainRead extends TestParallelReadUtil {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
       new File(sockDir.getDir(), "TestParallelLocalRead.%d.sock").getAbsolutePath());
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, false);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
     DomainSocket.disableBindPathValidation();
     setupCluster(1, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index 1a28404..09b9f83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -273,8 +273,8 @@ public class TestPread {
   public void testHedgedPreadDFSBasic() throws IOException {
     isHedgedRead = true;
     Configuration conf = new Configuration();
-    conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE, 5);
-    conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS, 1);
+    conf.setInt(HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, 5);
+    conf.setLong(HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY, 1);
     dfsPreadTest(conf, false, true); // normal pread
     dfsPreadTest(conf, true, true); // trigger read code path without
                                     // transferTo.
@@ -286,9 +286,9 @@ public class TestPread {
     int numHedgedReadPoolThreads = 5;
     final int hedgedReadTimeoutMillis = 50;
 
-    conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE,
+    conf.setInt(HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY,
         numHedgedReadPoolThreads);
-    conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS,
+    conf.setLong(HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY,
         hedgedReadTimeoutMillis);
     conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 0);
     // Set up the InjectionHandler
@@ -362,9 +362,9 @@ public class TestPread {
     int numHedgedReadPoolThreads = 5;
     final int initialHedgedReadTimeoutMillis = 50000;
     final int fixedSleepIntervalMillis = 50;
-    conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE,
+    conf.setInt(HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY,
         numHedgedReadPoolThreads);
-    conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS,
+    conf.setLong(HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY,
         initialHedgedReadTimeoutMillis);
 
     // Set up the InjectionHandler
@@ -404,7 +404,14 @@ public class TestPread {
        * that there were hedged reads. But, none of the reads had to run in the
        * current thread.
        */
-      dfsClient.setHedgedReadTimeout(50); // 50ms
+      {
+        Configuration conf2 =  new Configuration(cluster.getConfiguration(0));
+        conf2.setBoolean("fs.hdfs.impl.disable.cache", true);
+        conf2.setLong(HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY, 50);
+        fileSys.close();
+        fileSys = (DistributedFileSystem)FileSystem.get(cluster.getURI(0), conf2);
+        metrics = fileSys.getClient().getHedgedReadMetrics();
+      }
       pReadFile(fileSys, file1);
       // assert that there were hedged reads
       assertTrue(metrics.getHedgedReadOps() > 0);
@@ -439,7 +446,7 @@ public class TestPread {
   private void dfsPreadTest(Configuration conf, boolean disableTransferTo, boolean verifyChecksum)
       throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
-    conf.setLong(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 4096);
+    conf.setLong(HdfsClientConfigKeys.Read.PREFETCH_SIZE_KEY, 4096);
     // Set short retry timeouts so this test runs faster
     conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 0);
     if (simulatedStorage) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java
index d5531db..a6d972a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java
@@ -23,8 +23,6 @@ import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.EnumSet;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
@@ -35,6 +33,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -84,7 +83,7 @@ public class TestFsDatasetCacheRevocation {
     conf.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
         TestFsDatasetCache.CACHE_CAPACITY);
     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
     conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
       new File(sockDir.getDir(), "sock").getAbsolutePath());
     return conf;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index baa540d..6adec20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -17,6 +17,34 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
+import static org.apache.hadoop.fs.CreateFlag.CREATE;
+import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
+import static org.apache.hadoop.fs.StorageType.DEFAULT;
+import static org.apache.hadoop.fs.StorageType.RAM_DISK;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -24,12 +52,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
@@ -44,25 +73,6 @@ import org.junit.After;
 import org.junit.Rule;
 import org.junit.rules.Timeout;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
-
-import static org.apache.hadoop.fs.CreateFlag.CREATE;
-import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
-import static org.apache.hadoop.fs.StorageType.DEFAULT;
-import static org.apache.hadoop.fs.StorageType.RAM_DISK;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-
 public abstract class LazyPersistTestCase {
   static final byte LAZY_PERSIST_POLICY_ID = (byte) 15;
 
@@ -237,7 +247,7 @@ public abstract class LazyPersistTestCase {
                 EVICTION_LOW_WATERMARK * BLOCK_SIZE);
 
     if (useSCR) {
-      conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+      conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
       // Do not share a client context across tests.
       conf.set(DFS_CLIENT_CONTEXT, UUID.randomUUID().toString());
       if (useLegacyBlockReaderLocal) {
@@ -299,7 +309,7 @@ public abstract class LazyPersistTestCase {
 
     if (useSCR)
     {
-      conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY,useSCR);
+      conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY,useSCR);
       conf.set(DFS_CLIENT_CONTEXT, UUID.randomUUID().toString());
       sockDir = new TemporarySocketDirectory();
       conf.set(DFS_DOMAIN_SOCKET_PATH_KEY, new File(sockDir.getDir(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05eac262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index e38b97b..aafe1cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -20,10 +20,7 @@ package org.apache.hadoop.hdfs.shortcircuit;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY;
 import static org.hamcrest.CoreMatchers.equalTo;
 
 import java.io.DataOutputStream;
@@ -36,7 +33,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
-import com.google.common.collect.HashMultimap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -45,12 +41,12 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.BlockReaderTestUtil;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -61,8 +57,8 @@ import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.PerDatanodeVisito
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -76,6 +72,7 @@ import org.junit.Test;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
+import com.google.common.collect.HashMultimap;
 
 public class TestShortCircuitCache {
   static final Log LOG = LogFactory.getLog(TestShortCircuitCache.class);
@@ -388,8 +385,8 @@ public class TestShortCircuitCache {
     conf.setLong(DFS_BLOCK_SIZE_KEY, 4096);
     conf.set(DFS_DOMAIN_SOCKET_PATH_KEY, new File(sockDir.getDir(),
         testName).getAbsolutePath());
-    conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
-    conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY,
         false);
     conf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
     DFSInputStream.tcpReadsDisabledForTesting = true;
@@ -541,7 +538,7 @@ public class TestShortCircuitCache {
         "testUnlinkingReplicasInFileDescriptorCache", sockDir);
     // We don't want the CacheCleaner to time out short-circuit shared memory
     // segments during the test, so set the timeout really high.
-    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+    conf.setLong(HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_EXPIRY_MS_KEY,
         1000000000L);
     MiniDFSCluster cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
@@ -637,7 +634,8 @@ public class TestShortCircuitCache {
     TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
     Configuration conf = createShortCircuitConf(
         "testDataXceiverCleansUpSlotsOnFailure", sockDir);
-    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+    conf.setLong(
+        HdfsClientConfigKeys.Read.ShortCircuit.STREAMS_CACHE_EXPIRY_MS_KEY,
         1000000000L);
     MiniDFSCluster cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(1).build();


[02/50] [abbrv] hadoop git commit: HDFS-8100. Refactor DFSClient.Conf to a standalone class and separates short-circuit related conf to ShortCircuitConf.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
new file mode 100644
index 0000000..e781b16
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
@@ -0,0 +1,738 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client.impl;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.BlockReaderFactory;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.util.DataChecksum;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * DFSClient configuration 
+ */
+public class DfsClientConf {
+
+  private final int hdfsTimeout;    // timeout value for a DFS operation.
+
+  private final int maxFailoverAttempts;
+  private final int maxRetryAttempts;
+  private final int failoverSleepBaseMillis;
+  private final int failoverSleepMaxMillis;
+  private final int maxBlockAcquireFailures;
+  private final int datanodeSocketWriteTimeout;
+  private final int ioBufferSize;
+  private final ChecksumOpt defaultChecksumOpt;
+  private final int writePacketSize;
+  private final int writeMaxPackets;
+  private final ByteArrayManager.Conf writeByteArrayManagerConf;
+  private final int socketTimeout;
+  private final long excludedNodesCacheExpiry;
+  /** Wait time window (in msec) if BlockMissingException is caught */
+  private final int timeWindow;
+  private final int numCachedConnRetry;
+  private final int numBlockWriteRetry;
+  private final int numBlockWriteLocateFollowingRetry;
+  private final int blockWriteLocateFollowingInitialDelayMs;
+  private final long defaultBlockSize;
+  private final long prefetchSize;
+  private final short defaultReplication;
+  private final String taskId;
+  private final FsPermission uMask;
+  private final boolean connectToDnViaHostname;
+  private final boolean hdfsBlocksMetadataEnabled;
+  private final int fileBlockStorageLocationsNumThreads;
+  private final int fileBlockStorageLocationsTimeoutMs;
+  private final int retryTimesForGetLastBlockLength;
+  private final int retryIntervalForGetLastBlockLength;
+  private final long datanodeRestartTimeout;
+  private final long slowIoWarningThresholdMs;
+
+  private final ShortCircuitConf shortCircuitConf;
+
+  public DfsClientConf(Configuration conf) {
+    // The hdfsTimeout is currently the same as the ipc timeout 
+    hdfsTimeout = Client.getTimeout(conf);
+
+    maxFailoverAttempts = conf.getInt(
+        DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
+        DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
+    maxRetryAttempts = conf.getInt(
+        HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
+        HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
+    failoverSleepBaseMillis = conf.getInt(
+        DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
+        DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
+    failoverSleepMaxMillis = conf.getInt(
+        DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
+        DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
+
+    maxBlockAcquireFailures = conf.getInt(
+        DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
+        DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
+    datanodeSocketWriteTimeout = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
+        HdfsServerConstants.WRITE_TIMEOUT);
+    ioBufferSize = conf.getInt(
+        CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+        CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+    defaultChecksumOpt = getChecksumOptFromConf(conf);
+    socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
+        HdfsServerConstants.READ_TIMEOUT);
+    /** dfs.write.packet.size is an internal config variable */
+    writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
+        DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
+    writeMaxPackets = conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_KEY,
+        DFSConfigKeys.DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_DEFAULT);
+    
+    final boolean byteArrayManagerEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_ENABLED_KEY,
+        DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_ENABLED_DEFAULT);
+    if (!byteArrayManagerEnabled) {
+      writeByteArrayManagerConf = null;
+    } else {
+      final int countThreshold = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_KEY,
+          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_DEFAULT);
+      final int countLimit = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_LIMIT_KEY,
+          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_LIMIT_DEFAULT);
+      final long countResetTimePeriodMs = conf.getLong(
+          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_KEY,
+          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_DEFAULT);
+      writeByteArrayManagerConf = new ByteArrayManager.Conf(
+          countThreshold, countLimit, countResetTimePeriodMs); 
+    }
+    
+    defaultBlockSize = conf.getLongBytes(DFS_BLOCK_SIZE_KEY,
+        DFS_BLOCK_SIZE_DEFAULT);
+    defaultReplication = (short) conf.getInt(
+        DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT);
+    taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
+    excludedNodesCacheExpiry = conf.getLong(
+        DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
+        DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
+    prefetchSize = conf.getLong(DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
+        10 * defaultBlockSize);
+    timeWindow = conf.getInt(
+        HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY,
+        HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT);
+    numCachedConnRetry = conf.getInt(DFS_CLIENT_CACHED_CONN_RETRY_KEY,
+        DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
+    numBlockWriteRetry = conf.getInt(DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
+        DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT);
+    numBlockWriteLocateFollowingRetry = conf.getInt(
+        DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
+        DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
+    blockWriteLocateFollowingInitialDelayMs = conf.getInt(
+        DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY,
+        DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_DEFAULT);
+    uMask = FsPermission.getUMask(conf);
+    connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
+        DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
+    hdfsBlocksMetadataEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, 
+        DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
+    fileBlockStorageLocationsNumThreads = conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS,
+        DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS_DEFAULT);
+    fileBlockStorageLocationsTimeoutMs = conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS,
+        DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT);
+    retryTimesForGetLastBlockLength = conf.getInt(
+        HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY,
+        HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
+    retryIntervalForGetLastBlockLength = conf.getInt(
+        HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_KEY,
+        HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
+
+
+    datanodeRestartTimeout = conf.getLong(
+        DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,
+        DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT) * 1000;
+    slowIoWarningThresholdMs = conf.getLong(
+        DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
+        DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
+    
+    shortCircuitConf = new ShortCircuitConf(conf);
+  }
+
+  private DataChecksum.Type getChecksumType(Configuration conf) {
+    final String checksum = conf.get(
+        DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY,
+        DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT);
+    try {
+      return DataChecksum.Type.valueOf(checksum);
+    } catch(IllegalArgumentException iae) {
+      DFSClient.LOG.warn("Bad checksum type: " + checksum + ". Using default "
+          + DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT);
+      return DataChecksum.Type.valueOf(
+          DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT); 
+    }
+  }
+
+  // Construct a checksum option from conf
+  private ChecksumOpt getChecksumOptFromConf(Configuration conf) {
+    DataChecksum.Type type = getChecksumType(conf);
+    int bytesPerChecksum = conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY,
+        DFS_BYTES_PER_CHECKSUM_DEFAULT);
+    return new ChecksumOpt(type, bytesPerChecksum);
+  }
+
+  /** create a DataChecksum with the given option. */
+  public DataChecksum createChecksum(ChecksumOpt userOpt) {
+    // Fill in any missing field with the default.
+    ChecksumOpt opt = ChecksumOpt.processChecksumOpt(
+        defaultChecksumOpt, userOpt);
+    DataChecksum dataChecksum = DataChecksum.newDataChecksum(
+        opt.getChecksumType(),
+        opt.getBytesPerChecksum());
+    if (dataChecksum == null) {
+      throw new HadoopIllegalArgumentException("Invalid checksum type: userOpt="
+          + userOpt + ", default=" + defaultChecksumOpt
+          + ", effective=null");
+    }
+    return dataChecksum;
+  }
+
+  @VisibleForTesting
+  public int getBlockWriteLocateFollowingInitialDelayMs() {
+    return blockWriteLocateFollowingInitialDelayMs;
+  }
+
+  /**
+   * @return the hdfsTimeout
+   */
+  public int getHdfsTimeout() {
+    return hdfsTimeout;
+  }
+
+  /**
+   * @return the maxFailoverAttempts
+   */
+  public int getMaxFailoverAttempts() {
+    return maxFailoverAttempts;
+  }
+
+  /**
+   * @return the maxRetryAttempts
+   */
+  public int getMaxRetryAttempts() {
+    return maxRetryAttempts;
+  }
+
+  /**
+   * @return the failoverSleepBaseMillis
+   */
+  public int getFailoverSleepBaseMillis() {
+    return failoverSleepBaseMillis;
+  }
+
+  /**
+   * @return the failoverSleepMaxMillis
+   */
+  public int getFailoverSleepMaxMillis() {
+    return failoverSleepMaxMillis;
+  }
+
+  /**
+   * @return the maxBlockAcquireFailures
+   */
+  public int getMaxBlockAcquireFailures() {
+    return maxBlockAcquireFailures;
+  }
+
+  /**
+   * @return the datanodeSocketWriteTimeout
+   */
+  public int getDatanodeSocketWriteTimeout() {
+    return datanodeSocketWriteTimeout;
+  }
+
+  /**
+   * @return the ioBufferSize
+   */
+  public int getIoBufferSize() {
+    return ioBufferSize;
+  }
+
+  /**
+   * @return the defaultChecksumOpt
+   */
+  public ChecksumOpt getDefaultChecksumOpt() {
+    return defaultChecksumOpt;
+  }
+
+  /**
+   * @return the writePacketSize
+   */
+  public int getWritePacketSize() {
+    return writePacketSize;
+  }
+
+  /**
+   * @return the writeMaxPackets
+   */
+  public int getWriteMaxPackets() {
+    return writeMaxPackets;
+  }
+
+  /**
+   * @return the writeByteArrayManagerConf
+   */
+  public ByteArrayManager.Conf getWriteByteArrayManagerConf() {
+    return writeByteArrayManagerConf;
+  }
+
+  /**
+   * @return the socketTimeout
+   */
+  public int getSocketTimeout() {
+    return socketTimeout;
+  }
+
+  /**
+   * @return the excludedNodesCacheExpiry
+   */
+  public long getExcludedNodesCacheExpiry() {
+    return excludedNodesCacheExpiry;
+  }
+
+  /**
+   * @return the timeWindow
+   */
+  public int getTimeWindow() {
+    return timeWindow;
+  }
+
+  /**
+   * @return the numCachedConnRetry
+   */
+  public int getNumCachedConnRetry() {
+    return numCachedConnRetry;
+  }
+
+  /**
+   * @return the numBlockWriteRetry
+   */
+  public int getNumBlockWriteRetry() {
+    return numBlockWriteRetry;
+  }
+
+  /**
+   * @return the numBlockWriteLocateFollowingRetry
+   */
+  public int getNumBlockWriteLocateFollowingRetry() {
+    return numBlockWriteLocateFollowingRetry;
+  }
+
+  /**
+   * @return the defaultBlockSize
+   */
+  public long getDefaultBlockSize() {
+    return defaultBlockSize;
+  }
+
+  /**
+   * @return the prefetchSize
+   */
+  public long getPrefetchSize() {
+    return prefetchSize;
+  }
+
+  /**
+   * @return the defaultReplication
+   */
+  public short getDefaultReplication() {
+    return defaultReplication;
+  }
+
+  /**
+   * @return the taskId
+   */
+  public String getTaskId() {
+    return taskId;
+  }
+
+  /**
+   * @return the uMask
+   */
+  public FsPermission getUMask() {
+    return uMask;
+  }
+
+  /**
+   * @return the connectToDnViaHostname
+   */
+  public boolean isConnectToDnViaHostname() {
+    return connectToDnViaHostname;
+  }
+
+  /**
+   * @return the hdfsBlocksMetadataEnabled
+   */
+  public boolean isHdfsBlocksMetadataEnabled() {
+    return hdfsBlocksMetadataEnabled;
+  }
+
+  /**
+   * @return the fileBlockStorageLocationsNumThreads
+   */
+  public int getFileBlockStorageLocationsNumThreads() {
+    return fileBlockStorageLocationsNumThreads;
+  }
+
+  /**
+   * @return the getFileBlockStorageLocationsTimeoutMs
+   */
+  public int getFileBlockStorageLocationsTimeoutMs() {
+    return fileBlockStorageLocationsTimeoutMs;
+  }
+
+  /**
+   * @return the retryTimesForGetLastBlockLength
+   */
+  public int getRetryTimesForGetLastBlockLength() {
+    return retryTimesForGetLastBlockLength;
+  }
+
+  /**
+   * @return the retryIntervalForGetLastBlockLength
+   */
+  public int getRetryIntervalForGetLastBlockLength() {
+    return retryIntervalForGetLastBlockLength;
+  }
+
+  /**
+   * @return the datanodeRestartTimeout
+   */
+  public long getDatanodeRestartTimeout() {
+    return datanodeRestartTimeout;
+  }
+
+  /**
+   * @return the slowIoWarningThresholdMs
+   */
+  public long getSlowIoWarningThresholdMs() {
+    return slowIoWarningThresholdMs;
+  }
+
+  /**
+   * @return the shortCircuitConf
+   */
+  public ShortCircuitConf getShortCircuitConf() {
+    return shortCircuitConf;
+  }
+
+  public static class ShortCircuitConf {
+    private static final Log LOG = LogFactory.getLog(ShortCircuitConf.class);
+
+    private final int socketCacheCapacity;
+    private final long socketCacheExpiry;
+
+    private final boolean useLegacyBlockReader;
+    private final boolean useLegacyBlockReaderLocal;
+    private final String domainSocketPath;
+    private final boolean skipShortCircuitChecksums;
+
+    private final int shortCircuitBufferSize;
+    private final boolean shortCircuitLocalReads;
+    private final boolean domainSocketDataTraffic;
+    private final int shortCircuitStreamsCacheSize;
+    private final long shortCircuitStreamsCacheExpiryMs; 
+    private final int shortCircuitSharedMemoryWatcherInterruptCheckMs;
+    
+    private final boolean shortCircuitMmapEnabled;
+    private final int shortCircuitMmapCacheSize;
+    private final long shortCircuitMmapCacheExpiryMs;
+    private final long shortCircuitMmapCacheRetryTimeout;
+    private final long shortCircuitCacheStaleThresholdMs;
+
+    private final long keyProviderCacheExpiryMs;
+
+    @VisibleForTesting
+    public BlockReaderFactory.FailureInjector brfFailureInjector =
+        new BlockReaderFactory.FailureInjector();
+
+    public ShortCircuitConf(Configuration conf) {
+      socketCacheCapacity = conf.getInt(
+          DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
+          DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT);
+      socketCacheExpiry = conf.getLong(
+          DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY,
+          DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT);
+
+      useLegacyBlockReader = conf.getBoolean(
+          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER,
+          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT);
+      useLegacyBlockReaderLocal = conf.getBoolean(
+          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
+          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT);
+      shortCircuitLocalReads = conf.getBoolean(
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
+      domainSocketDataTraffic = conf.getBoolean(
+          DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+          DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT);
+      domainSocketPath = conf.getTrimmed(
+          DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+          DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL
+            + " = " + useLegacyBlockReaderLocal);
+        LOG.debug(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY
+            + " = " + shortCircuitLocalReads);
+        LOG.debug(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC
+            + " = " + domainSocketDataTraffic);
+        LOG.debug(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY
+            + " = " + domainSocketPath);
+      }
+
+      skipShortCircuitChecksums = conf.getBoolean(
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
+      shortCircuitBufferSize = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
+      shortCircuitStreamsCacheSize = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY,
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT);
+      shortCircuitStreamsCacheExpiryMs = conf.getLong(
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT);
+      shortCircuitMmapEnabled = conf.getBoolean(
+          DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED,
+          DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED_DEFAULT);
+      shortCircuitMmapCacheSize = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE,
+          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT);
+      shortCircuitMmapCacheExpiryMs = conf.getLong(
+          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
+          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT);
+      shortCircuitMmapCacheRetryTimeout = conf.getLong(
+          DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
+          DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT);
+      shortCircuitCacheStaleThresholdMs = conf.getLong(
+          DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
+          DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT);
+      shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
+          DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
+          DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
+
+      keyProviderCacheExpiryMs = conf.getLong(
+          DFSConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_MS,
+          DFSConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT);
+    }
+
+    /**
+     * @return the socketCacheCapacity
+     */
+    public int getSocketCacheCapacity() {
+      return socketCacheCapacity;
+    }
+
+    /**
+     * @return the socketCacheExpiry
+     */
+    public long getSocketCacheExpiry() {
+      return socketCacheExpiry;
+    }
+
+    public boolean isUseLegacyBlockReaderLocal() {
+      return useLegacyBlockReaderLocal;
+    }
+
+    public String getDomainSocketPath() {
+      return domainSocketPath;
+    }
+
+    public boolean isShortCircuitLocalReads() {
+      return shortCircuitLocalReads;
+    }
+
+    public boolean isDomainSocketDataTraffic() {
+      return domainSocketDataTraffic;
+    }
+    /**
+     * @return the useLegacyBlockReader
+     */
+    public boolean isUseLegacyBlockReader() {
+      return useLegacyBlockReader;
+    }
+
+    /**
+     * @return the skipShortCircuitChecksums
+     */
+    public boolean isSkipShortCircuitChecksums() {
+      return skipShortCircuitChecksums;
+    }
+
+    /**
+     * @return the shortCircuitBufferSize
+     */
+    public int getShortCircuitBufferSize() {
+      return shortCircuitBufferSize;
+    }
+
+    /**
+     * @return the shortCircuitStreamsCacheSize
+     */
+    public int getShortCircuitStreamsCacheSize() {
+      return shortCircuitStreamsCacheSize;
+    }
+
+    /**
+     * @return the shortCircuitStreamsCacheExpiryMs
+     */
+    public long getShortCircuitStreamsCacheExpiryMs() {
+      return shortCircuitStreamsCacheExpiryMs;
+    }
+
+    /**
+     * @return the shortCircuitSharedMemoryWatcherInterruptCheckMs
+     */
+    public int getShortCircuitSharedMemoryWatcherInterruptCheckMs() {
+      return shortCircuitSharedMemoryWatcherInterruptCheckMs;
+    }
+
+    /**
+     * @return the shortCircuitMmapEnabled
+     */
+    public boolean isShortCircuitMmapEnabled() {
+      return shortCircuitMmapEnabled;
+    }
+
+    /**
+     * @return the shortCircuitMmapCacheSize
+     */
+    public int getShortCircuitMmapCacheSize() {
+      return shortCircuitMmapCacheSize;
+    }
+
+    /**
+     * @return the shortCircuitMmapCacheExpiryMs
+     */
+    public long getShortCircuitMmapCacheExpiryMs() {
+      return shortCircuitMmapCacheExpiryMs;
+    }
+
+    /**
+     * @return the shortCircuitMmapCacheRetryTimeout
+     */
+    public long getShortCircuitMmapCacheRetryTimeout() {
+      return shortCircuitMmapCacheRetryTimeout;
+    }
+
+    /**
+     * @return the shortCircuitCacheStaleThresholdMs
+     */
+    public long getShortCircuitCacheStaleThresholdMs() {
+      return shortCircuitCacheStaleThresholdMs;
+    }
+
+    /**
+     * @return the keyProviderCacheExpiryMs
+     */
+    public long getKeyProviderCacheExpiryMs() {
+      return keyProviderCacheExpiryMs;
+    }
+
+    public String confAsString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("shortCircuitStreamsCacheSize = ").
+        append(shortCircuitStreamsCacheSize).
+        append(", shortCircuitStreamsCacheExpiryMs = ").
+        append(shortCircuitStreamsCacheExpiryMs).
+        append(", shortCircuitMmapCacheSize = ").
+        append(shortCircuitMmapCacheSize).
+        append(", shortCircuitMmapCacheExpiryMs = ").
+        append(shortCircuitMmapCacheExpiryMs).
+        append(", shortCircuitMmapCacheRetryTimeout = ").
+        append(shortCircuitMmapCacheRetryTimeout).
+        append(", shortCircuitCacheStaleThresholdMs = ").
+        append(shortCircuitCacheStaleThresholdMs).
+        append(", socketCacheCapacity = ").
+        append(socketCacheCapacity).
+        append(", socketCacheExpiry = ").
+        append(socketCacheExpiry).
+        append(", shortCircuitLocalReads = ").
+        append(shortCircuitLocalReads).
+        append(", useLegacyBlockReaderLocal = ").
+        append(useLegacyBlockReaderLocal).
+        append(", domainSocketDataTraffic = ").
+        append(domainSocketDataTraffic).
+        append(", shortCircuitSharedMemoryWatcherInterruptCheckMs = ").
+        append(shortCircuitSharedMemoryWatcherInterruptCheckMs).
+        append(", keyProviderCacheExpiryMs = ").
+        append(keyProviderCacheExpiryMs);
+
+      return builder.toString();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DomainSocketFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DomainSocketFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DomainSocketFactory.java
index 5fd31a9..fadb2f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DomainSocketFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DomainSocketFactory.java
@@ -26,14 +26,14 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSClient.Conf;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.util.PerformanceAdvisory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
-import org.apache.hadoop.util.PerformanceAdvisory;
 
 public class DomainSocketFactory {
   private static final Log LOG = LogFactory.getLog(DomainSocketFactory.class);
@@ -95,7 +95,7 @@ public class DomainSocketFactory {
       .expireAfterWrite(10, TimeUnit.MINUTES)
       .build();
 
-  public DomainSocketFactory(Conf conf) {
+  public DomainSocketFactory(ShortCircuitConf conf) {
     final String feature;
     if (conf.isShortCircuitLocalReads() && (!conf.isUseLegacyBlockReaderLocal())) {
       feature = "The short-circuit local reads feature";
@@ -129,7 +129,7 @@ public class DomainSocketFactory {
    *
    * @return             Information about the socket path.
    */
-  public PathInfo getPathInfo(InetSocketAddress addr, DFSClient.Conf conf) {
+  public PathInfo getPathInfo(InetSocketAddress addr, ShortCircuitConf conf) {
     // If there is no domain socket path configured, we can't use domain
     // sockets.
     if (conf.getDomainSocketPath().isEmpty()) return PathInfo.NOT_CONFIGURED;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
index d1ec3b8..27a9ef2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -359,6 +360,17 @@ public class ShortCircuitCache implements Closeable {
             DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT));
   }
 
+  public static ShortCircuitCache fromConf(ShortCircuitConf conf) {
+    return new ShortCircuitCache(
+        conf.getShortCircuitStreamsCacheSize(),
+        conf.getShortCircuitStreamsCacheExpiryMs(),
+        conf.getShortCircuitMmapCacheSize(),
+        conf.getShortCircuitMmapCacheExpiryMs(),
+        conf.getShortCircuitMmapCacheRetryTimeout(),
+        conf.getShortCircuitCacheStaleThresholdMs(),
+        conf.getShortCircuitSharedMemoryWatcherInterruptCheckMs());
+  }
+
   public ShortCircuitCache(int maxTotalSize, long maxNonMmappedEvictableLifespanMs,
       int maxEvictableMmapedSize, long maxEvictableMmapedLifespanMs,
       long mmapRetryTimeoutMs, long staleThresholdMs, int shmInterruptCheckMs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
index 296c8d2..6d644ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
@@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.ClientContext;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -50,6 +49,7 @@ import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -359,7 +359,7 @@ public class TestEnhancedByteBufferAccess {
     fsIn.close();
     fsIn = fs.open(TEST_PATH);
     final ShortCircuitCache cache = ClientContext.get(
-        CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache();
+        CONTEXT, new DfsClientConf(conf)). getShortCircuitCache();
     cache.accept(new CountingVisitor(0, 5, 5, 0));
     results[0] = fsIn.read(null, BLOCK_SIZE,
         EnumSet.of(ReadOption.SKIP_CHECKSUMS));
@@ -662,7 +662,7 @@ public class TestEnhancedByteBufferAccess {
     final ExtendedBlock firstBlock =
         DFSTestUtil.getFirstBlock(fs, TEST_PATH);
     final ShortCircuitCache cache = ClientContext.get(
-        CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache();
+        CONTEXT, new DfsClientConf(conf)). getShortCircuitCache();
     waitForReplicaAnchorStatus(cache, firstBlock, true, true, 1);
     // Uncache the replica
     fs.removeCacheDirective(directiveId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
index 29c32f5..ab3515e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
@@ -187,7 +188,7 @@ public class TestBlockReaderLocal {
               Time.now(), shm.allocAndRegisterSlot(
                   ExtendedBlockId.fromExtendedBlock(block)));
       blockReaderLocal = new BlockReaderLocal.Builder(
-              new DFSClient.Conf(conf)).
+              new DfsClientConf.ShortCircuitConf(conf)).
           setFilename(TEST_PATH.getName()).
           setBlock(block).
           setShortCircuitReplica(replica).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 5d95a8b..1c55d5d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -300,7 +300,7 @@ public class TestDFSClientRetries {
       NamenodeProtocols preSpyNN = cluster.getNameNodeRpc();
       NamenodeProtocols spyNN = spy(preSpyNN);
       DFSClient client = new DFSClient(null, spyNN, conf, null);
-      int maxBlockAcquires = client.getMaxBlockAcquireFailures();
+      int maxBlockAcquires = client.getConf().getMaxBlockAcquireFailures();
       assertTrue(maxBlockAcquires > 0);
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index a410e74..478f7e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -113,7 +114,7 @@ public class TestDFSOutputStream {
 
   @Test
   public void testCongestionBackoff() throws IOException {
-    DFSClient.Conf dfsClientConf = mock(DFSClient.Conf.class);
+    DfsClientConf dfsClientConf = mock(DfsClientConf.class);
     DFSClient client = mock(DFSClient.class);
     when(client.getConf()).thenReturn(dfsClientConf);
     client.clientRunning = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
index 11cbcad..f091db7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertSame;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
@@ -59,13 +60,13 @@ public class TestLeaseRenewer {
 }
  
   private DFSClient createMockClient() {
+    final DfsClientConf mockConf = Mockito.mock(DfsClientConf.class);
+    Mockito.doReturn((int)FAST_GRACE_PERIOD).when(mockConf).getHdfsTimeout();
+
     DFSClient mock = Mockito.mock(DFSClient.class);
-    Mockito.doReturn(true)
-      .when(mock).isClientRunning();
-    Mockito.doReturn((int)FAST_GRACE_PERIOD)
-      .when(mock).getHdfsTimeout();
-    Mockito.doReturn("myclient")
-      .when(mock).getClientName();
+    Mockito.doReturn(true).when(mock).isClientRunning();
+    Mockito.doReturn(mockConf).when(mock).getConf();
+    Mockito.doReturn("myclient").when(mock).getClientName();
     return mock;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
index 23e2a7a..05698ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
-import org.junit.Assume;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -325,7 +324,7 @@ public class TestParallelReadUtil {
       testInfo.filepath = new Path("/TestParallelRead.dat." + i);
       testInfo.authenticData = util.writeFile(testInfo.filepath, FILE_SIZE_K);
       testInfo.dis = dfsClient.open(testInfo.filepath.toString(),
-          dfsClient.getConf().ioBufferSize, verifyChecksums);
+          dfsClient.getConf().getIoBufferSize(), verifyChecksums);
 
       for (int j = 0; j < nWorkerEach; ++j) {
         workers[nWorkers++] = new ReadWorker(testInfo, nWorkers, helper);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index 84e5c82..c280027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.RemotePeerFactory;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -146,7 +147,7 @@ public class TestBlockTokenWithDFS {
       DatanodeInfo[] nodes = lblock.getLocations();
       targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr());
 
-      blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
+      blockReader = new BlockReaderFactory(new DfsClientConf(conf)).
           setFileName(BlockReaderFactory.getFileName(targetAddr, 
                         "test-blockpoolid", block.getBlockId())).
           setBlock(block).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index 41e8d7b..0a90947 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -40,12 +40,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.ClientContext;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.RemotePeerFactory;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -405,7 +405,7 @@ public class TestDataNodeVolumeFailure {
    
     targetAddr = NetUtils.createSocketAddr(datanode.getXferAddr());
 
-    BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
+    BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)).
       setInetSocketAddress(targetAddr).
       setBlock(block).
       setFileName(BlockReaderFactory.getFileName(targetAddr,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index 7d26dee..e38b97b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -657,7 +657,7 @@ public class TestShortCircuitCache {
 
     // The second read should fail, and we should only have 1 segment and 1 slot
     // left.
-    fs.getClient().getConf().brfFailureInjector =
+    fs.getClient().getConf().getShortCircuitConf().brfFailureInjector =
         new TestCleanupFailureInjector();
     try {
       DFSTestUtil.readFileBuffer(fs, TEST_PATH2);


[49/50] [abbrv] hadoop git commit: YARN-1402. Update related Web UI and CLI with exposing client API to check log aggregation status. Contributed by Xuan Gong.

Posted by zj...@apache.org.
YARN-1402. Update related Web UI and CLI with exposing client API to check log aggregation status. Contributed by Xuan Gong.

Conflicts:
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/23536766
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/23536766
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/23536766

Branch: refs/heads/YARN-2928
Commit: 23536766ef890a4f3478491b11b186c98a63eb5a
Parents: f846cc5
Author: Junping Du <ju...@apache.org>
Authored: Fri Apr 17 13:18:59 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:31:14 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/api/records/ApplicationReport.java     |  14 ++-
 .../yarn/api/records/LogAggregationStatus.java  |  55 ++++++++++
 .../src/main/proto/yarn_protos.proto            |  10 ++
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   3 +
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |   3 +
 .../impl/pb/ApplicationReportPBImpl.java        |  33 ++++++
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  17 +++
 .../protocolrecords/LogAggregationReport.java   |   2 +-
 .../impl/pb/LogAggregationReportPBImpl.java     |  13 +--
 .../api/records/LogAggregationStatus.java       |  31 ------
 .../main/proto/yarn_server_common_protos.proto  |  11 +-
 .../logaggregation/AppLogAggregatorImpl.java    |   4 +-
 .../server/resourcemanager/rmapp/RMApp.java     |   3 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |  99 +++++++++++++++--
 .../webapp/RMAppLogAggregationStatusBlock.java  |  11 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |   9 +-
 .../applicationsmanager/MockAsm.java            |   6 ++
 .../TestRMAppLogAggregationStatus.java          | 106 ++++++++++++++++++-
 .../server/resourcemanager/rmapp/MockRMApp.java |   6 ++
 .../webapp/TestRMWebServicesApps.java           |  14 ++-
 21 files changed, 375 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index cf64609..6749b4e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -143,6 +143,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3354. Add node label expression in ContainerTokenIdentifier to support
     RM recovery. (Wangda Tan via jianhe)
 
+    YARN-1402. Update related Web UI and CLI with exposing client API to check
+    log aggregation status. (Xuan Gong via junping_du)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
index ff4fb52..e5d7254 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
@@ -361,5 +361,17 @@ public abstract class ApplicationReport {
   @Public
   @Stable
   public abstract Token getAMRMToken();
-  
+
+  /**
+   * Get log aggregation status for the application
+   * @return Application's log aggregation status
+   */
+  @Public
+  @Stable
+  public abstract LogAggregationStatus getLogAggregationStatus();
+
+  @Private
+  @Unstable
+  public abstract void setLogAggregationStatus(
+      LogAggregationStatus logAggregationStatus);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java
new file mode 100644
index 0000000..da1230c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+/**
+ * <p>Status of Log aggregation.</p>
+ */
+public enum LogAggregationStatus {
+
+  /** Log Aggregation is Disabled. */
+  DISABLED,
+
+  /** Log Aggregation does not Start. */
+  NOT_START,
+
+  /** Log Aggregation is Running. */
+  RUNNING,
+
+  /**
+   * Log Aggregation is Succeeded. All of the logs have been aggregated
+   * successfully.
+   */
+  SUCCEEDED,
+
+  /**
+   * Log Aggregation is completed. But at least one of the logs have not been
+   * aggregated.
+   */
+  FAILED,
+
+  /**
+   * The application is finished, but the log aggregation status is not updated
+   * for a long time. 
+   * @see YarnConfiguration#LOG_AGGREGATION_STATUS_TIME_OUT_MS
+   */
+  TIME_OUT
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 7781d65..a0491fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -194,6 +194,16 @@ message ApplicationReportProto {
   optional string applicationType = 18;
   optional hadoop.common.TokenProto am_rm_token = 19;
   repeated string applicationTags = 20;
+  optional LogAggregationStatusProto log_aggregation_status = 21;
+}
+
+enum LogAggregationStatusProto {
+  LOG_DISABLED = 1;
+  LOG_NOT_START = 2;
+  LOG_RUNNING = 3;
+  LOG_SUCCEEDED = 4;
+  LOG_FAILED = 5;
+  LOG_TIME_OUT = 6;
 }
 
 message ApplicationAttemptReportProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index dd4a949..8ef88c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -530,6 +530,9 @@ public class ApplicationCLI extends YarnCLI {
       } else {
         appReportStr.println("N/A");
       }
+      appReportStr.print("\tLog Aggregation Status : ");
+      appReportStr.println(appReport.getLogAggregationStatus() == null ? "N/A"
+          : appReport.getLogAggregationStatus());
       appReportStr.print("\tDiagnostics : ");
       appReportStr.print(appReport.getDiagnostics());
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index b8be88d..003f086 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -104,6 +105,7 @@ public class TestYarnCLI {
           YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
           FinalApplicationStatus.SUCCEEDED, usageReport, "N/A", 0.53789f, "YARN",
           null);
+      newApplicationReport.setLogAggregationStatus(LogAggregationStatus.SUCCEEDED);
       when(client.getApplicationReport(any(ApplicationId.class))).thenReturn(
           newApplicationReport);
       int result = cli.run(new String[] { "application", "-status", applicationId.toString() });
@@ -127,6 +129,7 @@ public class TestYarnCLI {
       pw.println("\tAM Host : host");
       pw.println("\tAggregate Resource Allocation : " +
           (i == 0 ? "N/A" : "123456 MB-seconds, 4567 vcore-seconds"));
+      pw.println("\tLog Aggregation Status : SUCCEEDED");
       pw.println("\tDiagnostics : diagnostics");
       pw.close();
       String appReportStr = baos.toString("UTF-8");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
index dd3e2bc..751dd90 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
@@ -34,6 +35,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationReportProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationResourceUsageReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 
 import com.google.protobuf.TextFormat;
@@ -548,4 +550,35 @@ public class ApplicationReportPBImpl extends ApplicationReport {
   private TokenProto convertToProtoFormat(Token t) {
     return ((TokenPBImpl)t).getProto();
   }
+
+  @Override
+  public LogAggregationStatus getLogAggregationStatus() {
+    ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasLogAggregationStatus()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getLogAggregationStatus());
+  }
+
+  @Override
+  public void setLogAggregationStatus(
+      LogAggregationStatus logAggregationStatus) {
+    maybeInitBuilder();
+    if (logAggregationStatus == null) {
+      builder.clearLogAggregationStatus();
+      return;
+    }
+    builder.setLogAggregationStatus(
+        convertToProtoFormat(logAggregationStatus));
+  }
+
+  private LogAggregationStatus convertFromProtoFormat(
+      LogAggregationStatusProto s) {
+    return ProtoUtils.convertFromProtoFormat(s);
+  }
+
+  private LogAggregationStatusProto
+      convertToProtoFormat(LogAggregationStatus s) {
+    return ProtoUtils.convertToProtoFormat(s);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index 586e9dd..4e8a19c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -44,6 +45,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceTypeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceVisibilityProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
@@ -253,4 +255,19 @@ public class ProtoUtils {
     return ReservationRequestInterpreter.valueOf(e.name());
   }
 
+  /*
+   * Log Aggregation Status
+   */
+  private static final String LOG_AGGREGATION_STATUS_PREFIX = "LOG_";
+  public static LogAggregationStatusProto convertToProtoFormat(
+      LogAggregationStatus e) {
+    return LogAggregationStatusProto.valueOf(LOG_AGGREGATION_STATUS_PREFIX
+        + e.name());
+  }
+
+  public static LogAggregationStatus convertFromProtoFormat(
+      LogAggregationStatusProto e) {
+    return LogAggregationStatus.valueOf(e.name().replace(
+      LOG_AGGREGATION_STATUS_PREFIX, ""));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java
index 808804b..b2270d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java
@@ -21,8 +21,8 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.server.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.util.Records;
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java
index 7999fa7..75b6eab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java
@@ -21,16 +21,17 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
-import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.LogAggregationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
-import org.apache.hadoop.yarn.server.api.records.LogAggregationStatus;
 
 import com.google.protobuf.TextFormat;
 
@@ -43,8 +44,6 @@ public class LogAggregationReportPBImpl extends LogAggregationReport {
   LogAggregationReportProto.Builder builder = null;
   boolean viaProto = false;
 
-  private static final String LOGAGGREGATION_STATUS_PREFIX = "LOG_";
-
   private ApplicationId applicationId;
   private NodeId nodeId;
 
@@ -166,14 +165,12 @@ public class LogAggregationReportPBImpl extends LogAggregationReport {
 
   private LogAggregationStatus convertFromProtoFormat(
       LogAggregationStatusProto s) {
-    return LogAggregationStatus.valueOf(s.name().replace(
-      LOGAGGREGATION_STATUS_PREFIX, ""));
+    return ProtoUtils.convertFromProtoFormat(s);
   }
 
   private LogAggregationStatusProto
       convertToProtoFormat(LogAggregationStatus s) {
-    return LogAggregationStatusProto.valueOf(LOGAGGREGATION_STATUS_PREFIX
-        + s.name());
+    return ProtoUtils.convertToProtoFormat(s);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/LogAggregationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/LogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/LogAggregationStatus.java
deleted file mode 100644
index 496767f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/LogAggregationStatus.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.api.records;
-
-/**
- * <p>Status of Log aggregation.</p>
- */
-public enum LogAggregationStatus {
-  DISABLED,
-  NOT_START,
-  RUNNING,
-  FINISHED,
-  FAILED,
-  TIME_OUT
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
index 6e9f4cb..99149ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
@@ -52,13 +52,4 @@ message NodeHealthStatusProto {
 message VersionProto {
   optional int32 major_version = 1;
   optional int32 minor_version = 2;
-}
-
-enum LogAggregationStatusProto {
-  LOG_DISABLED = 1;
-  LOG_NOT_START = 2;
-  LOG_RUNNING = 3;
-  LOG_FINISHED = 4;
-  LOG_TIME_OUT = 5;
-}
-
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index bf7d5f8..3f09e5d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -58,7 +59,6 @@ import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogWriter;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy;
 import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
-import org.apache.hadoop.yarn.server.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
@@ -347,7 +347,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
       report.setDiagnosticMessage(diagnosticMessage);
       if (appFinished) {
         report.setLogAggregationStatus(renameTemporaryLogFileFailed
-            ? LogAggregationStatus.FAILED : LogAggregationStatus.FINISHED);
+            ? LogAggregationStatus.FAILED : LogAggregationStatus.SUCCEEDED);
       } else {
         report.setLogAggregationStatus(LogAggregationStatus.RUNNING);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index 8234bb1..d4b758e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -262,4 +263,6 @@ public interface RMApp extends EventHandler<RMAppEvent> {
   ResourceRequest getAMResourceRequest();
 
   Map<NodeId, LogAggregationReport> getLogAggregationReportsForApp();
+
+  LogAggregationStatus getLogAggregationStatusForAppReport();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 7945262..fba391c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.ReservationId;
@@ -64,7 +65,6 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
-import org.apache.hadoop.yarn.server.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
@@ -152,6 +152,7 @@ public class RMAppImpl implements RMApp, Recoverable {
   private final long logAggregationStatusTimeout;
   private final Map<NodeId, LogAggregationReport> logAggregationStatus =
       new HashMap<NodeId, LogAggregationReport>();
+  private LogAggregationStatus logAggregationStatusForAppReport;
 
   // These states stored are only valid when app is at killing or final_saving.
   private RMAppState stateBeforeKilling;
@@ -610,6 +611,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       String trackingUrl = UNAVAILABLE;
       String host = UNAVAILABLE;
       String origTrackingUrl = UNAVAILABLE;
+      LogAggregationStatus logAggregationStatus = null;
       int rpcPort = -1;
       ApplicationResourceUsageReport appUsageReport =
           RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT;
@@ -640,6 +642,7 @@ public class RMAppImpl implements RMApp, Recoverable {
           rpcPort = this.currentAttempt.getRpcPort();
           appUsageReport = currentAttempt.getApplicationResourceUsageReport();
           progress = currentAttempt.getProgress();
+          logAggregationStatus = this.getLogAggregationStatusForAppReport();
         }
         diags = this.diagnostics.toString();
 
@@ -667,13 +670,15 @@ public class RMAppImpl implements RMApp, Recoverable {
                 DUMMY_APPLICATION_ATTEMPT_NUMBER);
       }
 
-      return BuilderUtils.newApplicationReport(this.applicationId,
-          currentApplicationAttemptId, this.user, this.queue,
-          this.name, host, rpcPort, clientToAMToken,
+      ApplicationReport report = BuilderUtils.newApplicationReport(
+          this.applicationId, currentApplicationAttemptId, this.user,
+          this.queue, this.name, host, rpcPort, clientToAMToken,
           createApplicationState(), diags,
           trackingUrl, this.startTime, this.finishTime, finishState,
           appUsageReport, origTrackingUrl, progress, this.applicationType, 
           amrmToken, applicationTags);
+      report.setLogAggregationStatus(logAggregationStatus);
+      return report;
     } finally {
       this.readLock.unlock();
     }
@@ -876,11 +881,13 @@ public class RMAppImpl implements RMApp, Recoverable {
       // otherwise, add it to ranNodes for further process
       app.ranNodes.add(nodeAddedEvent.getNodeId());
 
-      app.logAggregationStatus.put(nodeAddedEvent.getNodeId(),
-        LogAggregationReport.newInstance(app.applicationId, nodeAddedEvent
-          .getNodeId(), app.logAggregationEnabled
-            ? LogAggregationStatus.NOT_START : LogAggregationStatus.DISABLED,
-          ""));
+      if (!app.logAggregationStatus.containsKey(nodeAddedEvent.getNodeId())) {
+        app.logAggregationStatus.put(nodeAddedEvent.getNodeId(),
+          LogAggregationReport.newInstance(app.applicationId, nodeAddedEvent
+            .getNodeId(), app.logAggregationEnabled
+              ? LogAggregationStatus.NOT_START : LogAggregationStatus.DISABLED,
+            ""));
+      }
     };
   }
 
@@ -1447,7 +1454,9 @@ public class RMAppImpl implements RMApp, Recoverable {
         if (!output.getValue().getLogAggregationStatus()
           .equals(LogAggregationStatus.TIME_OUT)
             && !output.getValue().getLogAggregationStatus()
-              .equals(LogAggregationStatus.FINISHED)
+              .equals(LogAggregationStatus.SUCCEEDED)
+            && !output.getValue().getLogAggregationStatus()
+              .equals(LogAggregationStatus.FAILED)
             && isAppInFinalState(this)
             && System.currentTimeMillis() > this.logAggregationStartTime
                 + this.logAggregationStatusTimeout) {
@@ -1472,7 +1481,9 @@ public class RMAppImpl implements RMApp, Recoverable {
           if (curReport.getLogAggregationStatus().equals(
             LogAggregationStatus.TIME_OUT)) {
             if (report.getLogAggregationStatus().equals(
-              LogAggregationStatus.FINISHED)) {
+              LogAggregationStatus.SUCCEEDED)
+                || report.getLogAggregationStatus().equals(
+                  LogAggregationStatus.FAILED)) {
               curReport.setLogAggregationStatus(report
                 .getLogAggregationStatus());
             }
@@ -1493,4 +1504,70 @@ public class RMAppImpl implements RMApp, Recoverable {
       this.writeLock.unlock();
     }
   }
+
+  @Override
+  public LogAggregationStatus getLogAggregationStatusForAppReport() {
+    if (!logAggregationEnabled) {
+      return LogAggregationStatus.DISABLED;
+    }
+    if (this.logAggregationStatusForAppReport == LogAggregationStatus.FAILED
+        || this.logAggregationStatusForAppReport == LogAggregationStatus.SUCCEEDED) {
+      return this.logAggregationStatusForAppReport;
+    }
+    try {
+      this.readLock.lock();
+      Map<NodeId, LogAggregationReport> reports =
+          getLogAggregationReportsForApp();
+      if (reports.size() == 0) {
+        return null;
+      }
+      int logNotStartCount = 0;
+      int logCompletedCount = 0;
+      int logTimeOutCount = 0;
+      int logFailedCount = 0;
+      for (Entry<NodeId, LogAggregationReport> report : reports.entrySet()) {
+        switch (report.getValue().getLogAggregationStatus()) {
+          case NOT_START:
+            logNotStartCount++;
+            break;
+          case SUCCEEDED:
+            logCompletedCount++;
+            break;
+          case FAILED:
+            logFailedCount++;
+            logCompletedCount++;
+            break;
+          case TIME_OUT:
+            logTimeOutCount++;
+            logCompletedCount++;
+            break;
+          default:
+            break;
+        }
+      }
+      if (logNotStartCount == reports.size()) {
+        return LogAggregationStatus.NOT_START;
+      } else if (logCompletedCount == reports.size()) {
+        // We should satisfy two condition in order to return SUCCEEDED or FAILED
+        // 1) make sure the application is in final state
+        // 2) logs status from all NMs are SUCCEEDED/FAILED/TIMEOUT
+        // The SUCCEEDED/FAILED status is the final status which means
+        // the log aggregation is finished. And the log aggregation status will
+        // not be updated anymore.
+        if (logFailedCount > 0 && isAppInFinalState(this)) {
+          this.logAggregationStatusForAppReport = LogAggregationStatus.FAILED;
+          return LogAggregationStatus.FAILED;
+        } else if (logTimeOutCount > 0) {
+          return LogAggregationStatus.TIME_OUT;
+        }
+        if (isAppInFinalState(this)) {
+          this.logAggregationStatusForAppReport = LogAggregationStatus.SUCCEEDED;
+          return LogAggregationStatus.SUCCEEDED;
+        }
+      }
+      return LogAggregationStatus.RUNNING;
+    } finally {
+      this.readLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
index a95f76f..a2f61e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
@@ -30,10 +30,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
-import org.apache.hadoop.yarn.server.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.util.Apps;
@@ -93,15 +93,16 @@ public class RMAppLogAggregationStatusBlock extends HtmlBlock {
       .td("Log Aggregation does not Start.")._();
     table_description.tr().td(LogAggregationStatus.RUNNING.name())
       .td("Log Aggregation is Running.")._();
-    table_description.tr().td(LogAggregationStatus.FINISHED.name())
-      .td("Log Aggregation is Finished. All of the logs have been "
+    table_description.tr().td(LogAggregationStatus.SUCCEEDED.name())
+      .td("Log Aggregation is Succeeded. All of the logs have been "
           + "aggregated successfully.")._();
     table_description.tr().td(LogAggregationStatus.FAILED.name())
       .td("Log Aggregation is Failed. At least one of the logs "
           + "have not been aggregated.")._();
     table_description.tr().td(LogAggregationStatus.TIME_OUT.name())
-      .td("Does not get the Log aggregation status for a long time. "
-          + "Not sure what is the current Log Aggregation Status.")._();
+      .td("The application is finished, but the log aggregation status is "
+          + "not updated for a long time. Not sure whether the log aggregation "
+          + "is finished or not.")._();
     table_description._();
     div_description._();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
index 79b2248..bd3b046 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -94,6 +95,8 @@ public class AppInfo {
 
   protected List<ResourceRequest> resourceRequests;
 
+  protected LogAggregationStatus logAggregationStatus;
+
   public AppInfo() {
   } // JAXB needs this
 
@@ -141,7 +144,7 @@ public class AppInfo {
         this.finishedTime = app.getFinishTime();
         this.elapsedTime = Times.elapsed(app.getStartTime(),
             app.getFinishTime());
-
+        this.logAggregationStatus = app.getLogAggregationStatusForAppReport();
         RMAppAttempt attempt = app.getCurrentAppAttempt();
         if (attempt != null) {
           Container masterContainer = attempt.getMasterContainer();
@@ -314,4 +317,8 @@ public class AppInfo {
   public List<ResourceRequest> getResourceRequests() {
     return this.resourceRequests;
   }
+
+  public LogAggregationStatus getLogAggregationStatus() {
+    return this.logAggregationStatus;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
index df919bc..e52b054 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -208,6 +209,11 @@ public abstract class MockAsm extends MockApps {
     public Map<NodeId, LogAggregationReport> getLogAggregationReportsForApp() {
       throw new UnsupportedOperationException("Not supported yet.");
     }
+
+    @Override
+    public LogAggregationStatus getLogAggregationStatusForAppReport() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
   }
 
   public static RMApp newApplication(int i) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
index 7397d38..4eec63f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -38,7 +39,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
-import org.apache.hadoop.yarn.server.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
@@ -267,7 +267,7 @@ public class TestRMAppLogAggregationStatus {
 
     // Finally, node1 finished its log aggregation and sent out its final
     // log aggregation status. The log aggregation status for node1 should
-    // be changed from TIME_OUT to Finished
+    // be changed from TIME_OUT to SUCCEEDED
     Map<ApplicationId, LogAggregationReport> node1ReportForApp3 =
         new HashMap<ApplicationId, LogAggregationReport>();
     String messageForNode1_3 =
@@ -275,7 +275,7 @@ public class TestRMAppLogAggregationStatus {
             + System.currentTimeMillis();
     LogAggregationReport report1_3 =
         LogAggregationReport.newInstance(appId, nodeId1,
-          LogAggregationStatus.FINISHED, messageForNode1_3);
+          LogAggregationStatus.SUCCEEDED, messageForNode1_3);
     node1ReportForApp3.put(appId, report1_3);
     node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
@@ -288,7 +288,7 @@ public class TestRMAppLogAggregationStatus {
     for (Entry<NodeId, LogAggregationReport> report : logAggregationStatus
       .entrySet()) {
       if (report.getKey().equals(node1.getNodeID())) {
-        Assert.assertEquals(LogAggregationStatus.FINISHED, report.getValue()
+        Assert.assertEquals(LogAggregationStatus.SUCCEEDED, report.getValue()
           .getLogAggregationStatus());
         Assert.assertEquals(messageForNode1_1 + messageForNode1_2
             + messageForNode1_3, report.getValue().getDiagnosticMessage());
@@ -303,6 +303,104 @@ public class TestRMAppLogAggregationStatus {
     }
   }
 
+  @Test (timeout = 10000)
+  public void testGetLogAggregationStatusForAppReport() {
+    YarnConfiguration conf = new YarnConfiguration();
+
+    // Disable the log aggregation
+    conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false);
+    RMAppImpl rmApp = (RMAppImpl)createRMApp(conf);
+    // The log aggregation status should be DISABLED.
+    Assert.assertEquals(LogAggregationStatus.DISABLED,
+      rmApp.getLogAggregationStatusForAppReport());
+
+    // Enable the log aggregation
+    conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
+    rmApp = (RMAppImpl)createRMApp(conf);
+    // If we do not know any NodeManagers for this application ,
+    // the log aggregation status will return null
+    Assert.assertNull(rmApp.getLogAggregationStatusForAppReport());
+
+    NodeId nodeId1 = NodeId.newInstance("localhost", 1111);
+    NodeId nodeId2 = NodeId.newInstance("localhost", 2222);
+    NodeId nodeId3 = NodeId.newInstance("localhost", 3333);
+    NodeId nodeId4 = NodeId.newInstance("localhost", 4444);
+
+    // If the log aggregation status for all NMs are NOT_START,
+    // the log aggregation status for this app will return NOT_START
+    rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+    Assert.assertEquals(LogAggregationStatus.NOT_START,
+      rmApp.getLogAggregationStatusForAppReport());
+
+    rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.RUNNING, ""));
+    rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.FAILED, ""));
+    Assert.assertEquals(LogAggregationStatus.RUNNING,
+      rmApp.getLogAggregationStatusForAppReport());
+
+    rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL));
+    Assert.assertTrue(RMAppImpl.isAppInFinalState(rmApp));
+
+    // If at least of one log aggregation status for one NM is TIME_OUT,
+    // others are SUCCEEDED, the log aggregation status for this app will
+    // return TIME_OUT
+    rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.TIME_OUT, ""));
+    rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    Assert.assertEquals(LogAggregationStatus.TIME_OUT,
+      rmApp.getLogAggregationStatusForAppReport());
+
+    // If the log aggregation status for all NMs are SUCCEEDED and Application
+    // is at the final state, the log aggregation status for this app will
+    // return SUCCEEDED
+    rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    Assert.assertEquals(LogAggregationStatus.SUCCEEDED,
+      rmApp.getLogAggregationStatusForAppReport());
+
+    rmApp = (RMAppImpl)createRMApp(conf);
+    rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL));
+    Assert.assertTrue(RMAppImpl.isAppInFinalState(rmApp));
+    // If at least of one log aggregation status for one NM is FAILED,
+    // others are either SUCCEEDED or TIME_OUT, and this application is
+    // at the final state, the log aggregation status for this app
+    // will return FAILED
+    rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.TIME_OUT, ""));
+    rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.FAILED, ""));
+    rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+    Assert.assertEquals(LogAggregationStatus.FAILED,
+      rmApp.getLogAggregationStatusForAppReport());
+
+  }
+
   private RMApp createRMApp(Configuration conf) {
     ApplicationSubmissionContext submissionContext =
         ApplicationSubmissionContext.newInstance(appId, "test", "default",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
index cc90ce8..aec5379 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -279,6 +280,11 @@ public class MockRMApp implements RMApp {
   }
 
   @Override
+  public LogAggregationStatus getLogAggregationStatusForAppReport() {
+    return null;
+  }
+
+  @Override
   public String getCollectorAddr() {
     throw new UnsupportedOperationException("Not supported yet.");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23536766/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
index bd43c55..549b9e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
@@ -1307,14 +1307,15 @@ public class TestRMWebServicesApps extends JerseyTestBase {
           WebServicesTestUtils.getXmlInt(element, "preemptedResourceMB"),
           WebServicesTestUtils.getXmlInt(element, "preemptedResourceVCores"),
           WebServicesTestUtils.getXmlInt(element, "numNonAMContainerPreempted"),
-          WebServicesTestUtils.getXmlInt(element, "numAMContainerPreempted"));
+          WebServicesTestUtils.getXmlInt(element, "numAMContainerPreempted"),
+          WebServicesTestUtils.getXmlString(element, "logAggregationStatus"));
     }
   }
 
   public void verifyAppInfo(JSONObject info, RMApp app) throws JSONException,
       Exception {
 
-    assertEquals("incorrect number of elements", 27, info.length());
+    assertEquals("incorrect number of elements", 28, info.length());
 
     verifyAppInfoGeneric(app, info.getString("id"), info.getString("user"),
         info.getString("name"), info.getString("applicationType"),
@@ -1329,7 +1330,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         info.getInt("preemptedResourceMB"),
         info.getInt("preemptedResourceVCores"),
         info.getInt("numNonAMContainerPreempted"),
-        info.getInt("numAMContainerPreempted"));
+        info.getInt("numAMContainerPreempted"),
+        info.getString("logAggregationStatus"));
   }
 
   public void verifyAppInfoGeneric(RMApp app, String id, String user,
@@ -1339,7 +1341,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       long elapsedTime, String amHostHttpAddress, String amContainerLogs,
       int allocatedMB, int allocatedVCores, int numContainers,
       int preemptedResourceMB, int preemptedResourceVCores,
-      int numNonAMContainerPreempted, int numAMContainerPreempted) throws JSONException,
+      int numNonAMContainerPreempted, int numAMContainerPreempted,
+      String logAggregationStatus) throws JSONException,
       Exception {
 
     WebServicesTestUtils.checkStringMatch("id", app.getApplicationId()
@@ -1386,6 +1389,9 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     assertEquals("numAMContainerPreempted doesn't match", app
         .getRMAppMetrics().getNumAMContainersPreempted(),
         numAMContainerPreempted);
+    assertEquals("Log aggregation Status doesn't match", app
+        .getLogAggregationStatusForAppReport().toString(),
+        logAggregationStatus);
   }
 
   @Test


[21/50] [abbrv] hadoop git commit: YARN-3266. RMContext#inactiveNodes should have NodeId as map key. Contributed by Chengbing Liu

Posted by zj...@apache.org.
YARN-3266. RMContext#inactiveNodes should have NodeId as map key. Contributed by Chengbing Liu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/77714c84
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/77714c84
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/77714c84

Branch: refs/heads/YARN-2928
Commit: 77714c849e55da113eac14112121638d451d9f20
Parents: 793bc2f
Author: Jian He <ji...@apache.org>
Authored: Tue Apr 14 10:54:22 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:43 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../resourcemanager/RMActiveServiceContext.java |  6 +--
 .../yarn/server/resourcemanager/RMContext.java  |  2 +-
 .../server/resourcemanager/RMContextImpl.java   |  2 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      | 10 ++--
 .../resourcemanager/webapp/RMWebServices.java   |  2 +-
 .../resourcemanager/TestRMNodeTransitions.java  | 53 ++++++++++++++++++--
 .../resourcemanager/webapp/TestRMWebApp.java    |  6 +--
 .../webapp/TestRMWebServicesNodes.java          |  9 ++--
 9 files changed, 72 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 517b55f..6ca51c9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -251,6 +251,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3472. Fixed possible leak in DelegationTokenRenewer#allTokens.
     (Rohith Sharmaks via jianhe)
 
+    YARN-3266. RMContext#inactiveNodes should have NodeId as map key.
+    (Chengbing Liu via jianhe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index cbb0a8b..1d95204 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -69,8 +69,8 @@ public class RMActiveServiceContext {
   private final ConcurrentMap<NodeId, RMNode> nodes =
       new ConcurrentHashMap<NodeId, RMNode>();
 
-  private final ConcurrentMap<String, RMNode> inactiveNodes =
-      new ConcurrentHashMap<String, RMNode>();
+  private final ConcurrentMap<NodeId, RMNode> inactiveNodes =
+      new ConcurrentHashMap<NodeId, RMNode>();
 
   private final ConcurrentMap<ApplicationId, ByteBuffer> systemCredentials =
       new ConcurrentHashMap<ApplicationId, ByteBuffer>();
@@ -188,7 +188,7 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
-  public ConcurrentMap<String, RMNode> getInactiveRMNodes() {
+  public ConcurrentMap<NodeId, RMNode> getInactiveRMNodes() {
     return this.inactiveNodes;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index b96601c..05fee99 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -62,7 +62,7 @@ public interface RMContext {
   
   ConcurrentMap<ApplicationId, ByteBuffer> getSystemCredentialsForApps();
 
-  ConcurrentMap<String, RMNode> getInactiveRMNodes();
+  ConcurrentMap<NodeId, RMNode> getInactiveRMNodes();
 
   ConcurrentMap<NodeId, RMNode> getRMNodes();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 531d4c5..fb6907c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -150,7 +150,7 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
-  public ConcurrentMap<String, RMNode> getInactiveRMNodes() {
+  public ConcurrentMap<NodeId, RMNode> getInactiveRMNodes() {
     return activeServiceContext.getInactiveRMNodes();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index ace2cf7..3be1867 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -524,11 +524,11 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       RMNodeStartedEvent startEvent = (RMNodeStartedEvent) event;
       List<NMContainerStatus> containers = null;
 
-      String host = rmNode.nodeId.getHost();
-      if (rmNode.context.getInactiveRMNodes().containsKey(host)) {
+      NodeId nodeId = rmNode.nodeId;
+      if (rmNode.context.getInactiveRMNodes().containsKey(nodeId)) {
         // Old node rejoining
-        RMNode previouRMNode = rmNode.context.getInactiveRMNodes().get(host);
-        rmNode.context.getInactiveRMNodes().remove(host);
+        RMNode previouRMNode = rmNode.context.getInactiveRMNodes().get(nodeId);
+        rmNode.context.getInactiveRMNodes().remove(nodeId);
         rmNode.updateMetricsForRejoinedNode(previouRMNode.getState());
       } else {
         // Increment activeNodes explicitly because this is a new node.
@@ -737,7 +737,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       rmNode.context.getRMNodes().remove(rmNode.nodeId);
       LOG.info("Deactivating Node " + rmNode.nodeId + " as it is now "
           + finalState);
-      rmNode.context.getInactiveRMNodes().put(rmNode.nodeId.getHost(), rmNode);
+      rmNode.context.getInactiveRMNodes().put(rmNode.nodeId, rmNode);
 
       //Update the metrics
       rmNode.updateMetricsForDeactivatedNode(initialState, finalState);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 8c63d27..967fbf1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -320,7 +320,7 @@ public class RMWebServices {
     RMNode ni = this.rm.getRMContext().getRMNodes().get(nid);
     boolean isInactive = false;
     if (ni == null) {
-      ni = this.rm.getRMContext().getInactiveRMNodes().get(nid.getHost());
+      ni = this.rm.getRMContext().getInactiveRMNodes().get(nid);
       if (ni == null) {
         throw new NotFoundException("nodeId, " + nodeId + ", is not found");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
index c6da3fd..fb9d2ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
@@ -307,6 +307,47 @@ public class TestRMNodeTransitions {
   }
 
   @Test
+  public void testRunningExpireMultiple() {
+    RMNodeImpl node1 = getRunningNode(null, 10001);
+    RMNodeImpl node2 = getRunningNode(null, 10002);
+    ClusterMetrics cm = ClusterMetrics.getMetrics();
+    int initialActive = cm.getNumActiveNMs();
+    int initialLost = cm.getNumLostNMs();
+    int initialUnhealthy = cm.getUnhealthyNMs();
+    int initialDecommissioned = cm.getNumDecommisionedNMs();
+    int initialRebooted = cm.getNumRebootedNMs();
+    node1.handle(new RMNodeEvent(node1.getNodeID(), RMNodeEventType.EXPIRE));
+    Assert.assertEquals("Active Nodes", initialActive - 1, cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost + 1, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes", initialUnhealthy,
+        cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes", initialDecommissioned,
+        cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes", initialRebooted,
+        cm.getNumRebootedNMs());
+    Assert.assertEquals(NodeState.LOST, node1.getState());
+    Assert.assertTrue("Node " + node1.toString() + " should be inactive",
+        rmContext.getInactiveRMNodes().containsKey(node1.getNodeID()));
+    Assert.assertFalse("Node " + node2.toString() + " should not be inactive",
+        rmContext.getInactiveRMNodes().containsKey(node2.getNodeID()));
+
+    node2.handle(new RMNodeEvent(node1.getNodeID(), RMNodeEventType.EXPIRE));
+    Assert.assertEquals("Active Nodes", initialActive - 2, cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost + 2, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes", initialUnhealthy,
+        cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes", initialDecommissioned,
+        cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes", initialRebooted,
+        cm.getNumRebootedNMs());
+    Assert.assertEquals(NodeState.LOST, node2.getState());
+    Assert.assertTrue("Node " + node1.toString() + " should be inactive",
+        rmContext.getInactiveRMNodes().containsKey(node1.getNodeID()));
+    Assert.assertTrue("Node " + node2.toString() + " should be inactive",
+        rmContext.getInactiveRMNodes().containsKey(node2.getNodeID()));
+  }
+
+  @Test
   public void testUnhealthyExpire() {
     RMNodeImpl node = getUnhealthyNode();
     ClusterMetrics cm = ClusterMetrics.getMetrics();
@@ -458,14 +499,18 @@ public class TestRMNodeTransitions {
   }
 
   private RMNodeImpl getRunningNode() {
-    return getRunningNode(null);
+    return getRunningNode(null, 0);
   }
 
   private RMNodeImpl getRunningNode(String nmVersion) {
-    NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
+    return getRunningNode(nmVersion, 0);
+  }
+
+  private RMNodeImpl getRunningNode(String nmVersion, int port) {
+    NodeId nodeId = BuilderUtils.newNodeId("localhost", port);
     Resource capability = Resource.newInstance(4096, 4);
-    RMNodeImpl node = new RMNodeImpl(nodeId, rmContext,null, 0, 0,
-        null, capability, nmVersion);
+    RMNodeImpl node = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null,
+        capability, nmVersion);
     node.handle(new RMNodeStartedEvent(node.getNodeID(), null, null));
     Assert.assertEquals(NodeState.RUNNING, node.getState());
     return node;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
index 481a53b..0290421 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
@@ -173,10 +173,10 @@ public class TestRMWebApp {
     
     final List<RMNode> deactivatedNodes =
         MockNodes.deactivatedNodes(racks, numNodes, newResource(mbsPerNode));
-    final ConcurrentMap<String, RMNode> deactivatedNodesMap =
+    final ConcurrentMap<NodeId, RMNode> deactivatedNodesMap =
         Maps.newConcurrentMap();
     for (RMNode node : deactivatedNodes) {
-      deactivatedNodesMap.put(node.getHostName(), node);
+      deactivatedNodesMap.put(node.getNodeID(), node);
     }
 
     RMContextImpl rmContext = new RMContextImpl(null, null, null, null,
@@ -186,7 +186,7 @@ public class TestRMWebApp {
          return applicationsMaps;
        }
        @Override
-       public ConcurrentMap<String, RMNode> getInactiveRMNodes() {
+       public ConcurrentMap<NodeId, RMNode> getInactiveRMNodes() {
          return deactivatedNodesMap;
        }
        @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77714c84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
index f507e17..ec20bba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
@@ -32,6 +32,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -263,8 +264,9 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     assertEquals("incorrect number of elements", 2, nodeArray.length());
     for (int i = 0; i < nodeArray.length(); ++i) {
       JSONObject info = nodeArray.getJSONObject(i);
-      String host = info.get("id").toString().split(":")[0];
-      RMNode rmNode = rm.getRMContext().getInactiveRMNodes().get(host);
+      String[] node = info.get("id").toString().split(":");
+      NodeId nodeId = NodeId.newInstance(node[0], Integer.parseInt(node[1]));
+      RMNode rmNode = rm.getRMContext().getInactiveRMNodes().get(nodeId);
       WebServicesTestUtils.checkStringMatch("nodeHTTPAddress", "",
           info.getString("nodeHTTPAddress"));
       WebServicesTestUtils.checkStringMatch("state", rmNode.getState()
@@ -295,7 +297,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
 
     assertEquals("Incorrect Node Information.", "h2:1234", id);
 
-    RMNode rmNode = rm.getRMContext().getInactiveRMNodes().get("h2");
+    NodeId nodeId = NodeId.newInstance("h2", 1234);
+    RMNode rmNode = rm.getRMContext().getInactiveRMNodes().get(nodeId);
     WebServicesTestUtils.checkStringMatch("nodeHTTPAddress", "",
         info.getString("nodeHTTPAddress"));
     WebServicesTestUtils.checkStringMatch("state",


[20/50] [abbrv] hadoop git commit: YARN-3347. Improve YARN log command to get AMContainer logs as well as running containers logs. Contributed by Xuan Gong.

Posted by zj...@apache.org.
YARN-3347. Improve YARN log command to get AMContainer logs as well as running containers logs. Contributed by Xuan Gong.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0b8b4077
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0b8b4077
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0b8b4077

Branch: refs/heads/YARN-2928
Commit: 0b8b40771e6f7b5876f07f1eb6023625f9795baf
Parents: 3d744e3
Author: Junping Du <ju...@apache.org>
Authored: Mon Apr 13 17:25:32 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:42 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  | 484 +++++++++++++++++--
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |  85 +++-
 .../logaggregation/AggregatedLogFormat.java     |  54 +++
 .../yarn/logaggregation/LogCLIHelpers.java      |  57 ++-
 5 files changed, 627 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b8b4077/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8a02ef8..1214dfd 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -125,6 +125,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3348. Add a 'yarn top' tool to help understand cluster usage. (Varun
     Vasudev via jianhe)
 
+    YARN-3347. Improve YARN log command to get AMContainer logs as well as 
+    running containers logs. (Xuan Gong via junping_du)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b8b4077/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index d3b44a7..25481f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -19,6 +19,12 @@
 package org.apache.hadoop.yarn.client.cli;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import javax.ws.rs.core.MediaType;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -27,6 +33,7 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
@@ -35,13 +42,25 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.logaggregation.LogCLIHelpers;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
 
 @Public
 @Evolving
@@ -51,6 +70,8 @@ public class LogsCLI extends Configured implements Tool {
   private static final String APPLICATION_ID_OPTION = "applicationId";
   private static final String NODE_ADDRESS_OPTION = "nodeAddress";
   private static final String APP_OWNER_OPTION = "appOwner";
+  private static final String AM_CONTAINER_OPTION = "am";
+  private static final String CONTAINER_LOG_FILES = "logFiles";
   public static final String HELP_CMD = "help";
 
   @Override
@@ -62,22 +83,47 @@ public class LogsCLI extends Configured implements Tool {
         new Option(APPLICATION_ID_OPTION, true, "ApplicationId (required)");
     appIdOpt.setRequired(true);
     opts.addOption(appIdOpt);
-    opts.addOption(CONTAINER_ID_OPTION, true,
-      "ContainerId (must be specified if node address is specified)");
+    opts.addOption(CONTAINER_ID_OPTION, true, "ContainerId. "
+        + "By default, it will only print syslog if the application is runing."
+        + " Work with -logFiles to get other logs.");
     opts.addOption(NODE_ADDRESS_OPTION, true, "NodeAddress in the format "
-      + "nodename:port (must be specified if container id is specified)");
+      + "nodename:port");
     opts.addOption(APP_OWNER_OPTION, true,
       "AppOwner (assumed to be current user if not specified)");
+    Option amOption = new Option(AM_CONTAINER_OPTION, true, 
+      "Prints the AM Container logs for this application. "
+      + "Specify comma-separated value to get logs for related AM Container. "
+      + "For example, If we specify -am 1,2, we will get the logs for "
+      + "the first AM Container as well as the second AM Container. "
+      + "To get logs for all AM Containers, use -am ALL. "
+      + "To get logs for the latest AM Container, use -am -1. "
+      + "By default, it will only print out syslog. Work with -logFiles "
+      + "to get other logs");
+    amOption.setValueSeparator(',');
+    amOption.setArgs(Option.UNLIMITED_VALUES);
+    amOption.setArgName("AM Containers");
+    opts.addOption(amOption);
+    Option logFileOpt = new Option(CONTAINER_LOG_FILES, true,
+      "Work with -am/-containerId and specify comma-separated value "
+      + "to get specified Container log files");
+    logFileOpt.setValueSeparator(',');
+    logFileOpt.setArgs(Option.UNLIMITED_VALUES);
+    logFileOpt.setArgName("Log File Name");
+    opts.addOption(logFileOpt);
+
     opts.getOption(APPLICATION_ID_OPTION).setArgName("Application ID");
     opts.getOption(CONTAINER_ID_OPTION).setArgName("Container ID");
     opts.getOption(NODE_ADDRESS_OPTION).setArgName("Node Address");
     opts.getOption(APP_OWNER_OPTION).setArgName("Application Owner");
+    opts.getOption(AM_CONTAINER_OPTION).setArgName("AM Containers");
 
     Options printOpts = new Options();
     printOpts.addOption(opts.getOption(HELP_CMD));
     printOpts.addOption(opts.getOption(CONTAINER_ID_OPTION));
     printOpts.addOption(opts.getOption(NODE_ADDRESS_OPTION));
     printOpts.addOption(opts.getOption(APP_OWNER_OPTION));
+    printOpts.addOption(opts.getOption(AM_CONTAINER_OPTION));
+    printOpts.addOption(opts.getOption(CONTAINER_LOG_FILES));
 
     if (args.length < 1) {
       printHelpMessage(printOpts);
@@ -92,12 +138,46 @@ public class LogsCLI extends Configured implements Tool {
     String containerIdStr = null;
     String nodeAddress = null;
     String appOwner = null;
+    boolean getAMContainerLogs = false;
+    String[] logFiles = null;
+    List<String> amContainersList = new ArrayList<String>();
     try {
       CommandLine commandLine = parser.parse(opts, args, true);
       appIdStr = commandLine.getOptionValue(APPLICATION_ID_OPTION);
       containerIdStr = commandLine.getOptionValue(CONTAINER_ID_OPTION);
       nodeAddress = commandLine.getOptionValue(NODE_ADDRESS_OPTION);
       appOwner = commandLine.getOptionValue(APP_OWNER_OPTION);
+      getAMContainerLogs = commandLine.hasOption(AM_CONTAINER_OPTION);
+      if (getAMContainerLogs) {
+        String[] amContainers = commandLine.getOptionValues(AM_CONTAINER_OPTION);
+        for (String am : amContainers) {
+          boolean errorInput = false;
+          if (!am.trim().equalsIgnoreCase("ALL")) {
+            try {
+              int id = Integer.parseInt(am.trim());
+              if (id != -1 && id <= 0) {
+                errorInput = true;
+              }
+            } catch (NumberFormatException ex) {
+              errorInput = true;
+            }
+            if (errorInput) {
+              System.err.println(
+                "Invalid input for option -am. Valid inputs are 'ALL', -1 "
+                + "and any other integer which is larger than 0.");
+              printHelpMessage(printOpts);
+              return -1;
+            }
+            amContainersList.add(am.trim());
+          } else {
+            amContainersList.add("ALL");
+            break;
+          }
+        }
+      }
+      if (commandLine.hasOption(CONTAINER_LOG_FILES)) {
+        logFiles = commandLine.getOptionValues(CONTAINER_LOG_FILES);
+      }
     } catch (ParseException e) {
       System.err.println("options parsing failed: " + e.getMessage());
       printHelpMessage(printOpts);
@@ -118,64 +198,137 @@ public class LogsCLI extends Configured implements Tool {
       return -1;
     }
 
+    LogCLIHelpers logCliHelper = new LogCLIHelpers();
+    logCliHelper.setConf(getConf());
+
+    if (appOwner == null || appOwner.isEmpty()) {
+      appOwner = UserGroupInformation.getCurrentUser().getShortUserName();
+    }
+
+    YarnApplicationState appState = YarnApplicationState.NEW;
     try {
-      int resultCode = verifyApplicationState(appId);
-      if (resultCode != 0) {
+      appState = getApplicationState(appId);
+      if (appState == YarnApplicationState.NEW
+          || appState == YarnApplicationState.NEW_SAVING
+          || appState == YarnApplicationState.SUBMITTED) {
         System.out.println("Logs are not avaiable right now.");
-        return resultCode;
+        return -1;
       }
-    } catch (Exception e) {
+    } catch (IOException | YarnException e) {
       System.err.println("Unable to get ApplicationState."
           + " Attempting to fetch logs directly from the filesystem.");
     }
 
-    LogCLIHelpers logCliHelper = new LogCLIHelpers();
-    logCliHelper.setConf(getConf());
-    
-    if (appOwner == null || appOwner.isEmpty()) {
-      appOwner = UserGroupInformation.getCurrentUser().getShortUserName();
+    // To get am logs
+    if (getAMContainerLogs) {
+      // if we do not specify the value for CONTAINER_LOG_FILES option,
+      // we will only output syslog
+      if (logFiles == null || logFiles.length == 0) {
+        logFiles = new String[] { "syslog" };
+      }
+      // If the application is running, we will call the RM WebService
+      // to get the AppAttempts which includes the nodeHttpAddress
+      // and containerId for all the AM Containers.
+      // After that, we will call NodeManager webService to get the
+      // related logs
+      if (appState == YarnApplicationState.ACCEPTED
+          || appState == YarnApplicationState.RUNNING) {
+        return printAMContainerLogs(getConf(), appIdStr, amContainersList,
+          logFiles, logCliHelper, appOwner, false);
+      } else {
+        // If the application is in the final state, we will call RM webservice
+        // to get all AppAttempts information first. If we get nothing,
+        // we will try to call AHS webservice to get related AppAttempts
+        // which includes nodeAddress for the AM Containers.
+        // After that, we will use nodeAddress and containerId
+        // to get logs from HDFS directly.
+        if (getConf().getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
+          YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
+          return printAMContainerLogs(getConf(), appIdStr, amContainersList,
+            logFiles, logCliHelper, appOwner, true);
+        } else {
+          System.out
+            .println("Can not get AMContainers logs for the application:"
+                + appId);
+          System.out.println("This application:" + appId + " is finished."
+              + " Please enable the application history service. Or Using "
+              + "yarn logs -applicationId <appId> -containerId <containerId> "
+              + "--nodeAddress <nodeHttpAddress> to get the container logs");
+          return -1;
+        }
+      }
     }
+
     int resultCode = 0;
-    if (containerIdStr == null && nodeAddress == null) {
-      resultCode = logCliHelper.dumpAllContainersLogs(appId, appOwner, System.out);
-    } else if ((containerIdStr == null && nodeAddress != null)
-        || (containerIdStr != null && nodeAddress == null)) {
-      System.out.println("ContainerId or NodeAddress cannot be null!");
-      printHelpMessage(printOpts);
-      resultCode = -1;
+    if (containerIdStr != null) {
+      // if we provide the node address and the application is in the final
+      // state, we could directly get logs from HDFS.
+      if (nodeAddress != null && isApplicationFinished(appState)) {
+        return logCliHelper.dumpAContainersLogsForALogType(appIdStr,
+            containerIdStr, nodeAddress, appOwner, logFiles == null ? null
+                : Arrays.asList(logFiles));
+      }
+      try {
+        // If the nodeAddress is not provided, we will try to get
+        // the ContainerReport. In the containerReport, we could get
+        // nodeAddress and nodeHttpAddress
+        ContainerReport report = getContainerReport(containerIdStr);
+        String nodeHttpAddress =
+            report.getNodeHttpAddress().replaceFirst(
+              WebAppUtils.getHttpSchemePrefix(getConf()), "");
+        String nodeId = report.getAssignedNode().toString();
+        // If the application is not in the final state,
+        // we will provide the NodeHttpAddress and get the container logs
+        // by calling NodeManager webservice.
+        if (!isApplicationFinished(appState)) {
+          if (logFiles == null || logFiles.length == 0) {
+            logFiles = new String[] { "syslog" };
+          }
+          printContainerLogsFromRunningApplication(getConf(), appIdStr,
+            containerIdStr, nodeHttpAddress, nodeId, logFiles, logCliHelper,
+            appOwner);
+        } else {
+          // If the application is in the final state, we will directly
+          // get the container logs from HDFS.
+          printContainerLogsForFinishedApplication(appIdStr, containerIdStr,
+            nodeId, logFiles, logCliHelper, appOwner);
+        }
+        return resultCode;
+      } catch (IOException | YarnException ex) {
+        System.err.println("Unable to get logs for this container:"
+            + containerIdStr + "for the application:" + appId);
+        if (!getConf().getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
+          YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
+          System.out.println("Please enable the application history service. Or ");
+        }
+        System.out.println("Using "
+            + "yarn logs -applicationId <appId> -containerId <containerId> "
+            + "--nodeAddress <nodeHttpAddress> to get the container logs");
+        return -1;
+      }
     } else {
-      resultCode =
-          logCliHelper.dumpAContainersLogs(appIdStr, containerIdStr,
-            nodeAddress, appOwner);
+      if (nodeAddress == null) {
+        resultCode =
+            logCliHelper.dumpAllContainersLogs(appId, appOwner, System.out);
+      } else {
+        System.out.println("Should at least provide ContainerId!");
+        printHelpMessage(printOpts);
+        resultCode = -1;
+      }
     }
-
     return resultCode;
   }
 
-  private int verifyApplicationState(ApplicationId appId) throws IOException,
-      YarnException {
+  private YarnApplicationState getApplicationState(ApplicationId appId)
+      throws IOException, YarnException {
     YarnClient yarnClient = createYarnClient();
 
     try {
       ApplicationReport appReport = yarnClient.getApplicationReport(appId);
-      switch (appReport.getYarnApplicationState()) {
-      case NEW:
-      case NEW_SAVING:
-      case SUBMITTED:
-        return -1;
-      case ACCEPTED:
-      case RUNNING:
-      case FAILED:
-      case FINISHED:
-      case KILLED:
-      default:
-        break;
-
-      }
+      return appReport.getYarnApplicationState();
     } finally {
       yarnClient.close();
     }
-    return 0;
   }
   
   @VisibleForTesting
@@ -201,4 +354,255 @@ public class LogsCLI extends Configured implements Tool {
     formatter.setSyntaxPrefix("");
     formatter.printHelp("general options are:", options);
   }
+
+  private List<JSONObject> getAMContainerInfoForRMWebService(
+      Configuration conf, String appId) throws ClientHandlerException,
+      UniformInterfaceException, JSONException {
+    Client webServiceClient = Client.create();
+    String webAppAddress =
+        WebAppUtils.getWebAppBindURL(conf, YarnConfiguration.RM_BIND_HOST,
+          WebAppUtils.getRMWebAppURLWithScheme(conf));
+    WebResource webResource = webServiceClient.resource(webAppAddress);
+
+    ClientResponse response =
+        webResource.path("ws").path("v1").path("cluster").path("apps")
+          .path(appId).path("appattempts").accept(MediaType.APPLICATION_JSON)
+          .get(ClientResponse.class);
+    JSONObject json =
+        response.getEntity(JSONObject.class).getJSONObject("appAttempts");
+    JSONArray requests = json.getJSONArray("appAttempt");
+    List<JSONObject> amContainersList = new ArrayList<JSONObject>();
+    for (int i = 0; i < requests.length(); i++) {
+      amContainersList.add(requests.getJSONObject(i));
+    }
+    return amContainersList;
+  }
+
+  private List<JSONObject> getAMContainerInfoForAHSWebService(Configuration conf,
+      String appId) throws ClientHandlerException, UniformInterfaceException,
+      JSONException {
+    Client webServiceClient = Client.create();
+    String webAppAddress =
+        WebAppUtils.getHttpSchemePrefix(conf)
+            + WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
+    WebResource webResource = webServiceClient.resource(webAppAddress);
+
+    ClientResponse response =
+        webResource.path("ws").path("v1").path("applicationhistory").path("apps")
+          .path(appId).path("appattempts").accept(MediaType.APPLICATION_JSON)
+          .get(ClientResponse.class);
+    JSONObject json = response.getEntity(JSONObject.class);
+    JSONArray requests = json.getJSONArray("appAttempt");
+    List<JSONObject> amContainersList = new ArrayList<JSONObject>();
+    for (int i = 0; i < requests.length(); i++) {
+      amContainersList.add(requests.getJSONObject(i));
+    }
+    Collections.reverse(amContainersList);
+    return amContainersList;
+  }
+
+  private void printContainerLogsFromRunningApplication(Configuration conf,
+      String appId, String containerIdStr, String nodeHttpAddress,
+      String nodeId, String[] logFiles, LogCLIHelpers logCliHelper,
+      String appOwner) throws IOException {
+    Client webServiceClient = Client.create();
+    String containerString = "\n\nContainer: " + containerIdStr;
+    System.out.println(containerString);
+    System.out.println(StringUtils.repeat("=", containerString.length()));
+    for (String logFile : logFiles) {
+      System.out.println("LogType:" + logFile);
+      System.out.println("Log Upload Time:"
+          + Times.format(System.currentTimeMillis()));
+      System.out.println("Log Contents:");
+      try {
+        WebResource webResource =
+            webServiceClient.resource(WebAppUtils.getHttpSchemePrefix(conf)
+                + nodeHttpAddress);
+        ClientResponse response =
+            webResource.path("ws").path("v1").path("node")
+              .path("containerlogs").path(containerIdStr).path(logFile)
+              .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
+        System.out.println(response.getEntity(String.class));
+        System.out.println("End of LogType:" + logFile);
+      } catch (ClientHandlerException | UniformInterfaceException ex) {
+        System.out.println("Can not find the log file:" + logFile
+            + " for the container:" + containerIdStr + " in NodeManager:"
+            + nodeId);
+      }
+    }
+    // for the case, we have already uploaded partial logs in HDFS
+    logCliHelper.dumpAContainersLogsForALogType(appId, containerIdStr, nodeId,
+      appOwner, Arrays.asList(logFiles));
+  }
+
+  private void printContainerLogsForFinishedApplication(String appId,
+      String containerId, String nodeAddress, String[] logFiles,
+      LogCLIHelpers logCliHelper, String appOwner) throws IOException {
+    String containerString = "\n\nContainer: " + containerId;
+    System.out.println(containerString);
+    System.out.println(StringUtils.repeat("=", containerString.length()));
+    logCliHelper.dumpAContainersLogsForALogType(appId, containerId,
+      nodeAddress, appOwner, logFiles != null ? Arrays.asList(logFiles) : null);
+  }
+
+  private ContainerReport getContainerReport(String containerIdStr)
+      throws YarnException, IOException {
+    YarnClient yarnClient = createYarnClient();
+    try {
+      return yarnClient.getContainerReport(ConverterUtils
+        .toContainerId(containerIdStr));
+    } finally {
+      yarnClient.close();
+    }
+  }
+
+  private boolean isApplicationFinished(YarnApplicationState appState) {
+    return appState == YarnApplicationState.FINISHED
+        || appState == YarnApplicationState.FAILED
+        || appState == YarnApplicationState.KILLED; 
+  }
+
+  private int printAMContainerLogs(Configuration conf, String appId,
+      List<String> amContainers, String[] logFiles, LogCLIHelpers logCliHelper,
+      String appOwner, boolean applicationFinished) throws Exception {
+    List<JSONObject> amContainersList = null;
+    List<AMLogsRequest> requests = new ArrayList<AMLogsRequest>();
+    boolean getAMContainerLists = false;
+    String errorMessage = "";
+    try {
+      amContainersList = getAMContainerInfoForRMWebService(conf, appId);
+      if (amContainersList != null && !amContainersList.isEmpty()) {
+        getAMContainerLists = true;
+        for (JSONObject amContainer : amContainersList) {
+          AMLogsRequest request = new AMLogsRequest(applicationFinished);
+          request.setAmContainerId(amContainer.getString("containerId"));
+          request.setNodeHttpAddress(amContainer.getString("nodeHttpAddress"));
+          request.setNodeId(amContainer.getString("nodeId"));
+          requests.add(request);
+        }
+      }
+    } catch (Exception ex) {
+      errorMessage = ex.getMessage();
+      if (applicationFinished) {
+        try {
+          amContainersList = getAMContainerInfoForAHSWebService(conf, appId);
+          if (amContainersList != null && !amContainersList.isEmpty()) {
+            getAMContainerLists = true;
+            for (JSONObject amContainer : amContainersList) {
+              AMLogsRequest request = new AMLogsRequest(applicationFinished);
+              request.setAmContainerId(amContainer.getString("amContainerId"));
+              requests.add(request);
+            }
+          }
+        } catch (Exception e) {
+          errorMessage = e.getMessage();
+        }
+      }
+    }
+
+    if (!getAMContainerLists) {
+      System.err.println("Unable to get AM container informations "
+          + "for the application:" + appId);
+      System.err.println(errorMessage);
+      return -1;
+    }
+
+    if (amContainers.contains("ALL")) {
+      for (AMLogsRequest request : requests) {
+        outputAMContainerLogs(request, conf, appId, logFiles, logCliHelper,
+          appOwner);
+      }
+      System.out.println();      
+      System.out.println("Specified ALL for -am option. "
+          + "Printed logs for all am containers.");
+    } else {
+      for (String amContainer : amContainers) {
+        int amContainerId = Integer.parseInt(amContainer.trim());
+        if (amContainerId == -1) {
+          outputAMContainerLogs(requests.get(requests.size() - 1), conf, appId,
+            logFiles, logCliHelper, appOwner);
+        } else {
+          if (amContainerId <= requests.size()) {
+            outputAMContainerLogs(requests.get(amContainerId - 1), conf, appId,
+              logFiles, logCliHelper, appOwner);
+          }
+        }
+      }
+    }
+    return 0;
+  }
+
+  private void outputAMContainerLogs(AMLogsRequest request, Configuration conf,
+      String appId, String[] logFiles, LogCLIHelpers logCliHelper,
+      String appOwner) throws Exception {
+    String nodeHttpAddress = request.getNodeHttpAddress();
+    String containerId = request.getAmContainerId();
+    String nodeId = request.getNodeId();
+
+    if (request.isAppFinished()) {
+      if (containerId != null && !containerId.isEmpty()) {
+        if (nodeId == null || nodeId.isEmpty()) {
+          try {
+            nodeId =
+                getContainerReport(containerId).getAssignedNode().toString();
+          } catch (Exception ex) {
+            System.err.println(ex);
+            nodeId = null;
+          }
+        }
+        if (nodeId != null && !nodeId.isEmpty()) {
+          printContainerLogsForFinishedApplication(appId, containerId, nodeId,
+            logFiles, logCliHelper, appOwner);
+        }
+      }
+    } else {
+      if (nodeHttpAddress != null && containerId != null
+          && !nodeHttpAddress.isEmpty() && !containerId.isEmpty()) {
+        printContainerLogsFromRunningApplication(conf, appId, containerId,
+          nodeHttpAddress, nodeId, logFiles, logCliHelper, appOwner);
+      }
+    }
+  }
+
+  private static class AMLogsRequest {
+    private String amContainerId;
+    private String nodeId;
+    private String nodeHttpAddress;
+    private final boolean isAppFinished;
+
+    AMLogsRequest(boolean isAppFinished) {
+      this.isAppFinished = isAppFinished;
+      this.setAmContainerId("");
+      this.setNodeId("");
+      this.setNodeHttpAddress("");
+    }
+
+    public String getAmContainerId() {
+      return amContainerId;
+    }
+
+    public void setAmContainerId(String amContainerId) {
+      this.amContainerId = amContainerId;
+    }
+
+    public String getNodeId() {
+      return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+      this.nodeId = nodeId;
+    }
+
+    public String getNodeHttpAddress() {
+      return nodeHttpAddress;
+    }
+
+    public void setNodeHttpAddress(String nodeHttpAddress) {
+      this.nodeHttpAddress = nodeHttpAddress;
+    }
+
+    public boolean isAppFinished() {
+      return isAppFinished;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b8b4077/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index 7ee918e..7d20cf2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -32,6 +32,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.Writer;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -145,14 +146,28 @@ public class TestLogsCLI {
     pw.println("usage: yarn logs -applicationId <application ID> [OPTIONS]");
     pw.println();
     pw.println("general options are:");
+    pw.println(" -am <AM Containers>             Prints the AM Container logs for this");
+    pw.println("                                 application. Specify comma-separated");
+    pw.println("                                 value to get logs for related AM");
+    pw.println("                                 Container. For example, If we specify -am");
+    pw.println("                                 1,2, we will get the logs for the first");
+    pw.println("                                 AM Container as well as the second AM");
+    pw.println("                                 Container. To get logs for all AM");
+    pw.println("                                 Containers, use -am ALL. To get logs for");
+    pw.println("                                 the latest AM Container, use -am -1. By");
+    pw.println("                                 default, it will only print out syslog.");
+    pw.println("                                 Work with -logFiles to get other logs");
     pw.println(" -appOwner <Application Owner>   AppOwner (assumed to be current user if");
     pw.println("                                 not specified)");
-    pw.println(" -containerId <Container ID>     ContainerId (must be specified if node");
-    pw.println("                                 address is specified)");
+    pw.println(" -containerId <Container ID>     ContainerId. By default, it will only");
+    pw.println("                                 print syslog if the application is");
+    pw.println("                                 runing. Work with -logFiles to get other");
+    pw.println("                                 logs.");
     pw.println(" -help                           Displays help for all commands.");
+    pw.println(" -logFiles <Log File Name>       Work with -am/-containerId and specify");
+    pw.println("                                 comma-separated value to get specified");
+    pw.println("                                 Container log files");
     pw.println(" -nodeAddress <Node Address>     NodeAddress in the format nodename:port");
-    pw.println("                                 (must be specified if container id is");
-    pw.println("                                 specified)");
     pw.close();
     String appReportStr = baos.toString("UTF-8");
     Assert.assertEquals(appReportStr, sysOutStream.toString());
@@ -176,6 +191,7 @@ public class TestLogsCLI {
     ContainerId containerId0 = ContainerIdPBImpl.newContainerId(appAttemptId, 0);
     ContainerId containerId1 = ContainerIdPBImpl.newContainerId(appAttemptId, 1);
     ContainerId containerId2 = ContainerIdPBImpl.newContainerId(appAttemptId, 2);
+    ContainerId containerId3 = ContainerIdPBImpl.newContainerId(appAttemptId, 3);
     NodeId nodeId = NodeId.newInstance("localhost", 1234);
 
     // create local logs
@@ -193,9 +209,15 @@ public class TestLogsCLI {
     assertTrue(fs.mkdirs(appLogsDir));
     List<String> rootLogDirs = Arrays.asList(rootLogDir);
 
+    List<String> logTypes = new ArrayList<String>();
+    logTypes.add("syslog");
     // create container logs in localLogDir
-    createContainerLogInLocalDir(appLogsDir, containerId1, fs);
-    createContainerLogInLocalDir(appLogsDir, containerId2, fs);
+    createContainerLogInLocalDir(appLogsDir, containerId1, fs, logTypes);
+    createContainerLogInLocalDir(appLogsDir, containerId2, fs, logTypes);
+
+    // create two logs for container3 in localLogDir
+    logTypes.add("stdout");
+    createContainerLogInLocalDir(appLogsDir, containerId3, fs, logTypes);
 
     Path path =
         new Path(remoteLogRootDir + ugi.getShortUserName()
@@ -217,6 +239,8 @@ public class TestLogsCLI {
       containerId1, path, fs);
     uploadContainerLogIntoRemoteDir(ugi, configuration, rootLogDirs, nodeId,
       containerId2, path, fs);
+    uploadContainerLogIntoRemoteDir(ugi, configuration, rootLogDirs, nodeId,
+      containerId3, path, fs);
 
     YarnClient mockYarnClient =
         createMockYarnClient(YarnApplicationState.FINISHED);
@@ -226,9 +250,13 @@ public class TestLogsCLI {
     int exitCode = cli.run(new String[] { "-applicationId", appId.toString() });
     assertTrue(exitCode == 0);
     assertTrue(sysOutStream.toString().contains(
-      "Hello container_0_0001_01_000001!"));
+      "Hello container_0_0001_01_000001 in syslog!"));
+    assertTrue(sysOutStream.toString().contains(
+      "Hello container_0_0001_01_000002 in syslog!"));
     assertTrue(sysOutStream.toString().contains(
-      "Hello container_0_0001_01_000002!"));
+      "Hello container_0_0001_01_000003 in syslog!"));
+    assertTrue(sysOutStream.toString().contains(
+      "Hello container_0_0001_01_000003 in stdout!"));
     sysOutStream.reset();
 
     // uploaded two logs for container1. The first log is empty.
@@ -240,7 +268,7 @@ public class TestLogsCLI {
             containerId1.toString() });
     assertTrue(exitCode == 0);
     assertTrue(sysOutStream.toString().contains(
-        "Hello container_0_0001_01_000001!"));
+        "Hello container_0_0001_01_000001 in syslog!"));
     assertTrue(sysOutStream.toString().contains("Log Upload Time"));
     assertTrue(!sysOutStream.toString().contains(
       "Logs for container " + containerId1.toString()
@@ -258,22 +286,51 @@ public class TestLogsCLI {
     assertTrue(sysOutStream.toString().contains(
       "Logs for container " + containerId0.toString()
           + " are not present in this log-file."));
+    sysOutStream.reset();
+
+    // uploaded two logs for container3. The first log is named as syslog.
+    // The second one is named as stdout.
+    exitCode =
+        cli.run(new String[] { "-applicationId", appId.toString(),
+            "-nodeAddress", nodeId.toString(), "-containerId",
+            containerId3.toString() });
+    assertTrue(exitCode == 0);
+    assertTrue(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000003 in syslog!"));
+    assertTrue(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000003 in stdout!"));
+    sysOutStream.reset();
+
+    // set -logFiles option as stdout
+    // should only print log with the name as stdout
+    exitCode =
+        cli.run(new String[] { "-applicationId", appId.toString(),
+            "-nodeAddress", nodeId.toString(), "-containerId",
+            containerId3.toString() , "-logFiles", "stdout"});
+    assertTrue(exitCode == 0);
+    assertTrue(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000003 in stdout!"));
+    assertTrue(!sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000003 in syslog!"));
+    sysOutStream.reset();
 
     fs.delete(new Path(remoteLogRootDir), true);
     fs.delete(new Path(rootLogDir), true);
   }
 
   private static void createContainerLogInLocalDir(Path appLogsDir,
-      ContainerId containerId, FileSystem fs) throws Exception {
+      ContainerId containerId, FileSystem fs, List<String> logTypes) throws Exception {
     Path containerLogsDir = new Path(appLogsDir, containerId.toString());
     if (fs.exists(containerLogsDir)) {
       fs.delete(containerLogsDir, true);
     }
     assertTrue(fs.mkdirs(containerLogsDir));
-    Writer writer =
-        new FileWriter(new File(containerLogsDir.toString(), "sysout"));
-    writer.write("Hello " + containerId + "!");
-    writer.close();
+    for (String logType : logTypes) {
+      Writer writer =
+          new FileWriter(new File(containerLogsDir.toString(), logType));
+      writer.write("Hello " + containerId + " in " + logType + "!");
+      writer.close();
+    }
   }
 
   private static void uploadContainerLogIntoRemoteDir(UserGroupInformation ugi,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b8b4077/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 57f655b..debe770 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -745,6 +745,60 @@ public class AggregatedLogFormat {
       readAContainerLogsForALogType(valueStream, out, -1);
     }
 
+    /**
+     * Keep calling this till you get a {@link EOFException} for getting logs of
+     * the specific types for a single container.
+     * @param valueStream
+     * @param out
+     * @param logUploadedTime
+     * @param logType
+     * @throws IOException
+     */
+    public static int readContainerLogsForALogType(
+        DataInputStream valueStream, PrintStream out, long logUploadedTime,
+        List<String> logType) throws IOException {
+      byte[] buf = new byte[65535];
+
+      String fileType = valueStream.readUTF();
+      String fileLengthStr = valueStream.readUTF();
+      long fileLength = Long.parseLong(fileLengthStr);
+      if (logType.contains(fileType)) {
+        out.print("LogType:");
+        out.println(fileType);
+        if (logUploadedTime != -1) {
+          out.print("Log Upload Time:");
+          out.println(Times.format(logUploadedTime));
+        }
+        out.print("LogLength:");
+        out.println(fileLengthStr);
+        out.println("Log Contents:");
+
+        long curRead = 0;
+        long pendingRead = fileLength - curRead;
+        int toRead = pendingRead > buf.length ? buf.length : (int) pendingRead;
+        int len = valueStream.read(buf, 0, toRead);
+        while (len != -1 && curRead < fileLength) {
+          out.write(buf, 0, len);
+          curRead += len;
+
+          pendingRead = fileLength - curRead;
+          toRead = pendingRead > buf.length ? buf.length : (int) pendingRead;
+          len = valueStream.read(buf, 0, toRead);
+        }
+        out.println("End of LogType:" + fileType);
+        out.println("");
+        return 0;
+      } else {
+        long totalSkipped = 0;
+        long currSkipped = 0;
+        while (currSkipped != -1 && totalSkipped < fileLength) {
+          currSkipped = valueStream.skip(fileLength - totalSkipped);
+          totalSkipped += currSkipped;
+        }
+        return -1;
+      }
+    }
+
     public void close() {
       IOUtils.cleanup(LOG, scanner, reader, fsDataIStream);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b8b4077/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
index df9bd32..39fd95e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
@@ -23,6 +23,7 @@ import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -48,6 +49,14 @@ public class LogCLIHelpers implements Configurable {
   @VisibleForTesting
   public int dumpAContainersLogs(String appId, String containerId,
       String nodeId, String jobOwner) throws IOException {
+    return dumpAContainersLogsForALogType(appId, containerId, nodeId, jobOwner,
+      null);
+  }
+
+  @Private
+  @VisibleForTesting
+  public int dumpAContainersLogsForALogType(String appId, String containerId,
+      String nodeId, String jobOwner, List<String> logType) throws IOException {
     Path remoteRootLogDir = new Path(getConf().get(
         YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
         YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR));
@@ -78,9 +87,16 @@ public class LogCLIHelpers implements Configurable {
           reader =
               new AggregatedLogFormat.LogReader(getConf(),
                 thisNodeFile.getPath());
-          if (dumpAContainerLogs(containerId, reader, System.out,
+          if (logType == null) {
+            if (dumpAContainerLogs(containerId, reader, System.out,
               thisNodeFile.getModificationTime()) > -1) {
-            foundContainerLogs = true;
+              foundContainerLogs = true;
+            }
+          } else {
+            if (dumpAContainerLogsForALogType(containerId, reader, System.out,
+              thisNodeFile.getModificationTime(), logType) > -1) {
+              foundContainerLogs = true;
+            }
           }
         } finally {
           if (reader != null) {
@@ -131,6 +147,43 @@ public class LogCLIHelpers implements Configurable {
   }
 
   @Private
+  public int dumpAContainerLogsForALogType(String containerIdStr,
+      AggregatedLogFormat.LogReader reader, PrintStream out,
+      long logUploadedTime, List<String> logType) throws IOException {
+    DataInputStream valueStream;
+    LogKey key = new LogKey();
+    valueStream = reader.next(key);
+
+    while (valueStream != null && !key.toString().equals(containerIdStr)) {
+      // Next container
+      key = new LogKey();
+      valueStream = reader.next(key);
+    }
+
+    if (valueStream == null) {
+      return -1;
+    }
+
+    boolean foundContainerLogs = false;
+    while (true) {
+      try {
+        int result = LogReader.readContainerLogsForALogType(
+            valueStream, out, logUploadedTime, logType);
+        if (result == 0) {
+          foundContainerLogs = true;
+        }
+      } catch (EOFException eof) {
+        break;
+      }
+    }
+
+    if (foundContainerLogs) {
+      return 0;
+    }
+    return -1;
+  }
+
+  @Private
   public int dumpAllContainersLogs(ApplicationId appId, String appOwner,
       PrintStream out) throws IOException {
     Path remoteRootLogDir = new Path(getConf().get(


[11/50] [abbrv] hadoop git commit: YARN-3472. Fixed possible leak in DelegationTokenRenewer#allTokens. Contributed by Rohith Sharmaks

Posted by zj...@apache.org.
YARN-3472. Fixed possible leak in DelegationTokenRenewer#allTokens. Contributed by Rohith Sharmaks


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b5ca2773
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b5ca2773
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b5ca2773

Branch: refs/heads/YARN-2928
Commit: b5ca2773462e71cff9b3e4371c7ecf2613b95bc0
Parents: 3c9cf4c
Author: Jian He <ji...@apache.org>
Authored: Mon Apr 13 14:07:17 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:41 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                          |  3 +++
 .../resourcemanager/security/DelegationTokenRenewer.java |  1 +
 .../security/TestDelegationTokenRenewer.java             | 11 ++++++++++-
 3 files changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5ca2773/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 044a7a9..88d0a18 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -239,6 +239,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3382. Some of UserMetricsInfo metrics are incorrectly set to root
     queue metrics. (Rohit Agarwal via jianhe)
 
+    YARN-3472. Fixed possible leak in DelegationTokenRenewer#allTokens.
+    (Rohith Sharmaks via jianhe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5ca2773/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index d49ecfc..e307645 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -577,6 +577,7 @@ public class DelegationTokenRenewer extends AbstractService {
             DelegationTokenToRenew t = iter.next();
             if (t.token.getKind().equals(new Text("HDFS_DELEGATION_TOKEN"))) {
               iter.remove();
+              allTokens.remove(t.token);
               t.cancelTimer();
               LOG.info("Removed expiring token " + t);
             }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5ca2773/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
index bc9c295..f2c0a30 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
@@ -906,7 +906,16 @@ public class TestDelegationTokenRenewer {
           new HashMap<ApplicationAccessType, String>(), false, "default", 1,
           credentials);
 
-    // wait for the initial expiring hdfs token to be removed.
+    // wait for the initial expiring hdfs token to be removed from allTokens
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      public Boolean get() {
+        return
+            rm.getRMContext().getDelegationTokenRenewer().getAllTokens()
+            .get(token1) == null;
+      }
+    }, 1000, 20000);
+
+    // wait for the initial expiring hdfs token to be removed from appTokens
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       public Boolean get() {
         return !rm.getRMContext().getDelegationTokenRenewer()


[35/50] [abbrv] hadoop git commit: YARN-3326. Support RESTful API for getLabelsToNodes. Contributed by Naganarasimha G R.

Posted by zj...@apache.org.
YARN-3326. Support RESTful API for getLabelsToNodes. Contributed by Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fb794639
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fb794639
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fb794639

Branch: refs/heads/YARN-2928
Commit: fb7946397be69e845529f52ea49d7db0137a5491
Parents: 89ed0e7
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Apr 15 14:03:55 2015 -0500
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:45 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../resourcemanager/webapp/NodeIDsInfo.java     | 47 +++++++++++++++
 .../resourcemanager/webapp/RMWebServices.java   | 61 +++++++++++++++-----
 .../webapp/dao/LabelsToNodesInfo.java           | 43 ++++++++++++++
 .../webapp/TestRMWebServicesNodeLabels.java     | 55 ++++++++++++++++++
 5 files changed, 194 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb794639/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e4755f6..565627f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -137,6 +137,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3318. Create Initial OrderingPolicy Framework and FifoOrderingPolicy.
     (Craig Welch via wangda)
 
+    YARN-3326. Support RESTful API for getLabelsToNodes. (Naganarasimha G R
+    via ozawa)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb794639/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java
new file mode 100644
index 0000000..39d636d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "labelsToNodesInfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class NodeIDsInfo {
+
+  /**
+   * Set doesn't support default no arg constructor which is req by JAXB
+   */
+  protected ArrayList<String> nodeIDsList = new ArrayList<String>();
+
+  public NodeIDsInfo() {
+  } // JAXB needs this
+
+  public NodeIDsInfo(List<String> nodeIdsList) {
+    this.nodeIDsList.addAll(nodeIdsList);
+  }
+
+  public ArrayList<String> getNodeIDs() {
+    return nodeIDsList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb794639/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 967fbf1..fae0806 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -20,10 +20,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
-import java.security.AccessControlException;
 import java.nio.ByteBuffer;
+import java.security.AccessControlException;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
@@ -31,6 +32,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
@@ -64,25 +66,25 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
 import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -116,12 +118,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedule
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
@@ -130,15 +131,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CredentialsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.DelegationToken;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FifoSchedulerInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.AdHocLogDumper;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -781,7 +784,35 @@ public class RMWebServices {
 
     return ntl;
   }
-  
+
+  @GET
+  @Path("/label-mappings")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public LabelsToNodesInfo getLabelsToNodes(
+      @QueryParam("labels") Set<String> labels) throws IOException {
+    init();
+
+    LabelsToNodesInfo lts = new LabelsToNodesInfo();
+    Map<String, NodeIDsInfo> ltsMap = lts.getLabelsToNodes();
+    Map<String, Set<NodeId>> labelsToNodeId = null;
+    if (labels == null || labels.size() == 0) {
+      labelsToNodeId =
+          rm.getRMContext().getNodeLabelManager().getLabelsToNodes();
+    } else {
+      labelsToNodeId =
+          rm.getRMContext().getNodeLabelManager().getLabelsToNodes(labels);
+    }
+
+    for (Entry<String, Set<NodeId>> entry : labelsToNodeId.entrySet()) {
+      List<String> nodeIdStrList = new ArrayList<String>();
+      for (NodeId nodeId : entry.getValue()) {
+        nodeIdStrList.add(nodeId.toString());
+      }
+      ltsMap.put(entry.getKey(), new NodeIDsInfo(nodeIdStrList));
+    }
+    return lts;
+  }
+
   @POST
   @Path("/replace-node-to-labels")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb794639/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java
new file mode 100644
index 0000000..625fedd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.NodeIDsInfo;
+
+@XmlRootElement(name = "labelsToNodesInfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class LabelsToNodesInfo {
+
+  protected Map<String, NodeIDsInfo> labelsToNodes =
+    new HashMap<String, NodeIDsInfo>();
+
+  public LabelsToNodesInfo() {
+  } // JAXB needs this
+
+  public Map<String, NodeIDsInfo> getLabelsToNodes() {
+   return labelsToNodes;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb794639/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
index 0cc576c..40c54a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
@@ -51,6 +52,7 @@ import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.json.JSONJAXBContext;
 import com.sun.jersey.api.json.JSONMarshaller;
 import com.sun.jersey.api.json.JSONUnmarshaller;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
 import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.test.framework.WebAppDescriptor;
 
@@ -160,6 +162,59 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
 
+    // Add labels to another node
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nid1:0")
+            .path("replace-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\": [\"b\"]}",
+              MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    LOG.info("posted node nodelabel");
+
+    // Add labels to another node
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nid2:0")
+            .path("replace-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\": [\"b\"]}",
+              MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    LOG.info("posted node nodelabel");
+
+    // Verify, using get-labels-to-Nodes
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("label-mappings").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    LabelsToNodesInfo ltni = response.getEntity(LabelsToNodesInfo.class);
+    assertEquals(2, ltni.getLabelsToNodes().size());
+    NodeIDsInfo nodes = ltni.getLabelsToNodes().get("b");
+    assertTrue(nodes.getNodeIDs().contains("nid2:0"));
+    assertTrue(nodes.getNodeIDs().contains("nid1:0"));
+    nodes = ltni.getLabelsToNodes().get("a");
+    assertTrue(nodes.getNodeIDs().contains("nid:0"));
+
+    // Verify, using get-labels-to-Nodes for specifiedset of labels
+    MultivaluedMapImpl params = new MultivaluedMapImpl();
+    params.add("labels", "a");
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("label-mappings").queryParam("user.name", userName)
+            .queryParams(params)
+            .accept(MediaType.APPLICATION_JSON)
+            .get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    ltni = response.getEntity(LabelsToNodesInfo.class);
+    assertEquals(1, ltni.getLabelsToNodes().size());
+    nodes = ltni.getLabelsToNodes().get("a");
+    assertTrue(nodes.getNodeIDs().contains("nid:0"));
+
     // Verify
     response =
         r.path("ws").path("v1").path("cluster")


[08/50] [abbrv] hadoop git commit: Revert "MAPREDUCE-6300. Task list sort by task id broken. Contributed by Siqi Li."

Posted by zj...@apache.org.
Revert "MAPREDUCE-6300. Task list sort by task id broken. Contributed by Siqi Li."

This reverts commit 6ac98b48042eca43811cdad4e0da820f719ddc23.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/72df8153
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/72df8153
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/72df8153

Branch: refs/heads/YARN-2928
Commit: 72df815359b513bdaddb902c9e83720888aa3f2d
Parents: 38b6bef
Author: Akira Ajisaka <aa...@apache.org>
Authored: Sat Apr 11 12:37:12 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:40 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                               | 2 --
 .../java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java  | 2 +-
 .../java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java | 2 +-
 3 files changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/72df8153/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index c81868d..a0854c1 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -346,8 +346,6 @@ Release 2.7.1 - UNRELEASED
 
   BUG FIXES
 
-    MAPREDUCE-6300. Task list sort by task id broken. (Siqi Li via aajisaka)
-
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72df8153/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
index 9648527..0212ae4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
@@ -43,7 +43,7 @@ public class TasksPage extends AppView {
       .append(", bProcessing: true")
 
       .append("\n, aoColumnDefs: [\n")
-      .append("{'sType':'string', 'aTargets': [0]")
+      .append("{'sType':'numeric', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }")
 
       .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets': [1]")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72df8153/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
index 2619dff..4e2b687 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
@@ -76,7 +76,7 @@ public class HsTasksPage extends HsView {
     .append(", bProcessing: true")
 
     .append("\n, aoColumnDefs: [\n")
-    .append("{'sType':'string', 'aTargets': [ 0 ]")
+    .append("{'sType':'numeric', 'aTargets': [ 0 ]")
     .append(", 'mRender': parseHadoopID }")
 
     .append(", {'sType':'numeric', 'aTargets': [ 4")


[41/50] [abbrv] hadoop git commit: HDFS-8135. Remove the deprecated FSConstants class. Contributed by Li Lu.

Posted by zj...@apache.org.
HDFS-8135. Remove the deprecated FSConstants class. Contributed by Li Lu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b9788aee
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b9788aee
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b9788aee

Branch: refs/heads/YARN-2928
Commit: b9788aee3526c26f54ec774060eeabcea4ec9547
Parents: 1e2ab61
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Apr 16 11:26:05 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:46 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 ++
 .../hadoop/hdfs/protocol/FSConstants.java       | 27 --------------------
 2 files changed, 2 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9788aee/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index cc9d901..488eefc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -319,6 +319,8 @@ Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
+    HDFS-8135. Remove the deprecated FSConstants class. (Li Lu via wheat9)
+
   NEW FEATURES
 
   IMPROVEMENTS

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9788aee/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FSConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FSConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FSConstants.java
deleted file mode 100644
index 17b2e58..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FSConstants.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-/**
- * @deprecated Please use {@link HdfsConstants}. This class
- * is left only for other ecosystem projects which depended on
- * it for SafemodeAction and DatanodeReport types.
- */
-@Deprecated
-public abstract class FSConstants extends HdfsConstants {
-}


[18/50] [abbrv] hadoop git commit: HADOOP-7713. Update CHANGES.txt

Posted by zj...@apache.org.
HADOOP-7713. Update CHANGES.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8bb1209a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8bb1209a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8bb1209a

Branch: refs/heads/YARN-2928
Commit: 8bb1209a4982857d687243a9561710a6f44ceb3a
Parents: 40b7248
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Apr 13 21:20:16 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:42 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8bb1209a/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index b80d5cb..0fb6e92 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -170,9 +170,6 @@ Trunk (Unreleased)
     HADOOP-11346. Rewrite sls/rumen to use new shell framework (John Smith
     via aw)
 
-    HADOOP-7713. dfs -count -q should label output column (Jonathan Allen
-    via aw)
-
     HADOOP-6964. Allow compact property description in xml (Kengo Seki
     via aw)
 
@@ -490,6 +487,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11814. Reformat hadoop-annotations, o.a.h.classification.tools.
     (Li Lu via wheat9)
 
+    HADOOP-7713. dfs -count -q should label output column (Jonathan Allen
+    via aw)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp


[09/50] [abbrv] hadoop git commit: HDFS-8084. Move dfs.client.failover.* confs from DFSConfigKeys to HdfsClientConfigKeys.Failover and fix typos in the dfs.http.client.* configuration keys.

Posted by zj...@apache.org.
HDFS-8084. Move dfs.client.failover.* confs from DFSConfigKeys to HdfsClientConfigKeys.Failover and fix typos in the dfs.http.client.* configuration keys.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/38b6befd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/38b6befd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/38b6befd

Branch: refs/heads/YARN-2928
Commit: 38b6befd44bace02e0e69ff6734b02c5a97f33dd
Parents: b5e18f0
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Apr 10 19:38:43 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:40 2015 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |  83 ++++++++-------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   4 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  72 ++++++++-----
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |   6 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |  25 ++---
 .../hadoop/hdfs/client/impl/DfsClientConf.java  |  40 ++++----
 .../ha/ConfiguredFailoverProxyProvider.java     |  11 +-
 .../namenode/ha/IPFailoverProxyProvider.java    |  11 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  24 ++---
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 100 +++++++++++++------
 .../hadoop/hdfs/TestDFSClientFailover.java      |  16 ++-
 .../hadoop/hdfs/TestDFSClientRetries.java       |   2 +-
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     |   4 +-
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |   4 +-
 .../hdfs/server/namenode/ha/HATestUtil.java     |   4 +-
 .../namenode/ha/TestPipelinesFailover.java      |   4 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |   9 +-
 17 files changed, 238 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 7316e3b..f8f615e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -28,53 +28,64 @@ public interface HdfsClientConfigKeys {
       "^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
 
   static final String PREFIX = "dfs.client.";
+
   /** Client retry configuration properties */
   public interface Retry {
-    static final String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
+    String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
+
+    String  POLICY_ENABLED_KEY = PREFIX + "policy.enabled";
+    boolean POLICY_ENABLED_DEFAULT = false; 
+    String  POLICY_SPEC_KEY = PREFIX + "policy.spec";
+    String  POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,... 
 
-    public static final String  POLICY_ENABLED_KEY
-        = PREFIX + "policy.enabled";
-    public static final boolean POLICY_ENABLED_DEFAULT
-        = false; 
-    public static final String  POLICY_SPEC_KEY
-        = PREFIX + "policy.spec";
-    public static final String  POLICY_SPEC_DEFAULT
-        = "10000,6,60000,10"; //t1,n1,t2,n2,... 
+    String  TIMES_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "times.get-last-block-length";
+    int     TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT = 3;
+    String  INTERVAL_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "interval-ms.get-last-block-length";
+    int     INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
 
-    public static final String  TIMES_GET_LAST_BLOCK_LENGTH_KEY
-        = PREFIX + "times.get-last-block-length";
-    public static final int     TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT
-        = 3;
-    public static final String  INTERVAL_GET_LAST_BLOCK_LENGTH_KEY
-        = PREFIX + "interval-ms.get-last-block-length";
-    public static final int     INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT
-        = 4000;
+    String  MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
+    int     MAX_ATTEMPTS_DEFAULT = 10;
+
+    String  WINDOW_BASE_KEY = PREFIX + "window.base";
+    int     WINDOW_BASE_DEFAULT = 3000;
+  }
 
-    public static final String  MAX_ATTEMPTS_KEY
-        = PREFIX + "max.attempts";
-    public static final int     MAX_ATTEMPTS_DEFAULT
-        = 10;
+  /** Client failover configuration properties */
+  interface Failover {
+    String PREFIX = HdfsClientConfigKeys.PREFIX + "failover.";
 
-    public static final String  WINDOW_BASE_KEY
-        = PREFIX + "window.base";
-    public static final int     WINDOW_BASE_DEFAULT
-        = 3000;
+    String  PROXY_PROVIDER_KEY_PREFIX = PREFIX + "proxy.provider";
+    String  MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
+    int     MAX_ATTEMPTS_DEFAULT = 15;
+    String  SLEEPTIME_BASE_KEY = PREFIX + "sleep.base.millis";
+    int     SLEEPTIME_BASE_DEFAULT = 500;
+    String  SLEEPTIME_MAX_KEY = PREFIX + "sleep.max.millis";
+    int     SLEEPTIME_MAX_DEFAULT = 15000;
+    String  CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
+    int     CONNECTION_RETRIES_DEFAULT = 0;
+    String  CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = PREFIX + "connection.retries.on.timeouts";
+    int     CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
+    
   }
 
-  // WebHDFS retry configuration policy
-  interface WebHdfsRetry {
-    String  PREFIX = HdfsClientConfigKeys.PREFIX + "http.client.";
-    String  RETRY_POLICY_ENABLED_KEY = PREFIX + "dfs.http.client.retry.policy.enabled";
+  /** HTTP client configuration properties */
+  interface HttpClient {
+    String  PREFIX = "dfs.http.client.";
+
+    // retry
+    String  RETRY_POLICY_ENABLED_KEY = PREFIX + "retry.policy.enabled";
     boolean RETRY_POLICY_ENABLED_DEFAULT = false;
-    String  RETRY_POLICY_SPEC_KEY = PREFIX + "dfs.http.client.retry.policy.spec";
+    String  RETRY_POLICY_SPEC_KEY = PREFIX + "retry.policy.spec";
     String  RETRY_POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
-    String  FAILOVER_MAX_ATTEMPTS_KEY = PREFIX + "dfs.http.client.failover.max.attempts";
-    int     FAILOVER_MAX_ATTEMPTS_DEFAULT =  15;
-    String  RETRY_MAX_ATTEMPTS_KEY = PREFIX + "dfs.http.client.retry.max.attempts";
+    String  RETRY_MAX_ATTEMPTS_KEY = PREFIX + "retry.max.attempts";
     int     RETRY_MAX_ATTEMPTS_DEFAULT = 10;
-    String  FAILOVER_SLEEPTIME_BASE_KEY = PREFIX + "dfs.http.client.failover.sleep.base.millis";
+    
+    // failover
+    String  FAILOVER_MAX_ATTEMPTS_KEY = PREFIX + "failover.max.attempts";
+    int     FAILOVER_MAX_ATTEMPTS_DEFAULT =  15;
+    String  FAILOVER_SLEEPTIME_BASE_KEY = PREFIX + "failover.sleep.base.millis";
     int     FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
-    String  FAILOVER_SLEEPTIME_MAX_KEY = PREFIX + "dfs.http.client.failover.sleep.max.millis";
+    String  FAILOVER_SLEEPTIME_MAX_KEY = PREFIX + "failover.sleep.max.millis";
     int     FAILOVER_SLEEPTIME_MAX_DEFAULT =  15000;
-  }
+  }  
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index be5c238..6e30990 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -421,6 +421,10 @@ Release 2.8.0 - UNRELEASED
     HDFS-8103. Move BlockTokenSecretManager.AccessMode into
     BlockTokenIdentifier. (wheat9)
 
+    HDFS-8084. Move dfs.client.failover.* confs from DFSConfigKeys to
+    HdfsClientConfigKeys.Failover and fix typos in the dfs.http.client.*
+    configuration keys.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index ce08075..eb3bd4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -603,40 +603,40 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   // WebHDFS retry policy
   @Deprecated
   public static final String  DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_KEY =
-      HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_KEY;
+      HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_KEY;
   @Deprecated
   public static final boolean DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_DEFAULT =
-      HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_DEFAULT;
+      HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_DEFAULT;
   @Deprecated
   public static final String  DFS_HTTP_CLIENT_RETRY_POLICY_SPEC_KEY =
-      HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_SPEC_KEY;
+      HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_KEY;
   @Deprecated
   public static final String  DFS_HTTP_CLIENT_RETRY_POLICY_SPEC_DEFAULT =
-      HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_SPEC_DEFAULT;
+      HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_DEFAULT;
   @Deprecated
   public static final String  DFS_HTTP_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY =
-      HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_MAX_ATTEMPTS_KEY;
+      HdfsClientConfigKeys.HttpClient.FAILOVER_MAX_ATTEMPTS_KEY;
   @Deprecated
   public static final int     DFS_HTTP_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT =
-      HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_MAX_ATTEMPTS_DEFAULT;
+      HdfsClientConfigKeys.HttpClient.FAILOVER_MAX_ATTEMPTS_DEFAULT;
   @Deprecated
   public static final String  DFS_HTTP_CLIENT_RETRY_MAX_ATTEMPTS_KEY =
-      HdfsClientConfigKeys.WebHdfsRetry.RETRY_MAX_ATTEMPTS_KEY;
+      HdfsClientConfigKeys.HttpClient.RETRY_MAX_ATTEMPTS_KEY;
   @Deprecated
   public static final int     DFS_HTTP_CLIENT_RETRY_MAX_ATTEMPTS_DEFAULT =
-      HdfsClientConfigKeys.WebHdfsRetry.RETRY_MAX_ATTEMPTS_DEFAULT;
+      HdfsClientConfigKeys.HttpClient.RETRY_MAX_ATTEMPTS_DEFAULT;
   @Deprecated
   public static final String  DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY =
-      HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_BASE_KEY;
+      HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_BASE_KEY;
   @Deprecated
   public static final int     DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT =
-      HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_BASE_DEFAULT;
+      HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_BASE_DEFAULT;
   @Deprecated
   public static final String  DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY =
-      HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_MAX_KEY;
+      HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_MAX_KEY;
   @Deprecated
   public static final int     DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT
-      = HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_MAX_DEFAULT;
+      = HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_MAX_DEFAULT;
 
   // Handling unresolved DN topology mapping
   public static final String  DFS_REJECT_UNRESOLVED_DN_TOPOLOGY_MAPPING_KEY = 
@@ -688,7 +688,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
   
   
-  // client retry confs are moved to HdfsConfigKeys.Client.Retry 
+  // client retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY
       = HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY;
@@ -726,6 +726,40 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_CLIENT_RETRY_WINDOW_BASE_DEFAULT
       = HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT;
 
+  // client failover confs are moved to HdfsClientConfigKeys.Failover 
+  @Deprecated
+  public static final String  DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX
+      = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX;
+  @Deprecated
+  public static final String  DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY
+      = HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT
+      = HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY
+      = HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT
+      = HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY
+      = HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT
+      = HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY
+      = HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT
+      = HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT;
+  @Deprecated
+  public static final String  DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY
+      = HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY;
+  @Deprecated
+  public static final int     DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT
+      = HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT;
 
   
   
@@ -779,18 +813,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS_DEFAULT = 10;
   public static final String  DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS = "dfs.client.file-block-storage-locations.timeout.millis";
   public static final int     DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT = 1000;
-
-  public static final String  DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX = "dfs.client.failover.proxy.provider";
-  public static final String  DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY = "dfs.client.failover.max.attempts";
-  public static final int     DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT = 15;
-  public static final String  DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY = "dfs.client.failover.sleep.base.millis";
-  public static final int     DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
-  public static final String  DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY = "dfs.client.failover.sleep.max.millis";
-  public static final int     DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT = 15000;
-  public static final String  DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY = "dfs.client.failover.connection.retries";
-  public static final int     DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT = 0;
-  public static final String  DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = "dfs.client.failover.connection.retries.on.timeouts";
-  public static final int     DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
   
   public static final String  DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY = "dfs.client.datanode-restart.timeout";
   public static final long    DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT = 30;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
index e1401c1..240256c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
@@ -39,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.NameNodeProxies.ProxyAndInfo;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -227,8 +227,8 @@ public class HAUtil {
   public static boolean isClientFailoverConfigured(
       Configuration conf, URI nameNodeUri) {
     String host = nameNodeUri.getHost();
-    String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
-        + host;
+    String configKey = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
+        + "." + host;
     return conf.get(configKey) != null;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
index 5a929fc..ee3290b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
@@ -17,14 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY;
-
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationHandler;
@@ -230,14 +222,14 @@ public class NameNodeProxies {
 
     if (failoverProxyProvider != null) { // HA case
       int delay = config.getInt(
-          DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
-          DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
+          HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
+          HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
       int maxCap = config.getInt(
-          DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
-          DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
+          HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
+          HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
       int maxFailoverAttempts = config.getInt(
-          DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
-          DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
+          HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
+          HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
       int maxRetryAttempts = config.getInt(
           HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
           HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
@@ -469,9 +461,8 @@ public class NameNodeProxies {
       return null;
     }
     String host = nameNodeUri.getHost();
-  
-    String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
-        + host;
+    String configKey = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
+        + "." + host;
     try {
       @SuppressWarnings("unchecked")
       Class<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>) conf

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
index e781b16..8ba43d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
@@ -31,12 +31,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY;
@@ -119,18 +113,28 @@ public class DfsClientConf {
     // The hdfsTimeout is currently the same as the ipc timeout 
     hdfsTimeout = Client.getTimeout(conf);
 
-    maxFailoverAttempts = conf.getInt(
-        DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
-        DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
     maxRetryAttempts = conf.getInt(
         HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
         HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
+    timeWindow = conf.getInt(
+        HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY,
+        HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT);
+    retryTimesForGetLastBlockLength = conf.getInt(
+        HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY,
+        HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
+    retryIntervalForGetLastBlockLength = conf.getInt(
+        HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_KEY,
+        HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
+
+    maxFailoverAttempts = conf.getInt(
+        HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
+        HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
     failoverSleepBaseMillis = conf.getInt(
-        DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
-        DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
+        HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
+        HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
     failoverSleepMaxMillis = conf.getInt(
-        DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
-        DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
+        HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
+        HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
 
     maxBlockAcquireFailures = conf.getInt(
         DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
@@ -179,9 +183,6 @@ public class DfsClientConf {
         DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
     prefetchSize = conf.getLong(DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
         10 * defaultBlockSize);
-    timeWindow = conf.getInt(
-        HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY,
-        HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT);
     numCachedConnRetry = conf.getInt(DFS_CLIENT_CACHED_CONN_RETRY_KEY,
         DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
     numBlockWriteRetry = conf.getInt(DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
@@ -204,13 +205,6 @@ public class DfsClientConf {
     fileBlockStorageLocationsTimeoutMs = conf.getInt(
         DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS,
         DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT);
-    retryTimesForGetLastBlockLength = conf.getInt(
-        HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY,
-        HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
-    retryIntervalForGetLastBlockLength = conf.getInt(
-        HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_KEY,
-        HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
-
 
     datanodeRestartTimeout = conf.getLong(
         DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
index 06aa8fa..235c886 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
@@ -30,11 +30,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
-import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -69,15 +68,15 @@ public class ConfiguredFailoverProxyProvider<T> extends
     
     this.conf = new Configuration(conf);
     int maxRetries = this.conf.getInt(
-        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY,
-        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT);
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
     this.conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
         maxRetries);
     
     int maxRetriesOnSocketTimeouts = this.conf.getInt(
-        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
     this.conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
         maxRetriesOnSocketTimeouts);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
index 4ccec16..bc4e726 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
@@ -24,11 +24,10 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.io.retry.FailoverProxyProvider;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -67,15 +66,15 @@ public class IPFailoverProxyProvider<T> extends
 
     this.conf = new Configuration(conf);
     int maxRetries = this.conf.getInt(
-        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY,
-        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT);
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
     this.conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
         maxRetries);
     
     int maxRetriesOnSocketTimeouts = this.conf.getInt(
-        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
     this.conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
         maxRetriesOnSocketTimeouts);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 044403e..4c21229 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -173,25 +173,25 @@ public class WebHdfsFileSystem extends FileSystem
       this.retryPolicy =
           RetryUtils.getDefaultRetryPolicy(
               conf,
-              HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_KEY,
-              HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_DEFAULT,
-              HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_SPEC_KEY,
-              HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_SPEC_DEFAULT,
+              HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_KEY,
+              HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_DEFAULT,
+              HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_KEY,
+              HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_DEFAULT,
               SafeModeException.class);
     } else {
 
       int maxFailoverAttempts = conf.getInt(
-          HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_MAX_ATTEMPTS_KEY,
-          HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_MAX_ATTEMPTS_DEFAULT);
+          HdfsClientConfigKeys.HttpClient.FAILOVER_MAX_ATTEMPTS_KEY,
+          HdfsClientConfigKeys.HttpClient.FAILOVER_MAX_ATTEMPTS_DEFAULT);
       int maxRetryAttempts = conf.getInt(
-          HdfsClientConfigKeys.WebHdfsRetry.RETRY_MAX_ATTEMPTS_KEY,
-          HdfsClientConfigKeys.WebHdfsRetry.RETRY_MAX_ATTEMPTS_DEFAULT);
+          HdfsClientConfigKeys.HttpClient.RETRY_MAX_ATTEMPTS_KEY,
+          HdfsClientConfigKeys.HttpClient.RETRY_MAX_ATTEMPTS_DEFAULT);
       int failoverSleepBaseMillis = conf.getInt(
-          HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_BASE_KEY,
-          HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_BASE_DEFAULT);
+          HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_BASE_KEY,
+          HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_BASE_DEFAULT);
       int failoverSleepMaxMillis = conf.getInt(
-          HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_MAX_KEY,
-          HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_MAX_DEFAULT);
+          HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_MAX_KEY,
+          HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_MAX_DEFAULT);
 
       this.retryPolicy = RetryPolicies
           .failoverOnNetworkException(RetryPolicies.TRY_ONCE_THEN_FAIL,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 5fc78d1..1b3b62d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -18,13 +18,52 @@
 
 package org.apache.hadoop.hdfs;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Charsets;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import static org.apache.hadoop.fs.CreateFlag.CREATE;
+import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
+import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.PrintStream;
+import java.io.RandomAccessFile;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.URL;
+import java.net.URLConnection;
+import java.nio.ByteBuffer;
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
@@ -35,24 +74,33 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -75,8 +123,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.ha
-        .ConfiguredFailoverProxyProvider;
+import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
@@ -97,21 +144,12 @@ import org.apache.log4j.Level;
 import org.junit.Assume;
 import org.mockito.internal.util.reflection.Whitebox;
 
-import java.io.*;
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-import java.net.*;
-import java.nio.ByteBuffer;
-import java.security.NoSuchAlgorithmException;
-import java.security.PrivilegedExceptionAction;
-import java.util.*;
-import java.util.concurrent.TimeoutException;
-
-import static org.apache.hadoop.fs.CreateFlag.*;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
@@ -194,7 +232,7 @@ public class DFSTestUtil {
     }
     conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
             logicalName), "nn1,nn2");
-    conf.set(DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "" +
+    conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX +
             "." + logicalName,
             ConfiguredFailoverProxyProvider.class.getName());
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
@@ -963,8 +1001,8 @@ public class DFSTestUtil {
     for (Map.Entry<String, List<String>> entry : nameservices.entrySet()) {
       conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX,
           entry.getKey()), Joiner.on(",").join(entry.getValue()));
-      conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + entry
-          .getKey(), ConfiguredFailoverProxyProvider.class.getName());
+      conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "."
+          + entry.getKey(), ConfiguredFailoverProxyProvider.class.getName());
     }
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, Joiner.on(",")
         .join(nameservices.keySet()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
index 5f136a9..aa14909 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
@@ -17,18 +17,17 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.SocketAddress;
-import java.lang.reflect.Field;
-import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.List;
@@ -42,13 +41,12 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
-import org.apache.hadoop.hdfs.server.namenode.ha.IPFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.hdfs.server.namenode.ha.IPFailoverProxyProvider;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
 import org.apache.hadoop.io.retry.FailoverProxyProvider;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.StandardSocketFactory;
@@ -206,7 +204,7 @@ public class TestDFSClientFailover {
   public void testFailureWithMisconfiguredHaNNs() throws Exception {
     String logicalHost = "misconfigured-ha-uri";
     Configuration conf = new Configuration();
-    conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalHost,
+    conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalHost,
         ConfiguredFailoverProxyProvider.class.getName());
     
     URI uri = new URI("hdfs://" + logicalHost + "/test");
@@ -334,7 +332,7 @@ public class TestDFSClientFailover {
     Configuration config = new HdfsConfiguration(conf);
     String logicalName = HATestUtil.getLogicalHostname(cluster);
     HATestUtil.setFailoverConfigurations(cluster, config, logicalName);
-    config.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
+    config.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
         DummyLegacyFailoverProxyProvider.class.getName());
     Path p = new Path("hdfs://" + logicalName + "/");
 
@@ -354,7 +352,7 @@ public class TestDFSClientFailover {
     // setup the config with the IP failover proxy provider class
     Configuration config = new HdfsConfiguration(conf);
     URI nnUri = cluster.getURI(0);
-    config.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." +
+    config.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." +
         nnUri.getHost(),
         IPFailoverProxyProvider.class.getName());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 1c55d5d..51add1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -875,7 +875,7 @@ public class TestDFSClientRetries {
     final Path dir = new Path("/testNamenodeRestart");
 
     if (isWebHDFS) {
-      conf.setBoolean(HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_KEY, true);
+      conf.setBoolean(HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_KEY, true);
     } else {
       conf.setBoolean(HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY, true);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index 7de121f..ed913f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_INTERNAL_NAMESERVICES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
@@ -60,6 +59,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -603,7 +603,7 @@ public class TestDFSUtil {
     conf.set(DFSUtil.addKeySuffixes(
         DFS_NAMENODE_HTTP_ADDRESS_KEY, "ns1", "nn2"), nnaddr2);
 
-    conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalHostName,
+    conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalHostName,
         ConfiguredFailoverProxyProvider.class.getName());
     return conf;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
index 9380701..ef4c559 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.qjournal;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 
@@ -33,6 +32,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
@@ -137,7 +137,7 @@ public class MiniQJMHACluster {
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, NAMESERVICE);
     conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, NAMESERVICE),
         NN1 + "," + NN2);
-    conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + NAMESERVICE,
+    conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + NAMESERVICE,
         ConfiguredFailoverProxyProvider.class.getName());
     conf.set("fs.defaultFS", "hdfs://" + NAMESERVICE);
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
index c57efa9..c7c4a77 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 
@@ -35,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
@@ -189,7 +189,7 @@ public abstract class HATestUtil {
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, logicalName);
     conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, logicalName),
         nameNodeId1 + "," + nameNodeId2);
-    conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
+    conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
         ConfiguredFailoverProxyProvider.class.getName());
     conf.set("fs.defaultFS", "hdfs://" + logicalName);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
index 18d6dfc..76a62ff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
@@ -448,8 +449,7 @@ public class TestPipelinesFailover {
     // This test triggers rapid NN failovers.  The client retry policy uses an
     // exponential backoff.  This can quickly lead to long sleep times and even
     // timeout the whole test.  Cap the sleep time at 1s to prevent this.
-    harness.conf.setInt(DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
-      1000);
+    harness.conf.setInt(HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY, 1000);
 
     final MiniDFSCluster cluster = harness.startCluster();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38b6befd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index c0d320c..e3572ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -58,8 +58,11 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -69,8 +72,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -201,8 +202,8 @@ public class TestRetryCacheWithHA {
         failoverProxyProvider, RetryPolicies
         .failoverOnNetworkException(RetryPolicies.TRY_ONCE_THEN_FAIL,
             Integer.MAX_VALUE,
-            DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT,
-            DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT));
+            HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT,
+            HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT));
     ClientProtocol proxy = (ClientProtocol) Proxy.newProxyInstance(
         failoverProxyProvider.getInterface().getClassLoader(),
         new Class[] { ClientProtocol.class }, dummyHandler);


[03/50] [abbrv] hadoop git commit: HDFS-8100. Refactor DFSClient.Conf to a standalone class and separates short-circuit related conf to ShortCircuitConf.

Posted by zj...@apache.org.
HDFS-8100. Refactor DFSClient.Conf to a standalone class and separates short-circuit related conf to ShortCircuitConf.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ffe48015
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ffe48015
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ffe48015

Branch: refs/heads/YARN-2928
Commit: ffe48015c341d5db908e8efed96876bd5ad53c1a
Parents: f5d1172
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Apr 10 14:48:45 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:39 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  31 +-
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |  10 +-
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |  27 +-
 .../org/apache/hadoop/hdfs/ClientContext.java   |  75 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 446 +----------
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  21 +-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  22 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  34 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |   4 +-
 .../org/apache/hadoop/hdfs/LeaseRenewer.java    |  15 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |  10 +-
 .../hadoop/hdfs/client/impl/DfsClientConf.java  | 738 +++++++++++++++++++
 .../hdfs/shortcircuit/DomainSocketFactory.java  |   8 +-
 .../hdfs/shortcircuit/ShortCircuitCache.java    |  12 +
 .../hadoop/fs/TestEnhancedByteBufferAccess.java |   6 +-
 .../hadoop/hdfs/TestBlockReaderLocal.java       |   3 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |   2 +-
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |   3 +-
 .../apache/hadoop/hdfs/TestLeaseRenewer.java    |  13 +-
 .../hadoop/hdfs/TestParallelReadUtil.java       |   3 +-
 .../blockmanagement/TestBlockTokenWithDFS.java  |   3 +-
 .../datanode/TestDataNodeVolumeFailure.java     |   4 +-
 .../shortcircuit/TestShortCircuitCache.java     |   2 +-
 24 files changed, 930 insertions(+), 565 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 333a1b1..c2f0363 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -415,6 +415,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8102. Separate webhdfs retry configuration keys from DFSConfigKeys.
     (wheat9)
 
+    HDFS-8100. Refactor DFSClient.Conf to a standalone class and separates
+    short-circuit related conf to ShortCircuitConf.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
index 8f33899..5175a87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -32,6 +32,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -81,7 +83,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   static ShortCircuitReplicaCreator
       createShortCircuitReplicaInfoCallback = null;
 
-  private final DFSClient.Conf conf;
+  private final DfsClientConf conf;
 
   /**
    * Injects failures into specific operations during unit tests.
@@ -180,10 +182,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    */
   private int remainingCacheTries;
 
-  public BlockReaderFactory(DFSClient.Conf conf) {
+  public BlockReaderFactory(DfsClientConf conf) {
     this.conf = conf;
-    this.failureInjector = conf.brfFailureInjector;
-    this.remainingCacheTries = conf.nCachedConnRetry;
+    this.failureInjector = conf.getShortCircuitConf().brfFailureInjector;
+    this.remainingCacheTries = conf.getNumCachedConnRetry();
   }
 
   public BlockReaderFactory setFileName(String fileName) {
@@ -317,7 +319,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     BlockReader reader = null;
 
     Preconditions.checkNotNull(configuration);
-    if (conf.shortCircuitLocalReads && allowShortCircuitLocalReads) {
+    final ShortCircuitConf scConf = conf.getShortCircuitConf();
+    if (scConf.isShortCircuitLocalReads() && allowShortCircuitLocalReads) {
       if (clientContext.getUseLegacyBlockReaderLocal()) {
         reader = getLegacyBlockReaderLocal();
         if (reader != null) {
@@ -336,7 +339,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         }
       }
     }
-    if (conf.domainSocketDataTraffic) {
+    if (scConf.isDomainSocketDataTraffic()) {
       reader = getRemoteBlockReaderFromDomain();
       if (reader != null) {
         if (LOG.isTraceEnabled()) {
@@ -406,8 +409,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           "for short-circuit reads.");
     }
     if (pathInfo == null) {
-      pathInfo = clientContext.getDomainSocketFactory().
-                      getPathInfo(inetSocketAddress, conf);
+      pathInfo = clientContext.getDomainSocketFactory()
+          .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
     }
     if (!pathInfo.getPathState().getUsableForShortCircuit()) {
       PerformanceAdvisory.LOG.debug("{}: {} is not usable for short circuit; " +
@@ -431,7 +434,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           "BlockReaderLocal via {}", this, pathInfo.getPath());
       return null;
     }
-    return new BlockReaderLocal.Builder(conf).
+    return new BlockReaderLocal.Builder(conf.getShortCircuitConf()).
         setFilename(fileName).
         setBlock(block).
         setStartOffset(startOffset).
@@ -604,8 +607,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    */
   private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
     if (pathInfo == null) {
-      pathInfo = clientContext.getDomainSocketFactory().
-                      getPathInfo(inetSocketAddress, conf);
+      pathInfo = clientContext.getDomainSocketFactory()
+          .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
     }
     if (!pathInfo.getPathState().getUsableForDataTransfer()) {
       PerformanceAdvisory.LOG.debug("{}: not trying to create a " +
@@ -744,7 +747,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       }
     }
     DomainSocket sock = clientContext.getDomainSocketFactory().
-        createSocket(pathInfo, conf.socketTimeout);
+        createSocket(pathInfo, conf.getSocketTimeout());
     if (sock == null) return null;
     return new BlockReaderPeer(new DomainPeer(sock), false);
   }
@@ -803,9 +806,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
 
   @SuppressWarnings("deprecation")
   private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
-    if (conf.useLegacyBlockReader) {
+    if (conf.getShortCircuitConf().isUseLegacyBlockReader()) {
       return RemoteBlockReader.newBlockReader(fileName,
-          block, token, startOffset, length, conf.ioBufferSize,
+          block, token, startOffset, length, conf.getIoBufferSize(),
           verifyChecksum, clientName, peer, datanode,
           clientContext.getPeerCache(), cachingStrategy);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index ab93441..d913f3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
@@ -27,14 +27,14 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.DFSClient.Conf;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
-import org.apache.hadoop.util.DirectBufferPool;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DirectBufferPool;
 import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
@@ -74,10 +74,10 @@ class BlockReaderLocal implements BlockReader {
     private ExtendedBlock block;
     private StorageType storageType;
 
-    public Builder(Conf conf) {
+    public Builder(ShortCircuitConf conf) {
       this.maxReadahead = Integer.MAX_VALUE;
-      this.verifyChecksum = !conf.skipShortCircuitChecksums;
-      this.bufferSize = conf.shortCircuitBufferSize;
+      this.verifyChecksum = !conf.isSkipShortCircuitChecksums();
+      this.bufferSize = conf.getShortCircuitBufferSize();
     }
 
     public Builder setVerifyChecksum(boolean verifyChecksum) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index 0c9ec45..8df44f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -42,12 +44,12 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
-import org.apache.hadoop.util.DirectBufferPool;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DirectBufferPool;
 import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
@@ -180,12 +182,13 @@ class BlockReaderLocalLegacy implements BlockReader {
   /**
    * The only way this object can be instantiated.
    */
-  static BlockReaderLocalLegacy newBlockReader(DFSClient.Conf conf,
+  static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
       UserGroupInformation userGroupInformation,
       Configuration configuration, String file, ExtendedBlock blk,
       Token<BlockTokenIdentifier> token, DatanodeInfo node, 
       long startOffset, long length, StorageType storageType)
       throws IOException {
+    final ShortCircuitConf scConf = conf.getShortCircuitConf();
     LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
         .getIpcPort());
     // check the cache first
@@ -195,8 +198,8 @@ class BlockReaderLocalLegacy implements BlockReader {
         userGroupInformation = UserGroupInformation.getCurrentUser();
       }
       pathinfo = getBlockPathInfo(userGroupInformation, blk, node,
-          configuration, conf.socketTimeout, token,
-          conf.connectToDnViaHostname, storageType);
+          configuration, conf.getSocketTimeout(), token,
+          conf.isConnectToDnViaHostname(), storageType);
     }
 
     // check to see if the file exists. It may so happen that the
@@ -208,8 +211,8 @@ class BlockReaderLocalLegacy implements BlockReader {
     FileInputStream dataIn = null;
     FileInputStream checksumIn = null;
     BlockReaderLocalLegacy localBlockReader = null;
-    boolean skipChecksumCheck = conf.skipShortCircuitChecksums ||
-        storageType.isTransient();
+    final boolean skipChecksumCheck = scConf.isSkipShortCircuitChecksums()
+        || storageType.isTransient();
     try {
       // get a local file system
       File blkfile = new File(pathinfo.getBlockPath());
@@ -230,11 +233,11 @@ class BlockReaderLocalLegacy implements BlockReader {
             new DataInputStream(checksumIn), blk);
         long firstChunkOffset = startOffset
             - (startOffset % checksum.getBytesPerChecksum());
-        localBlockReader = new BlockReaderLocalLegacy(conf, file, blk, token,
+        localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
             startOffset, length, pathinfo, checksum, true, dataIn,
             firstChunkOffset, checksumIn);
       } else {
-        localBlockReader = new BlockReaderLocalLegacy(conf, file, blk, token,
+        localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
             startOffset, length, pathinfo, dataIn);
       }
     } catch (IOException e) {
@@ -312,7 +315,7 @@ class BlockReaderLocalLegacy implements BlockReader {
     return bufferSizeBytes / bytesPerChecksum;
   }
 
-  private BlockReaderLocalLegacy(DFSClient.Conf conf, String hdfsfile,
+  private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
       ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
       long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
       throws IOException {
@@ -321,7 +324,7 @@ class BlockReaderLocalLegacy implements BlockReader {
         dataIn, startOffset, null);
   }
 
-  private BlockReaderLocalLegacy(DFSClient.Conf conf, String hdfsfile,
+  private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
       ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
       long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
       boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
@@ -339,8 +342,8 @@ class BlockReaderLocalLegacy implements BlockReader {
     this.checksumIn = checksumIn;
     this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
 
-    int chunksPerChecksumRead = getSlowReadBufferNumChunks(
-        conf.shortCircuitBufferSize, bytesPerChecksum);
+    final int chunksPerChecksumRead = getSlowReadBufferNumChunks(
+        conf.getShortCircuitBufferSize(), bytesPerChecksum);
     slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
     checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
     // Initially the buffers have nothing to read.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
index af7c095..6359def 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
@@ -23,13 +23,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSClient.Conf;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.shortcircuit.DomainSocketFactory;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.cache.Cache;
 
 /**
  * ClientContext contains context information for a client.
@@ -99,59 +99,24 @@ public class ClientContext {
    */
   private boolean printedConfWarning = false;
 
-  private ClientContext(String name, Conf conf) {
-    this.name = name;
-    this.confString = confAsString(conf);
-    this.shortCircuitCache = new ShortCircuitCache(
-        conf.shortCircuitStreamsCacheSize,
-        conf.shortCircuitStreamsCacheExpiryMs,
-        conf.shortCircuitMmapCacheSize,
-        conf.shortCircuitMmapCacheExpiryMs,
-        conf.shortCircuitMmapCacheRetryTimeout,
-        conf.shortCircuitCacheStaleThresholdMs,
-        conf.shortCircuitSharedMemoryWatcherInterruptCheckMs);
-    this.peerCache =
-          new PeerCache(conf.socketCacheCapacity, conf.socketCacheExpiry);
-    this.keyProviderCache = new KeyProviderCache(conf.keyProviderCacheExpiryMs);
-    this.useLegacyBlockReaderLocal = conf.useLegacyBlockReaderLocal;
-    this.domainSocketFactory = new DomainSocketFactory(conf);
-
-    this.byteArrayManager = ByteArrayManager.newInstance(conf.writeByteArrayManagerConf);
-  }
+  private ClientContext(String name, DfsClientConf conf) {
+    final ShortCircuitConf scConf = conf.getShortCircuitConf();
 
-  public static String confAsString(Conf conf) {
-    StringBuilder builder = new StringBuilder();
-    builder.append("shortCircuitStreamsCacheSize = ").
-      append(conf.shortCircuitStreamsCacheSize).
-      append(", shortCircuitStreamsCacheExpiryMs = ").
-      append(conf.shortCircuitStreamsCacheExpiryMs).
-      append(", shortCircuitMmapCacheSize = ").
-      append(conf.shortCircuitMmapCacheSize).
-      append(", shortCircuitMmapCacheExpiryMs = ").
-      append(conf.shortCircuitMmapCacheExpiryMs).
-      append(", shortCircuitMmapCacheRetryTimeout = ").
-      append(conf.shortCircuitMmapCacheRetryTimeout).
-      append(", shortCircuitCacheStaleThresholdMs = ").
-      append(conf.shortCircuitCacheStaleThresholdMs).
-      append(", socketCacheCapacity = ").
-      append(conf.socketCacheCapacity).
-      append(", socketCacheExpiry = ").
-      append(conf.socketCacheExpiry).
-      append(", shortCircuitLocalReads = ").
-      append(conf.shortCircuitLocalReads).
-      append(", useLegacyBlockReaderLocal = ").
-      append(conf.useLegacyBlockReaderLocal).
-      append(", domainSocketDataTraffic = ").
-      append(conf.domainSocketDataTraffic).
-      append(", shortCircuitSharedMemoryWatcherInterruptCheckMs = ").
-      append(conf.shortCircuitSharedMemoryWatcherInterruptCheckMs).
-      append(", keyProviderCacheExpiryMs = ").
-      append(conf.keyProviderCacheExpiryMs);
-
-    return builder.toString();
+    this.name = name;
+    this.confString = scConf.confAsString();
+    this.shortCircuitCache = ShortCircuitCache.fromConf(scConf);
+    this.peerCache = new PeerCache(scConf.getSocketCacheCapacity(),
+        scConf.getSocketCacheExpiry());
+    this.keyProviderCache = new KeyProviderCache(
+        scConf.getKeyProviderCacheExpiryMs());
+    this.useLegacyBlockReaderLocal = scConf.isUseLegacyBlockReaderLocal();
+    this.domainSocketFactory = new DomainSocketFactory(scConf);
+
+    this.byteArrayManager = ByteArrayManager.newInstance(
+        conf.getWriteByteArrayManagerConf());
   }
 
-  public static ClientContext get(String name, Conf conf) {
+  public static ClientContext get(String name, DfsClientConf conf) {
     ClientContext context;
     synchronized(ClientContext.class) {
       context = CACHES.get(name);
@@ -175,12 +140,12 @@ public class ClientContext {
   public static ClientContext getFromConf(Configuration conf) {
     return get(conf.get(DFSConfigKeys.DFS_CLIENT_CONTEXT,
         DFSConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT),
-            new DFSClient.Conf(conf));
+            new DfsClientConf(conf));
   }
 
-  private void printConfWarningIfNeeded(Conf conf) {
+  private void printConfWarningIfNeeded(DfsClientConf conf) {
     String existing = this.getConfString();
-    String requested = confAsString(conf);
+    String requested = conf.getShortCircuitConf().confAsString();
     if (!existing.equals(requested)) {
       if (!printedConfWarning) {
         printedConfWarning = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index d43e7de..f79d160 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -18,48 +18,11 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -109,7 +72,6 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersi
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
 import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -136,9 +98,9 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.AclException;
@@ -195,14 +157,12 @@ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
-import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.DNS;
@@ -250,7 +210,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
 
   private final Configuration conf;
-  private final Conf dfsClientConf;
+  private final DfsClientConf dfsClientConf;
   final ClientProtocol namenode;
   /* The service used for delegation tokens */
   private Text dtService;
@@ -278,307 +238,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
   private final Sampler<?> traceSampler;
 
-  /**
-   * DFSClient configuration 
-   */
-  public static class Conf {
-    final int hdfsTimeout;    // timeout value for a DFS operation.
-
-    final int maxFailoverAttempts;
-    final int maxRetryAttempts;
-    final int failoverSleepBaseMillis;
-    final int failoverSleepMaxMillis;
-    final int maxBlockAcquireFailures;
-    final int confTime;
-    final int ioBufferSize;
-    final ChecksumOpt defaultChecksumOpt;
-    final int writePacketSize;
-    final int writeMaxPackets;
-    final ByteArrayManager.Conf writeByteArrayManagerConf;
-    final int socketTimeout;
-    final int socketCacheCapacity;
-    final long socketCacheExpiry;
-    final long excludedNodesCacheExpiry;
-    /** Wait time window (in msec) if BlockMissingException is caught */
-    final int timeWindow;
-    final int nCachedConnRetry;
-    final int nBlockWriteRetry;
-    final int nBlockWriteLocateFollowingRetry;
-    final int blockWriteLocateFollowingInitialDelayMs;
-    final long defaultBlockSize;
-    final long prefetchSize;
-    final short defaultReplication;
-    final String taskId;
-    final FsPermission uMask;
-    final boolean connectToDnViaHostname;
-    final boolean getHdfsBlocksMetadataEnabled;
-    final int getFileBlockStorageLocationsNumThreads;
-    final int getFileBlockStorageLocationsTimeoutMs;
-    final int retryTimesForGetLastBlockLength;
-    final int retryIntervalForGetLastBlockLength;
-    final long datanodeRestartTimeout;
-    final long dfsclientSlowIoWarningThresholdMs;
-
-    final boolean useLegacyBlockReader;
-    final boolean useLegacyBlockReaderLocal;
-    final String domainSocketPath;
-    final boolean skipShortCircuitChecksums;
-    final int shortCircuitBufferSize;
-    final boolean shortCircuitLocalReads;
-    final boolean domainSocketDataTraffic;
-    final int shortCircuitStreamsCacheSize;
-    final long shortCircuitStreamsCacheExpiryMs; 
-    final int shortCircuitSharedMemoryWatcherInterruptCheckMs;
-    
-    final boolean shortCircuitMmapEnabled;
-    final int shortCircuitMmapCacheSize;
-    final long shortCircuitMmapCacheExpiryMs;
-    final long shortCircuitMmapCacheRetryTimeout;
-    final long shortCircuitCacheStaleThresholdMs;
-
-    final long keyProviderCacheExpiryMs;
-    public BlockReaderFactory.FailureInjector brfFailureInjector =
-      new BlockReaderFactory.FailureInjector();
-
-    public Conf(Configuration conf) {
-      // The hdfsTimeout is currently the same as the ipc timeout 
-      hdfsTimeout = Client.getTimeout(conf);
-      maxFailoverAttempts = conf.getInt(
-          DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
-          DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
-      maxRetryAttempts = conf.getInt(
-          HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
-          HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
-      failoverSleepBaseMillis = conf.getInt(
-          DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
-          DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
-      failoverSleepMaxMillis = conf.getInt(
-          DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
-          DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
-
-      maxBlockAcquireFailures = conf.getInt(
-          DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
-          DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
-      confTime = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
-          HdfsServerConstants.WRITE_TIMEOUT);
-      ioBufferSize = conf.getInt(
-          CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
-          CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
-      defaultChecksumOpt = getChecksumOptFromConf(conf);
-      socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
-          HdfsServerConstants.READ_TIMEOUT);
-      /** dfs.write.packet.size is an internal config variable */
-      writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
-          DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
-      writeMaxPackets = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_KEY,
-          DFSConfigKeys.DFS_CLIENT_WRITE_MAX_PACKETS_IN_FLIGHT_DEFAULT);
-      
-      final boolean byteArrayManagerEnabled = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_ENABLED_KEY,
-          DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_ENABLED_DEFAULT);
-      if (!byteArrayManagerEnabled) {
-        writeByteArrayManagerConf = null;
-      } else {
-        final int countThreshold = conf.getInt(
-            DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_KEY,
-            DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_THRESHOLD_DEFAULT);
-        final int countLimit = conf.getInt(
-            DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_LIMIT_KEY,
-            DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_LIMIT_DEFAULT);
-        final long countResetTimePeriodMs = conf.getLong(
-            DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_KEY,
-            DFSConfigKeys.DFS_CLIENT_WRITE_BYTE_ARRAY_MANAGER_COUNT_RESET_TIME_PERIOD_MS_DEFAULT);
-        writeByteArrayManagerConf = new ByteArrayManager.Conf(
-            countThreshold, countLimit, countResetTimePeriodMs); 
-      }
-      
-      
-      defaultBlockSize = conf.getLongBytes(DFS_BLOCK_SIZE_KEY,
-          DFS_BLOCK_SIZE_DEFAULT);
-      defaultReplication = (short) conf.getInt(
-          DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT);
-      taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
-      socketCacheCapacity = conf.getInt(DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
-          DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT);
-      socketCacheExpiry = conf.getLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY,
-          DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT);
-      excludedNodesCacheExpiry = conf.getLong(
-          DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
-          DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
-      prefetchSize = conf.getLong(DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
-          10 * defaultBlockSize);
-      timeWindow = conf.getInt(
-          HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY,
-          HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT);
-      nCachedConnRetry = conf.getInt(DFS_CLIENT_CACHED_CONN_RETRY_KEY,
-          DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
-      nBlockWriteRetry = conf.getInt(DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
-          DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT);
-      nBlockWriteLocateFollowingRetry = conf.getInt(
-          DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
-          DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
-      blockWriteLocateFollowingInitialDelayMs = conf.getInt(
-          DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY,
-          DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_DEFAULT);
-      uMask = FsPermission.getUMask(conf);
-      connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
-          DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
-      getHdfsBlocksMetadataEnabled = conf.getBoolean(
-          DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, 
-          DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
-      getFileBlockStorageLocationsNumThreads = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS,
-          DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS_DEFAULT);
-      getFileBlockStorageLocationsTimeoutMs = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS,
-          DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT);
-      retryTimesForGetLastBlockLength = conf.getInt(
-          HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY,
-          HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
-      retryIntervalForGetLastBlockLength = conf.getInt(
-          HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_KEY,
-          HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
-
-      useLegacyBlockReader = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER,
-          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT);
-      useLegacyBlockReaderLocal = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
-          DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT);
-      shortCircuitLocalReads = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
-      domainSocketDataTraffic = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
-          DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT);
-      domainSocketPath = conf.getTrimmed(
-          DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
-          DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT);
-
-      if (BlockReaderLocal.LOG.isDebugEnabled()) {
-        BlockReaderLocal.LOG.debug(
-            DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL
-            + " = " + useLegacyBlockReaderLocal);
-        BlockReaderLocal.LOG.debug(
-            DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY
-            + " = " + shortCircuitLocalReads);
-        BlockReaderLocal.LOG.debug(
-            DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC
-            + " = " + domainSocketDataTraffic);
-        BlockReaderLocal.LOG.debug(
-            DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY
-            + " = " + domainSocketPath);
-      }
-
-      skipShortCircuitChecksums = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
-      shortCircuitBufferSize = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
-      shortCircuitStreamsCacheSize = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT);
-      shortCircuitStreamsCacheExpiryMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
-          DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT);
-      shortCircuitMmapEnabled = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED,
-          DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED_DEFAULT);
-      shortCircuitMmapCacheSize = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE,
-          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT);
-      shortCircuitMmapCacheExpiryMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
-          DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT);
-      shortCircuitMmapCacheRetryTimeout = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
-          DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT);
-      shortCircuitCacheStaleThresholdMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
-          DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT);
-      shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
-          DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
-          DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
-
-      datanodeRestartTimeout = conf.getLong(
-          DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,
-          DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT) * 1000;
-      dfsclientSlowIoWarningThresholdMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
-          DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
-
-      keyProviderCacheExpiryMs = conf.getLong(
-          DFSConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_MS,
-          DFSConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT);
-    }
-
-    public boolean isUseLegacyBlockReaderLocal() {
-      return useLegacyBlockReaderLocal;
-    }
-
-    public String getDomainSocketPath() {
-      return domainSocketPath;
-    }
-
-    public boolean isShortCircuitLocalReads() {
-      return shortCircuitLocalReads;
-    }
-
-    public boolean isDomainSocketDataTraffic() {
-      return domainSocketDataTraffic;
-    }
-
-    private DataChecksum.Type getChecksumType(Configuration conf) {
-      final String checksum = conf.get(
-          DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY,
-          DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT);
-      try {
-        return DataChecksum.Type.valueOf(checksum);
-      } catch(IllegalArgumentException iae) {
-        LOG.warn("Bad checksum type: " + checksum + ". Using default "
-            + DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT);
-        return DataChecksum.Type.valueOf(
-            DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT); 
-      }
-    }
-
-    // Construct a checksum option from conf
-    private ChecksumOpt getChecksumOptFromConf(Configuration conf) {
-      DataChecksum.Type type = getChecksumType(conf);
-      int bytesPerChecksum = conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY,
-          DFS_BYTES_PER_CHECKSUM_DEFAULT);
-      return new ChecksumOpt(type, bytesPerChecksum);
-    }
-
-    // create a DataChecksum with the default option.
-    private DataChecksum createChecksum() throws IOException {
-      return createChecksum(null);
-    }
-
-    private DataChecksum createChecksum(ChecksumOpt userOpt) {
-      // Fill in any missing field with the default.
-      ChecksumOpt myOpt = ChecksumOpt.processChecksumOpt(
-          defaultChecksumOpt, userOpt);
-      DataChecksum dataChecksum = DataChecksum.newDataChecksum(
-          myOpt.getChecksumType(),
-          myOpt.getBytesPerChecksum());
-      if (dataChecksum == null) {
-        throw new HadoopIllegalArgumentException("Invalid checksum type: userOpt="
-            + userOpt + ", default=" + defaultChecksumOpt
-            + ", effective=null");
-      }
-      return dataChecksum;
-    }
-
-    @VisibleForTesting
-    public int getBlockWriteLocateFollowingInitialDelayMs() {
-      return blockWriteLocateFollowingInitialDelayMs;
-    }
-  }
- 
-  public Conf getConf() {
+  public DfsClientConf getConf() {
     return dfsClientConf;
   }
 
@@ -642,10 +302,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     SpanReceiverHost.getInstance(conf);
     traceSampler = new SamplerBuilder(TraceUtils.wrapHadoopConf(conf)).build();
     // Copy only the required DFSClient configuration
-    this.dfsClientConf = new Conf(conf);
-    if (this.dfsClientConf.useLegacyBlockReaderLocal) {
-      LOG.debug("Using legacy short-circuit local reads.");
-    }
+    this.dfsClientConf = new DfsClientConf(conf);
     this.conf = conf;
     this.stats = stats;
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
@@ -654,7 +311,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     this.ugi = UserGroupInformation.getCurrentUser();
     
     this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
-    this.clientName = "DFSClient_" + dfsClientConf.taskId + "_" + 
+    this.clientName = "DFSClient_" + dfsClientConf.getTaskId() + "_" + 
         DFSUtil.getRandom().nextInt()  + "_" + Thread.currentThread().getId();
     int numResponseToDrop = conf.getInt(
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
@@ -779,30 +436,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /**
-   * Return the number of times the client should go back to the namenode
-   * to retrieve block locations when reading.
-   */
-  int getMaxBlockAcquireFailures() {
-    return dfsClientConf.maxBlockAcquireFailures;
-  }
-
-  /**
    * Return the timeout that clients should use when writing to datanodes.
    * @param numNodes the number of nodes in the pipeline.
    */
   int getDatanodeWriteTimeout(int numNodes) {
-    return (dfsClientConf.confTime > 0) ?
-      (dfsClientConf.confTime + HdfsServerConstants.WRITE_TIMEOUT_EXTENSION * numNodes) : 0;
+    final int t = dfsClientConf.getDatanodeSocketWriteTimeout();
+    return t > 0? t + HdfsServerConstants.WRITE_TIMEOUT_EXTENSION*numNodes: 0;
   }
 
   int getDatanodeReadTimeout(int numNodes) {
-    return dfsClientConf.socketTimeout > 0 ?
-        (HdfsServerConstants.READ_TIMEOUT_EXTENSION * numNodes +
-            dfsClientConf.socketTimeout) : 0;
-  }
-  
-  int getHdfsTimeout() {
-    return dfsClientConf.hdfsTimeout;
+    final int t = dfsClientConf.getSocketTimeout();
+    return t > 0? HdfsServerConstants.READ_TIMEOUT_EXTENSION*numNodes + t: 0;
   }
   
   @VisibleForTesting
@@ -992,14 +636,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /**
-   * Get the default block size for this cluster
-   * @return the default block size in bytes
-   */
-  public long getDefaultBlockSize() {
-    return dfsClientConf.defaultBlockSize;
-  }
-    
-  /**
    * @see ClientProtocol#getPreferredBlockSize(String)
    */
   public long getBlockSize(String f) throws IOException {
@@ -1211,13 +847,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     namenode.reportBadBlocks(blocks);
   }
   
-  public short getDefaultReplication() {
-    return dfsClientConf.defaultReplication;
-  }
-  
   public LocatedBlocks getLocatedBlocks(String src, long start)
       throws IOException {
-    return getLocatedBlocks(src, start, dfsClientConf.prefetchSize);
+    return getLocatedBlocks(src, start, dfsClientConf.getPrefetchSize());
   }
 
   /*
@@ -1319,7 +951,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public BlockStorageLocation[] getBlockStorageLocations(
       List<BlockLocation> blockLocations) throws IOException,
       UnsupportedOperationException, InvalidBlockTokenException {
-    if (!getConf().getHdfsBlocksMetadataEnabled) {
+    if (!getConf().isHdfsBlocksMetadataEnabled()) {
       throw new UnsupportedOperationException("Datanode-side support for " +
           "getVolumeBlockLocations() must also be enabled in the client " +
           "configuration.");
@@ -1356,9 +988,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try {
       metadatas = BlockStorageLocationUtil.
           queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
-              getConf().getFileBlockStorageLocationsNumThreads,
-              getConf().getFileBlockStorageLocationsTimeoutMs,
-              getConf().connectToDnViaHostname);
+              getConf().getFileBlockStorageLocationsNumThreads(),
+              getConf().getFileBlockStorageLocationsTimeoutMs(),
+              getConf().isConnectToDnViaHostname());
       if (LOG.isTraceEnabled()) {
         LOG.trace("metadata returned: "
             + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
@@ -1512,7 +1144,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public DFSInputStream open(String src) 
       throws IOException, UnresolvedLinkException {
-    return open(src, dfsClientConf.ioBufferSize, true, null);
+    return open(src, dfsClientConf.getIoBufferSize(), true, null);
   }
 
   /**
@@ -1563,8 +1195,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public OutputStream create(String src, boolean overwrite) 
       throws IOException {
-    return create(src, overwrite, dfsClientConf.defaultReplication,
-        dfsClientConf.defaultBlockSize, null);
+    return create(src, overwrite, dfsClientConf.getDefaultReplication(),
+        dfsClientConf.getDefaultBlockSize(), null);
   }
     
   /**
@@ -1574,8 +1206,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public OutputStream create(String src, 
                              boolean overwrite,
                              Progressable progress) throws IOException {
-    return create(src, overwrite, dfsClientConf.defaultReplication,
-        dfsClientConf.defaultBlockSize, progress);
+    return create(src, overwrite, dfsClientConf.getDefaultReplication(),
+        dfsClientConf.getDefaultBlockSize(), progress);
   }
     
   /**
@@ -1596,7 +1228,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public OutputStream create(String src, boolean overwrite, short replication,
       long blockSize, Progressable progress) throws IOException {
     return create(src, overwrite, replication, blockSize, progress,
-        dfsClientConf.ioBufferSize);
+        dfsClientConf.getIoBufferSize());
   }
 
   /**
@@ -1678,6 +1310,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
         progress, buffersize, checksumOpt, null);
   }
 
+  private FsPermission applyUMask(FsPermission permission) {
+    if (permission == null) {
+      permission = FsPermission.getFileDefault();
+    }
+    return permission.applyUMask(dfsClientConf.getUMask());
+  }
+
   /**
    * Same as {@link #create(String, FsPermission, EnumSet, boolean, short, long,
    * Progressable, int, ChecksumOpt)} with the addition of favoredNodes that is
@@ -1698,10 +1337,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                              ChecksumOpt checksumOpt,
                              InetSocketAddress[] favoredNodes) throws IOException {
     checkOpen();
-    if (permission == null) {
-      permission = FsPermission.getFileDefault();
-    }
-    FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
+    final FsPermission masked = applyUMask(permission);
     if(LOG.isDebugEnabled()) {
       LOG.debug(src + ": masked=" + masked);
     }
@@ -1783,8 +1419,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throws IOException {
     TraceScope scope = getPathTraceScope("createSymlink", target);
     try {
-      FsPermission dirPerm = 
-          FsPermission.getDefault().applyUMask(dfsClientConf.uMask); 
+      final FsPermission dirPerm = applyUMask(null);
       namenode.createSymlink(target, link, dirPerm, createParent);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
@@ -1828,7 +1463,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           new EnumSetWritable<>(flag, CreateFlag.class));
       return DFSOutputStream.newStreamForAppend(this, src, flag, buffersize,
           progress, blkWithStatus.getLastBlock(),
-          blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(),
+          blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(null),
           favoredNodes);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
@@ -2253,7 +1888,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       final DatanodeInfo[] datanodes = lb.getLocations();
       
       //try each datanode location of the block
-      final int timeout = 3000 * datanodes.length + dfsClientConf.socketTimeout;
+      final int timeout = 3000*datanodes.length + dfsClientConf.getSocketTimeout();
       boolean done = false;
       for(int j = 0; !done && j < datanodes.length; j++) {
         DataOutputStream out = null;
@@ -2391,7 +2026,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     Socket sock = null;
     try {
       sock = socketFactory.createSocket();
-      String dnAddr = dn.getXferAddr(getConf().connectToDnViaHostname);
+      String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
       if (LOG.isDebugEnabled()) {
         LOG.debug("Connecting to datanode " + dnAddr);
       }
@@ -2424,7 +2059,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   private Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
       throws IOException {
-    IOStreamPair pair = connectToDN(dn, dfsClientConf.socketTimeout, lb);
+    IOStreamPair pair = connectToDN(dn, dfsClientConf.getSocketTimeout(), lb);
 
     try {
       DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
@@ -2979,10 +2614,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean mkdirs(String src, FsPermission permission,
       boolean createParent) throws IOException {
-    if (permission == null) {
-      permission = FsPermission.getDefault();
-    }
-    FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
+    final FsPermission masked = applyUMask(permission);
     return primitiveMkdir(src, masked, createParent);
   }
 
@@ -3004,8 +2636,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     throws IOException {
     checkOpen();
     if (absPermission == null) {
-      absPermission = 
-        FsPermission.getDefault().applyUMask(dfsClientConf.uMask);
+      absPermission = applyUMask(null);
     } 
 
     if(LOG.isDebugEnabled()) {
@@ -3447,14 +3078,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     Peer peer = null;
     boolean success = false;
     Socket sock = null;
+    final int socketTimeout = dfsClientConf.getSocketTimeout(); 
     try {
       sock = socketFactory.createSocket();
-      NetUtils.connect(sock, addr,
-        getRandomLocalInterfaceAddr(),
-        dfsClientConf.socketTimeout);
+      NetUtils.connect(sock, addr, getRandomLocalInterfaceAddr(), socketTimeout);
       peer = TcpPeerServer.peerFromSocketAndKey(saslClient, sock, this,
           blockToken, datanodeId);
-      peer.setReadTimeout(dfsClientConf.socketTimeout);
+      peer.setReadTimeout(socketTimeout);
       success = true;
       return peer;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 41b9d50..dd0f6fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -52,14 +52,15 @@ import org.apache.hadoop.fs.CanSetReadahead;
 import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
@@ -265,9 +266,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Grab the open-file info from namenode
    */
   void openInfo() throws IOException, UnresolvedLinkException {
+    final DfsClientConf conf = dfsClient.getConf();
     synchronized(infoLock) {
       lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
-      int retriesForLastBlockLength = dfsClient.getConf().retryTimesForGetLastBlockLength;
+      int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
       while (retriesForLastBlockLength > 0) {
         // Getting last block length as -1 is a special case. When cluster
         // restarts, DNs may not report immediately. At this time partial block
@@ -277,7 +279,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           DFSClient.LOG.warn("Last block locations not available. "
               + "Datanodes might not have reported blocks completely."
               + " Will retry for " + retriesForLastBlockLength + " times");
-          waitFor(dfsClient.getConf().retryIntervalForGetLastBlockLength);
+          waitFor(conf.getRetryIntervalForGetLastBlockLength());
           lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
         } else {
           break;
@@ -346,13 +348,14 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     assert locatedblock != null : "LocatedBlock cannot be null";
     int replicaNotFoundCount = locatedblock.getLocations().length;
     
+    final DfsClientConf conf = dfsClient.getConf();
     for(DatanodeInfo datanode : locatedblock.getLocations()) {
       ClientDatanodeProtocol cdp = null;
       
       try {
         cdp = DFSUtil.createClientDatanodeProtocolProxy(datanode,
-            dfsClient.getConfiguration(), dfsClient.getConf().socketTimeout,
-            dfsClient.getConf().connectToDnViaHostname, locatedblock);
+            dfsClient.getConfiguration(), conf.getSocketTimeout(),
+            conf.isConnectToDnViaHostname(), locatedblock);
         
         final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
         
@@ -938,7 +941,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
           deadNodes, ignoredNodes);
         String blockInfo = block.getBlock() + " file=" + src;
-        if (failures >= dfsClient.getMaxBlockAcquireFailures()) {
+        if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
           String description = "Could not obtain block: " + blockInfo;
           DFSClient.LOG.warn(description + errMsg
               + ". Throwing a BlockMissingException");
@@ -963,7 +966,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // alleviating the request rate from the server. Similarly the 3rd retry
           // will wait 6000ms grace period before retry and the waiting window is
           // expanded to 9000ms. 
-          final int timeWindow = dfsClient.getConf().timeWindow;
+          final int timeWindow = dfsClient.getConf().getTimeWindow();
           double waitTime = timeWindow * failures +       // grace period for the last round of attempt
             timeWindow * (failures + 1) * DFSUtil.getRandom().nextDouble(); // expanding time window for each failure
           DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
@@ -1012,7 +1015,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           ", ignoredNodes = " + ignoredNodes);
     }
     final String dnAddr =
-        chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
+        chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
     if (DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
     }
@@ -1706,7 +1709,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       }
     }
     ByteBuffer buffer = null;
-    if (dfsClient.getConf().shortCircuitMmapEnabled) {
+    if (dfsClient.getConf().getShortCircuitConf().isShortCircuitMmapEnabled()) {
       buffer = tryReadZeroCopy(maxLength, opts);
     }
     if (buffer != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index f6733e3..8cde274 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -33,10 +33,11 @@ import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -211,7 +212,7 @@ public class DFSOutputStream extends FSOutputSummer
     this(dfsClient, src, progress, stat, checksum);
     this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
 
-    computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum);
+    computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
 
     streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
         cachingStrategy, byteArrayManager);
@@ -297,7 +298,7 @@ public class DFSOutputStream extends FSOutputSummer
       adjustPacketChunkSize(stat);
       streamer.setPipelineInConstruction(lastBlock);
     } else {
-      computePacketChunkSize(dfsClient.getConf().writePacketSize,
+      computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
           bytesPerChecksum);
       streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
           dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager);
@@ -334,7 +335,8 @@ public class DFSOutputStream extends FSOutputSummer
       // that expected size of of a packet, then create
       // smaller size packet.
       //
-      computePacketChunkSize(Math.min(dfsClient.getConf().writePacketSize, freeInLastBlock),
+      computePacketChunkSize(
+          Math.min(dfsClient.getConf().getWritePacketSize(), freeInLastBlock),
           bytesPerChecksum);
     }
   }
@@ -445,7 +447,7 @@ public class DFSOutputStream extends FSOutputSummer
 
     if (!streamer.getAppendChunk()) {
       int psize = Math.min((int)(blockSize- streamer.getBytesCurBlock()),
-          dfsClient.getConf().writePacketSize);
+          dfsClient.getConf().getWritePacketSize());
       computePacketChunkSize(psize, bytesPerChecksum);
     }
   }
@@ -717,7 +719,7 @@ public class DFSOutputStream extends FSOutputSummer
       return;
     }
     streamer.setLastException(new IOException("Lease timeout of "
-        + (dfsClient.getHdfsTimeout()/1000) + " seconds expired."));
+        + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
     closeThreads(true);
     dfsClient.endFileLease(fileId);
   }
@@ -806,15 +808,15 @@ public class DFSOutputStream extends FSOutputSummer
   // be called during unit tests
   protected void completeFile(ExtendedBlock last) throws IOException {
     long localstart = Time.monotonicNow();
-    long sleeptime = dfsClient.getConf().
-        blockWriteLocateFollowingInitialDelayMs;
+    final DfsClientConf conf = dfsClient.getConf();
+    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
     boolean fileComplete = false;
-    int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
+    int retries = conf.getNumBlockWriteLocateFollowingRetry();
     while (!fileComplete) {
       fileComplete =
           dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
       if (!fileComplete) {
-        final int hdfsTimeout = dfsClient.getHdfsTimeout();
+        final int hdfsTimeout = conf.getHdfsTimeout();
         if (!dfsClient.clientRunning
             || (hdfsTimeout > 0
                 && localstart + hdfsTimeout < Time.monotonicNow())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 0c6b4a3..405f775 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -43,6 +43,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -84,6 +85,7 @@ import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceInfo;
 import org.apache.htrace.TraceScope;
+
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -123,15 +125,15 @@ class DataStreamer extends Daemon {
    */
   static Socket createSocketForPipeline(final DatanodeInfo first,
       final int length, final DFSClient client) throws IOException {
-    final String dnAddr = first.getXferAddr(
-        client.getConf().connectToDnViaHostname);
+    final DfsClientConf conf = client.getConf();
+    final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
     if (DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
     }
     final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
     final Socket sock = client.socketFactory.createSocket();
     final int timeout = client.getDatanodeReadTimeout(length);
-    NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), client.getConf().socketTimeout);
+    NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
     sock.setSoTimeout(timeout);
     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
     if(DFSClient.LOG.isDebugEnabled()) {
@@ -244,7 +246,7 @@ class DataStreamer extends Daemon {
     this.byteArrayManager = byteArrayManage;
     isLazyPersistFile = isLazyPersist(stat);
     this.dfsclientSlowLogThresholdMs =
-        dfsClient.getConf().dfsclientSlowIoWarningThresholdMs;
+        dfsClient.getConf().getSlowIoWarningThresholdMs();
     excludedNodes = initExcludedNodes();
   }
 
@@ -368,6 +370,7 @@ class DataStreamer extends Daemon {
           doSleep = processDatanodeError();
         }
 
+        final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; 
         synchronized (dataQueue) {
           // wait for a packet to be sent.
           long now = Time.monotonicNow();
@@ -375,8 +378,8 @@ class DataStreamer extends Daemon {
               && dataQueue.size() == 0 &&
               (stage != BlockConstructionStage.DATA_STREAMING ||
                   stage == BlockConstructionStage.DATA_STREAMING &&
-                      now - lastPacket < dfsClient.getConf().socketTimeout/2)) || doSleep ) {
-            long timeout = dfsClient.getConf().socketTimeout/2 - (now-lastPacket);
+                      now - lastPacket < halfSocketTimeout)) || doSleep ) {
+            long timeout = halfSocketTimeout - (now-lastPacket);
             timeout = timeout <= 0 ? 1000 : timeout;
             timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
                 timeout : 1000;
@@ -627,7 +630,7 @@ class DataStreamer extends Daemon {
         boolean firstWait = true;
         try {
           while (!streamerClosed && dataQueue.size() + ackQueue.size() >
-              dfsClient.getConf().writeMaxPackets) {
+              dfsClient.getConf().getWriteMaxPackets()) {
             if (firstWait) {
               Span span = Trace.currentSpan();
               if (span != null) {
@@ -842,7 +845,7 @@ class DataStreamer extends Daemon {
             // the local node or the only one in the pipeline.
             if (PipelineAck.isRestartOOBStatus(reply) &&
                 shouldWaitForRestart(i)) {
-              restartDeadline = dfsClient.getConf().datanodeRestartTimeout
+              restartDeadline = dfsClient.getConf().getDatanodeRestartTimeout()
                   + Time.monotonicNow();
               setRestartingNodeIndex(i);
               String message = "A datanode is restarting: " + targets[i];
@@ -1158,7 +1161,7 @@ class DataStreamer extends Daemon {
         // 4 seconds or the configured deadline period, whichever is shorter.
         // This is the retry interval and recovery will be retried in this
         // interval until timeout or success.
-        long delay = Math.min(dfsClient.getConf().datanodeRestartTimeout,
+        long delay = Math.min(dfsClient.getConf().getDatanodeRestartTimeout(),
             4000L);
         try {
           Thread.sleep(delay);
@@ -1311,7 +1314,7 @@ class DataStreamer extends Daemon {
     LocatedBlock lb = null;
     DatanodeInfo[] nodes = null;
     StorageType[] storageTypes = null;
-    int count = dfsClient.getConf().nBlockWriteRetry;
+    int count = dfsClient.getConf().getNumBlockWriteRetry();
     boolean success = false;
     ExtendedBlock oldBlock = block;
     do {
@@ -1471,7 +1474,7 @@ class DataStreamer extends Daemon {
         }
         // Check whether there is a restart worth waiting for.
         if (checkRestart && shouldWaitForRestart(errorIndex)) {
-          restartDeadline = dfsClient.getConf().datanodeRestartTimeout
+          restartDeadline = dfsClient.getConf().getDatanodeRestartTimeout()
               + Time.monotonicNow();
           restartingNodeIndex.set(errorIndex);
           errorIndex = -1;
@@ -1524,9 +1527,9 @@ class DataStreamer extends Daemon {
 
   protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
       throws IOException {
-    int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
-    long sleeptime = dfsClient.getConf().
-        blockWriteLocateFollowingInitialDelayMs;
+    final DfsClientConf conf = dfsClient.getConf(); 
+    int retries = conf.getNumBlockWriteLocateFollowingRetry();
+    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
     while (true) {
       long localstart = Time.monotonicNow();
       while (true) {
@@ -1674,7 +1677,8 @@ class DataStreamer extends Daemon {
 
   private LoadingCache<DatanodeInfo, DatanodeInfo> initExcludedNodes() {
     return CacheBuilder.newBuilder().expireAfterWrite(
-        dfsClient.getConf().excludedNodesCacheExpiry, TimeUnit.MILLISECONDS)
+        dfsClient.getConf().getExcludedNodesCacheExpiry(),
+        TimeUnit.MILLISECONDS)
         .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
           @Override
           public void onRemoval(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 3edab48..21f5107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -160,12 +160,12 @@ public class DistributedFileSystem extends FileSystem {
 
   @Override
   public long getDefaultBlockSize() {
-    return dfs.getDefaultBlockSize();
+    return dfs.getConf().getDefaultBlockSize();
   }
 
   @Override
   public short getDefaultReplication() {
-    return dfs.getDefaultReplication();
+    return dfs.getConf().getDefaultReplication();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
index 3e0abce..511bddb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
@@ -225,8 +225,9 @@ class LeaseRenewer {
     dfsclients.add(dfsc);
 
     //update renewal time
-    if (dfsc.getHdfsTimeout() > 0) {
-      final long half = dfsc.getHdfsTimeout()/2;
+    final int hdfsTimeout = dfsc.getConf().getHdfsTimeout();
+    if (hdfsTimeout > 0) {
+      final long half = hdfsTimeout/2;
       if (half < renewal) {
         this.renewal = half;
       }
@@ -368,14 +369,12 @@ class LeaseRenewer {
     }
 
     //update renewal time
-    if (renewal == dfsc.getHdfsTimeout()/2) {
+    if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
       long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
       for(DFSClient c : dfsclients) {
-        if (c.getHdfsTimeout() > 0) {
-          final long timeout = c.getHdfsTimeout();
-          if (timeout < min) {
-            min = timeout;
-          }
+        final int timeout = c.getConf().getHdfsTimeout();
+        if (timeout > 0 && timeout < min) {
+          min = timeout;
         }
       }
       renewal = min/2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffe48015/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
index ec2223f..5a929fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
@@ -40,8 +40,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSClient.Conf;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -178,12 +178,12 @@ public class NameNodeProxies {
           UserGroupInformation.getCurrentUser(), true, fallbackToSimpleAuth);
     } else {
       // HA case
-      Conf config = new Conf(conf);
+      DfsClientConf config = new DfsClientConf(conf);
       T proxy = (T) RetryProxy.create(xface, failoverProxyProvider,
           RetryPolicies.failoverOnNetworkException(
-              RetryPolicies.TRY_ONCE_THEN_FAIL, config.maxFailoverAttempts,
-              config.maxRetryAttempts, config.failoverSleepBaseMillis,
-              config.failoverSleepMaxMillis));
+              RetryPolicies.TRY_ONCE_THEN_FAIL, config.getMaxFailoverAttempts(),
+              config.getMaxRetryAttempts(), config.getFailoverSleepBaseMillis(),
+              config.getFailoverSleepMaxMillis()));
 
       Text dtService;
       if (failoverProxyProvider.useLogicalURI()) {


[24/50] [abbrv] hadoop git commit: HDFS-6666. Abort NameNode and DataNode startup if security is enabled but block access token is not enabled. Contributed by Vijay Bhat.

Posted by zj...@apache.org.
HDFS-6666. Abort NameNode and DataNode startup if security is enabled but block access token is not enabled. Contributed by Vijay Bhat.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/904cf42b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/904cf42b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/904cf42b

Branch: refs/heads/YARN-2928
Commit: 904cf42bf0960d21aae74c8f98d2ca06e60a9351
Parents: ef4baa3
Author: cnauroth <cn...@apache.org>
Authored: Tue Apr 14 09:59:01 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:43 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../server/blockmanagement/BlockManager.java    |  7 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   | 14 +++
 .../hdfs/server/namenode/FSNamesystem.java      | 10 ++-
 .../sasl/SaslDataTransferTestCase.java          | 41 +++++++--
 .../server/namenode/TestSecureNameNode.java     | 90 ++++++++++++--------
 6 files changed, 115 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/904cf42b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7414d33..f9b27da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -494,6 +494,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7701. Support reporting per storage type quota and usage
     with hadoop/hdfs shell. (Peter Shi via Arpit Agarwal)
 
+    HDFS-6666. Abort NameNode and DataNode startup if security is enabled but
+    block access token is not enabled. (Vijay Bhat via cnauroth)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/904cf42b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 8540dc1..4c9582c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -359,7 +359,7 @@ public class BlockManager {
   }
 
   private static BlockTokenSecretManager createBlockTokenSecretManager(
-      final Configuration conf) {
+      final Configuration conf) throws IOException {
     final boolean isEnabled = conf.getBoolean(
         DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, 
         DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT);
@@ -367,10 +367,11 @@ public class BlockManager {
 
     if (!isEnabled) {
       if (UserGroupInformation.isSecurityEnabled()) {
-        LOG.error("Security is enabled but block access tokens " +
+        String errMessage = "Security is enabled but block access tokens " +
             "(via " + DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY + ") " +
             "aren't enabled. This may cause issues " +
-            "when clients attempt to talk to a DataNode.");
+            "when clients attempt to connect to a DataNode. Aborting NameNode";
+        throw new IOException(errMessage);
       }
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/904cf42b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index b0e79e3..cb22b73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1183,6 +1183,20 @@ public class DataNode extends ReconfigurableBase
     if (!UserGroupInformation.isSecurityEnabled()) {
       return;
     }
+
+    // Abort out of inconsistent state if Kerberos is enabled
+    // but block access tokens are not enabled.
+    boolean isEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY,
+        DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT);
+    if (!isEnabled) {
+      String errMessage = "Security is enabled but block access tokens " +
+          "(via " + DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY + ") " +
+          "aren't enabled. This may cause issues " +
+          "when clients attempt to connect to a DataNode. Aborting DataNode";
+      throw new RuntimeException(errMessage);
+    }
+
     SaslPropertiesResolver saslPropsResolver = dnConf.getSaslPropsResolver();
     if (resources != null && saslPropsResolver == null) {
       return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/904cf42b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 449b7e0..35abb5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1242,10 +1242,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         cacheManager.stopMonitorThread();
         cacheManager.clearDirectiveStats();
       }
-      blockManager.getDatanodeManager().clearPendingCachingCommands();
-      blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
-      // Don't want to keep replication queues when not in Active.
-      blockManager.clearQueues();
+      if (blockManager != null) {
+        blockManager.getDatanodeManager().clearPendingCachingCommands();
+        blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
+        // Don't want to keep replication queues when not in Active.
+        blockManager.clearQueues();
+      }
       initializedReplQueues = false;
     } finally {
       writeUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/904cf42b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
index 6f2b3aa..98e4b1e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
@@ -33,6 +33,7 @@ import static org.junit.Assert.*;
 import java.io.File;
 import java.util.Properties;
 
+import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.http.HttpConfig;
@@ -48,10 +49,28 @@ public abstract class SaslDataTransferTestCase {
 
   private static File baseDir;
   private static String hdfsPrincipal;
+  private static String userPrincipal;
   private static MiniKdc kdc;
-  private static String keytab;
+  private static String hdfsKeytab;
+  private static String userKeyTab;
   private static String spnegoPrincipal;
 
+  public static String getUserKeyTab() {
+    return userKeyTab;
+  }
+
+  public static String getUserPrincipal() {
+    return userPrincipal;
+  }
+
+  public static String getHdfsPrincipal() {
+    return hdfsPrincipal;
+  }
+
+  public static String getHdfsKeytab() {
+    return hdfsKeytab;
+  }
+
   @BeforeClass
   public static void initKdc() throws Exception {
     baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
@@ -63,11 +82,17 @@ public abstract class SaslDataTransferTestCase {
     kdc = new MiniKdc(kdcConf, baseDir);
     kdc.start();
 
-    String userName = UserGroupInformation.getLoginUser().getShortUserName();
-    File keytabFile = new File(baseDir, userName + ".keytab");
-    keytab = keytabFile.getAbsolutePath();
-    kdc.createPrincipal(keytabFile, userName + "/localhost", "HTTP/localhost");
-    hdfsPrincipal = userName + "/localhost@" + kdc.getRealm();
+    String userName = RandomStringUtils.randomAlphabetic(8);
+    File userKeytabFile = new File(baseDir, userName + ".keytab");
+    userKeyTab = userKeytabFile.getAbsolutePath();
+    kdc.createPrincipal(userKeytabFile, userName + "/localhost");
+    userPrincipal = userName + "/localhost@" + kdc.getRealm();
+
+    String superUserName = "hdfs";
+    File hdfsKeytabFile = new File(baseDir, superUserName + ".keytab");
+    hdfsKeytab = hdfsKeytabFile.getAbsolutePath();
+    kdc.createPrincipal(hdfsKeytabFile, superUserName + "/localhost", "HTTP/localhost");
+    hdfsPrincipal = superUserName + "/localhost@" + kdc.getRealm();
     spnegoPrincipal = "HTTP/localhost@" + kdc.getRealm();
   }
 
@@ -91,9 +116,9 @@ public abstract class SaslDataTransferTestCase {
     HdfsConfiguration conf = new HdfsConfiguration();
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
     conf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
-    conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, hdfsKeytab);
     conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
-    conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, hdfsKeytab);
     conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoPrincipal);
     conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     conf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, dataTransferProtection);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/904cf42b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java
index 100dcb2e..6b6ce53 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java
@@ -24,71 +24,63 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.security.TestUGIWithSecurityOn;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferTestCase;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import org.junit.Assume;
-import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
-public class TestSecureNameNode {
+
+public class TestSecureNameNode extends SaslDataTransferTestCase {
   final static private int NUM_OF_DATANODES = 0;
 
-  @Before
-  public void testKdcRunning() {
-    // Tests are skipped if KDC is not running
-    Assume.assumeTrue(TestUGIWithSecurityOn.isKdcRunning());
-  }
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
 
   @Test
-  public void testName() throws IOException, InterruptedException {
+  public void testName() throws Exception {
     MiniDFSCluster cluster = null;
+    HdfsConfiguration conf = createSecureConfig(
+        "authentication,privacy");
     try {
-      String keyTabDir = System.getProperty("kdc.resource.dir") + "/keytabs";
-      String nn1KeytabPath = keyTabDir + "/nn1.keytab";
-      String user1KeyTabPath = keyTabDir + "/user1.keytab";
-      Configuration conf = new HdfsConfiguration();
-      conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
-          "kerberos");
-      conf.set(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY,
-          "nn1/localhost@EXAMPLE.COM");
-      conf.set(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY, nn1KeytabPath);
-
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_OF_DATANODES)
           .build();
       final MiniDFSCluster clusterRef = cluster;
       cluster.waitActive();
-      FileSystem fsForCurrentUser = cluster.getFileSystem();
-      fsForCurrentUser.mkdirs(new Path("/tmp"));
-      fsForCurrentUser.setPermission(new Path("/tmp"), new FsPermission(
+      FileSystem fsForSuperUser = UserGroupInformation
+          .loginUserFromKeytabAndReturnUGI(getHdfsPrincipal(), getHdfsKeytab()).doAs(new PrivilegedExceptionAction<FileSystem>() {
+            @Override
+            public FileSystem run() throws Exception {
+              return clusterRef.getFileSystem();
+            }
+          });
+      fsForSuperUser.mkdirs(new Path("/tmp"));
+      fsForSuperUser.setPermission(new Path("/tmp"), new FsPermission(
           (short) 511));
 
       UserGroupInformation ugi = UserGroupInformation
-          .loginUserFromKeytabAndReturnUGI("user1@EXAMPLE.COM", user1KeyTabPath);
+          .loginUserFromKeytabAndReturnUGI(getUserPrincipal(), getUserKeyTab());
       FileSystem fs = ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
         @Override
         public FileSystem run() throws Exception {
           return clusterRef.getFileSystem();
         }
       });
-      try {
-        Path p = new Path("/users");
-        fs.mkdirs(p);
-        fail("user1 must not be allowed to write in /");
-      } catch (IOException expected) {
-      }
-
-      Path p = new Path("/tmp/alpha");
+      Path p = new Path("/mydir");
+      exception.expect(IOException.class);
       fs.mkdirs(p);
-      assertNotNull(fs.listStatus(p));
+
+      Path tmp = new Path("/tmp/alpha");
+      fs.mkdirs(tmp);
+      assertNotNull(fs.listStatus(tmp));
       assertEquals(AuthenticationMethod.KERBEROS,
           ugi.getAuthenticationMethod());
     } finally {
@@ -97,4 +89,32 @@ public class TestSecureNameNode {
       }
     }
   }
+
+  /**
+   * Verify the following scenario.
+   * 1. Kerberos is enabled.
+   * 2. HDFS block tokens are not enabled.
+   * 3. Start the NN.
+   * 4. NN should throw an IOException and abort
+   * @throws Exception
+   */
+  @Test
+  public void testKerberosHdfsBlockTokenInconsistencyNNStartup() throws Exception {
+    MiniDFSCluster dfsCluster = null;
+    HdfsConfiguration conf = createSecureConfig(
+        "authentication,privacy");
+    try {
+      conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, false);
+      exception.expect(IOException.class);
+      exception.expectMessage("Security is enabled but block access tokens");
+      dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      dfsCluster.waitActive();
+    } finally {
+      if (dfsCluster != null) {
+        dfsCluster.shutdown();
+      }
+    }
+    return;
+  }
+
 }


[17/50] [abbrv] hadoop git commit: YARN-3443. Create a 'ResourceHandler' subsystem to ease addition of support for new resource types on the NM. Contributed by Sidharta Seethana.

Posted by zj...@apache.org.
YARN-3443. Create a 'ResourceHandler' subsystem to ease addition of support for new resource types on the NM. Contributed by Sidharta Seethana.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ae7a5ffd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ae7a5ffd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ae7a5ffd

Branch: refs/heads/YARN-2928
Commit: ae7a5ffdc2a9f1a09675f0d5fff10481bdc59e77
Parents: 0b8b407
Author: Junping Du <ju...@apache.org>
Authored: Mon Apr 13 18:35:56 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:42 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../linux/privileged/PrivilegedOperation.java   | 119 +++++
 .../PrivilegedOperationException.java           |  43 ++
 .../privileged/PrivilegedOperationExecutor.java | 255 +++++++++++
 .../linux/resources/CGroupsHandler.java         | 132 ++++++
 .../linux/resources/CGroupsHandlerImpl.java     | 436 +++++++++++++++++++
 .../linux/resources/ResourceHandler.java        |  91 ++++
 .../linux/resources/ResourceHandlerChain.java   | 142 ++++++
 .../resources/ResourceHandlerException.java     |  47 ++
 .../TestPrivilegedOperationExecutor.java        | 233 ++++++++++
 .../linux/resources/TestCGroupsHandlerImpl.java | 235 ++++++++++
 11 files changed, 1736 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1214dfd..517b55f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -128,6 +128,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3347. Improve YARN log command to get AMContainer logs as well as 
     running containers logs. (Xuan Gong via junping_du)
 
+    YARN-3443. Create a 'ResourceHandler' subsystem to ease addition of support 
+    for new resource types on the NM. (Sidharta Seethana via junping_du)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
new file mode 100644
index 0000000..74556a8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
@@ -0,0 +1,119 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Represents operations that require higher system privileges - e.g
+ * creating cgroups, launching containers as specified users, 'tc' commands etc
+ * that are completed using the container-executor binary
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class PrivilegedOperation {
+
+  public enum OperationType {
+    CHECK_SETUP("--checksetup"),
+    MOUNT_CGROUPS("--mount-cgroups"),
+    INITIALIZE_CONTAINER(""), //no CLI switch supported yet
+    LAUNCH_CONTAINER(""), //no CLI switch supported yet
+    SIGNAL_CONTAINER(""), //no CLI switch supported yet
+    DELETE_AS_USER(""), //no CLI switch supported yet
+    TC_MODIFY_STATE("--tc-modify-state"),
+    TC_READ_STATE("--tc-read-state"),
+    TC_READ_STATS("--tc-read-stats"),
+    ADD_PID_TO_CGROUP(""); //no CLI switch supported yet.
+
+    private final String option;
+
+    OperationType(String option) {
+      this.option = option;
+    }
+
+    public String getOption() {
+      return option;
+    }
+  }
+
+  public static final String CGROUP_ARG_PREFIX = "cgroups=";
+
+  private final OperationType opType;
+  private final List<String> args;
+
+  public PrivilegedOperation(OperationType opType, String arg) {
+    this.opType = opType;
+    this.args = new ArrayList<String>();
+
+    if (arg != null) {
+      this.args.add(arg);
+    }
+  }
+
+  public PrivilegedOperation(OperationType opType, List<String> args) {
+    this.opType = opType;
+    this.args = new ArrayList<String>();
+
+    if (args != null) {
+      this.args.addAll(args);
+    }
+  }
+
+  public void appendArgs(String... args) {
+    for (String arg : args) {
+      this.args.add(arg);
+    }
+  }
+
+  public void appendArgs(List<String> args) {
+    this.args.addAll(args);
+  }
+
+  public OperationType getOperationType() {
+    return opType;
+  }
+
+  public List<String> getArguments() {
+    return Collections.unmodifiableList(this.args);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null || !(other instanceof PrivilegedOperation)) {
+      return false;
+    }
+
+    PrivilegedOperation otherOp = (PrivilegedOperation) other;
+
+    return otherOp.opType.equals(opType) && otherOp.args.equals(args);
+  }
+
+  @Override
+  public int hashCode() {
+    return opType.hashCode() + 97 * args.hashCode();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperationException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperationException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperationException.java
new file mode 100644
index 0000000..20c234d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperationException.java
@@ -0,0 +1,43 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+public class PrivilegedOperationException extends YarnException {
+  private static final long serialVersionUID = 1L;
+
+  public PrivilegedOperationException() {
+    super();
+  }
+
+  public PrivilegedOperationException(String message) {
+    super(message);
+  }
+
+  public PrivilegedOperationException(Throwable cause) {
+    super(cause);
+  }
+
+  public PrivilegedOperationException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperationExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperationExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperationExecutor.java
new file mode 100644
index 0000000..1c4a51c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperationExecutor.java
@@ -0,0 +1,255 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.util.Shell.ExitCodeException;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * provides mechanisms to execute PrivilegedContainerOperations *
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class PrivilegedOperationExecutor {
+  private static final Log LOG = LogFactory.getLog(PrivilegedOperationExecutor
+      .class);
+  private volatile static PrivilegedOperationExecutor instance;
+
+  private String containerExecutorExe;
+
+  public static String getContainerExecutorExecutablePath(Configuration conf) {
+    String yarnHomeEnvVar =
+        System.getenv(ApplicationConstants.Environment.HADOOP_YARN_HOME.key());
+    File hadoopBin = new File(yarnHomeEnvVar, "bin");
+    String defaultPath =
+        new File(hadoopBin, "container-executor").getAbsolutePath();
+    return null == conf
+        ? defaultPath
+        : conf.get(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH,
+        defaultPath);
+  }
+
+  private void init(Configuration conf) {
+    containerExecutorExe = getContainerExecutorExecutablePath(conf);
+  }
+
+  private PrivilegedOperationExecutor(Configuration conf) {
+    init(conf);
+  }
+
+  public static PrivilegedOperationExecutor getInstance(Configuration conf) {
+    if (instance == null) {
+      synchronized (PrivilegedOperationExecutor.class) {
+        if (instance == null) {
+          instance = new PrivilegedOperationExecutor(conf);
+        }
+      }
+    }
+
+    return instance;
+  }
+
+  /**
+   * @param prefixCommands in some cases ( e.g priorities using nice ),
+   *                       prefix commands are necessary
+   * @param operation      the type and arguments for the operation to be
+   *                       executed
+   * @return execution string array for priviledged operation
+   */
+
+  public String[] getPrivilegedOperationExecutionCommand(List<String>
+      prefixCommands,
+      PrivilegedOperation operation) {
+    List<String> fullCommand = new ArrayList<String>();
+
+    if (prefixCommands != null && !prefixCommands.isEmpty()) {
+      fullCommand.addAll(prefixCommands);
+    }
+
+    fullCommand.add(containerExecutorExe);
+    fullCommand.add(operation.getOperationType().getOption());
+    fullCommand.addAll(operation.getArguments());
+
+    String[] fullCommandArray =
+        fullCommand.toArray(new String[fullCommand.size()]);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Privileged Execution Command Array: " +
+          Arrays.toString(fullCommandArray));
+    }
+
+    return fullCommandArray;
+  }
+
+  /**
+   * Executes a privileged operation. It is up to the callers to ensure that
+   * each privileged operation's parameters are constructed correctly. The
+   * parameters are passed verbatim to the container-executor binary.
+   *
+   * @param prefixCommands in some cases ( e.g priorities using nice ),
+   *                       prefix commands are necessary
+   * @param operation      the type and arguments for the operation to be executed
+   * @param workingDir     (optional) working directory for execution
+   * @param env            (optional) env of the command will include specified vars
+   * @param grabOutput     return (possibly large) shell command output
+   * @return stdout contents from shell executor - useful for some privileged
+   * operations - e.g --tc_read
+   * @throws org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException
+   */
+  public String executePrivilegedOperation(List<String> prefixCommands,
+      PrivilegedOperation operation, File workingDir,
+      Map<String, String> env, boolean grabOutput)
+      throws PrivilegedOperationException {
+    String[] fullCommandArray = getPrivilegedOperationExecutionCommand
+        (prefixCommands, operation);
+    ShellCommandExecutor exec = new ShellCommandExecutor(fullCommandArray,
+        workingDir, env);
+
+    try {
+      exec.execute();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Privileged Execution Operation Output:");
+        LOG.debug(exec.getOutput());
+      }
+    } catch (ExitCodeException e) {
+      String logLine = new StringBuffer("Shell execution returned exit code: ")
+          .append(exec.getExitCode())
+          .append(". Privileged Execution Operation Output: ")
+          .append(System.lineSeparator()).append(exec.getOutput()).toString();
+
+      LOG.warn(logLine);
+      throw new PrivilegedOperationException(e);
+    } catch (IOException e) {
+      LOG.warn("IOException executing command: ", e);
+      throw new PrivilegedOperationException(e);
+    }
+
+    if (grabOutput) {
+      return exec.getOutput();
+    }
+
+    return null;
+  }
+
+  /**
+   * Executes a privileged operation. It is up to the callers to ensure that
+   * each privileged operation's parameters are constructed correctly. The
+   * parameters are passed verbatim to the container-executor binary.
+   *
+   * @param operation  the type and arguments for the operation to be executed
+   * @param grabOutput return (possibly large) shell command output
+   * @return stdout contents from shell executor - useful for some privileged
+   * operations - e.g --tc_read
+   * @throws org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException
+   */
+  public String executePrivilegedOperation(PrivilegedOperation operation,
+      boolean grabOutput) throws PrivilegedOperationException {
+    return executePrivilegedOperation(null, operation, null, null, grabOutput);
+  }
+
+  //Utility functions for squashing together operations in supported ways
+  //At some point, we need to create a generalized mechanism that uses a set
+  //of squashing 'rules' to squash an set of PrivilegedOperations of varying
+  //types - e.g Launch Container + Add Pid to CGroup(s) + TC rules
+
+  /**
+   * Squash operations for cgroups - e.g mount, add pid to cgroup etc .,
+   * For now, we only implement squashing for 'add pid to cgroup' since this
+   * is the only optimization relevant to launching containers
+   *
+   * @return single squashed cgroup operation. Null on failure.
+   */
+
+  public static PrivilegedOperation squashCGroupOperations
+  (List<PrivilegedOperation> ops) throws PrivilegedOperationException {
+    if (ops.size() == 0) {
+      return null;
+    }
+
+    StringBuffer finalOpArg = new StringBuffer(PrivilegedOperation
+        .CGROUP_ARG_PREFIX);
+    boolean noneArgsOnly = true;
+
+    for (PrivilegedOperation op : ops) {
+      if (!op.getOperationType()
+          .equals(PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP)) {
+        LOG.warn("Unsupported operation type: " + op.getOperationType());
+        throw new PrivilegedOperationException("Unsupported operation type:"
+            + op.getOperationType());
+      }
+
+      List<String> args = op.getArguments();
+      if (args.size() != 1) {
+        LOG.warn("Invalid number of args: " + args.size());
+        throw new PrivilegedOperationException("Invalid number of args: "
+            + args.size());
+      }
+
+      String arg = args.get(0);
+      String tasksFile = StringUtils.substringAfter(arg,
+          PrivilegedOperation.CGROUP_ARG_PREFIX);
+      if (tasksFile == null || tasksFile.isEmpty()) {
+        LOG.warn("Invalid argument: " + arg);
+        throw new PrivilegedOperationException("Invalid argument: " + arg);
+      }
+
+      if (tasksFile.equals("none")) {
+        //Don't append to finalOpArg
+        continue;
+      }
+
+      if (noneArgsOnly == false) {
+        //We have already appended at least one tasks file.
+        finalOpArg.append(",");
+        finalOpArg.append(tasksFile);
+      } else {
+        finalOpArg.append(tasksFile);
+        noneArgsOnly = false;
+      }
+    }
+
+    if (noneArgsOnly) {
+      finalOpArg.append("none"); //there were no tasks file to append
+    }
+
+    PrivilegedOperation finalOp = new PrivilegedOperation(
+        PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP, finalOpArg
+        .toString());
+
+    return finalOp;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
new file mode 100644
index 0000000..34429d3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
@@ -0,0 +1,132 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Provides CGroups functionality. Implementations are expected to be
+ * thread-safe
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface CGroupsHandler {
+  public enum CGroupController {
+    CPU("cpu"),
+    NET_CLS("net_cls");
+
+    private final String name;
+
+    CGroupController(String name) {
+      this.name = name;
+    }
+
+    String getName() {
+      return name;
+    }
+  }
+
+  public static final String CGROUP_FILE_TASKS = "tasks";
+  public static final String CGROUP_PARAM_CLASSID = "classid";
+
+  /**
+   * Mounts a cgroup controller
+   * @param controller - the controller being mounted
+   * @throws ResourceHandlerException
+   */
+  public void mountCGroupController(CGroupController controller)
+      throws ResourceHandlerException;
+
+  /**
+   * Creates a cgroup for a given controller
+   * @param controller - controller type for which the cgroup is being created
+   * @param cGroupId - id of the cgroup being created
+   * @return full path to created cgroup
+   * @throws ResourceHandlerException
+   */
+  public String createCGroup(CGroupController controller, String cGroupId)
+      throws ResourceHandlerException;
+
+  /**
+   * Deletes the specified cgroup
+   * @param controller - controller type for the cgroup
+   * @param cGroupId - id of the cgroup being deleted
+   * @throws ResourceHandlerException
+   */
+  public void deleteCGroup(CGroupController controller, String cGroupId) throws
+      ResourceHandlerException;
+
+  /**
+   * Gets the full path for the cgroup, given a controller and a cgroup id
+   * @param controller - controller type for the cgroup
+   * @param cGroupId - id of the cgroup
+   * @return full path for the cgroup
+   */
+  public String getPathForCGroup(CGroupController controller, String
+      cGroupId);
+
+  /**
+   * Gets the full path for the cgroup's tasks file, given a controller and a
+   * cgroup id
+   * @param controller - controller type for the cgroup
+   * @param cGroupId - id of the cgroup
+   * @return full path for the cgroup's tasks file
+   */
+  public String getPathForCGroupTasks(CGroupController controller, String
+      cGroupId);
+
+  /**
+   * Gets the full path for a cgroup parameter, given a controller,
+   * cgroup id and parameter name
+   * @param controller - controller type for the cgroup
+   * @param cGroupId - id of the cgroup
+   * @param param - cgroup parameter ( e.g classid )
+   * @return full path for the cgroup parameter
+   */
+  public String getPathForCGroupParam(CGroupController controller, String
+      cGroupId, String param);
+
+  /**
+   * updates a cgroup parameter, given a controller, cgroup id, parameter name
+   * and a parameter value
+   * @param controller - controller type for the cgroup
+   * @param cGroupId - id of the cgroup
+   * @param param - cgroup parameter ( e.g classid )
+   * @param value - value to be written to the parameter file
+   * @throws ResourceHandlerException
+   */
+  public void updateCGroupParam(CGroupController controller, String cGroupId,
+      String param, String value) throws ResourceHandlerException;
+
+  /**
+   * reads a cgroup parameter value, given a controller, cgroup id, parameter
+   * name
+   * @param controller - controller type for the cgroup
+   * @param cGroupId - id of the cgroup
+   * @param param - cgroup parameter ( e.g classid )
+   * @return parameter value as read from the parameter file
+   * @throws ResourceHandlerException
+   */
+  public String getCGroupParam(CGroupController controller, String cGroupId,
+      String param) throws ResourceHandlerException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
new file mode 100644
index 0000000..9a4230f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
@@ -0,0 +1,436 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+
+import java.io.*;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Support for interacting with various CGroup subsystems. Thread-safe.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class CGroupsHandlerImpl implements CGroupsHandler {
+
+  private static final Log LOG = LogFactory.getLog(CGroupsHandlerImpl.class);
+  private static final String MTAB_FILE = "/proc/mounts";
+  private static final String CGROUPS_FSTYPE = "cgroup";
+
+  private final String cGroupPrefix;
+  private final boolean enableCGroupMount;
+  private final String cGroupMountPath;
+  private final long deleteCGroupTimeout;
+  private final long deleteCGroupDelay;
+  private final Map<CGroupController, String> controllerPaths;
+  private final ReadWriteLock rwLock;
+  private final PrivilegedOperationExecutor privilegedOperationExecutor;
+  private final Clock clock;
+
+  public CGroupsHandlerImpl(Configuration conf, PrivilegedOperationExecutor
+      privilegedOperationExecutor) throws ResourceHandlerException {
+    this.cGroupPrefix = conf.get(YarnConfiguration.
+        NM_LINUX_CONTAINER_CGROUPS_HIERARCHY, "/hadoop-yarn")
+        .replaceAll("^/", "").replaceAll("$/", "");
+    this.enableCGroupMount = conf.getBoolean(YarnConfiguration.
+        NM_LINUX_CONTAINER_CGROUPS_MOUNT, false);
+    this.cGroupMountPath = conf.get(YarnConfiguration.
+        NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH, null);
+    this.deleteCGroupTimeout = conf.getLong(
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_DELETE_TIMEOUT,
+        YarnConfiguration.DEFAULT_NM_LINUX_CONTAINER_CGROUPS_DELETE_TIMEOUT);
+    this.deleteCGroupDelay =
+        conf.getLong(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_DELETE_DELAY,
+            YarnConfiguration.DEFAULT_NM_LINUX_CONTAINER_CGROUPS_DELETE_DELAY);
+    this.controllerPaths = new HashMap<>();
+    this.rwLock = new ReentrantReadWriteLock();
+    this.privilegedOperationExecutor = privilegedOperationExecutor;
+    this.clock = new SystemClock();
+
+    init();
+  }
+
+  private void init() throws ResourceHandlerException {
+    initializeControllerPaths();
+  }
+
+  private String getControllerPath(CGroupController controller) {
+    try {
+      rwLock.readLock().lock();
+      return controllerPaths.get(controller);
+    } finally {
+      rwLock.readLock().unlock();
+    }
+  }
+
+  private void initializeControllerPaths() throws ResourceHandlerException {
+    if (enableCGroupMount) {
+      //nothing to do here - we support 'deferred' mounting of specific
+      //controllers - we'll populate the path for a given controller when an
+      //explicit mountCGroupController request is issued.
+      LOG.info("CGroup controller mounting enabled.");
+    } else {
+      //cluster admins are expected to have mounted controllers in specific
+      //locations - we'll attempt to figure out mount points
+      initializeControllerPathsFromMtab();
+    }
+  }
+
+  private void initializeControllerPathsFromMtab()
+      throws ResourceHandlerException {
+    try {
+      Map<String, List<String>> parsedMtab = parseMtab();
+
+      //we want to do a bulk update without the paths changing concurrently
+      rwLock.writeLock().lock();
+
+      for (CGroupController controller : CGroupController.values()) {
+        String name = controller.getName();
+        String controllerPath = findControllerInMtab(name, parsedMtab);
+
+        if (controllerPath != null) {
+          File f = new File(controllerPath + "/" + this.cGroupPrefix);
+
+          if (FileUtil.canWrite(f)) {
+            controllerPaths.put(controller, controllerPath);
+          } else {
+            String error =
+                new StringBuffer("Mount point Based on mtab file: ")
+                    .append(MTAB_FILE).append(
+                    ". Controller mount point not writable for: ")
+                    .append(name).toString();
+
+            LOG.error(error);
+            throw new ResourceHandlerException(error);
+          }
+        } else {
+
+            LOG.warn("Controller not mounted but automount disabled: " + name);
+        }
+      }
+    } catch (IOException e) {
+      LOG.warn("Failed to initialize controller paths! Exception: " + e);
+      throw new ResourceHandlerException(
+          "Failed to initialize controller paths!");
+    } finally {
+      rwLock.writeLock().unlock();
+    }
+  }
+
+  /* We are looking for entries of the form:
+   * none /cgroup/path/mem cgroup rw,memory 0 0
+   *
+   * Use a simple pattern that splits on the five spaces, and
+   * grabs the 2, 3, and 4th fields.
+   */
+
+  private static final Pattern MTAB_FILE_FORMAT = Pattern.compile(
+      "^[^\\s]+\\s([^\\s]+)\\s([^\\s]+)\\s([^\\s]+)\\s[^\\s]+\\s[^\\s]+$");
+
+  /*
+   * Returns a map: path -> mount options
+   * for mounts with type "cgroup". Cgroup controllers will
+   * appear in the list of options for a path.
+   */
+  private Map<String, List<String>> parseMtab() throws IOException {
+    Map<String, List<String>> ret = new HashMap<String, List<String>>();
+    BufferedReader in = null;
+
+    try {
+      FileInputStream fis = new FileInputStream(new File(getMtabFileName()));
+      in = new BufferedReader(new InputStreamReader(fis, "UTF-8"));
+
+      for (String str = in.readLine(); str != null;
+           str = in.readLine()) {
+        Matcher m = MTAB_FILE_FORMAT.matcher(str);
+        boolean mat = m.find();
+        if (mat) {
+          String path = m.group(1);
+          String type = m.group(2);
+          String options = m.group(3);
+
+          if (type.equals(CGROUPS_FSTYPE)) {
+            List<String> value = Arrays.asList(options.split(","));
+            ret.put(path, value);
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new IOException("Error while reading " + getMtabFileName(), e);
+    } finally {
+      IOUtils.cleanup(LOG, in);
+    }
+
+    return ret;
+  }
+
+  private String findControllerInMtab(String controller,
+      Map<String, List<String>> entries) {
+    for (Map.Entry<String, List<String>> e : entries.entrySet()) {
+      if (e.getValue().contains(controller))
+        return e.getKey();
+    }
+
+    return null;
+  }
+
+  String getMtabFileName() {
+    return MTAB_FILE;
+  }
+
+  @Override
+  public void mountCGroupController(CGroupController controller)
+      throws ResourceHandlerException {
+    if (!enableCGroupMount) {
+      LOG.warn("CGroup mounting is disabled - ignoring mount request for: " +
+          controller.getName());
+      return;
+    }
+
+    String path = getControllerPath(controller);
+
+    if (path == null) {
+      try {
+        //lock out other readers/writers till we are done
+        rwLock.writeLock().lock();
+
+        String hierarchy = cGroupPrefix;
+        StringBuffer controllerPath = new StringBuffer()
+            .append(cGroupMountPath).append('/').append(controller.getName());
+        StringBuffer cGroupKV = new StringBuffer()
+            .append(controller.getName()).append('=').append(controllerPath);
+        PrivilegedOperation.OperationType opType = PrivilegedOperation
+            .OperationType.MOUNT_CGROUPS;
+        PrivilegedOperation op = new PrivilegedOperation(opType, (String) null);
+
+        op.appendArgs(hierarchy, cGroupKV.toString());
+        LOG.info("Mounting controller " + controller.getName() + " at " +
+              controllerPath);
+        privilegedOperationExecutor.executePrivilegedOperation(op, false);
+
+        //if privileged operation succeeds, update controller paths
+        controllerPaths.put(controller, controllerPath.toString());
+
+        return;
+      } catch (PrivilegedOperationException e) {
+        LOG.error("Failed to mount controller: " + controller.getName());
+        throw new ResourceHandlerException("Failed to mount controller: "
+            + controller.getName());
+      } finally {
+        rwLock.writeLock().unlock();
+      }
+    } else {
+      LOG.info("CGroup controller already mounted at: " + path);
+      return;
+    }
+  }
+
+  @Override
+  public String getPathForCGroup(CGroupController controller, String cGroupId) {
+    return new StringBuffer(getControllerPath(controller))
+        .append('/').append(cGroupPrefix).append("/")
+        .append(cGroupId).toString();
+  }
+
+  @Override
+  public String getPathForCGroupTasks(CGroupController controller,
+      String cGroupId) {
+    return new StringBuffer(getPathForCGroup(controller, cGroupId))
+        .append('/').append(CGROUP_FILE_TASKS).toString();
+  }
+
+  @Override
+  public String getPathForCGroupParam(CGroupController controller,
+      String cGroupId, String param) {
+    return new StringBuffer(getPathForCGroup(controller, cGroupId))
+        .append('/').append(controller.getName()).append('.')
+        .append(param).toString();
+  }
+
+  @Override
+  public String createCGroup(CGroupController controller, String cGroupId)
+      throws ResourceHandlerException {
+    String path = getPathForCGroup(controller, cGroupId);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("createCgroup: " + path);
+    }
+
+    if (!new File(path).mkdir()) {
+      throw new ResourceHandlerException("Failed to create cgroup at " + path);
+    }
+
+    return path;
+  }
+
+  /*
+  * Utility routine to print first line from cgroup tasks file
+  */
+  private void logLineFromTasksFile(File cgf) {
+    String str;
+    if (LOG.isDebugEnabled()) {
+      try (BufferedReader inl =
+          new BufferedReader(new InputStreamReader(new FileInputStream(cgf
+              + "/tasks"), "UTF-8"))) {
+        if ((str = inl.readLine()) != null) {
+          LOG.debug("First line in cgroup tasks file: " + cgf + " " + str);
+        }
+      } catch (IOException e) {
+        LOG.warn("Failed to read cgroup tasks file. ", e);
+      }
+    }
+  }
+
+  /**
+   * If tasks file is empty, delete the cgroup.
+   *
+   * @param cgf object referring to the cgroup to be deleted
+   * @return Boolean indicating whether cgroup was deleted
+   */
+  boolean checkAndDeleteCgroup(File cgf) throws InterruptedException {
+    boolean deleted = false;
+    // FileInputStream in = null;
+    try (FileInputStream in = new FileInputStream(cgf + "/tasks")) {
+      if (in.read() == -1) {
+        /*
+         * "tasks" file is empty, sleep a bit more and then try to delete the
+         * cgroup. Some versions of linux will occasionally panic due to a race
+         * condition in this area, hence the paranoia.
+         */
+        Thread.sleep(deleteCGroupDelay);
+        deleted = cgf.delete();
+        if (!deleted) {
+          LOG.warn("Failed attempt to delete cgroup: " + cgf);
+        }
+      } else {
+        logLineFromTasksFile(cgf);
+      }
+    } catch (IOException e) {
+      LOG.warn("Failed to read cgroup tasks file. ", e);
+    }
+    return deleted;
+  }
+
+  @Override
+  public void deleteCGroup(CGroupController controller, String cGroupId)
+      throws ResourceHandlerException {
+    boolean deleted = false;
+    String cGroupPath = getPathForCGroup(controller, cGroupId);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("deleteCGroup: " + cGroupPath);
+    }
+
+    long start = clock.getTime();
+
+    do {
+      try {
+        deleted = checkAndDeleteCgroup(new File(cGroupPath));
+        if (!deleted) {
+          Thread.sleep(deleteCGroupDelay);
+        }
+      } catch (InterruptedException ex) {
+        // NOP
+      }
+    } while (!deleted && (clock.getTime() - start) < deleteCGroupTimeout);
+
+    if (!deleted) {
+      LOG.warn("Unable to delete  " + cGroupPath +
+          ", tried to delete for " + deleteCGroupTimeout + "ms");
+    }
+  }
+
+  @Override
+  public void updateCGroupParam(CGroupController controller, String cGroupId,
+      String param, String value) throws ResourceHandlerException {
+    String cGroupParamPath = getPathForCGroupParam(controller, cGroupId, param);
+    PrintWriter pw = null;
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          "updateCGroupParam for path: " + cGroupParamPath + " with value " +
+              value);
+    }
+
+    try {
+      File file = new File(cGroupParamPath);
+      Writer w = new OutputStreamWriter(new FileOutputStream(file), "UTF-8");
+      pw = new PrintWriter(w);
+      pw.write(value);
+    } catch (IOException e) {
+      throw new ResourceHandlerException(new StringBuffer("Unable to write to ")
+          .append(cGroupParamPath).append(" with value: ").append(value)
+          .toString(), e);
+    } finally {
+      if (pw != null) {
+        boolean hasError = pw.checkError();
+        pw.close();
+        if (hasError) {
+          throw new ResourceHandlerException(
+              new StringBuffer("Unable to write to ")
+                  .append(cGroupParamPath).append(" with value: ").append(value)
+                  .toString());
+        }
+        if (pw.checkError()) {
+          throw new ResourceHandlerException("Error while closing cgroup file" +
+              " " + cGroupParamPath);
+        }
+      }
+    }
+  }
+
+  @Override
+  public String getCGroupParam(CGroupController controller, String cGroupId,
+      String param) throws ResourceHandlerException {
+    String cGroupParamPath = getPathForCGroupParam(controller, cGroupId, param);
+
+    try {
+      byte[] contents = Files.readAllBytes(Paths.get(cGroupParamPath));
+      return new String(contents, "UTF-8").trim();
+    } catch (IOException e) {
+      throw new ResourceHandlerException(
+          "Unable to read from " + cGroupParamPath);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandler.java
new file mode 100644
index 0000000..3dfc86b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandler.java
@@ -0,0 +1,91 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+
+import java.util.List;
+
+/**
+ * Handler interface for resource subsystems' isolation and enforcement. e.g cpu, memory, network, disks etc
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface ResourceHandler {
+
+  /**
+   * Bootstrap resource susbsystem.
+   *
+   * @return (possibly empty) list of operations that require elevated
+   * privileges
+   */
+  List<PrivilegedOperation> bootstrap(Configuration configuration)
+      throws ResourceHandlerException;
+
+  /**
+   * Prepare a resource environment for container launch
+   *
+   * @param container Container being launched
+   * @return (possibly empty) list of operations that require elevated
+   * privileges e.g a) create a custom cgroup b) add pid for container to tasks
+   * file for a cgroup.
+   * @throws ResourceHandlerException
+   */
+  List<PrivilegedOperation> preStart(Container container)
+      throws ResourceHandlerException;
+
+  /**
+   * Require state for container that was already launched
+   *
+   * @param containerId if of the container being reacquired.
+   * @return (possibly empty) list of operations that require elevated
+   * privileges
+   * @throws ResourceHandlerException
+   */
+
+  List<PrivilegedOperation> reacquireContainer(ContainerId containerId)
+      throws ResourceHandlerException;
+
+  /**
+   * Perform any tasks necessary after container completion
+   * @param containerId of the container that was completed.
+   * @return (possibly empty) list of operations that require elevated
+   * privileges
+   * @throws ResourceHandlerException
+   */
+  List<PrivilegedOperation> postComplete(ContainerId containerId) throws
+      ResourceHandlerException;
+
+  /**
+   * Teardown environment for resource subsystem if requested. This method
+   * needs to be used with care since it could impact running containers.
+   *
+   * @return (possibly empty) list of operations that require elevated
+   * privileges
+   */
+  List<PrivilegedOperation> teardown() throws ResourceHandlerException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerChain.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerChain.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerChain.java
new file mode 100644
index 0000000..955d216
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerChain.java
@@ -0,0 +1,142 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A helper class to delegate funcationality to a 'chain' of
+ * ResourceHandler(s)
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ResourceHandlerChain implements ResourceHandler {
+  private final List<ResourceHandler> resourceHandlers;
+
+  public ResourceHandlerChain(List<ResourceHandler> resourceHandlers) {
+    this.resourceHandlers = resourceHandlers;
+  }
+
+  @Override
+  public List<PrivilegedOperation> bootstrap(Configuration configuration)
+      throws ResourceHandlerException {
+
+    List<PrivilegedOperation> allOperations = new
+        ArrayList<PrivilegedOperation>();
+
+    for (ResourceHandler resourceHandler : resourceHandlers) {
+      List<PrivilegedOperation> handlerOperations =
+          resourceHandler.bootstrap(configuration);
+      if (handlerOperations != null) {
+        allOperations.addAll(handlerOperations);
+      }
+
+    }
+    return allOperations;
+  }
+
+  @Override
+  public List<PrivilegedOperation> preStart(Container container)
+      throws ResourceHandlerException {
+    List<PrivilegedOperation> allOperations = new
+        ArrayList<PrivilegedOperation>();
+
+    for (ResourceHandler resourceHandler : resourceHandlers) {
+      List<PrivilegedOperation> handlerOperations =
+          resourceHandler.preStart(container);
+
+      if (handlerOperations != null) {
+        allOperations.addAll(handlerOperations);
+      }
+
+    }
+    return allOperations;
+  }
+
+  @Override
+  public List<PrivilegedOperation> reacquireContainer(ContainerId containerId)
+      throws ResourceHandlerException {
+    List<PrivilegedOperation> allOperations = new
+        ArrayList<PrivilegedOperation>();
+
+    for (ResourceHandler resourceHandler : resourceHandlers) {
+      List<PrivilegedOperation> handlerOperations =
+          resourceHandler.reacquireContainer(containerId);
+
+      if (handlerOperations != null) {
+        allOperations.addAll(handlerOperations);
+      }
+
+    }
+    return allOperations;
+  }
+
+  @Override
+  public List<PrivilegedOperation> postComplete(ContainerId containerId)
+      throws ResourceHandlerException {
+    List<PrivilegedOperation> allOperations = new
+        ArrayList<PrivilegedOperation>();
+
+    for (ResourceHandler resourceHandler : resourceHandlers) {
+      List<PrivilegedOperation> handlerOperations =
+          resourceHandler.postComplete(containerId);
+
+      if (handlerOperations != null) {
+        allOperations.addAll(handlerOperations);
+      }
+
+    }
+    return allOperations;
+  }
+
+  @Override
+  public List<PrivilegedOperation> teardown()
+      throws ResourceHandlerException {
+    List<PrivilegedOperation> allOperations = new
+        ArrayList<PrivilegedOperation>();
+
+    for (ResourceHandler resourceHandler : resourceHandlers) {
+      List<PrivilegedOperation> handlerOperations =
+          resourceHandler.teardown();
+
+      if (handlerOperations != null) {
+        allOperations.addAll(handlerOperations);
+      }
+
+    }
+    return allOperations;
+  }
+
+  List<ResourceHandler> getResourceHandlerList() {
+    return Collections.unmodifiableList(resourceHandlers);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerException.java
new file mode 100644
index 0000000..3ab7548
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerException.java
@@ -0,0 +1,47 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ResourceHandlerException extends YarnException {
+  private static final long serialVersionUID = 1L;
+
+  public ResourceHandlerException() {
+    super();
+  }
+
+  public ResourceHandlerException(String message) {
+    super(message);
+  }
+
+  public ResourceHandlerException(Throwable cause) {
+    super(cause);
+  }
+
+  public ResourceHandlerException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java
new file mode 100644
index 0000000..7154d03
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java
@@ -0,0 +1,233 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class TestPrivilegedOperationExecutor {
+  private static final Log LOG = LogFactory
+      .getLog(TestPrivilegedOperationExecutor.class);
+  private String localDataDir;
+  private String customExecutorPath;
+  private Configuration nullConf = null;
+  private Configuration emptyConf;
+  private Configuration confWithExecutorPath;
+
+  private String cGroupTasksNone;
+  private String cGroupTasksInvalid;
+  private String cGroupTasks1;
+  private String cGroupTasks2;
+  private String cGroupTasks3;
+  private PrivilegedOperation opDisallowed;
+  private PrivilegedOperation opTasksNone;
+  private PrivilegedOperation opTasksInvalid;
+  private PrivilegedOperation opTasks1;
+  private PrivilegedOperation opTasks2;
+  private PrivilegedOperation opTasks3;
+
+  @Before
+  public void setup() {
+    localDataDir = System.getProperty("test.build.data");
+    customExecutorPath = localDataDir + "/bin/container-executor";
+    emptyConf = new YarnConfiguration();
+    confWithExecutorPath = new YarnConfiguration();
+    confWithExecutorPath.set(YarnConfiguration
+        .NM_LINUX_CONTAINER_EXECUTOR_PATH, customExecutorPath);
+
+    cGroupTasksNone = "none";
+    cGroupTasksInvalid = "invalid_string";
+    cGroupTasks1 = "cpu/hadoop_yarn/container_01/tasks";
+    cGroupTasks2 = "net_cls/hadoop_yarn/container_01/tasks";
+    cGroupTasks3 = "blkio/hadoop_yarn/container_01/tasks";
+    opDisallowed = new PrivilegedOperation
+        (PrivilegedOperation.OperationType.DELETE_AS_USER, (String) null);
+    opTasksNone = new PrivilegedOperation
+        (PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+            PrivilegedOperation.CGROUP_ARG_PREFIX + cGroupTasksNone);
+    opTasksInvalid = new PrivilegedOperation
+        (PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+            cGroupTasksInvalid);
+    opTasks1 = new PrivilegedOperation
+        (PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+            PrivilegedOperation.CGROUP_ARG_PREFIX + cGroupTasks1);
+    opTasks2 = new PrivilegedOperation
+        (PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+            PrivilegedOperation.CGROUP_ARG_PREFIX + cGroupTasks2);
+    opTasks3 = new PrivilegedOperation
+        (PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+            PrivilegedOperation.CGROUP_ARG_PREFIX + cGroupTasks3);
+  }
+
+  @Test
+  public void testExecutorPath() {
+    String containerExePath = PrivilegedOperationExecutor
+        .getContainerExecutorExecutablePath(nullConf);
+
+    //In case HADOOP_YARN_HOME isn't set, CWD is used. If conf is null or
+    //NM_LINUX_CONTAINER_EXECUTOR_PATH is not set, then a defaultPath is
+    //constructed.
+    String yarnHomeEnvVar = System.getenv("HADOOP_YARN_HOME");
+    String yarnHome = yarnHomeEnvVar != null ? yarnHomeEnvVar
+        : new File("").getAbsolutePath();
+    String expectedPath = yarnHome + "/bin/container-executor";
+
+    Assert.assertEquals(expectedPath, containerExePath);
+
+    containerExePath = PrivilegedOperationExecutor
+        .getContainerExecutorExecutablePath(emptyConf);
+    Assert.assertEquals(expectedPath, containerExePath);
+
+    //if NM_LINUX_CONTAINER_EXECUTOR_PATH is set, this must be returned
+    expectedPath = customExecutorPath;
+    containerExePath = PrivilegedOperationExecutor
+        .getContainerExecutorExecutablePath(confWithExecutorPath);
+    Assert.assertEquals(expectedPath, containerExePath);
+  }
+
+  @Test
+  public void testExecutionCommand() {
+    PrivilegedOperationExecutor exec = PrivilegedOperationExecutor
+        .getInstance(confWithExecutorPath);
+    PrivilegedOperation op = new PrivilegedOperation(PrivilegedOperation
+        .OperationType.LAUNCH_CONTAINER, (String) null);
+    String[] cmdArray = exec.getPrivilegedOperationExecutionCommand(null, op);
+
+    //No arguments added - so the resulting array should consist of
+    //1)full path to executor 2) cli switch
+    Assert.assertEquals(2, cmdArray.length);
+    Assert.assertEquals(customExecutorPath, cmdArray[0]);
+    Assert.assertEquals(op.getOperationType().getOption(), cmdArray[1]);
+
+    //other (dummy) arguments to launch container
+    String[] additionalArgs = { "test_user", "yarn", "1", "app_01",
+        "container_01", "workdir", "launch_script.sh", "tokens", "pidfile",
+        "nm-local-dirs", "nm-log-dirs", "resource-spec" };
+
+    op.appendArgs(additionalArgs);
+    cmdArray = exec.getPrivilegedOperationExecutionCommand(null, op);
+
+    //Resulting array should be of length 2 greater than the number of
+    //additional arguments added.
+
+    Assert.assertEquals(2 + additionalArgs.length, cmdArray.length);
+    Assert.assertEquals(customExecutorPath, cmdArray[0]);
+    Assert.assertEquals(op.getOperationType().getOption(), cmdArray[1]);
+
+    //Rest of args should be same as additional args.
+    for (int i = 0; i < additionalArgs.length; ++i) {
+      Assert.assertEquals(additionalArgs[i], cmdArray[2 + i]);
+    }
+
+    //Now test prefix commands
+    List<String> prefixCommands = Arrays.asList("nice", "-10");
+    cmdArray = exec.getPrivilegedOperationExecutionCommand(prefixCommands, op);
+    int prefixLength = prefixCommands.size();
+    //Resulting array should be of length of prefix command args + 2 (exec
+    // path + switch) + length of additional args.
+    Assert.assertEquals(prefixLength + 2 + additionalArgs.length,
+        cmdArray.length);
+
+    //Prefix command array comes first
+    for (int i = 0; i < prefixLength; ++i) {
+      Assert.assertEquals(prefixCommands.get(i), cmdArray[i]);
+    }
+
+    //Followed by the container executor path and the cli switch
+    Assert.assertEquals(customExecutorPath, cmdArray[prefixLength]);
+    Assert.assertEquals(op.getOperationType().getOption(),
+        cmdArray[prefixLength + 1]);
+
+    //Followed by the rest of the args
+    //Rest of args should be same as additional args.
+    for (int i = 0; i < additionalArgs.length; ++i) {
+      Assert.assertEquals(additionalArgs[i], cmdArray[prefixLength + 2 + i]);
+    }
+  }
+
+  @Test
+  public void testSquashCGroupOperationsWithInvalidOperations() {
+    List<PrivilegedOperation> ops = new ArrayList<>();
+
+    //Ensure that disallowed ops are rejected
+    ops.add(opTasksNone);
+    ops.add(opDisallowed);
+
+    try {
+      PrivilegedOperationExecutor.squashCGroupOperations(ops);
+      Assert.fail("Expected squash operation to fail with an exception!");
+    } catch (PrivilegedOperationException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+
+    //Ensure that invalid strings are rejected
+    ops.clear();
+    ops.add(opTasksNone);
+    ops.add(opTasksInvalid);
+
+    try {
+      PrivilegedOperationExecutor.squashCGroupOperations(ops);
+      Assert.fail("Expected squash operation to fail with an exception!");
+    } catch (PrivilegedOperationException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
+  public void testSquashCGroupOperationsWithValidOperations() {
+    List<PrivilegedOperation> ops = new ArrayList<>();
+    //Test squashing, including 'none'
+    ops.clear();
+    ops.add(opTasks1);
+    //this is expected to be ignored
+    ops.add(opTasksNone);
+    ops.add(opTasks2);
+    ops.add(opTasks3);
+
+    try {
+      PrivilegedOperation op = PrivilegedOperationExecutor
+          .squashCGroupOperations(ops);
+      String expected = new StringBuffer
+          (PrivilegedOperation.CGROUP_ARG_PREFIX)
+          .append(cGroupTasks1).append(',')
+          .append(cGroupTasks2).append(',')
+          .append(cGroupTasks3).toString();
+
+      //We expect axactly one argument
+      Assert.assertEquals(1, op.getArguments().size());
+      //Squashed list of tasks files
+      Assert.assertEquals(expected, op.getArguments().get(0));
+    } catch (PrivilegedOperationException e) {
+      LOG.info("Caught unexpected exception : " + e);
+      Assert.fail("Caught unexpected exception: " + e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae7a5ffd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
new file mode 100644
index 0000000..0717447
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
@@ -0,0 +1,235 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements. See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership. The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.isNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.verifyZeroInteractions;
+
+public class TestCGroupsHandlerImpl {
+  private static final Log LOG =
+      LogFactory.getLog(TestCGroupsHandlerImpl.class);
+
+  private PrivilegedOperationExecutor privilegedOperationExecutorMock;
+  private Configuration conf;
+  private String tmpPath;
+  private String hierarchy;
+  private CGroupsHandler.CGroupController controller;
+  private String controllerPath;
+
+  @Before
+  public void setup() {
+    privilegedOperationExecutorMock = mock(PrivilegedOperationExecutor.class);
+    conf = new YarnConfiguration();
+    tmpPath = System.getProperty("test.build.data") + "/cgroups";
+    //no leading or trailing slashes here
+    hierarchy = "test-hadoop-yarn";
+
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_HIERARCHY, hierarchy);
+    conf.setBoolean(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_MOUNT, true);
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH, tmpPath);
+    controller = CGroupsHandler.CGroupController.NET_CLS;
+    controllerPath = new StringBuffer(tmpPath).append('/')
+        .append(controller.getName()).append('/').append(hierarchy).toString();
+  }
+
+  @Test
+  public void testMountController() {
+    CGroupsHandler cGroupsHandler = null;
+    //Since we enabled (deferred) cgroup controller mounting, no interactions
+    //should have occurred, with this mock
+    verifyZeroInteractions(privilegedOperationExecutorMock);
+
+    try {
+      cGroupsHandler = new CGroupsHandlerImpl(conf,
+          privilegedOperationExecutorMock);
+      PrivilegedOperation expectedOp = new PrivilegedOperation
+          (PrivilegedOperation.OperationType.MOUNT_CGROUPS, (String) null);
+      //This is expected to be of the form :
+      //net_cls=<mount_path>/net_cls
+      StringBuffer controllerKV = new StringBuffer(controller.getName())
+          .append('=').append(tmpPath).append('/').append(controller.getName());
+      expectedOp.appendArgs(hierarchy, controllerKV.toString());
+
+      cGroupsHandler.mountCGroupController(controller);
+      try {
+        ArgumentCaptor<PrivilegedOperation> opCaptor = ArgumentCaptor.forClass
+            (PrivilegedOperation.class);
+        verify(privilegedOperationExecutorMock)
+            .executePrivilegedOperation(opCaptor.capture(), eq(false));
+
+        //we'll explicitly capture and assert that the
+        //captured op and the expected op are identical.
+        Assert.assertEquals(expectedOp, opCaptor.getValue());
+        verifyNoMoreInteractions(privilegedOperationExecutorMock);
+
+        //Try mounting the same controller again - this should be a no-op
+        cGroupsHandler.mountCGroupController(controller);
+        verifyNoMoreInteractions(privilegedOperationExecutorMock);
+      } catch (PrivilegedOperationException e) {
+        LOG.error("Caught exception: " + e);
+        Assert.assertTrue("Unexpected PrivilegedOperationException from mock!",
+            false);
+      }
+    } catch (ResourceHandlerException e) {
+      LOG.error("Caught exception: " + e);
+      Assert.assertTrue("Unexpected ResourceHandler Exception!", false);
+    }
+  }
+
+  @Test
+  public void testCGroupPaths() {
+    //As per junit behavior, we expect a new mock object to be available
+    //in this test.
+    verifyZeroInteractions(privilegedOperationExecutorMock);
+    CGroupsHandler cGroupsHandler = null;
+    try {
+      cGroupsHandler = new CGroupsHandlerImpl(conf,
+          privilegedOperationExecutorMock);
+      cGroupsHandler.mountCGroupController(controller);
+    } catch (ResourceHandlerException e) {
+      LOG.error("Caught exception: " + e);
+      Assert.assertTrue(
+          "Unexpected ResourceHandlerException when mounting controller!",
+          false);
+    }
+
+    String testCGroup = "container_01";
+    String expectedPath = new StringBuffer(controllerPath).append('/')
+        .append(testCGroup).toString();
+    String path = cGroupsHandler.getPathForCGroup(controller, testCGroup);
+    Assert.assertEquals(expectedPath, path);
+
+    String expectedPathTasks = new StringBuffer(expectedPath).append('/')
+        .append(CGroupsHandler.CGROUP_FILE_TASKS).toString();
+    path = cGroupsHandler.getPathForCGroupTasks(controller, testCGroup);
+    Assert.assertEquals(expectedPathTasks, path);
+
+    String param = CGroupsHandler.CGROUP_PARAM_CLASSID;
+    String expectedPathParam = new StringBuffer(expectedPath).append('/')
+        .append(controller.getName()).append('.').append(param).toString();
+    path = cGroupsHandler.getPathForCGroupParam(controller, testCGroup, param);
+    Assert.assertEquals(expectedPathParam, path);
+  }
+
+  @Test
+  public void testCGroupOperations() {
+    //As per junit behavior, we expect a new mock object to be available
+    //in this test.
+    verifyZeroInteractions(privilegedOperationExecutorMock);
+    CGroupsHandler cGroupsHandler = null;
+
+    try {
+      cGroupsHandler = new CGroupsHandlerImpl(conf,
+          privilegedOperationExecutorMock);
+      cGroupsHandler.mountCGroupController(controller);
+    } catch (ResourceHandlerException e) {
+      LOG.error("Caught exception: " + e);
+      Assert.assertTrue(
+          "Unexpected ResourceHandlerException when mounting controller!",
+          false);
+    }
+    //Lets manually create a path to (partially) simulate a mounted controller
+    //this is required because the handler uses a mocked privileged operation
+    //executor
+    new File(controllerPath).mkdirs();
+
+    String testCGroup = "container_01";
+    String expectedPath = new StringBuffer(controllerPath).append('/')
+        .append(testCGroup).toString();
+    try {
+      String path = cGroupsHandler.createCGroup(controller, testCGroup);
+
+      Assert.assertTrue(new File(expectedPath).exists());
+      Assert.assertEquals(expectedPath, path);
+
+      //update param and read param tests.
+      //We don't use net_cls.classid because as a test param here because
+      //cgroups provides very specific read/write semantics for classid (only
+      //numbers can be written - potentially as hex but can be read out only
+      //as decimal)
+      String param = "test_param";
+      String paramValue = "test_param_value";
+
+      cGroupsHandler
+          .updateCGroupParam(controller, testCGroup, param, paramValue);
+      String paramPath = new StringBuffer(expectedPath).append('/')
+          .append(controller.getName()).append('.').append(param).toString();
+      File paramFile = new File(paramPath);
+
+      Assert.assertTrue(paramFile.exists());
+      try {
+        Assert.assertEquals(paramValue, new String(Files.readAllBytes
+            (paramFile
+                .toPath())));
+      } catch (IOException e) {
+        LOG.error("Caught exception: " + e);
+        Assert.assertTrue("Unexpected IOException trying to read cgroup param!",
+            false);
+      }
+
+      Assert.assertEquals(paramValue, cGroupsHandler.getCGroupParam
+          (controller, testCGroup, param));
+
+      //We can't really do a delete test here. Linux cgroups
+      //implementation provides additional semantics - the cgroup cannot be
+      //deleted if there are any tasks still running in the cgroup even if
+      //the user attempting the delete has the file permissions to do so - we
+      //cannot simulate that here. Even if we create a dummy 'tasks' file, we
+      //wouldn't be able to simulate the delete behavior we need, since a cgroup
+      //can be deleted using using 'rmdir' if the tasks file is empty. Such a
+      //delete is not possible with a regular non-empty directory.
+    } catch (ResourceHandlerException e) {
+      LOG.error("Caught exception: " + e);
+      Assert.assertTrue(
+          "Unexpected ResourceHandlerException during cgroup operations!",
+          false);
+    }
+  }
+
+  @After
+  public void teardown() {
+    FileUtil.fullyDelete(new File(tmpPath));
+  }
+}
\ No newline at end of file


[39/50] [abbrv] hadoop git commit: HDFS-8142. DistributedFileSystem encryption zone commands should resolve relative paths. Contributed by Rakesh R.

Posted by zj...@apache.org.
HDFS-8142. DistributedFileSystem encryption zone commands should resolve relative paths. Contributed by Rakesh R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/223c80e2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/223c80e2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/223c80e2

Branch: refs/heads/YARN-2928
Commit: 223c80e25862a3a8ae7c7d502fcf3d79874b7243
Parents: 25366bc
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Apr 16 07:14:44 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:46 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hadoop/hdfs/DistributedFileSystem.java      | 51 ++++++++++++++++++--
 .../apache/hadoop/hdfs/TestEncryptionZones.java | 18 +++++++
 3 files changed, 68 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/223c80e2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6523423..cc9d901 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -502,6 +502,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8055. NullPointerException when topology script is missing.
     (Anu Engineer via cnauroth)
 
+    HDFS-8142. DistributedFileSystem encryption zone commands should resolve
+    relative paths. (Rakesh R via wang)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/223c80e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 21f5107..8e7daf3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2046,16 +2046,59 @@ public class DistributedFileSystem extends FileSystem {
   }
   
   /* HDFS only */
-  public void createEncryptionZone(Path path, String keyName)
+  public void createEncryptionZone(final Path path, final String keyName)
     throws IOException {
-    dfs.createEncryptionZone(getPathName(path), keyName);
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        dfs.createEncryptionZone(getPathName(p), keyName);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          myDfs.createEncryptionZone(p, keyName);
+          return null;
+        } else {
+          throw new UnsupportedOperationException(
+              "Cannot call createEncryptionZone"
+                  + " on a symlink to a non-DistributedFileSystem: " + path
+                  + " -> " + p);
+        }
+      }
+    }.resolve(this, absF);
   }
 
   /* HDFS only */
-  public EncryptionZone getEZForPath(Path path)
+  public EncryptionZone getEZForPath(final Path path)
           throws IOException {
     Preconditions.checkNotNull(path);
-    return dfs.getEZForPath(getPathName(path));
+    Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<EncryptionZone>() {
+      @Override
+      public EncryptionZone doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        return dfs.getEZForPath(getPathName(p));
+      }
+
+      @Override
+      public EncryptionZone next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          return myDfs.getEZForPath(p);
+        } else {
+          throw new UnsupportedOperationException(
+              "Cannot call getEZForPath"
+                  + " on a symlink to a non-DistributedFileSystem: " + path
+                  + " -> " + p);
+        }
+      }
+    }.resolve(this, absF);
   }
 
   /* HDFS only */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/223c80e2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index dbb7ea5..e43593d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -1310,4 +1310,22 @@ public class TestEncryptionZones {
         true, fs.getFileStatus(zoneFile).isEncrypted());
     DFSTestUtil.verifyFilesNotEqual(fs, zoneFile, rawFile, len);
   }
+
+  @Test(timeout = 60000)
+  public void testEncryptionZonesOnRelativePath() throws Exception {
+    final int len = 8196;
+    final Path baseDir = new Path("/somewhere/base");
+    final Path zoneDir = new Path("zone");
+    final Path zoneFile = new Path("file");
+    fs.setWorkingDirectory(baseDir);
+    fs.mkdirs(zoneDir);
+    dfsAdmin.createEncryptionZone(zoneDir, TEST_KEY);
+    DFSTestUtil.createFile(fs, zoneFile, len, (short) 1, 0xFEED);
+
+    assertNumZones(1);
+    assertZonePresent(TEST_KEY, "/somewhere/base/zone");
+
+    assertEquals("Got unexpected ez path", "/somewhere/base/zone", dfsAdmin
+        .getEncryptionZoneForPath(zoneDir).getPath().toString());
+  }
 }


[26/50] [abbrv] hadoop git commit: YARN-3361. CapacityScheduler side changes to support non-exclusive node labels. Contributed by Wangda Tan

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
new file mode 100644
index 0000000..cf1b26f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
@@ -0,0 +1,1027 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+public class TestNodeLabelContainerAllocation {
+  private final int GB = 1024;
+
+  private YarnConfiguration conf;
+  
+  RMNodeLabelsManager mgr;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+      ResourceScheduler.class);
+    mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+  }
+  
+  private Configuration getConfigurationWithQueueLabels(Configuration config) {
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+    
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+    conf.setAccessibleNodeLabels(A, toSet("x"));
+    conf.setCapacityByLabel(A, "x", 100);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 20);
+    conf.setAccessibleNodeLabels(B, toSet("y"));
+    conf.setCapacityByLabel(B, "y", 100);
+    
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    conf.setCapacity(C, 70);
+    conf.setMaximumCapacity(C, 70);
+    conf.setAccessibleNodeLabels(C, RMNodeLabelsManager.EMPTY_STRING_SET);
+    
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    conf.setQueues(A, new String[] {"a1"});
+    conf.setCapacity(A1, 100);
+    conf.setMaximumCapacity(A1, 100);
+    conf.setCapacityByLabel(A1, "x", 100);
+    
+    final String B1 = B + ".b1";
+    conf.setQueues(B, new String[] {"b1"});
+    conf.setCapacity(B1, 100);
+    conf.setMaximumCapacity(B1, 100);
+    conf.setCapacityByLabel(B1, "y", 100);
+
+    final String C1 = C + ".c1";
+    conf.setQueues(C, new String[] {"c1"});
+    conf.setCapacity(C1, 100);
+    conf.setMaximumCapacity(C1, 100);
+    
+    return conf;
+  }
+  
+  private void checkTaskContainersHost(ApplicationAttemptId attemptId,
+      ContainerId containerId, ResourceManager rm, String host) {
+    YarnScheduler scheduler = rm.getRMContext().getScheduler();
+    SchedulerAppReport appReport = scheduler.getSchedulerAppInfo(attemptId);
+
+    Assert.assertTrue(appReport.getLiveContainers().size() > 0);
+    for (RMContainer c : appReport.getLiveContainers()) {
+      if (c.getContainerId().equals(containerId)) {
+        Assert.assertEquals(host, c.getAllocatedNode().getHost());
+      }
+    }
+  }
+  
+  @SuppressWarnings("unchecked")
+  private <E> Set<E> toSet(E... elements) {
+    Set<E> set = Sets.newHashSet(elements);
+    return set;
+  }
+  
+  
+  @Test (timeout = 300000)
+  public void testContainerAllocationWithSingleUserLimits() throws Exception {
+    final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
+        NodeId.newInstance("h2", 0), toSet("y")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithDefaultQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
+    rm1.registerNode("h2:1234", 8000); // label = y
+    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    
+    // A has only 10% of x, so it can only allocate one container in label=empty
+    ContainerId containerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+          RMContainerState.ALLOCATED, 10 * 1000));
+    // Cannot allocate 2nd label=empty container
+    containerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
+    Assert.assertFalse(rm1.waitForState(nm3, containerId,
+          RMContainerState.ALLOCATED, 10 * 1000));
+
+    // A has default user limit = 100, so it can use all resource in label = x
+    // We can allocate floor(8000 / 1024) = 7 containers
+    for (int id = 3; id <= 8; id++) {
+      containerId =
+          ContainerId.newContainerId(am1.getApplicationAttemptId(), id);
+      am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
+      Assert.assertTrue(rm1.waitForState(nm1, containerId,
+          RMContainerState.ALLOCATED, 10 * 1000));
+    }
+    rm1.close();
+  }
+  
+  @Test(timeout = 300000)
+  public void testContainerAllocateWithComplexLabels() throws Exception {
+    /*
+     * Queue structure:
+     *                      root (*)
+     *                  ________________
+     *                 /                \
+     *               a x(100%), y(50%)   b y(50%), z(100%)
+     *               ________________    ______________
+     *              /                   /              \
+     *             a1 (x,y)         b1(no)              b2(y,z)
+     *               100%                          y = 100%, z = 100%
+     *                           
+     * Node structure:
+     * h1 : x
+     * h2 : y
+     * h3 : y
+     * h4 : z
+     * h5 : NO
+     * 
+     * Total resource:
+     * x: 4G
+     * y: 6G
+     * z: 2G
+     * *: 2G
+     * 
+     * Resource of
+     * a1: x=4G, y=3G, NO=0.2G
+     * b1: NO=0.9G (max=1G)
+     * b2: y=3, z=2G, NO=0.9G (max=1G)
+     * 
+     * Each node can only allocate two containers
+     */
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y", "z"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0),
+        toSet("x"), NodeId.newInstance("h2", 0), toSet("y"),
+        NodeId.newInstance("h3", 0), toSet("y"), NodeId.newInstance("h4", 0),
+        toSet("z"), NodeId.newInstance("h5", 0),
+        RMNodeLabelsManager.EMPTY_STRING_SET));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(TestUtils.getComplexConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 2048);
+    MockNM nm2 = rm1.registerNode("h2:1234", 2048);
+    MockNM nm3 = rm1.registerNode("h3:1234", 2048);
+    MockNM nm4 = rm1.registerNode("h4:1234", 2048);
+    MockNM nm5 = rm1.registerNode("h5:1234", 2048);
+    
+    ContainerId containerId;
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(1024, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // request a container (label = y). can be allocated on nm2 
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
+    containerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2L);
+    Assert.assertTrue(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
+        "h2");
+
+    // launch an app to queue b1 (label = y), and check all container will
+    // be allocated in h5
+    RMApp app2 = rm1.submitApp(1024, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm5);
+
+    // request a container for AM, will succeed
+    // and now b1's queue capacity will be used, cannot allocate more containers
+    // (Maximum capacity reached)
+    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm4, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertFalse(rm1.waitForState(nm5, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    
+    // launch an app to queue b2
+    RMApp app3 = rm1.submitApp(1024, "app", "user", null, "b2");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm5);
+
+    // request a container. try to allocate on nm1 (label = x) and nm3 (label =
+    // y,z). Will successfully allocate on nm3
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
+        "h3");
+    
+    // try to allocate container (request label = z) on nm4 (label = y,z). 
+    // Will successfully allocate on nm4 only.
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "z");
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 3L);
+    Assert.assertTrue(rm1.waitForState(nm4, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
+        "h4");
+
+    rm1.close();
+  }
+
+  @Test (timeout = 120000)
+  public void testContainerAllocateWithLabels() throws Exception {
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
+        NodeId.newInstance("h2", 0), toSet("y")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
+    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
+    
+    ContainerId containerId;
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm3);
+
+    // request a container.
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
+    containerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
+        "h1");
+
+    // launch an app to queue b1 (label = y), and check all container will
+    // be allocated in h2
+    RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm3);
+
+    // request a container.
+    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
+    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am2.getApplicationAttemptId(), containerId, rm1,
+        "h2");
+    
+    // launch an app to queue c1 (label = ""), and check all container will
+    // be allocated in h3
+    RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
+
+    // request a container.
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
+        "h3");
+
+    rm1.close();
+  }
+  
+  @Test (timeout = 120000)
+  public void testContainerAllocateWithDefaultQueueLabels() throws Exception {
+    // This test is pretty much similar to testContainerAllocateWithLabel.
+    // Difference is, this test doesn't specify label expression in ResourceRequest,
+    // instead, it uses default queue label expression
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
+        NodeId.newInstance("h2", 0), toSet("y")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithDefaultQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
+    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
+    
+    ContainerId containerId;
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // request a container.
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
+        "h1");
+
+    // launch an app to queue b1 (label = y), and check all container will
+    // be allocated in h2
+    RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // request a container.
+    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am2.getApplicationAttemptId(), containerId, rm1,
+        "h2");
+    
+    // launch an app to queue c1 (label = ""), and check all container will
+    // be allocated in h3
+    RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
+
+    // request a container.
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
+        "h3");
+
+    rm1.close();
+  }
+  
+  private void checkPendingResource(MockRM rm, int priority,
+      ApplicationAttemptId attemptId, int memory) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler();
+    FiCaSchedulerApp app = cs.getApplicationAttempt(attemptId);
+    ResourceRequest rr =
+        app.getAppSchedulingInfo().getResourceRequest(
+            Priority.newInstance(priority), "*");
+    Assert.assertEquals(memory,
+        rr.getCapability().getMemory() * rr.getNumContainers());
+  }
+  
+  private void checkLaunchedContainerNumOnNode(MockRM rm, NodeId nodeId,
+      int numContainers) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler();
+    SchedulerNode node = cs.getSchedulerNode(nodeId);
+    Assert.assertEquals(numContainers, node.getNumContainers());
+  }
+  
+  @Test
+  public void testPreferenceOfNeedyAppsTowardsNodePartitions() throws Exception {
+    /**
+     * Test case: Submit two application to a queue (app1 first then app2), app1
+     * asked for no-label, app2 asked for label=x, when node1 has label=x
+     * doing heart beat, app2 will get allocation first, even if app2 submits later
+     * than app1
+     */
+    
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    // Makes y to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("y", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("y")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); // label = y
+    MockNM nm2 = rm1.registerNode("h2:1234", 100 * GB); // label = <empty>
+
+    // launch an app to queue b1 (label = y), AM container should be launched in nm2
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "b1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2);
+    
+    // launch another app to queue b1 (label = y), AM container should be launched in nm2
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // request container and nm1 do heartbeat (nm2 has label=y), note that app1
+    // request non-labeled container, and app2 request labeled container, app2
+    // will get allocated first even if app1 submitted first.  
+    am1.allocate("*", 1 * GB, 8, new ArrayList<ContainerId>());
+    am2.allocate("*", 1 * GB, 8, new ArrayList<ContainerId>(), "y");
+    
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+    
+    // Do node heartbeats many times
+    for (int i = 0; i < 50; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+    
+    // App2 will get preference to be allocated on node1, and node1 will be all
+    // used by App2.
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(am1.getApplicationAttemptId());
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(am2.getApplicationAttemptId());
+    // app1 get nothing in nm1 (partition=y)
+    checkNumOfContainersInAnAppOnGivenNode(0, nm1.getNodeId(), schedulerApp1);
+    checkNumOfContainersInAnAppOnGivenNode(9, nm2.getNodeId(), schedulerApp1);
+    // app2 get all resource in nm1 (partition=y)
+    checkNumOfContainersInAnAppOnGivenNode(8, nm1.getNodeId(), schedulerApp2);
+    checkNumOfContainersInAnAppOnGivenNode(1, nm2.getNodeId(), schedulerApp2);
+    
+    rm1.close();
+  }
+  
+  private void checkNumOfContainersInAnAppOnGivenNode(int expectedNum,
+      NodeId nodeId, FiCaSchedulerApp app) {
+    int num = 0;
+    for (RMContainer container : app.getLiveContainers()) {
+      if (container.getAllocatedNode().equals(nodeId)) {
+        num++;
+      }
+    }
+    Assert.assertEquals(expectedNum, num);
+  }
+  
+  @Test
+  public void
+      testPreferenceOfNeedyPrioritiesUnderSameAppTowardsNodePartitions()
+          throws Exception {
+    /**
+     * Test case: Submit one application, it asks label="" in priority=1 and
+     * label="x" in priority=2, when a node with label=x heartbeat, priority=2
+     * will get allocation first even if there're pending resource in priority=1
+     */
+    
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    // Makes y to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("y", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("y")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); // label = y
+    MockNM nm2 = rm1.registerNode("h2:1234", 100 * GB); // label = <empty>
+    
+    ContainerId nextContainerId;
+
+    // launch an app to queue b1 (label = y), AM container should be launched in nm3
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "b1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2);
+    
+    // request containers from am2, priority=1 asks for "" and priority=2 asks
+    // for "y", "y" container should be allocated first
+    nextContainerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    am1.allocate("*", 1 * GB, 1, 1, new ArrayList<ContainerId>(), "");
+    am1.allocate("*", 1 * GB, 1, 2, new ArrayList<ContainerId>(), "y");
+    Assert.assertTrue(rm1.waitForState(nm1, nextContainerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    
+    // Check pending resource for am2, priority=1 doesn't get allocated before
+    // priority=2 allocated
+    checkPendingResource(rm1, 1, am1.getApplicationAttemptId(), 1 * GB);
+    checkPendingResource(rm1, 2, am1.getApplicationAttemptId(), 0 * GB);
+    
+    rm1.close();
+  }
+  
+  @Test
+  public void testNonLabeledResourceRequestGetPreferrenceToNonLabeledNode()
+      throws Exception {
+    /**
+     * Test case: Submit one application, it asks 6 label="" containers, NM1
+     * with label=y and NM2 has no label, NM1/NM2 doing heartbeat together. Even
+     * if NM1 has idle resource, containers are all allocated to NM2 since
+     * non-labeled request should get allocation on non-labeled nodes first.
+     */
+    
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    // Makes x to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); // label = y
+    MockNM nm2 = rm1.registerNode("h2:1234", 100 * GB); // label = <empty>
+    
+    ContainerId nextContainerId;
+
+    // launch an app to queue b1 (label = y), AM container should be launched in nm3
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "b1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2);
+    
+    // request containers from am2, priority=1 asks for "" * 6 (id from 4 to 9),
+    // nm2/nm3 do
+    // heartbeat at the same time, check containers are always allocated to nm3.
+    // This is to verify when there's resource available in non-labeled
+    // partition, non-labeled resource should allocate to non-labeled partition
+    // first.
+    am1.allocate("*", 1 * GB, 6, 1, new ArrayList<ContainerId>(), "");
+    for (int i = 2; i < 2 + 6; i++) {
+      nextContainerId =
+          ContainerId.newContainerId(am1.getApplicationAttemptId(), i);
+      Assert.assertTrue(rm1.waitForState(Arrays.asList(nm1, nm2),
+          nextContainerId, RMContainerState.ALLOCATED, 10 * 1000));
+    }
+    // no more container allocated on nm1
+    checkLaunchedContainerNumOnNode(rm1, nm1.getNodeId(), 0);
+    // all 7 (1 AM container + 6 task container) containers allocated on nm2
+    checkLaunchedContainerNumOnNode(rm1, nm2.getNodeId(), 7);   
+    
+    rm1.close();
+  }
+
+  @Test
+  public void testPreferenceOfQueuesTowardsNodePartitions()
+      throws Exception {
+    /**
+     * Test case: have a following queue structure:
+     * 
+     * <pre>
+     *            root
+     *         /   |   \
+     *        a     b    c
+     *       / \   / \  /  \
+     *      a1 a2 b1 b2 c1 c2
+     *     (x)    (x)   (x)
+     * </pre>
+     * 
+     * Only a1, b1, c1 can access label=x, and their default label=x Each each
+     * has one application, asks for 5 containers. NM1 has label=x
+     * 
+     * NM1/NM2 doing heartbeat for 15 times, it should allocate all 15
+     * containers with label=x
+     */
+    
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(this.conf);
+    
+    // Define top-level queues
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});
+    csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    csConf.setCapacity(A, 33);
+    csConf.setAccessibleNodeLabels(A, toSet("x"));
+    csConf.setCapacityByLabel(A, "x", 33);
+    csConf.setQueues(A, new String[] {"a1", "a2"});
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(B, 33);
+    csConf.setAccessibleNodeLabels(B, toSet("x"));
+    csConf.setCapacityByLabel(B, "x", 33);
+    csConf.setQueues(B, new String[] {"b1", "b2"});
+    
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    csConf.setCapacity(C, 34);
+    csConf.setAccessibleNodeLabels(C, toSet("x"));
+    csConf.setCapacityByLabel(C, "x", 34);
+    csConf.setQueues(C, new String[] {"c1", "c2"});
+    
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    csConf.setCapacity(A1, 50);
+    csConf.setCapacityByLabel(A1, "x", 100);
+    csConf.setDefaultNodeLabelExpression(A1, "x");
+    
+    final String A2 = A + ".a2";
+    csConf.setCapacity(A2, 50);
+    csConf.setCapacityByLabel(A2, "x", 0);
+    
+    final String B1 = B + ".b1";
+    csConf.setCapacity(B1, 50);
+    csConf.setCapacityByLabel(B1, "x", 100);
+    csConf.setDefaultNodeLabelExpression(B1, "x");
+    
+    final String B2 = B + ".b2";
+    csConf.setCapacity(B2, 50);
+    csConf.setCapacityByLabel(B2, "x", 0);
+    
+    final String C1 = C + ".c1";
+    csConf.setCapacity(C1, 50);
+    csConf.setCapacityByLabel(C1, "x", 100);
+    csConf.setDefaultNodeLabelExpression(C1, "x");
+    
+    final String C2 = C + ".c2";
+    csConf.setCapacity(C2, 50);
+    csConf.setCapacityByLabel(C2, "x", 0);
+    
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    // Makes x to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 20 * GB); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 100 * GB); // label = <empty>
+
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    
+    // app2 -> a2
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "a2");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+    
+    // app3 -> b1
+    RMApp app3 = rm1.submitApp(1 * GB, "app", "user", null, "b1");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm1);
+    
+    // app4 -> b2
+    RMApp app4 = rm1.submitApp(1 * GB, "app", "user", null, "b2");
+    MockAM am4 = MockRM.launchAndRegisterAM(app4, rm1, nm2);
+    
+    // app5 -> c1
+    RMApp app5 = rm1.submitApp(1 * GB, "app", "user", null, "c1");
+    MockAM am5 = MockRM.launchAndRegisterAM(app5, rm1, nm1);
+    
+    // app6 -> b2
+    RMApp app6 = rm1.submitApp(1 * GB, "app", "user", null, "c2");
+    MockAM am6 = MockRM.launchAndRegisterAM(app6, rm1, nm2);
+    
+    // Each application request 5 * 1GB container
+    am1.allocate("*", 1 * GB, 5, new ArrayList<ContainerId>());
+    am2.allocate("*", 1 * GB, 5, new ArrayList<ContainerId>());
+    am3.allocate("*", 1 * GB, 5, new ArrayList<ContainerId>());
+    am4.allocate("*", 1 * GB, 5, new ArrayList<ContainerId>());
+    am5.allocate("*", 1 * GB, 5, new ArrayList<ContainerId>());
+    am6.allocate("*", 1 * GB, 5, new ArrayList<ContainerId>());
+    
+    // NM1 do 15 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    for (int i = 0; i < 15; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+    
+    // NM1 get 15 new containers (total is 18, 15 task containers and 3 AM
+    // containers)
+    checkLaunchedContainerNumOnNode(rm1, nm1.getNodeId(), 18);
+
+    // Check pending resource each application
+    // APP1/APP3/APP5 get satisfied, and APP2/APP2/APP3 get nothing.
+    checkPendingResource(rm1, 1, am1.getApplicationAttemptId(), 0 * GB);
+    checkPendingResource(rm1, 1, am2.getApplicationAttemptId(), 5 * GB);
+    checkPendingResource(rm1, 1, am3.getApplicationAttemptId(), 0 * GB);
+    checkPendingResource(rm1, 1, am4.getApplicationAttemptId(), 5 * GB);
+    checkPendingResource(rm1, 1, am5.getApplicationAttemptId(), 0 * GB);
+    checkPendingResource(rm1, 1, am6.getApplicationAttemptId(), 5 * GB);
+
+    rm1.close();
+  }
+  
+  @Test
+  public void testQueuesWithoutAccessUsingPartitionedNodes() throws Exception {
+    /**
+     * Test case: have a following queue structure:
+     * 
+     * <pre>
+     *            root
+     *         /      \
+     *        a        b
+     *        (x)
+     * </pre>
+     * 
+     * Only a can access label=x, two nodes in the cluster, n1 has x and n2 has
+     * no-label.
+     * 
+     * When user-limit-factor=5, submit one application in queue b and request
+     * for infinite containers should be able to use up all cluster resources.
+     */
+    
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(this.conf);
+    
+    // Define top-level queues
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+    csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    csConf.setCapacity(A, 50);
+    csConf.setAccessibleNodeLabels(A, toSet("x"));
+    csConf.setCapacityByLabel(A, "x", 100);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(B, 50);
+    csConf.setAccessibleNodeLabels(B, new HashSet<String>());
+    csConf.setUserLimitFactor(B, 5);
+    
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x"));
+    // Makes x to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 10 * GB); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 10 * GB); // label = <empty>
+
+    // app1 -> b
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "b");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2);
+    
+    // Each application request 5 * 1GB container
+    am1.allocate("*", 1 * GB, 50, new ArrayList<ContainerId>());
+    
+    // NM1 do 50 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+    
+    SchedulerNode schedulerNode1 = cs.getSchedulerNode(nm1.getNodeId());
+    
+    // How much cycles we waited to be allocated when available resource only on
+    // partitioned node
+    int cycleWaited = 0;
+    for (int i = 0; i < 50; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+      if (schedulerNode1.getNumContainers() == 0) {
+        cycleWaited++;
+      }
+    }
+    // We will will 10 cycles before get allocated on partitioned node
+    // NM2 can allocate 10 containers totally, exclude already allocated AM
+    // container, we will wait 9 to fulfill non-partitioned node, and need wait
+    // one more cycle before allocating to non-partitioned node 
+    Assert.assertEquals(10, cycleWaited);
+    
+    // Both NM1/NM2 launched 10 containers, cluster resource is exhausted
+    checkLaunchedContainerNumOnNode(rm1, nm1.getNodeId(), 10);
+    checkLaunchedContainerNumOnNode(rm1, nm2.getNodeId(), 10);
+
+    rm1.close();
+  }
+  
+  @Test
+  public void testAMContainerAllocationWillAlwaysBeExclusive()
+      throws Exception {
+    /**
+     * Test case: Submit one application without partition, trying to allocate a
+     * node has partition=x, it should fail to allocate since AM container will
+     * always respect exclusivity for partitions
+     */
+    
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    // Makes x to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); // label = x
+
+    // launch an app to queue b1 (label = y), AM container should be launched in nm3
+    rm1.submitApp(1 * GB, "app", "user", null, "b1");
+   
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    
+    // Heartbeat for many times, app1 should get nothing
+    for (int i = 0; i < 50; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+    
+    Assert.assertEquals(0, cs.getSchedulerNode(nm1.getNodeId())
+        .getNumContainers());
+    
+    rm1.close();
+  }
+  
+  @Test
+  public void
+      testQueueMaxCapacitiesWillNotBeHonoredWhenNotRespectingExclusivity()
+          throws Exception {
+    /**
+     * Test case: have a following queue structure:
+     * 
+     * <pre>
+     *            root
+     *         /      \
+     *        a        b
+     *        (x)     (x)
+     * </pre>
+     * 
+     * a/b can access x, both of them has max-capacity-on-x = 50
+     * 
+     * When doing non-exclusive allocation, app in a (or b) can use 100% of x
+     * resource.
+     */
+
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(this.conf);
+
+    // Define top-level queues
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a",
+        "b" });
+    csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    csConf.setCapacity(A, 50);
+    csConf.setAccessibleNodeLabels(A, toSet("x"));
+    csConf.setCapacityByLabel(A, "x", 50);
+    csConf.setMaximumCapacityByLabel(A, "x", 50);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(B, 50);
+    csConf.setAccessibleNodeLabels(B, toSet("x"));
+    csConf.setCapacityByLabel(B, "x", 50);
+    csConf.setMaximumCapacityByLabel(B, "x", 50);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x"));
+    // Makes x to be non-exclusive node labels
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 10 * GB); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 10 * GB); // label = <empty>
+
+    // app1 -> a
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2);
+
+    // app1 asks for 10 partition= containers
+    am1.allocate("*", 1 * GB, 10, new ArrayList<ContainerId>());
+
+    // NM1 do 50 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    SchedulerNode schedulerNode1 = cs.getSchedulerNode(nm1.getNodeId());
+
+    for (int i = 0; i < 50; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+    
+    // app1 gets all resource in partition=x
+    Assert.assertEquals(10, schedulerNode1.getNumContainers());
+
+    rm1.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
index 7da1c97..52d0bc1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doReturn;
@@ -45,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -146,7 +146,7 @@ public class TestParentQueue {
         final Resource allocatedResource = Resources.createResource(allocation);
         if (queue instanceof ParentQueue) {
           ((ParentQueue)queue).allocateResource(clusterResource, 
-              allocatedResource, null);
+              allocatedResource, RMNodeLabelsManager.NO_LABEL);
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
@@ -157,7 +157,7 @@ public class TestParentQueue {
         if (allocation > 0) {
           doReturn(new CSAssignment(Resources.none(), type)).when(queue)
               .assignContainers(eq(clusterResource), eq(node),
-                  any(ResourceLimits.class));
+                  any(ResourceLimits.class), any(SchedulingMode.class));
 
           // Mock the node's resource availability
           Resource available = node.getAvailableResource();
@@ -168,7 +168,7 @@ public class TestParentQueue {
         return new CSAssignment(allocatedResource, type);
       }
     }).when(queue).assignContainers(eq(clusterResource), eq(node),
-        any(ResourceLimits.class));
+        any(ResourceLimits.class), any(SchedulingMode.class));
   }
   
   private float computeQueueAbsoluteUsedCapacity(CSQueue queue, 
@@ -228,11 +228,16 @@ public class TestParentQueue {
     LeafQueue a = (LeafQueue)queues.get(A);
     LeafQueue b = (LeafQueue)queues.get(B);
     
+    a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
+    .incPending(Resources.createResource(1 * GB));
+    
     // Simulate B returning a container on node_0
     stubQueueAllocation(a, clusterResource, node_0, 0*GB);
     stubQueueAllocation(b, clusterResource, node_0, 1*GB);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyQueueMetrics(a, 0*GB, clusterResource);
     verifyQueueMetrics(b, 1*GB, clusterResource);
     
@@ -240,12 +245,12 @@ public class TestParentQueue {
     stubQueueAllocation(a, clusterResource, node_1, 2*GB);
     stubQueueAllocation(b, clusterResource, node_1, 1*GB);
     root.assignContainers(clusterResource, node_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     InOrder allocationOrder = inOrder(a, b);
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(a, 2*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
 
@@ -254,12 +259,12 @@ public class TestParentQueue {
     stubQueueAllocation(a, clusterResource, node_0, 1*GB);
     stubQueueAllocation(b, clusterResource, node_0, 2*GB);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     allocationOrder = inOrder(b, a);
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 4*GB, clusterResource);
 
@@ -268,12 +273,12 @@ public class TestParentQueue {
     stubQueueAllocation(a, clusterResource, node_0, 0*GB);
     stubQueueAllocation(b, clusterResource, node_0, 4*GB);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     allocationOrder = inOrder(b, a);
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 8*GB, clusterResource);
 
@@ -282,12 +287,12 @@ public class TestParentQueue {
     stubQueueAllocation(a, clusterResource, node_1, 1*GB);
     stubQueueAllocation(b, clusterResource, node_1, 1*GB);
     root.assignContainers(clusterResource, node_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     allocationOrder = inOrder(a, b);
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(a, 4*GB, clusterResource);
     verifyQueueMetrics(b, 9*GB, clusterResource);
   }
@@ -448,16 +453,27 @@ public class TestParentQueue {
 
     // Start testing
     CSQueue a = queues.get(A);
+    a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
     CSQueue b = queues.get(B);
+    b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
     CSQueue c = queues.get(C);
+    c.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
     CSQueue d = queues.get(D);
+    d.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
 
     CSQueue a1 = queues.get(A1);
+    a1.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
     CSQueue a2 = queues.get(A2);
+    a2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
 
     CSQueue b1 = queues.get(B1);
+    b1.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
     CSQueue b2 = queues.get(B2);
+    b2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
     CSQueue b3 = queues.get(B3);
+    b3.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
+    .incPending(Resources.createResource(1 * GB));
 
     // Simulate C returning a container on node_0
     stubQueueAllocation(a, clusterResource, node_0, 0*GB);
@@ -465,7 +481,7 @@ public class TestParentQueue {
     stubQueueAllocation(c, clusterResource, node_0, 1*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyQueueMetrics(a, 0*GB, clusterResource);
     verifyQueueMetrics(b, 0*GB, clusterResource);
     verifyQueueMetrics(c, 1*GB, clusterResource);
@@ -478,7 +494,7 @@ public class TestParentQueue {
     stubQueueAllocation(b2, clusterResource, node_1, 4*GB);
     stubQueueAllocation(c, clusterResource, node_1, 0*GB);
     root.assignContainers(clusterResource, node_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyQueueMetrics(a, 0*GB, clusterResource);
     verifyQueueMetrics(b, 4*GB, clusterResource);
     verifyQueueMetrics(c, 1*GB, clusterResource);
@@ -490,14 +506,14 @@ public class TestParentQueue {
     stubQueueAllocation(b3, clusterResource, node_0, 2*GB);
     stubQueueAllocation(c, clusterResource, node_0, 2*GB);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     InOrder allocationOrder = inOrder(a, c, b);
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(c).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(a, 1*GB, clusterResource);
     verifyQueueMetrics(b, 6*GB, clusterResource);
     verifyQueueMetrics(c, 3*GB, clusterResource);
@@ -517,16 +533,16 @@ public class TestParentQueue {
     stubQueueAllocation(b1, clusterResource, node_2, 1*GB);
     stubQueueAllocation(c, clusterResource, node_2, 1*GB);
     root.assignContainers(clusterResource, node_2, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     allocationOrder = inOrder(a, a2, a1, b, c);
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(a2).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(c).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 8*GB, clusterResource);
     verifyQueueMetrics(c, 4*GB, clusterResource);
@@ -622,12 +638,16 @@ public class TestParentQueue {
     // Start testing
     LeafQueue a = (LeafQueue)queues.get(A);
     LeafQueue b = (LeafQueue)queues.get(B);
+    a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
+    .incPending(Resources.createResource(1 * GB));
     
     // Simulate B returning a container on node_0
     stubQueueAllocation(a, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
     stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyQueueMetrics(a, 0*GB, clusterResource);
     verifyQueueMetrics(b, 1*GB, clusterResource);
     
@@ -636,12 +656,12 @@ public class TestParentQueue {
     stubQueueAllocation(a, clusterResource, node_1, 2*GB, NodeType.RACK_LOCAL);
     stubQueueAllocation(b, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
     root.assignContainers(clusterResource, node_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     InOrder allocationOrder = inOrder(a, b);
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(a, 2*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
     
@@ -651,12 +671,12 @@ public class TestParentQueue {
     stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
     stubQueueAllocation(b, clusterResource, node_0, 2*GB, NodeType.OFF_SWITCH);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     allocationOrder = inOrder(b, a);
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(a, 2*GB, clusterResource);
     verifyQueueMetrics(b, 4*GB, clusterResource);
 
@@ -691,12 +711,19 @@ public class TestParentQueue {
     // Start testing
     LeafQueue b3 = (LeafQueue)queues.get(B3);
     LeafQueue b2 = (LeafQueue)queues.get(B2);
+    b2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    b3.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
+    .incPending(Resources.createResource(1 * GB));
+    
+    CSQueue b = queues.get(B);
+    b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
     
     // Simulate B3 returning a container on node_0
     stubQueueAllocation(b2, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
     stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyQueueMetrics(b2, 0*GB, clusterResource);
     verifyQueueMetrics(b3, 1*GB, clusterResource);
     
@@ -705,12 +732,12 @@ public class TestParentQueue {
     stubQueueAllocation(b2, clusterResource, node_1, 1*GB, NodeType.RACK_LOCAL);
     stubQueueAllocation(b3, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
     root.assignContainers(clusterResource, node_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     InOrder allocationOrder = inOrder(b2, b3);
     allocationOrder.verify(b2).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(b3).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(b2, 1*GB, clusterResource);
     verifyQueueMetrics(b3, 2*GB, clusterResource);
     
@@ -720,12 +747,12 @@ public class TestParentQueue {
     stubQueueAllocation(b2, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
     stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
     root.assignContainers(clusterResource, node_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     allocationOrder = inOrder(b3, b2);
     allocationOrder.verify(b3).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     allocationOrder.verify(b2).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
     verifyQueueMetrics(b2, 1*GB, clusterResource);
     verifyQueueMetrics(b3, 3*GB, clusterResource);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
index e8a8243..47be618 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
@@ -48,10 +48,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -266,7 +266,7 @@ public class TestReservations {
     // Start testing...
     // Only AM
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -278,7 +278,7 @@ public class TestReservations {
 
     // Only 1 map - simulating reduce
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -290,7 +290,7 @@ public class TestReservations {
 
     // Only 1 map to other node - simulating reduce
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -305,7 +305,7 @@ public class TestReservations {
 
     // try to assign reducer (5G on node 0 and should reserve)
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(5 * GB, a.getMetrics().getReservedMB());
@@ -321,7 +321,7 @@ public class TestReservations {
 
     // assign reducer to node 2
     a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(18 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(5 * GB, a.getMetrics().getReservedMB());
@@ -338,7 +338,7 @@ public class TestReservations {
     // node_1 heartbeat and unreserves from node_0 in order to allocate
     // on node_1
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(18 * GB, a.getUsedResources().getMemory());
     assertEquals(18 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -422,7 +422,7 @@ public class TestReservations {
     // Start testing...
     // Only AM
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -434,7 +434,7 @@ public class TestReservations {
 
     // Only 1 map - simulating reduce
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -446,7 +446,7 @@ public class TestReservations {
 
     // Only 1 map to other node - simulating reduce
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -461,7 +461,7 @@ public class TestReservations {
 
     // try to assign reducer (5G on node 0 and should reserve)
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(5 * GB, a.getMetrics().getReservedMB());
@@ -477,7 +477,7 @@ public class TestReservations {
 
     // assign reducer to node 2
     a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(18 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(5 * GB, a.getMetrics().getReservedMB());
@@ -494,7 +494,7 @@ public class TestReservations {
     // node_1 heartbeat and won't unreserve from node_0, potentially stuck
     // if AM doesn't handle
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(18 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(5 * GB, a.getMetrics().getReservedMB());
@@ -570,7 +570,7 @@ public class TestReservations {
     // Start testing...
     // Only AM
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -581,7 +581,7 @@ public class TestReservations {
 
     // Only 1 map - simulating reduce
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -592,7 +592,7 @@ public class TestReservations {
 
     // Only 1 map to other node - simulating reduce
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -606,7 +606,7 @@ public class TestReservations {
 
     // try to assign reducer (5G on node 0 and should reserve)
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(5 * GB, a.getMetrics().getReservedMB());
@@ -621,7 +621,7 @@ public class TestReservations {
 
     // could allocate but told need to unreserve first
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -823,7 +823,7 @@ public class TestReservations {
     // Start testing...
     // Only AM
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -834,7 +834,7 @@ public class TestReservations {
 
     // Only 1 map - simulating reduce
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -845,7 +845,7 @@ public class TestReservations {
 
     // Only 1 map to other node - simulating reduce
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -860,15 +860,16 @@ public class TestReservations {
     Resource capability = Resources.createResource(32 * GB, 0);
     boolean res =
         a.canAssignToThisQueue(clusterResource,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
-                clusterResource), capability, Resources.none());
+            RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
+                clusterResource), capability, Resources.none(),
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertFalse(res);
 
     // now add in reservations and make sure it continues if config set
     // allocate to queue so that the potential new capacity is greater then
     // absoluteMaxCapacity
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(5 * GB, a.getMetrics().getReservedMB());
@@ -881,16 +882,17 @@ public class TestReservations {
     capability = Resources.createResource(5 * GB, 0);
     res =
         a.canAssignToThisQueue(clusterResource,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
-                clusterResource), capability, Resources
-                .createResource(5 * GB));
+            RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
+                clusterResource), capability, Resources.createResource(5 * GB),
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertTrue(res);
 
     // tell to not check reservations
     res =
         a.canAssignToThisQueue(clusterResource,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
-                clusterResource), capability, Resources.none());
+            RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
+                clusterResource), capability, Resources.none(),
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertFalse(res);
 
     refreshQueuesTurnOffReservationsContLook(a, csConf);
@@ -899,15 +901,16 @@ public class TestReservations {
     // in since feature is off
     res =
         a.canAssignToThisQueue(clusterResource,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
-                clusterResource), capability, Resources.none());
+            RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
+                clusterResource), capability, Resources.none(),
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertFalse(res);
 
     res =
         a.canAssignToThisQueue(clusterResource,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
-                clusterResource), capability, Resources
-                .createResource(5 * GB));
+            RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
+                clusterResource), capability, Resources.createResource(5 * GB),
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertFalse(res);
   }
 
@@ -1008,7 +1011,7 @@ public class TestReservations {
     // Start testing...
     // Only AM
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1019,7 +1022,7 @@ public class TestReservations {
 
     // Only 1 map - simulating reduce
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1030,7 +1033,7 @@ public class TestReservations {
 
     // Only 1 map to other node - simulating reduce
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1044,7 +1047,7 @@ public class TestReservations {
     // allocate to queue so that the potential new capacity is greater then
     // absoluteMaxCapacity
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(5 * GB, app_0.getCurrentReservation().getMemory());
@@ -1059,18 +1062,18 @@ public class TestReservations {
     // set limit so subtrace reservations it can continue
     Resource limit = Resources.createResource(12 * GB, 0);
     boolean res = a.canAssignToUser(clusterResource, user_0, limit, app_0,
-        true, null);
+        true, "");
     assertTrue(res);
 
     // tell it not to check for reservations and should fail as already over
     // limit
-    res = a.canAssignToUser(clusterResource, user_0, limit, app_0, false, null);
+    res = a.canAssignToUser(clusterResource, user_0, limit, app_0, false, "");
     assertFalse(res);
 
     refreshQueuesTurnOffReservationsContLook(a, csConf);
 
     // should now return false since feature off
-    res = a.canAssignToUser(clusterResource, user_0, limit, app_0, true, null);
+    res = a.canAssignToUser(clusterResource, user_0, limit, app_0, true, "");
     assertFalse(res);
   }
 
@@ -1143,7 +1146,7 @@ public class TestReservations {
     // Start testing...
     // Only AM
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1155,7 +1158,7 @@ public class TestReservations {
 
     // Only 1 map - simulating reduce
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1167,7 +1170,7 @@ public class TestReservations {
 
     // Only 1 map to other node - simulating reduce
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1183,7 +1186,7 @@ public class TestReservations {
     // some resource. Even with continous reservation looking, we don't allow 
     // unreserve resource to reserve container.
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(Resources.createResource(10 * GB)));
+        new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1199,7 +1202,7 @@ public class TestReservations {
     // used (8G) + required (5G). It will not reserved since it has to unreserve
     // some resource. Unfortunately, there's nothing to unreserve.
     a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(Resources.createResource(10 * GB)));
+        new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1213,7 +1216,7 @@ public class TestReservations {
 
     // let it assign 5G to node_2
     a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
@@ -1226,7 +1229,7 @@ public class TestReservations {
 
     // reserve 8G node_0
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(21 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(8 * GB, a.getMetrics().getReservedMB());
@@ -1241,7 +1244,7 @@ public class TestReservations {
     // continued to try due to having reservation above,
     // but hits queue limits so can't reserve anymore.
     a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(21 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(8 * GB, a.getMetrics().getReservedMB());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
index 62135b9..84abf4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
@@ -160,6 +160,7 @@ public class TestUtils {
     request.setCapability(capability);
     request.setRelaxLocality(relaxLocality);
     request.setPriority(priority);
+    request.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL);
     return request;
   }
   
@@ -273,6 +274,7 @@ public class TestUtils {
     conf.setCapacity(B1, 100);
     conf.setMaximumCapacity(B1, 100);
     conf.setCapacityByLabel(B1, "y", 100);
+    conf.setMaximumApplicationMasterResourcePerQueuePercent(B1, 1f);
 
     final String C1 = C + ".c1";
     conf.setQueues(C, new String[] {"c1"});


[12/50] [abbrv] hadoop git commit: Fix up author name to Jun Gong in CHANGES.txt for YARN-3469

Posted by zj...@apache.org.
Fix up author name to Jun Gong in CHANGES.txt for YARN-3469


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3c9cf4c6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3c9cf4c6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3c9cf4c6

Branch: refs/heads/YARN-2928
Commit: 3c9cf4c6e5775eeffe7fe32a34246a57df6eb31b
Parents: 45fa332
Author: Karthik Kambatla <ka...@apache.org>
Authored: Mon Apr 13 13:52:28 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:41 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c9cf4c6/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d90d79a..044a7a9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -178,7 +178,7 @@ Release 2.8.0 - UNRELEASED
     the entire centos repository. (Ravindra Kumar Naik via raviprak)
 
     YARN-3469. ZKRMStateStore: Avoid setting watches that are not required. 
-    (Jun Hong via kasha)
+    (Jun Gong via kasha)
 
   BUG FIXES
 


[10/50] [abbrv] hadoop git commit: YARN-3394. Enrich WebApplication proxy documentation. Contributed by Naganarasimha G R

Posted by zj...@apache.org.
YARN-3394. Enrich WebApplication proxy documentation. Contributed by Naganarasimha G R


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3d744e3c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3d744e3c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3d744e3c

Branch: refs/heads/YARN-2928
Commit: 3d744e3c24b9bebee4b78ba35923c845cebc42fd
Parents: b5ca277
Author: Jian He <ji...@apache.org>
Authored: Mon Apr 13 17:05:38 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:41 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../src/site/markdown/WebApplicationProxy.md    | 50 ++++++++++++++++++--
 2 files changed, 49 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d744e3c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 88d0a18..8a02ef8 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -172,6 +172,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3293. Track and display capacity scheduler health metrics
     in web UI. (Varun Vasudev via xgong)
 
+    YARN-3394. Enrich WebApplication proxy documentation. (Naganarasimha G R
+    via jianhe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d744e3c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
index 8d6187d..5fa9a9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
@@ -15,10 +15,52 @@
 Web Application Proxy
 =====================
 
-The Web Application Proxy is part of YARN. By default it will run as part of the Resource Manager(RM), but can be configured to run in stand alone mode. The reason for the proxy is to reduce the possibility of web based attacks through YARN.
+* [Overview](#Overview)
+    * [Introduction](#Introduction)
+    * [Current Status](#Current_Status)
+* [Deployment](#Deployment)
+    * [Configurations](#Configurations)
+    * [Running Web Application Proxy](#Running_Web_Proxy)
 
-In YARN the Application Master(AM) has the responsibility to provide a web UI and to send that link to the RM. This opens up a number of potential issues. The RM runs as a trusted user, and people visiting that web address will treat it, and links it provides to them as trusted, when in reality the AM is running as a non-trusted user, and the links it gives to the RM could point to anything malicious or otherwise. The Web Application Proxy mitigates this risk by warning users that do not own the given application that they are connecting to an untrusted site.
 
-In addition to this the proxy also tries to reduce the impact that a malicious AM could have on a user. It primarily does this by stripping out cookies from the user, and replacing them with a single cookie providing the user name of the logged in user. This is because most web based authentication systems will identify a user based off of a cookie. By providing this cookie to an untrusted application it opens up the potential for an exploit. If the cookie is designed properly that potential should be fairly minimal, but this is just to reduce that potential attack vector. The current proxy implementation does nothing to prevent the AM from providing links to malicious external sites, nor does it do anything to prevent malicious javascript code from running as well. In fact javascript can be used to get the cookies, so stripping the cookies from the request has minimal benefit at this time.
+Overview
+---------
 
-In the future we hope to address the attack vectors described above and make attaching to an AM's web UI safer.
+### Introduction 
+
+ The Web Application Proxy is part of YARN. By default it will run as part of the Resource Manager(RM), but can be configured to run in stand alone mode. The reason for the proxy is to reduce the possibility of web based attacks through YARN.
+
+ In YARN the Application Master(AM) has the responsibility to provide a web UI and to send that link to the RM. This opens up a number of potential issues. The RM runs as a trusted user, and people visiting that web address will treat it, and links it provides to them as trusted, when in reality the AM is running as a non-trusted user, and the links it gives to the RM could point to anything malicious or otherwise. The Web Application Proxy mitigates this risk by warning users that do not own the given application that they are connecting to an untrusted site.
+
+ In addition to this the proxy also tries to reduce the impact that a malicious AM could have on a user. It primarily does this by stripping out cookies from the user, and replacing them with a single cookie providing the user name of the logged in user. This is because most web based authentication systems will identify a user based off of a cookie. By providing this cookie to an untrusted application it opens up the potential for an exploit. If the cookie is designed properly that potential should be fairly minimal, but this is just to reduce that potential attack vector. 
+ 
+### Current Status
+ 
+ The current proxy implementation does nothing to prevent the AM from providing links to malicious external sites, nor does it do anything to prevent malicious javascript code from running as well. In fact javascript can be used to get the cookies, so stripping the cookies from the request has minimal benefit at this time. In the future we hope to address the attack vectors described above and make attaching to an AM's web UI safer.
+
+
+Deployment
+----------
+
+###Configurations
+
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.web-proxy.address` | The address for the web proxy as HOST:PORT, if this is not given then the proxy will run as part of the RM. |
+| `yarn.web-proxy.keytab` | Keytab for WebAppProxy, if the proxy is not running as part of the RM. |
+| `yarn.web-proxy.principal` | The kerberos principal for the proxy, if the proxy is not running as part of the RM. |
+
+
+### Running Web Application Proxy
+
+  Standalone Web application proxy server can be launched with the following command. 
+
+```
+  $ yarn proxyserver
+```
+
+  Or users can start the stand alone Web Application Proxy server as a daemon, with the following command
+
+```
+  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh start proxyserver
+```


[27/50] [abbrv] hadoop git commit: YARN-3361. CapacityScheduler side changes to support non-exclusive node labels. Contributed by Wangda Tan

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
index 882498a..eb64d43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
@@ -56,8 +56,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -377,16 +375,29 @@ public class ParentQueue extends AbstractCSQueue {
 
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
-      FiCaSchedulerNode node, ResourceLimits resourceLimits) {
-    CSAssignment assignment = 
-        new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
-    Set<String> nodeLabels = node.getLabels();
-    
+      FiCaSchedulerNode node, ResourceLimits resourceLimits,
+      SchedulingMode schedulingMode) {
     // if our queue cannot access this node, just return
-    if (!SchedulerUtils.checkQueueAccessToNode(accessibleLabels, nodeLabels)) {
-      return assignment;
+    if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
+        && !accessibleToPartition(node.getPartition())) {
+      return NULL_ASSIGNMENT;
+    }
+    
+    // Check if this queue need more resource, simply skip allocation if this
+    // queue doesn't need more resources.
+    if (!super.hasPendingResourceRequest(node.getPartition(),
+        clusterResource, schedulingMode)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skip this queue=" + getQueuePath()
+            + ", because it doesn't need more resource, schedulingMode="
+            + schedulingMode.name() + " node-partition=" + node.getPartition());
+      }
+      return NULL_ASSIGNMENT;
     }
     
+    CSAssignment assignment = 
+        new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
+    
     while (canAssign(clusterResource, node)) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Trying to assign containers to child-queue of "
@@ -396,15 +407,17 @@ public class ParentQueue extends AbstractCSQueue {
       // Are we over maximum-capacity for this queue?
       // This will also consider parent's limits and also continuous reservation
       // looking
-      if (!super.canAssignToThisQueue(clusterResource, nodeLabels, resourceLimits,
-          minimumAllocation, Resources.createResource(getMetrics()
-              .getReservedMB(), getMetrics().getReservedVirtualCores()))) {
+      if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
+          resourceLimits, minimumAllocation, Resources.createResource(
+              getMetrics().getReservedMB(), getMetrics()
+                  .getReservedVirtualCores()), schedulingMode)) {
         break;
       }
       
       // Schedule
-      CSAssignment assignedToChild = 
-          assignContainersToChildQueues(clusterResource, node, resourceLimits);
+      CSAssignment assignedToChild =
+          assignContainersToChildQueues(clusterResource, node, resourceLimits,
+              schedulingMode);
       assignment.setType(assignedToChild.getType());
       
       // Done if no child-queue assigned anything
@@ -413,7 +426,7 @@ public class ParentQueue extends AbstractCSQueue {
               assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
         super.allocateResource(clusterResource, assignedToChild.getResource(),
-            nodeLabels);
+            node.getPartition());
         
         // Track resource utilization in this pass of the scheduler
         Resources
@@ -510,7 +523,8 @@ public class ParentQueue extends AbstractCSQueue {
   }
   
   private synchronized CSAssignment assignContainersToChildQueues(
-      Resource cluster, FiCaSchedulerNode node, ResourceLimits limits) {
+      Resource cluster, FiCaSchedulerNode node, ResourceLimits limits,
+      SchedulingMode schedulingMode) {
     CSAssignment assignment = 
         new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
     
@@ -523,12 +537,13 @@ public class ParentQueue extends AbstractCSQueue {
         LOG.debug("Trying to assign to queue: " + childQueue.getQueuePath()
           + " stats: " + childQueue);
       }
-      
+
       // Get ResourceLimits of child queue before assign containers
       ResourceLimits childLimits =
           getResourceLimitsOfChild(childQueue, cluster, limits);
       
-      assignment = childQueue.assignContainers(cluster, node, childLimits);
+      assignment = childQueue.assignContainers(cluster, node, 
+          childLimits, schedulingMode);
       if(LOG.isDebugEnabled()) {
         LOG.debug("Assigned to queue: " + childQueue.getQueuePath() +
           " stats: " + childQueue + " --> " + 
@@ -584,7 +599,7 @@ public class ParentQueue extends AbstractCSQueue {
       // Book keeping
       synchronized (this) {
         super.releaseResource(clusterResource, rmContainer.getContainer()
-            .getResource(), node.getLabels());
+            .getResource(), node.getPartition());
 
         LOG.info("completedContainer" +
             " queue=" + getQueueName() + 
@@ -653,7 +668,7 @@ public class ParentQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       super.allocateResource(clusterResource, rmContainer.getContainer()
-          .getResource(), node.getLabels());
+          .getResource(), node.getPartition());
     }
     if (parent != null) {
       parent.recoverContainer(clusterResource, attempt, rmContainer);
@@ -681,7 +696,7 @@ public class ParentQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       super.allocateResource(clusterResource, rmContainer.getContainer()
-          .getResource(), node.getLabels());
+          .getResource(), node.getPartition());
       LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
           + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster="
@@ -701,7 +716,7 @@ public class ParentQueue extends AbstractCSQueue {
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       super.releaseResource(clusterResource,
           rmContainer.getContainer().getResource(),
-          node.getLabels());
+          node.getPartition());
       LOG.info("movedContainer" + " queueMoveOut=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
           + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster="

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/SchedulingMode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/SchedulingMode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/SchedulingMode.java
new file mode 100644
index 0000000..7e7dc37
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/SchedulingMode.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+/**
+ * Scheduling modes, see below for detailed explanations 
+ */
+public enum SchedulingMode {
+  /**
+   * <p>
+   * When a node has partition (say partition=x), only application in the queue
+   * can access to partition=x AND requires for partition=x resource can get
+   * chance to allocate on the node.
+   * </p>
+   * 
+   * <p>
+   * When a node has no partition, only application requires non-partitioned
+   * resource can get chance to allocate on the node.
+   * </p>
+   */
+  RESPECT_PARTITION_EXCLUSIVITY,
+  
+  /**
+   * Only used when a node has partition AND the partition isn't an exclusive
+   * partition AND application requires non-partitioned resource.
+   */
+  IGNORE_PARTITION_EXCLUSIVITY
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
index 76ede39..9b7eb84 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.Task.State;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -277,6 +278,9 @@ public class Application {
     } else {
       request.setNumContainers(request.getNumContainers() + 1);
     }
+    if (request.getNodeLabelExpression() == null) {
+      request.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL);
+    }
     
     // Note this down for next interaction with ResourceManager
     ask.remove(request);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
index f62fdb3..5c107aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
@@ -150,8 +150,14 @@ public class MockAM {
   public AllocateResponse allocate(
       String host, int memory, int numContainers,
       List<ContainerId> releases, String labelExpression) throws Exception {
+    return allocate(host, memory, numContainers, 1, releases, labelExpression);
+  }
+  
+  public AllocateResponse allocate(
+      String host, int memory, int numContainers, int priority,
+      List<ContainerId> releases, String labelExpression) throws Exception {
     List<ResourceRequest> reqs =
-        createReq(new String[] { host }, memory, 1, numContainers,
+        createReq(new String[] { host }, memory, priority, numContainers,
             labelExpression);
     return allocate(reqs, releases);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 06c6b32..f2b1d86 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
@@ -200,10 +202,18 @@ public class MockRM extends ResourceManager {
   
   public boolean waitForState(MockNM nm, ContainerId containerId,
       RMContainerState containerState, int timeoutMillisecs) throws Exception {
+    return waitForState(Arrays.asList(nm), containerId, containerState,
+        timeoutMillisecs);
+  }
+  
+  public boolean waitForState(Collection<MockNM> nms, ContainerId containerId,
+      RMContainerState containerState, int timeoutMillisecs) throws Exception {
     RMContainer container = getResourceScheduler().getRMContainer(containerId);
     int timeoutSecs = 0;
     while(container == null && timeoutSecs++ < timeoutMillisecs / 100) {
-      nm.nodeHeartbeat(true);
+      for (MockNM nm : nms) {
+        nm.nodeHeartbeat(true);
+      }
       container = getResourceScheduler().getRMContainer(containerId);
       System.out.println("Waiting for container " + containerId + " to be allocated.");
       Thread.sleep(100);
@@ -217,9 +227,11 @@ public class MockRM extends ResourceManager {
         && timeoutSecs++ < timeoutMillisecs / 100) {
       System.out.println("Container : " + containerId + " State is : "
           + container.getState() + " Waiting for state : " + containerState);
-      nm.nodeHeartbeat(true);
+      for (MockNM nm : nms) {
+        nm.nodeHeartbeat(true);
+      }
       Thread.sleep(100);
-      
+
       if (timeoutMillisecs <= timeoutSecs * 100) {
         return false;
       }
@@ -650,11 +662,28 @@ public class MockRM extends ResourceManager {
     am.waitForState(RMAppAttemptState.FINISHED);
     rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
   }
+  
+  @SuppressWarnings("rawtypes")
+  private static void waitForSchedulerAppAttemptAdded(
+      ApplicationAttemptId attemptId, MockRM rm) throws InterruptedException {
+    int tick = 0;
+    // Wait for at most 5 sec
+    while (null == ((AbstractYarnScheduler) rm.getResourceScheduler())
+        .getApplicationAttempt(attemptId) && tick < 50) {
+      Thread.sleep(100);
+      if (tick % 10 == 0) {
+        System.out.println("waiting for SchedulerApplicationAttempt="
+            + attemptId + " added.");
+      }
+      tick++;
+    }
+  }
 
   public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
       throws Exception {
     rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);
     RMAppAttempt attempt = app.getCurrentAppAttempt();
+    waitForSchedulerAppAttemptAdded(attempt.getAppAttemptId(), rm);
     System.out.println("Launch AM " + attempt.getAppAttemptId());
     nm.nodeHeartbeat(true);
     MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
index 1ca5c97..46167ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
@@ -612,7 +612,7 @@ public class TestApplicationLimits {
 
     // Schedule to compute 
     queue.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource));
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     Resource expectedHeadroom = Resources.createResource(10*16*GB, 1);
     assertEquals(expectedHeadroom, app_0_0.getHeadroom());
 
@@ -632,7 +632,7 @@ public class TestApplicationLimits {
 
     // Schedule to compute 
     queue.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource)); // Schedule to compute
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); // Schedule to compute
     assertEquals(expectedHeadroom, app_0_0.getHeadroom());
     assertEquals(expectedHeadroom, app_0_1.getHeadroom());// no change
     
@@ -652,7 +652,7 @@ public class TestApplicationLimits {
     
     // Schedule to compute 
     queue.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource)); // Schedule to compute
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); // Schedule to compute
     expectedHeadroom = Resources.createResource(10*16*GB / 2, 1); // changes
     assertEquals(expectedHeadroom, app_0_0.getHeadroom());
     assertEquals(expectedHeadroom, app_0_1.getHeadroom());
@@ -661,7 +661,7 @@ public class TestApplicationLimits {
     // Now reduce cluster size and check for the smaller headroom
     clusterResource = Resources.createResource(90*16*GB);
     queue.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource)); // Schedule to compute
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); // Schedule to compute
     expectedHeadroom = Resources.createResource(9*16*GB / 2, 1); // changes
     assertEquals(expectedHeadroom, app_0_0.getHeadroom());
     assertEquals(expectedHeadroom, app_0_1.getHeadroom());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
index 23b31fa..970a98a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -133,7 +134,7 @@ public class TestChildQueueOrder {
         final Resource allocatedResource = Resources.createResource(allocation);
         if (queue instanceof ParentQueue) {
           ((ParentQueue)queue).allocateResource(clusterResource, 
-              allocatedResource, null);
+              allocatedResource, RMNodeLabelsManager.NO_LABEL);
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
@@ -145,7 +146,7 @@ public class TestChildQueueOrder {
           doReturn(new CSAssignment(Resources.none(), type)).
           when(queue)
               .assignContainers(eq(clusterResource), eq(node),
-                  any(ResourceLimits.class));
+                  any(ResourceLimits.class), any(SchedulingMode.class));
 
           // Mock the node's resource availability
           Resource available = node.getAvailableResource();
@@ -157,7 +158,7 @@ public class TestChildQueueOrder {
       }
     }).
     when(queue).assignContainers(eq(clusterResource), eq(node), 
-        any(ResourceLimits.class));
+        any(ResourceLimits.class), any(SchedulingMode.class));
     doNothing().when(node).releaseContainer(any(Container.class));
   }
 
@@ -241,6 +242,14 @@ public class TestChildQueueOrder {
     CSQueue b = queues.get(B);
     CSQueue c = queues.get(C);
     CSQueue d = queues.get(D);
+    
+    // Make a/b/c/d has >0 pending resource, so that allocation will continue.
+    queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
+        .incPending(Resources.createResource(1 * GB));
+    a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    c.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
+    d.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
 
     final String user_0 = "user_0";
 
@@ -275,7 +284,7 @@ public class TestChildQueueOrder {
     stubQueueAllocation(c, clusterResource, node_0, 0*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
     root.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource));
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     for(int i=0; i < 2; i++)
     {
       stubQueueAllocation(a, clusterResource, node_0, 0*GB);
@@ -283,7 +292,7 @@ public class TestChildQueueOrder {
       stubQueueAllocation(c, clusterResource, node_0, 0*GB);
       stubQueueAllocation(d, clusterResource, node_0, 0*GB);
       root.assignContainers(clusterResource, node_0, new ResourceLimits(
-          clusterResource));
+          clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     } 
     for(int i=0; i < 3; i++)
     {
@@ -292,7 +301,7 @@ public class TestChildQueueOrder {
       stubQueueAllocation(c, clusterResource, node_0, 1*GB);
       stubQueueAllocation(d, clusterResource, node_0, 0*GB);
       root.assignContainers(clusterResource, node_0, new ResourceLimits(
-          clusterResource));
+          clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     }  
     for(int i=0; i < 4; i++)
     {
@@ -301,7 +310,7 @@ public class TestChildQueueOrder {
       stubQueueAllocation(c, clusterResource, node_0, 0*GB);
       stubQueueAllocation(d, clusterResource, node_0, 1*GB);
       root.assignContainers(clusterResource, node_0, new ResourceLimits(
-          clusterResource));
+          clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     }    
     verifyQueueMetrics(a, 1*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
@@ -335,7 +344,7 @@ public class TestChildQueueOrder {
       stubQueueAllocation(c, clusterResource, node_0, 0*GB);
       stubQueueAllocation(d, clusterResource, node_0, 0*GB);
       root.assignContainers(clusterResource, node_0, new ResourceLimits(
-          clusterResource));
+          clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     }
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
@@ -363,7 +372,7 @@ public class TestChildQueueOrder {
     stubQueueAllocation(c, clusterResource, node_0, 0*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
     root.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource));
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyQueueMetrics(a, 2*GB, clusterResource);
     verifyQueueMetrics(b, 3*GB, clusterResource);
     verifyQueueMetrics(c, 3*GB, clusterResource);
@@ -390,7 +399,7 @@ public class TestChildQueueOrder {
     stubQueueAllocation(c, clusterResource, node_0, 0*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
     root.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource));
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
     verifyQueueMetrics(c, 3*GB, clusterResource);
@@ -405,12 +414,14 @@ public class TestChildQueueOrder {
     stubQueueAllocation(c, clusterResource, node_0, 0*GB);
     stubQueueAllocation(d, clusterResource, node_0, 1*GB);
     root.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource));
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     InOrder allocationOrder = inOrder(d,b);
-    allocationOrder.verify(d).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), any(ResourceLimits.class));
-    allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), any(ResourceLimits.class));
+    allocationOrder.verify(d).assignContainers(eq(clusterResource),
+        any(FiCaSchedulerNode.class), any(ResourceLimits.class),
+        any(SchedulingMode.class));
+    allocationOrder.verify(b).assignContainers(eq(clusterResource),
+        any(FiCaSchedulerNode.class), any(ResourceLimits.class),
+        any(SchedulingMode.class));
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
     verifyQueueMetrics(c, 3*GB, clusterResource);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
index 03b8f5c..54ba617 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -32,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -51,9 +54,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
@@ -327,387 +334,4 @@ public class TestContainerAllocation {
     rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.ALLOCATED);
     MockRM.launchAndRegisterAM(app1, rm1, nm1);
   }
-  
-  private Configuration getConfigurationWithQueueLabels(Configuration config) {
-    CapacitySchedulerConfiguration conf =
-        new CapacitySchedulerConfiguration(config);
-    
-    // Define top-level queues
-    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});
-    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
-    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100);
-
-    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-    conf.setCapacity(A, 10);
-    conf.setMaximumCapacity(A, 15);
-    conf.setAccessibleNodeLabels(A, toSet("x"));
-    conf.setCapacityByLabel(A, "x", 100);
-    
-    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-    conf.setCapacity(B, 20);
-    conf.setAccessibleNodeLabels(B, toSet("y"));
-    conf.setCapacityByLabel(B, "y", 100);
-    
-    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
-    conf.setCapacity(C, 70);
-    conf.setMaximumCapacity(C, 70);
-    conf.setAccessibleNodeLabels(C, RMNodeLabelsManager.EMPTY_STRING_SET);
-    
-    // Define 2nd-level queues
-    final String A1 = A + ".a1";
-    conf.setQueues(A, new String[] {"a1"});
-    conf.setCapacity(A1, 100);
-    conf.setMaximumCapacity(A1, 100);
-    conf.setCapacityByLabel(A1, "x", 100);
-    
-    final String B1 = B + ".b1";
-    conf.setQueues(B, new String[] {"b1"});
-    conf.setCapacity(B1, 100);
-    conf.setMaximumCapacity(B1, 100);
-    conf.setCapacityByLabel(B1, "y", 100);
-
-    final String C1 = C + ".c1";
-    conf.setQueues(C, new String[] {"c1"});
-    conf.setCapacity(C1, 100);
-    conf.setMaximumCapacity(C1, 100);
-    
-    return conf;
-  }
-  
-  private void checkTaskContainersHost(ApplicationAttemptId attemptId,
-      ContainerId containerId, ResourceManager rm, String host) {
-    YarnScheduler scheduler = rm.getRMContext().getScheduler();
-    SchedulerAppReport appReport = scheduler.getSchedulerAppInfo(attemptId);
-
-    Assert.assertTrue(appReport.getLiveContainers().size() > 0);
-    for (RMContainer c : appReport.getLiveContainers()) {
-      if (c.getContainerId().equals(containerId)) {
-        Assert.assertEquals(host, c.getAllocatedNode().getHost());
-      }
-    }
-  }
-  
-  @SuppressWarnings("unchecked")
-  private <E> Set<E> toSet(E... elements) {
-    Set<E> set = Sets.newHashSet(elements);
-    return set;
-  }
-  
-  @Test (timeout = 300000)
-  public void testContainerAllocationWithSingleUserLimits() throws Exception {
-    final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
-    mgr.init(conf);
-
-    // set node -> label
-    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
-    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
-        NodeId.newInstance("h2", 0), toSet("y")));
-
-    // inject node label manager
-    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithDefaultQueueLabels(conf)) {
-      @Override
-      public RMNodeLabelsManager createNodeLabelManager() {
-        return mgr;
-      }
-    };
-
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
-    rm1.registerNode("h2:1234", 8000); // label = y
-    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
-
-    // launch an app to queue a1 (label = x), and check all container will
-    // be allocated in h1
-    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-    
-    // A has only 10% of x, so it can only allocate one container in label=empty
-    ContainerId containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
-    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
-    Assert.assertTrue(rm1.waitForState(nm3, containerId,
-          RMContainerState.ALLOCATED, 10 * 1000));
-    // Cannot allocate 2nd label=empty container
-    containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
-    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
-    Assert.assertFalse(rm1.waitForState(nm3, containerId,
-          RMContainerState.ALLOCATED, 10 * 1000));
-
-    // A has default user limit = 100, so it can use all resource in label = x
-    // We can allocate floor(8000 / 1024) = 7 containers
-    for (int id = 3; id <= 8; id++) {
-      containerId =
-          ContainerId.newContainerId(am1.getApplicationAttemptId(), id);
-      am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
-      Assert.assertTrue(rm1.waitForState(nm1, containerId,
-          RMContainerState.ALLOCATED, 10 * 1000));
-    }
-    rm1.close();
-  }
-  
-  @Test(timeout = 300000)
-  public void testContainerAllocateWithComplexLabels() throws Exception {
-    /*
-     * Queue structure:
-     *                      root (*)
-     *                  ________________
-     *                 /                \
-     *               a x(100%), y(50%)   b y(50%), z(100%)
-     *               ________________    ______________
-     *              /                   /              \
-     *             a1 (x,y)         b1(no)              b2(y,z)
-     *               100%                          y = 100%, z = 100%
-     *                           
-     * Node structure:
-     * h1 : x
-     * h2 : y
-     * h3 : y
-     * h4 : z
-     * h5 : NO
-     * 
-     * Total resource:
-     * x: 4G
-     * y: 6G
-     * z: 2G
-     * *: 2G
-     * 
-     * Resource of
-     * a1: x=4G, y=3G, NO=0.2G
-     * b1: NO=0.9G (max=1G)
-     * b2: y=3, z=2G, NO=0.9G (max=1G)
-     * 
-     * Each node can only allocate two containers
-     */
-
-    // set node -> label
-    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y", "z"));
-    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0),
-        toSet("x"), NodeId.newInstance("h2", 0), toSet("y"),
-        NodeId.newInstance("h3", 0), toSet("y"), NodeId.newInstance("h4", 0),
-        toSet("z"), NodeId.newInstance("h5", 0),
-        RMNodeLabelsManager.EMPTY_STRING_SET));
-
-    // inject node label manager
-    MockRM rm1 = new MockRM(TestUtils.getComplexConfigurationWithQueueLabels(conf)) {
-      @Override
-      public RMNodeLabelsManager createNodeLabelManager() {
-        return mgr;
-      }
-    };
-
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 2048);
-    MockNM nm2 = rm1.registerNode("h2:1234", 2048);
-    MockNM nm3 = rm1.registerNode("h3:1234", 2048);
-    MockNM nm4 = rm1.registerNode("h4:1234", 2048);
-    MockNM nm5 = rm1.registerNode("h5:1234", 2048);
-    
-    ContainerId containerId;
-
-    // launch an app to queue a1 (label = x), and check all container will
-    // be allocated in h1
-    RMApp app1 = rm1.submitApp(1024, "app", "user", null, "a1");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    // request a container (label = y). can be allocated on nm2 
-    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
-    containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2L);
-    Assert.assertTrue(rm1.waitForState(nm2, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
-        "h2");
-
-    // launch an app to queue b1 (label = y), and check all container will
-    // be allocated in h5
-    RMApp app2 = rm1.submitApp(1024, "app", "user", null, "b1");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm5);
-
-    // request a container for AM, will succeed
-    // and now b1's queue capacity will be used, cannot allocate more containers
-    // (Maximum capacity reached)
-    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm4, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    Assert.assertFalse(rm1.waitForState(nm5, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    
-    // launch an app to queue b2
-    RMApp app3 = rm1.submitApp(1024, "app", "user", null, "b2");
-    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm5);
-
-    // request a container. try to allocate on nm1 (label = x) and nm3 (label =
-    // y,z). Will successfully allocate on nm3
-    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
-    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm1, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    Assert.assertTrue(rm1.waitForState(nm3, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
-        "h3");
-    
-    // try to allocate container (request label = z) on nm4 (label = y,z). 
-    // Will successfully allocate on nm4 only.
-    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "z");
-    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 3L);
-    Assert.assertTrue(rm1.waitForState(nm4, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
-        "h4");
-
-    rm1.close();
-  }
-
-  @Test (timeout = 120000)
-  public void testContainerAllocateWithLabels() throws Exception {
-    // set node -> label
-    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
-    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
-        NodeId.newInstance("h2", 0), toSet("y")));
-
-    // inject node label manager
-    MockRM rm1 = new MockRM(getConfigurationWithQueueLabels(conf)) {
-      @Override
-      public RMNodeLabelsManager createNodeLabelManager() {
-        return mgr;
-      }
-    };
-
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
-    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
-    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
-    
-    ContainerId containerId;
-
-    // launch an app to queue a1 (label = x), and check all container will
-    // be allocated in h1
-    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm3);
-
-    // request a container.
-    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
-    containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm2, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    Assert.assertTrue(rm1.waitForState(nm1, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
-        "h1");
-
-    // launch an app to queue b1 (label = y), and check all container will
-    // be allocated in h2
-    RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm3);
-
-    // request a container.
-    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
-    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm1, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    Assert.assertTrue(rm1.waitForState(nm2, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am2.getApplicationAttemptId(), containerId, rm1,
-        "h2");
-    
-    // launch an app to queue c1 (label = ""), and check all container will
-    // be allocated in h3
-    RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
-    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
-
-    // request a container.
-    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm2, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    Assert.assertTrue(rm1.waitForState(nm3, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
-        "h3");
-
-    rm1.close();
-  }
-  
-  @Test (timeout = 120000)
-  public void testContainerAllocateWithDefaultQueueLabels() throws Exception {
-    // This test is pretty much similar to testContainerAllocateWithLabel.
-    // Difference is, this test doesn't specify label expression in ResourceRequest,
-    // instead, it uses default queue label expression
-
-    // set node -> label
-    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
-    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
-        NodeId.newInstance("h2", 0), toSet("y")));
-
-    // inject node label manager
-    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithDefaultQueueLabels(conf)) {
-      @Override
-      public RMNodeLabelsManager createNodeLabelManager() {
-        return mgr;
-      }
-    };
-
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
-    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
-    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
-    
-    ContainerId containerId;
-
-    // launch an app to queue a1 (label = x), and check all container will
-    // be allocated in h1
-    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    // request a container.
-    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm3, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    Assert.assertTrue(rm1.waitForState(nm1, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
-        "h1");
-
-    // launch an app to queue b1 (label = y), and check all container will
-    // be allocated in h2
-    RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // request a container.
-    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm3, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    Assert.assertTrue(rm1.waitForState(nm2, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am2.getApplicationAttemptId(), containerId, rm1,
-        "h2");
-    
-    // launch an app to queue c1 (label = ""), and check all container will
-    // be allocated in h3
-    RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
-    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
-
-    // request a container.
-    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
-    Assert.assertFalse(rm1.waitForState(nm2, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    Assert.assertTrue(rm1.waitForState(nm3, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
-        "h3");
-
-    rm1.close();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/025787b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 972cabb..0b5250b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -351,7 +351,7 @@ public class TestLeafQueue {
     
     // Only 1 container
     a.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource));
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(
         (int)(node_0.getTotalResource().getMemory() * a.getCapacity()) - (1*GB),
         a.getMetrics().getAvailableMB());
@@ -487,7 +487,7 @@ public class TestLeafQueue {
     
     // Only 1 container
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -498,7 +498,7 @@ public class TestLeafQueue {
     // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
     // you can get one container more than user-limit
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -507,7 +507,7 @@ public class TestLeafQueue {
     
     // Can't allocate 3rd due to user-limit
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -517,7 +517,7 @@ public class TestLeafQueue {
     // Bump up user-limit-factor, now allocate should work
     a.setUserLimitFactor(10);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(3*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -526,7 +526,7 @@ public class TestLeafQueue {
 
     // One more should work, for app_1, due to user-limit-factor
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(4*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@@ -537,7 +537,7 @@ public class TestLeafQueue {
     // Now - no more allocs since we are at max-cap
     a.setMaxCapacity(0.5f);
     a.assignContainers(clusterResource, node_0, new ResourceLimits(
-        clusterResource));
+        clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(4*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@@ -653,21 +653,21 @@ public class TestLeafQueue {
 
     // 1 container to user_0
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
 
     // Again one to user_0 since he hasn't exceeded user limit yet
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(3*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
 
     // One more to user_0 since he is the only active user
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(4*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(2*GB, app_1.getCurrentConsumption().getMemory());
@@ -719,10 +719,10 @@ public class TestLeafQueue {
         1, qb.getActiveUsersManager().getNumActiveUsers());
     //get headroom
     qb.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
         .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
-        null);
+        "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
 
     //maxqueue 16G, userlimit 13G, - 4G used = 9G
     assertEquals(9*GB,app_0.getHeadroom().getMemory());
@@ -739,10 +739,10 @@ public class TestLeafQueue {
             u1Priority, recordFactory)));
     qb.submitApplicationAttempt(app_2, user_1);
     qb.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
         .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
-        null);
+        "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
 
     assertEquals(8*GB, qb.getUsedResources().getMemory());
     assertEquals(4*GB, app_0.getCurrentConsumption().getMemory());
@@ -782,12 +782,12 @@ public class TestLeafQueue {
     qb.submitApplicationAttempt(app_1, user_0);
     qb.submitApplicationAttempt(app_3, user_1);
     qb.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     qb.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
         .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
-        null);
+        "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(4*GB, qb.getUsedResources().getMemory());
     //maxqueue 16G, userlimit 7G, used (by each user) 2G, headroom 5G (both)
     assertEquals(5*GB, app_3.getHeadroom().getMemory());
@@ -803,13 +803,13 @@ public class TestLeafQueue {
               TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true,
                       u0Priority, recordFactory)));
     qb.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     qb.computeUserLimitAndSetHeadroom(app_4, clusterResource, app_4
         .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
-        null);
+        "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
         .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
-        null);
+        "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     
     
     //app3 is user1, active from last test case
@@ -876,7 +876,7 @@ public class TestLeafQueue {
                 priority, recordFactory)));
 
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -893,7 +893,7 @@ public class TestLeafQueue {
             priority, recordFactory)));
 
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@@ -982,7 +982,7 @@ public class TestLeafQueue {
 
     // 1 container to user_0
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -993,7 +993,7 @@ public class TestLeafQueue {
 
     // Again one to user_0 since he hasn't exceeded user limit yet
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(3*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@@ -1010,7 +1010,7 @@ public class TestLeafQueue {
     // No more to user_0 since he is already over user-limit
     // and no more containers to queue since it's already at max-cap
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(3*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@@ -1024,7 +1024,7 @@ public class TestLeafQueue {
             priority, recordFactory)));
     assertEquals(1, a.getActiveUsersManager().getNumActiveUsers());
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(0*GB, app_2.getHeadroom().getMemory());   // hit queue max-cap 
   }
 
@@ -1095,7 +1095,7 @@ public class TestLeafQueue {
     
     // Only 1 container
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1103,7 +1103,7 @@ public class TestLeafQueue {
     // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
     // you can get one container more than user-limit
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1111,7 +1111,7 @@ public class TestLeafQueue {
     // Can't allocate 3rd due to user-limit
     a.setUserLimit(25);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1130,7 +1130,7 @@ public class TestLeafQueue {
     // user_0 is at limit inspite of high user-limit-factor
     a.setUserLimitFactor(10);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1140,7 +1140,7 @@ public class TestLeafQueue {
     // Now allocations should goto app_0 since 
     // user_0 is at user-limit not above it
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(6*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1151,7 +1151,7 @@ public class TestLeafQueue {
     // Now - no more allocs since we are at max-cap
     a.setMaxCapacity(0.5f);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(6*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1163,7 +1163,7 @@ public class TestLeafQueue {
     a.setMaxCapacity(1.0f);
     a.setUserLimitFactor(1);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(7*GB, a.getUsedResources().getMemory()); 
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1172,7 +1172,7 @@ public class TestLeafQueue {
 
     // Now we should assign to app_3 again since user_2 is under user-limit
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8*GB, a.getUsedResources().getMemory()); 
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1272,7 +1272,7 @@ public class TestLeafQueue {
     
     // Only 1 container
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1283,7 +1283,7 @@ public class TestLeafQueue {
     // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
     // you can get one container more than user-limit
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1292,7 +1292,7 @@ public class TestLeafQueue {
     
     // Now, reservation should kick in for app_1
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(6*GB, a.getUsedResources().getMemory()); 
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1309,7 +1309,7 @@ public class TestLeafQueue {
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5*GB, a.getUsedResources().getMemory()); 
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1326,7 +1326,7 @@ public class TestLeafQueue {
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(4*GB, a.getUsedResources().getMemory());
     assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(4*GB, app_1.getCurrentConsumption().getMemory());
@@ -1394,7 +1394,7 @@ public class TestLeafQueue {
     // Start testing...
 
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1404,7 +1404,7 @@ public class TestLeafQueue {
 
     // Now, reservation should kick in for app_1
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(6*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1418,7 +1418,7 @@ public class TestLeafQueue {
     doReturn(-1).when(a).getNodeLocalityDelay();
     
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(10*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(4*GB, app_1.getCurrentConsumption().getMemory());
@@ -1435,7 +1435,7 @@ public class TestLeafQueue {
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8*GB, a.getUsedResources().getMemory());
     assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(8*GB, app_1.getCurrentConsumption().getMemory());
@@ -1504,7 +1504,7 @@ public class TestLeafQueue {
     
     // Only 1 container
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1512,14 +1512,14 @@ public class TestLeafQueue {
     // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
     // you can get one container more than user-limit
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
     
     // Now, reservation should kick in for app_1
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(6*GB, a.getUsedResources().getMemory()); 
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1534,7 +1534,7 @@ public class TestLeafQueue {
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5*GB, a.getUsedResources().getMemory()); 
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1544,7 +1544,7 @@ public class TestLeafQueue {
 
     // Re-reserve
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(5*GB, a.getUsedResources().getMemory()); 
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@@ -1554,7 +1554,7 @@ public class TestLeafQueue {
     
     // Try to schedule on node_1 now, should *move* the reservation
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(9*GB, a.getUsedResources().getMemory()); 
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(4*GB, app_1.getCurrentConsumption().getMemory());
@@ -1572,7 +1572,7 @@ public class TestLeafQueue {
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
     CSAssignment assignment = a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(8*GB, a.getUsedResources().getMemory());
     assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(4*GB, app_1.getCurrentConsumption().getMemory());
@@ -1644,7 +1644,7 @@ public class TestLeafQueue {
     
     // Start with off switch, shouldn't allocate due to delay scheduling
     assignment = a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(1, app_0.getSchedulingOpportunities(priority));
@@ -1653,7 +1653,7 @@ public class TestLeafQueue {
 
     // Another off switch, shouldn't allocate due to delay scheduling
     assignment = a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(2, app_0.getSchedulingOpportunities(priority));
@@ -1662,7 +1662,7 @@ public class TestLeafQueue {
     
     // Another off switch, shouldn't allocate due to delay scheduling
     assignment = a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(3, app_0.getSchedulingOpportunities(priority));
@@ -1672,7 +1672,7 @@ public class TestLeafQueue {
     // Another off switch, now we should allocate 
     // since missedOpportunities=3 and reqdContainers=3
     assignment = a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(4, app_0.getSchedulingOpportunities(priority)); // should NOT reset
@@ -1681,7 +1681,7 @@ public class TestLeafQueue {
     
     // NODE_LOCAL - node_0
     assignment = a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@@ -1690,7 +1690,7 @@ public class TestLeafQueue {
     
     // NODE_LOCAL - node_1
     assignment = a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@@ -1719,14 +1719,14 @@ public class TestLeafQueue {
     
     // Shouldn't assign RACK_LOCAL yet
     assignment = a.assignContainers(clusterResource, node_3,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertEquals(1, app_0.getSchedulingOpportunities(priority));
     assertEquals(2, app_0.getTotalRequiredResources(priority));
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
 
     // Should assign RACK_LOCAL now
     assignment = a.assignContainers(clusterResource, node_3,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.RACK_LOCAL), eq(node_3), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@@ -1808,7 +1808,7 @@ public class TestLeafQueue {
     // Start with off switch, shouldn't allocate P1 due to delay scheduling
     // thus, no P2 either!
     a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
     assertEquals(1, app_0.getSchedulingOpportunities(priority_1));
@@ -1821,7 +1821,7 @@ public class TestLeafQueue {
     // Another off-switch, shouldn't allocate P1 due to delay scheduling
     // thus, no P2 either!
     a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
     assertEquals(2, app_0.getSchedulingOpportunities(priority_1));
@@ -1833,7 +1833,7 @@ public class TestLeafQueue {
 
     // Another off-switch, shouldn't allocate OFF_SWITCH P1
     a.assignContainers(clusterResource, node_2,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
     assertEquals(3, app_0.getSchedulingOpportunities(priority_1));
@@ -1845,7 +1845,7 @@ public class TestLeafQueue {
 
     // Now, DATA_LOCAL for P1
     a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority_1));
@@ -1857,7 +1857,7 @@ public class TestLeafQueue {
 
     // Now, OFF_SWITCH for P2
     a.assignContainers(clusterResource, node_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority_1));
@@ -1934,7 +1934,7 @@ public class TestLeafQueue {
     
     // NODE_LOCAL - node_0_1
     a.assignContainers(clusterResource, node_0_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@@ -1943,7 +1943,7 @@ public class TestLeafQueue {
     // No allocation on node_1_0 even though it's node/rack local since
     // required(ANY) == 0
     a.assignContainers(clusterResource, node_1_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // Still zero
@@ -1960,7 +1960,7 @@ public class TestLeafQueue {
     // No allocation on node_0_1 even though it's node/rack local since
     // required(rack_1) == 0
     a.assignContainers(clusterResource, node_0_1,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(1, app_0.getSchedulingOpportunities(priority)); 
@@ -1968,7 +1968,7 @@ public class TestLeafQueue {
     
     // NODE_LOCAL - node_1
     a.assignContainers(clusterResource, node_1_0,
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@@ -2221,7 +2221,7 @@ public class TestLeafQueue {
     // node_0_1  
     // Shouldn't allocate since RR(rack_0) = null && RR(ANY) = relax: false
     a.assignContainers(clusterResource, node_0_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
@@ -2244,7 +2244,7 @@ public class TestLeafQueue {
     // node_1_1  
     // Shouldn't allocate since RR(rack_1) = relax: false
     a.assignContainers(clusterResource, node_1_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
@@ -2275,7 +2275,7 @@ public class TestLeafQueue {
     // node_1_1  
     // Shouldn't allocate since node_1_1 is blacklisted
     a.assignContainers(clusterResource, node_1_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
@@ -2304,7 +2304,7 @@ public class TestLeafQueue {
     // node_1_1  
     // Shouldn't allocate since rack_1 is blacklisted
     a.assignContainers(clusterResource, node_1_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
@@ -2331,7 +2331,7 @@ public class TestLeafQueue {
 
     // Now, should allocate since RR(rack_1) = relax: true
     a.assignContainers(clusterResource, node_1_1, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0,never()).allocate(eq(NodeType.RACK_LOCAL), eq(node_1_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); 
@@ -2362,7 +2362,7 @@ public class TestLeafQueue {
     // host_1_1: 7G
 
     a.assignContainers(clusterResource, node_1_0, 
-        new ResourceLimits(clusterResource));
+        new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
     assertEquals(0, app_0.getSchedulingOpportunities(priority)); 
@@ -2445,7 +2445,7 @@ public class TestLeafQueue {
 
     try {
       a.assignContainers(clusterResource, node_0, 
-          new ResourceLimits(clusterResource));
+          new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     } catch (NullPointerException e) {
       Assert.fail("NPE when allocating container on node but "
           + "forget to set off-switch request should be handled");


[16/50] [abbrv] hadoop git commit: HDFS-7701. Support reporting per storage type quota and usage with hadoop/hdfs shell. (Contributed by Peter Shi)

Posted by zj...@apache.org.
HDFS-7701. Support reporting per storage type quota and usage with hadoop/hdfs shell. (Contributed by Peter Shi)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/40b72486
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/40b72486
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/40b72486

Branch: refs/heads/YARN-2928
Commit: 40b72486acc967e8650b4949f7ea4b1d0c2f22e6
Parents: ae7a5ff
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Apr 13 21:01:15 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:42 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/ContentSummary.java    |  89 ++++++++++--
 .../apache/hadoop/fs/shell/CommandFormat.java   |  49 ++++++-
 .../java/org/apache/hadoop/fs/shell/Count.java  |  73 +++++++++-
 .../org/apache/hadoop/fs/shell/TestCount.java   | 142 ++++++++++++++++++-
 .../src/test/resources/testConf.xml             |   2 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 6 files changed, 334 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b72486/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
index 66137d0..ccd6960 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.fs;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.List;
 
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.Writable;
@@ -255,6 +255,8 @@ public class ContentSummary implements Writable{
   private static final String QUOTA_SUMMARY_FORMAT = "%12s %15s ";
   private static final String SPACE_QUOTA_SUMMARY_FORMAT = "%15s %15s ";
 
+  private static final String STORAGE_TYPE_SUMMARY_FORMAT = "%13s %17s ";
+
   private static final String[] HEADER_FIELDS = new String[] { "DIR_COUNT",
       "FILE_COUNT", "CONTENT_SIZE"};
   private static final String[] QUOTA_HEADER_FIELDS = new String[] { "QUOTA",
@@ -268,7 +270,11 @@ public class ContentSummary implements Writable{
       QUOTA_SUMMARY_FORMAT + SPACE_QUOTA_SUMMARY_FORMAT,
       (Object[]) QUOTA_HEADER_FIELDS) +
       HEADER;
-  
+
+  /** default quota display string */
+  private static final String QUOTA_NONE = "none";
+  private static final String QUOTA_INF = "inf";
+
   /** Return the header of the output.
    * if qOption is false, output directory count, file count, and content size;
    * if qOption is true, output quota and remaining quota as well.
@@ -281,6 +287,26 @@ public class ContentSummary implements Writable{
   }
 
   /**
+   * return the header of with the StorageTypes
+   *
+   * @param storageTypes
+   * @return storage header string
+   */
+  public static String getStorageTypeHeader(List<StorageType> storageTypes) {
+    StringBuffer header = new StringBuffer();
+
+    for (StorageType st : storageTypes) {
+      /* the field length is 13/17 for quota and remain quota
+       * as the max length for quota name is ARCHIVE_QUOTA
+        * and remain quota name REM_ARCHIVE_QUOTA */
+      String storageName = st.toString();
+      header.append(String.format(STORAGE_TYPE_SUMMARY_FORMAT, storageName + "_QUOTA",
+          "REM_" + storageName + "_QUOTA"));
+    }
+    return header.toString();
+  }
+
+  /**
    * Returns the names of the fields from the summary header.
    * 
    * @return names of fields as displayed in the header
@@ -325,13 +351,49 @@ public class ContentSummary implements Writable{
    * @return the string representation of the object
    */
   public String toString(boolean qOption, boolean hOption) {
+    return toString(qOption, hOption, false, null);
+  }
+
+  /**
+   * Return the string representation of the object in the output format.
+   * if tOption is true, display the quota by storage types,
+   * Otherwise, same logic with #toString(boolean,boolean)
+   *
+   * @param qOption a flag indicating if quota needs to be printed or not
+   * @param hOption a flag indicating if human readable output if to be used
+   * @param tOption a flag indicating if display quota by storage types
+   * @param types Storage types to display
+   * @return the string representation of the object
+   */
+  public String toString(boolean qOption, boolean hOption,
+                         boolean tOption, List<StorageType> types) {
     String prefix = "";
+
+    if (tOption) {
+      StringBuffer content = new StringBuffer();
+      for (StorageType st : types) {
+        long typeQuota = getTypeQuota(st);
+        long typeConsumed = getTypeConsumed(st);
+        String quotaStr = QUOTA_NONE;
+        String quotaRem = QUOTA_INF;
+
+        if (typeQuota > 0) {
+          quotaStr = formatSize(typeQuota, hOption);
+          quotaRem = formatSize(typeQuota - typeConsumed, hOption);
+        }
+
+        content.append(String.format(STORAGE_TYPE_SUMMARY_FORMAT,
+            quotaStr, quotaRem));
+      }
+      return content.toString();
+    }
+
     if (qOption) {
-      String quotaStr = "none";
-      String quotaRem = "inf";
-      String spaceQuotaStr = "none";
-      String spaceQuotaRem = "inf";
-      
+      String quotaStr = QUOTA_NONE;
+      String quotaRem = QUOTA_INF;
+      String spaceQuotaStr = QUOTA_NONE;
+      String spaceQuotaRem = QUOTA_INF;
+
       if (quota>0) {
         quotaStr = formatSize(quota, hOption);
         quotaRem = formatSize(quota-(directoryCount+fileCount), hOption);
@@ -340,16 +402,17 @@ public class ContentSummary implements Writable{
         spaceQuotaStr = formatSize(spaceQuota, hOption);
         spaceQuotaRem = formatSize(spaceQuota - spaceConsumed, hOption);
       }
-      
+
       prefix = String.format(QUOTA_SUMMARY_FORMAT + SPACE_QUOTA_SUMMARY_FORMAT,
-                             quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
+          quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
     }
-    
+
     return prefix + String.format(SUMMARY_FORMAT,
-     formatSize(directoryCount, hOption),
-     formatSize(fileCount, hOption),
-     formatSize(length, hOption));
+        formatSize(directoryCount, hOption),
+        formatSize(fileCount, hOption),
+        formatSize(length, hOption));
   }
+
   /**
    * Formats a size to be human readable or in bytes
    * @param size value to be formatted

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b72486/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
index e1aeea9..0f9aa38 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
@@ -31,6 +31,7 @@ import java.util.Set;
 public class CommandFormat {
   final int minPar, maxPar;
   final Map<String, Boolean> options = new HashMap<String, Boolean>();
+  final Map<String, String> optionsWithValue = new HashMap<String, String>();
   boolean ignoreUnknownOpts = false;
   
   /**
@@ -64,6 +65,18 @@ public class CommandFormat {
     }
   }
 
+  /**
+   * add option with value
+   *
+   * @param option option name
+   */
+  public void addOptionWithValue(String option) {
+    if (options.containsKey(option)) {
+      throw new DuplicatedOptionException(option);
+    }
+    optionsWithValue.put(option, null);
+  }
+
   /** Parse parameters starting from the given position
    * Consider using the variant that directly takes a List
    * 
@@ -99,6 +112,17 @@ public class CommandFormat {
       if (options.containsKey(opt)) {
         args.remove(pos);
         options.put(opt, Boolean.TRUE);
+      } else if (optionsWithValue.containsKey(opt)) {
+        args.remove(pos);
+        if (pos < args.size() && (args.size() > minPar)) {
+          arg = args.get(pos);
+          args.remove(pos);
+        } else {
+          arg = "";
+        }
+        if (!arg.startsWith("-") || arg.equals("-")) {
+          optionsWithValue.put(opt, arg);
+        }
       } else if (ignoreUnknownOpts) {
         pos++;
       } else {
@@ -122,7 +146,19 @@ public class CommandFormat {
   public boolean getOpt(String option) {
     return options.containsKey(option) ? options.get(option) : false;
   }
-  
+
+  /**
+   * get the option's value
+   *
+   * @param option option name
+   * @return option value
+   * if option exists, but no value assigned, return ""
+   * if option not exists, return null
+   */
+  public String getOptValue(String option) {
+    return optionsWithValue.get(option);
+  }
+
   /** Returns all the options that are set
    * 
    * @return Set<String> of the enabled options
@@ -203,4 +239,15 @@ public class CommandFormat {
       return option;
     }
   }
+
+  /**
+   * Used when a duplicated option is supplied to a command.
+   */
+  public static class DuplicatedOptionException extends IllegalArgumentException {
+    private static final long serialVersionUID = 0L;
+
+    public DuplicatedOptionException(String duplicatedOption) {
+      super("option " + duplicatedOption + " already exsits!");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b72486/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
index dd7d168..c615876 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
@@ -18,8 +18,10 @@
 package org.apache.hadoop.fs.shell;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedList;
+import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -27,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.StorageType;
 
 /**
  * Count the number of directories, files, bytes, quota, and remaining quota.
@@ -46,11 +49,12 @@ public class Count extends FsCommand {
   private static final String OPTION_QUOTA = "q";
   private static final String OPTION_HUMAN = "h";
   private static final String OPTION_HEADER = "v";
+  private static final String OPTION_TYPE = "t";
 
   public static final String NAME = "count";
   public static final String USAGE =
       "[-" + OPTION_QUOTA + "] [-" + OPTION_HUMAN + "] [-" + OPTION_HEADER
-          + "] <path> ...";
+          + "] [-" + OPTION_TYPE + " [<storage type>]] <path> ...";
   public static final String DESCRIPTION =
       "Count the number of directories, files and bytes under the paths\n" +
           "that match the specified file pattern.  The output columns are:\n" +
@@ -63,10 +67,19 @@ public class Count extends FsCommand {
           " PATHNAME\n" +
           "The -" + OPTION_HUMAN +
           " option shows file sizes in human readable format.\n" +
-          "The -" + OPTION_HEADER + " option displays a header line.";
+          "The -" + OPTION_HEADER + " option displays a header line.\n" +
+          "The -" + OPTION_TYPE + " option displays quota by storage types.\n" +
+          "It must be used with -" + OPTION_QUOTA + " option.\n" +
+          "If a comma-separated list of storage types is given after the -" +
+          OPTION_TYPE + " option, \n" +
+          "it displays the quota and usage for the specified types. \n" +
+          "Otherwise, it displays the quota and usage for all the storage \n" +
+          "types that support quota";
 
   private boolean showQuotas;
   private boolean humanReadable;
+  private boolean showQuotabyType;
+  private List<StorageType> storageTypes = null;
 
   /** Constructor */
   public Count() {}
@@ -87,21 +100,54 @@ public class Count extends FsCommand {
   protected void processOptions(LinkedList<String> args) {
     CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE,
         OPTION_QUOTA, OPTION_HUMAN, OPTION_HEADER);
+    cf.addOptionWithValue(OPTION_TYPE);
     cf.parse(args);
     if (args.isEmpty()) { // default path is the current working directory
       args.add(".");
     }
     showQuotas = cf.getOpt(OPTION_QUOTA);
     humanReadable = cf.getOpt(OPTION_HUMAN);
+
+    if (showQuotas) {
+      String types = cf.getOptValue(OPTION_TYPE);
+
+      if (null != types) {
+        showQuotabyType = true;
+        storageTypes = getAndCheckStorageTypes(types);
+      } else {
+        showQuotabyType = false;
+      }
+    }
+
     if (cf.getOpt(OPTION_HEADER)) {
-      out.println(ContentSummary.getHeader(showQuotas) + "PATHNAME");
+      if (showQuotabyType) {
+        out.println(ContentSummary.getStorageTypeHeader(storageTypes) + "PATHNAME");
+      } else {
+        out.println(ContentSummary.getHeader(showQuotas) + "PATHNAME");
+      }
     }
   }
 
+  private List<StorageType> getAndCheckStorageTypes(String types) {
+    if ("".equals(types) || "all".equalsIgnoreCase(types)) {
+      return StorageType.getTypesSupportingQuota();
+    }
+
+    String[] typeArray = StringUtils.split(types, ',');
+    List<StorageType> stTypes = new ArrayList<>();
+
+    for (String t : typeArray) {
+      stTypes.add(StorageType.parseStorageType(t));
+    }
+
+    return stTypes;
+  }
+
   @Override
   protected void processPath(PathData src) throws IOException {
     ContentSummary summary = src.fs.getContentSummary(src.path);
-    out.println(summary.toString(showQuotas, isHumanReadable()) + src);
+    out.println(summary.toString(showQuotas, isHumanReadable(),
+        showQuotabyType, storageTypes) + src);
   }
   
   /**
@@ -121,4 +167,23 @@ public class Count extends FsCommand {
   boolean isHumanReadable() {
     return humanReadable;
   }
+
+  /**
+   * should print quota by storage types
+   * @return true if enables quota by storage types
+   */
+  @InterfaceAudience.Private
+  boolean isShowQuotabyType() {
+    return showQuotabyType;
+  }
+
+  /**
+   * show specified storage types
+   * @return specified storagetypes
+   */
+  @InterfaceAudience.Private
+  List<StorageType> getStorageTypes() {
+    return storageTypes;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b72486/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
index d5f097d..22d9a21 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
@@ -24,13 +24,15 @@ import java.io.PrintStream;
 import java.io.IOException;
 import java.net.URI;
 import java.util.LinkedList;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.shell.CommandFormat.NotEnoughArgumentsException;
 import org.junit.Test;
 import org.junit.Before;
@@ -79,11 +81,17 @@ public class TestCount {
     LinkedList<String> options = new LinkedList<String>();
     options.add("-q");
     options.add("-h");
+    options.add("-t");
+    options.add("SSD");
     options.add("dummy");
     Count count = new Count();
     count.processOptions(options);
     assertTrue(count.isShowQuotas());
     assertTrue(count.isHumanReadable());
+    assertTrue(count.isShowQuotabyType());
+    assertEquals(1, count.getStorageTypes().size());
+    assertEquals(StorageType.SSD, count.getStorageTypes().get(0));
+
   }
 
   // check no options is handled correctly
@@ -254,6 +262,112 @@ public class TestCount {
   }
 
   @Test
+  public void processPathWithQuotasByStorageTypesHeader() throws Exception {
+    Path path = new Path("mockfs:/test");
+
+    when(mockFs.getFileStatus(eq(path))).thenReturn(fileStat);
+
+    PrintStream out = mock(PrintStream.class);
+
+    Count count = new Count();
+    count.out = out;
+
+    LinkedList<String> options = new LinkedList<String>();
+    options.add("-q");
+    options.add("-v");
+    options.add("-t");
+    options.add("all");
+    options.add("dummy");
+    count.processOptions(options);
+    String withStorageTypeHeader =
+        // <----13---> <-------17------> <----13-----> <------17------->
+        "   DISK_QUOTA    REM_DISK_QUOTA     SSD_QUOTA     REM_SSD_QUOTA " +
+        // <----13---> <-------17------>
+        "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
+        "PATHNAME";
+    verify(out).println(withStorageTypeHeader);
+    verifyNoMoreInteractions(out);
+  }
+
+  @Test
+  public void processPathWithQuotasBySSDStorageTypesHeader() throws Exception {
+    Path path = new Path("mockfs:/test");
+
+    when(mockFs.getFileStatus(eq(path))).thenReturn(fileStat);
+
+    PrintStream out = mock(PrintStream.class);
+
+    Count count = new Count();
+    count.out = out;
+
+    LinkedList<String> options = new LinkedList<String>();
+    options.add("-q");
+    options.add("-v");
+    options.add("-t");
+    options.add("SSD");
+    options.add("dummy");
+    count.processOptions(options);
+    String withStorageTypeHeader =
+        // <----13---> <-------17------>
+        "    SSD_QUOTA     REM_SSD_QUOTA " +
+        "PATHNAME";
+    verify(out).println(withStorageTypeHeader);
+    verifyNoMoreInteractions(out);
+  }
+
+  @Test
+  public void processPathWithQuotasByMultipleStorageTypesContent() throws Exception {
+    Path path = new Path("mockfs:/test");
+
+    when(mockFs.getFileStatus(eq(path))).thenReturn(fileStat);
+    PathData pathData = new PathData(path.toString(), conf);
+
+    PrintStream out = mock(PrintStream.class);
+
+    Count count = new Count();
+    count.out = out;
+
+    LinkedList<String> options = new LinkedList<String>();
+    options.add("-q");
+    options.add("-t");
+    options.add("SSD,DISK");
+    options.add("dummy");
+    count.processOptions(options);
+    count.processPath(pathData);
+    String withStorageType = BYTES + StorageType.SSD.toString()
+        + " " + StorageType.DISK.toString() + " " + pathData.toString();
+    verify(out).println(withStorageType);
+    verifyNoMoreInteractions(out);
+  }
+
+  @Test
+  public void processPathWithQuotasByMultipleStorageTypes() throws Exception {
+    Path path = new Path("mockfs:/test");
+
+    when(mockFs.getFileStatus(eq(path))).thenReturn(fileStat);
+
+    PrintStream out = mock(PrintStream.class);
+
+    Count count = new Count();
+    count.out = out;
+
+    LinkedList<String> options = new LinkedList<String>();
+    options.add("-q");
+    options.add("-v");
+    options.add("-t");
+    options.add("SSD,DISK");
+    options.add("dummy");
+    count.processOptions(options);
+    String withStorageTypeHeader =
+        // <----13---> <------17------->
+        "    SSD_QUOTA     REM_SSD_QUOTA " +
+        "   DISK_QUOTA    REM_DISK_QUOTA " +
+        "PATHNAME";
+    verify(out).println(withStorageTypeHeader);
+    verifyNoMoreInteractions(out);
+  }
+
+  @Test
   public void getCommandName() {
     Count count = new Count();
     String actual = count.getCommandName();
@@ -289,7 +403,7 @@ public class TestCount {
   public void getUsage() {
     Count count = new Count();
     String actual = count.getUsage();
-    String expected = "-count [-q] [-h] [-v] <path> ...";
+    String expected = "-count [-q] [-h] [-v] [-t [<storage type>]] <path> ...";
     assertEquals("Count.getUsage", expected, actual);
   }
 
@@ -306,7 +420,13 @@ public class TestCount {
         + "QUOTA REM_QUOTA SPACE_QUOTA REM_SPACE_QUOTA\n"
         + "      DIR_COUNT FILE_COUNT CONTENT_SIZE PATHNAME\n"
         + "The -h option shows file sizes in human readable format.\n"
-        + "The -v option displays a header line.";
+        + "The -v option displays a header line.\n"
+        + "The -t option displays quota by storage types.\n"
+        + "It must be used with -q option.\n"
+        + "If a comma-separated list of storage types is given after the -t option, \n"
+        + "it displays the quota and usage for the specified types. \n"
+        + "Otherwise, it displays the quota and usage for all the storage \n"
+        + "types that support quota";
 
     assertEquals("Count.getDescription", expected, actual);
   }
@@ -321,7 +441,19 @@ public class TestCount {
     }
 
     @Override
-    public String toString(boolean qOption, boolean hOption) {
+    public String toString(boolean qOption, boolean hOption,
+                           boolean tOption, List<StorageType> types) {
+      if (tOption) {
+        StringBuffer result = new StringBuffer();
+        result.append(hOption ? HUMAN : BYTES);
+
+        for (StorageType type : types) {
+          result.append(type.toString());
+          result.append(" ");
+        }
+        return result.toString();
+      }
+
       if (qOption) {
         if (hOption) {
           return (HUMAN + WITH_QUOTAS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b72486/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
index ac28192..9b72960 100644
--- a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
+++ b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
@@ -262,7 +262,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-count \[-q\] \[-h\] \[-v\] &lt;path&gt; \.\.\. :( )*</expected-output>
+          <expected-output>^-count \[-q\] \[-h\] \[-v\] \[-t \[&lt;storage type&gt;\]\] &lt;path&gt; \.\.\. :( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b72486/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1aaf42c..7414d33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -491,6 +491,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8111. NPE thrown when invalid FSImage filename given for
     'hdfs oiv_legacy' cmd ( surendra singh lilhore via vinayakumarb )
 
+    HDFS-7701. Support reporting per storage type quota and usage
+    with hadoop/hdfs shell. (Peter Shi via Arpit Agarwal)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[32/50] [abbrv] hadoop git commit: HDFS-8149. The footer of the Web UI "Hadoop, 2014" is old. Contributed by Brahma Reddy Battula.

Posted by zj...@apache.org.
HDFS-8149. The footer of the Web UI "Hadoop, 2014" is old. Contributed by Brahma Reddy Battula.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/43e93200
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/43e93200
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/43e93200

Branch: refs/heads/YARN-2928
Commit: 43e932008f4c438b12f66947be5859f6d367dfe9
Parents: 504fab9
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Apr 15 15:31:04 2015 -0500
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hadoop-hdfs/src/main/webapps/datanode/index.html              | 2 +-
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html              | 2 +-
 .../hadoop-hdfs/src/main/webapps/hdfs/explorer.html               | 2 +-
 .../hadoop-hdfs/src/main/webapps/journal/index.html               | 2 +-
 .../hadoop-hdfs/src/main/webapps/secondary/status.html            | 2 +-
 6 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43e93200/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 60fff16..6523423 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -528,6 +528,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-7934. Update RollingUpgrade rollback documentation: should use
     bootstrapstandby for standby NN. (J. Andreina via jing9)
 
+    HDFS-8149. The footer of the Web UI "Hadoop, 2014" is old.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43e93200/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/index.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/index.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/index.html
index 04cb703..aeb61c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/index.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/index.html
@@ -47,7 +47,7 @@
 
 <div class="row">
   <hr />
-  <div class="col-xs-2"><p>Hadoop, 2014.</p></div>
+  <div class="col-xs-2"><p>Hadoop, 2015.</p></div>
 </div>
 </div>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43e93200/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
index 928431c..5a3a309 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
@@ -67,7 +67,7 @@
 
 <div class="row">
   <hr />
-  <div class="col-xs-2"><p>Hadoop, 2014.</p></div>
+  <div class="col-xs-2"><p>Hadoop, 2015.</p></div>
 </div>
 </div>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43e93200/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index cd6623c..fbea6ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -142,7 +142,7 @@
 
       <div class="row">
         <hr />
-        <div class="col-xs-2"><p>Hadoop, 2014.</p></div>
+        <div class="col-xs-2"><p>Hadoop, 2015.</p></div>
       </div>
 
     </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43e93200/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html
index 5eabf64..4d658c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html
@@ -47,7 +47,7 @@
 
 <div class="row">
   <hr />
-  <div class="col-xs-2"><p>Hadoop, 2014.</p></div>
+  <div class="col-xs-2"><p>Hadoop, 2015.</p></div>
 </div>
 </div>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43e93200/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
index d7d3cd4..7893a17 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
@@ -53,7 +53,7 @@
 
 <div class="row">
   <hr />
-  <div class="col-xs-2"><p>Hadoop, 2014.</p></div>
+  <div class="col-xs-2"><p>Hadoop, 2015.</p></div>
 </div>
 </div>
 


[38/50] [abbrv] hadoop git commit: Revert "HDFS-6153. Document "fileId" and "childrenNum" fields in the FileStatus Json schema"

Posted by zj...@apache.org.
Revert "HDFS-6153. Document "fileId" and "childrenNum" fields in the FileStatus Json schema"

This reverts commit eb1d4c39e00e2a962e1a00192b952e1ba8bc4633.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f150002c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f150002c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f150002c

Branch: refs/heads/YARN-2928
Commit: f150002ccb34fac246adca868d24213df2274e3b
Parents: 05eac26
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Apr 16 16:54:01 2015 -0500
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:46 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt       |  3 ---
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md      | 18 ------------------
 2 files changed, 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f150002c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c0472aa..dd0979b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -2940,9 +2940,6 @@ Release 2.5.0 - 2014-08-11
 
     HDFS-4909. Avoid protocol buffer RPC namespace clashes. (cmccabe)
 
-    HDFS-6153. Document "fileId" and "childrenNum" fields in the FileStatus Json schema
-    (Akira Ajisaka via vinayakumarb)
-
     HDFS-6178. Decommission on standby NN couldn't finish. (Ming Ma via jing9)
 
     HDFS-6213. TestDataNodeConfig failing on Jenkins runs due to DN web port

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f150002c/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
index 7afb0f5..6432c57a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
@@ -426,8 +426,6 @@ See also: [`newlength`](#New_Length), [FileSystem](../../api/org/apache/hadoop/f
           {
             "accessTime"      : 0,
             "blockSize"       : 0,
-            "childrenNum"     : 1,
-            "fileId"          : 16386,
             "group"           : "supergroup",
             "length"          : 0,             //in bytes, zero for directories
             "modificationTime": 1320173277227,
@@ -461,8 +459,6 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileSt
               {
                 "accessTime"      : 1320171722771,
                 "blockSize"       : 33554432,
-                "childrenNum"     : 0,
-                "fileId"          : 16387,
                 "group"           : "supergroup",
                 "length"          : 24930,
                 "modificationTime": 1320171722771,
@@ -475,8 +471,6 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileSt
               {
                 "accessTime"      : 0,
                 "blockSize"       : 0,
-                "childrenNum"     : 2,
-                "fileId"          : 16388,
                 "group"           : "supergroup",
                 "length"          : 0,
                 "modificationTime": 1320895981256,
@@ -1323,18 +1317,6 @@ var fileStatusProperties =
       "type"       : "integer",
       "required"   : true
     },
-    "childrenNum":
-    {
-      "description": "The number of children.",
-      "type"       : "integer",
-      "required"   : true
-    },
-    "fileId":
-    {
-      "description": "The inode ID.",
-      "type"       : "integer",
-      "required"   : true
-    },
     "group":
     {
       "description": "The group owner.",


[33/50] [abbrv] hadoop git commit: YARN-3404. Display queue name on application page. Contributed by Ryu Kobayashi

Posted by zj...@apache.org.
YARN-3404. Display queue name on application page. Contributed by Ryu Kobayashi


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1aa894ea
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1aa894ea
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1aa894ea

Branch: refs/heads/YARN-2928
Commit: 1aa894ea9d3b28b582b8cf84d06b960abfa271ab
Parents: 43e9320
Author: Jian He <ji...@apache.org>
Authored: Wed Apr 15 13:52:50 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:45 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                               | 2 ++
 .../java/org/apache/hadoop/yarn/server/webapp/AppBlock.java   | 7 ++++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1aa894ea/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 565627f..8b08f98 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -190,6 +190,8 @@ Release 2.8.0 - UNRELEASED
     YARN-3394. Enrich WebApplication proxy documentation. (Naganarasimha G R
     via jianhe)
 
+    YARN-3404. Display queue name on application page. (Ryu Kobayashi via jianhe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1aa894ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index d5a3dd8..0d80339 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -157,8 +158,11 @@ public class AppBlock extends HtmlBlock {
       html.script().$type("text/javascript")._(script.toString())._();
     }
 
+    String schedulerPath = WebAppUtils.getResolvedRMWebAppURLWithScheme(conf) +
+        "/cluster/scheduler?openQueues=" + app.getQueue();
+
     ResponseInfo overviewTable = info("Application Overview")
-      ._("User:", app.getUser())
+      ._("User:", schedulerPath, app.getUser())
       ._("Name:", app.getName())
       ._("Application Type:", app.getType())
       ._("Application Tags:",
@@ -167,6 +171,7 @@ public class AppBlock extends HtmlBlock {
         "YarnApplicationState:",
         app.getAppState() == null ? UNAVAILABLE : clarifyAppState(app
           .getAppState()))
+      ._("Queue:", schedulerPath, app.getQueue())
       ._("FinalStatus Reported by AM:",
         clairfyAppFinalStatus(app.getFinalAppStatus()))
       ._("Started:", Times.format(app.getStartedTime()))


[06/50] [abbrv] hadoop git commit: HDFS-8111. NPE thrown when invalid FSImage filename given for 'hdfs oiv_legacy' cmd ( Contributed by surendra singh lilhore )

Posted by zj...@apache.org.
HDFS-8111. NPE thrown when invalid FSImage filename given for 'hdfs oiv_legacy' cmd ( Contributed by surendra singh lilhore )


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/668d1581
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/668d1581
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/668d1581

Branch: refs/heads/YARN-2928
Commit: 668d1581a3885fd0f4ebf450558dafb92ba298cb
Parents: 8f7c5de
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Apr 13 17:16:05 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:40 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                    | 3 +++
 .../hdfs/tools/offlineImageViewer/OfflineImageViewer.java      | 6 +++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/668d1581/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 134bba0..abbfe6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -482,6 +482,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7939. Two fsimage_rollback_* files are created which are not deleted
     after rollback. (J.Andreina via vinayakumarb)
 
+    HDFS-8111. NPE thrown when invalid FSImage filename given for
+    'hdfs oiv_legacy' cmd ( surendra singh lilhore via vinayakumarb )
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/668d1581/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
index 316ebca..7f81ba8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
@@ -137,7 +137,11 @@ public class OfflineImageViewer {
       done = true;
     } finally {
       if (!done) {
-        LOG.error("image loading failed at offset " + tracker.getPos());
+        if (tracker != null) {
+          LOG.error("image loading failed at offset " + tracker.getPos());
+        } else {
+          LOG.error("Failed to load image file.");
+        }
       }
       IOUtils.cleanup(LOG, in, tracker);
     }


[04/50] [abbrv] hadoop git commit: HDFS-7931. DistributedFileSystem should not look for keyProvider in cache if Encryption is disabled (asuresh)

Posted by zj...@apache.org.
HDFS-7931. DistributedFileSystem should not look for keyProvider in cache if Encryption is disabled (asuresh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f5d11720
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f5d11720
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f5d11720

Branch: refs/heads/YARN-2928
Commit: f5d1172011647ed46a1d05a85934baba2e3524bf
Parents: 305da53
Author: Arun Suresh <as...@apache.org>
Authored: Fri Apr 10 11:40:25 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:39 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java          | 4 ++++
 .../main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java  | 2 +-
 3 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5d11720/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 5d5c8a6..333a1b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -479,6 +479,9 @@ Release 2.7.1 - UNRELEASED
 
     HDFS-8081. Split getAdditionalBlock() into two methods. (shv)
 
+    HDFS-7931. DistributedFileSystem should not look for keyProvider in
+    cache if Encryption is disabled (asuresh)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5d11720/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index c85c1ec..d43e7de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -3538,6 +3538,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  public boolean isHDFSEncryptionEnabled() {
+    return conf.get(
+        DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, null) != null;
+  }
   /**
    * Returns the SaslDataTransferClient configured for this DFSClient.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5d11720/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 090d884..3edab48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2191,7 +2191,7 @@ public class DistributedFileSystem extends FileSystem {
   public Token<?>[] addDelegationTokens(
       final String renewer, Credentials credentials) throws IOException {
     Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
-    if (dfs.getKeyProvider() != null) {
+    if (dfs.isHDFSEncryptionEnabled()) {
       KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension =
           KeyProviderDelegationTokenExtension.
               createKeyProviderDelegationTokenExtension(dfs.getKeyProvider());


[44/50] [abbrv] hadoop git commit: HDFS-7891. A block placement policy with best rack failure tolerance. Contributed by Walter Su

Posted by zj...@apache.org.
HDFS-7891. A block placement policy with best rack failure tolerance.  Contributed by Walter Su


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ee5bcb12
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ee5bcb12
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ee5bcb12

Branch: refs/heads/YARN-2928
Commit: ee5bcb1215e41951507bee42b061486f2477842b
Parents: f150002
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Apr 16 18:25:53 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:47 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../BlockPlacementPolicyDefault.java            |   2 +-
 .../BlockPlacementPolicyRackFaultTolarent.java  | 154 ++++++++++++++
 ...stBlockPlacementPolicyRackFaultTolarent.java | 209 +++++++++++++++++++
 4 files changed, 367 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee5bcb12/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index dd0979b..e977e6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -323,6 +323,9 @@ Release 2.8.0 - UNRELEASED
 
   NEW FEATURES
 
+    HDFS-7891. A block placement policy with best rack failure tolerance.
+    (Walter Su via szetszwo)
+
   IMPROVEMENTS
 
     HDFS-3918. EditLogTailer shouldn't log WARN when other node

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee5bcb12/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index 09db986..c2752ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -237,7 +237,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    *         is independent of the number of chosen nodes, as it is calculated
    *         using the target number of replicas.
    */
-  private int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) {
+  protected int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) {
     int clusterSize = clusterMap.getNumOfLeaves();
     int totalNumOfReplicas = numOfChosen + numOfReplicas;
     if (totalNumOfReplicas > clusterSize) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee5bcb12/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolarent.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolarent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolarent.java
new file mode 100644
index 0000000..4dbf384
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolarent.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.net.NodeBase;
+
+import java.util.*;
+
+/**
+ * The class is responsible for choosing the desired number of targets
+ * for placing block replicas.
+ * The strategy is that it tries its best to place the replicas to most racks.
+ */
+@InterfaceAudience.Private
+public class BlockPlacementPolicyRackFaultTolarent extends BlockPlacementPolicyDefault {
+
+  @Override
+  protected int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) {
+    int clusterSize = clusterMap.getNumOfLeaves();
+    int totalNumOfReplicas = numOfChosen + numOfReplicas;
+    if (totalNumOfReplicas > clusterSize) {
+      numOfReplicas -= (totalNumOfReplicas-clusterSize);
+      totalNumOfReplicas = clusterSize;
+    }
+    // No calculation needed when there is only one rack or picking one node.
+    int numOfRacks = clusterMap.getNumOfRacks();
+    if (numOfRacks == 1 || totalNumOfReplicas <= 1) {
+      return new int[] {numOfReplicas, totalNumOfReplicas};
+    }
+    if(totalNumOfReplicas<numOfRacks){
+      return new int[] {numOfReplicas, 1};
+    }
+    int maxNodesPerRack = (totalNumOfReplicas - 1) / numOfRacks + 1;
+    return new int[] {numOfReplicas, maxNodesPerRack};
+  }
+
+  /**
+   * Choose numOfReplicas in order:
+   * 1. If total replica expected is less than numOfRacks in cluster, it choose
+   * randomly.
+   * 2. If total replica expected is bigger than numOfRacks, it choose:
+   *  2a. Fill each rack exactly (maxNodesPerRack-1) replicas.
+   *  2b. For some random racks, place one more replica to each one of them, until
+   *  numOfReplicas have been chosen. <br>
+   * In the end, the difference of the numbers of replicas for each two racks
+   * is no more than 1.
+   * Either way it always prefer local storage.
+   * @return local node of writer
+   */
+  @Override
+  protected Node chooseTargetInOrder(int numOfReplicas,
+                                 Node writer,
+                                 final Set<Node> excludedNodes,
+                                 final long blocksize,
+                                 final int maxNodesPerRack,
+                                 final List<DatanodeStorageInfo> results,
+                                 final boolean avoidStaleNodes,
+                                 final boolean newBlock,
+                                 EnumMap<StorageType, Integer> storageTypes)
+                                 throws NotEnoughReplicasException {
+    int totalReplicaExpected = results.size() + numOfReplicas;
+    int numOfRacks = clusterMap.getNumOfRacks();
+    if (totalReplicaExpected < numOfRacks ||
+        totalReplicaExpected % numOfRacks == 0) {
+      writer = chooseOnce(numOfReplicas, writer, excludedNodes, blocksize,
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
+      return writer;
+    }
+
+    assert totalReplicaExpected > (maxNodesPerRack -1) * numOfRacks;
+
+    // Calculate numOfReplicas for filling each rack exactly (maxNodesPerRack-1)
+    // replicas.
+    HashMap<String, Integer> rackCounts = new HashMap<>();
+    for (DatanodeStorageInfo dsInfo : results) {
+      String rack = dsInfo.getDatanodeDescriptor().getNetworkLocation();
+      Integer count = rackCounts.get(rack);
+      if (count != null) {
+        rackCounts.put(rack, count + 1);
+      } else {
+        rackCounts.put(rack, 1);
+      }
+    }
+    int excess = 0; // Sum of the above (maxNodesPerRack-1) part of nodes in results
+    for (int count : rackCounts.values()) {
+      if (count > maxNodesPerRack -1) {
+        excess += count - (maxNodesPerRack -1);
+      }
+    }
+    numOfReplicas = Math.min(totalReplicaExpected - results.size(),
+        (maxNodesPerRack -1) * numOfRacks - (results.size() - excess));
+
+    // Fill each rack exactly (maxNodesPerRack-1) replicas.
+    writer = chooseOnce(numOfReplicas, writer, new HashSet<>(excludedNodes),
+        blocksize, maxNodesPerRack -1, results, avoidStaleNodes, storageTypes);
+
+    for (DatanodeStorageInfo resultStorage : results) {
+      addToExcludedNodes(resultStorage.getDatanodeDescriptor(), excludedNodes);
+    }
+
+    // For some racks, place one more replica to each one of them.
+    numOfReplicas = totalReplicaExpected - results.size();
+    chooseOnce(numOfReplicas, writer, excludedNodes, blocksize,
+        maxNodesPerRack, results, avoidStaleNodes, storageTypes);
+
+    return writer;
+  }
+
+  /**
+   * Randomly choose <i>numOfReplicas</i> targets from the given <i>scope</i>.
+   * Except that 1st replica prefer local storage.
+   * @return local node of writer.
+   */
+  private Node chooseOnce(int numOfReplicas,
+                            Node writer,
+                            final Set<Node> excludedNodes,
+                            final long blocksize,
+                            final int maxNodesPerRack,
+                            final List<DatanodeStorageInfo> results,
+                            final boolean avoidStaleNodes,
+                            EnumMap<StorageType, Integer> storageTypes)
+                            throws NotEnoughReplicasException {
+    if (numOfReplicas == 0) {
+      return writer;
+    }
+    writer = chooseLocalStorage(writer, excludedNodes, blocksize,
+        maxNodesPerRack, results, avoidStaleNodes, storageTypes, true)
+        .getDatanodeDescriptor();
+    if (--numOfReplicas == 0) {
+      return writer;
+    }
+    chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize,
+        maxNodesPerRack, results, avoidStaleNodes, storageTypes);
+    return writer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee5bcb12/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockPlacementPolicyRackFaultTolarent.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockPlacementPolicyRackFaultTolarent.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockPlacementPolicyRackFaultTolarent.java
new file mode 100644
index 0000000..d86a267
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockPlacementPolicyRackFaultTolarent.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolarent;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.net.StaticMapping;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestBlockPlacementPolicyRackFaultTolarent {
+
+  private static final int DEFAULT_BLOCK_SIZE = 1024;
+  private MiniDFSCluster cluster = null;
+  private NamenodeProtocols nameNodeRpc = null;
+  private FSNamesystem namesystem = null;
+  private PermissionStatus perm = null;
+
+  @Before
+  public void setup() throws IOException {
+    StaticMapping.resetMap();
+    Configuration conf = new HdfsConfiguration();
+    final ArrayList<String> rackList = new ArrayList<String>();
+    final ArrayList<String> hostList = new ArrayList<String>();
+    for (int i = 0; i < 10; i++) {
+      for (int j = 0; j < 2; j++) {
+        rackList.add("/rack" + i);
+        hostList.add("/host" + i + j);
+      }
+    }
+    conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
+        BlockPlacementPolicyRackFaultTolarent.class,
+        BlockPlacementPolicy.class);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE / 2);
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(hostList.size())
+        .racks(rackList.toArray(new String[rackList.size()]))
+        .hosts(hostList.toArray(new String[hostList.size()]))
+        .build();
+    cluster.waitActive();
+    nameNodeRpc = cluster.getNameNodeRpc();
+    namesystem = cluster.getNamesystem();
+    perm = new PermissionStatus("TestBlockPlacementPolicyEC", null,
+        FsPermission.getDefault());
+  }
+
+  @After
+  public void teardown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testChooseTarget() throws Exception {
+    doTestChooseTargetNormalCase();
+    doTestChooseTargetSpecialCase();
+  }
+
+  private void doTestChooseTargetNormalCase() throws Exception {
+    String clientMachine = "client.foo.com";
+    short[][] testSuite = {
+        {3, 2}, {3, 7}, {3, 8}, {3, 10}, {9, 1}, {10, 1}, {10, 6}, {11, 6},
+        {11, 9}
+    };
+    // Test 5 files
+    int fileCount = 0;
+    for (int i = 0; i < 5; i++) {
+      for (short[] testCase : testSuite) {
+        short replication = testCase[0];
+        short additionalReplication = testCase[1];
+        String src = "/testfile" + (fileCount++);
+        // Create the file with client machine
+        HdfsFileStatus fileStatus = namesystem.startFile(src, perm,
+            clientMachine, clientMachine, EnumSet.of(CreateFlag.CREATE), true,
+            replication, DEFAULT_BLOCK_SIZE, null, false);
+
+        //test chooseTarget for new file
+        LocatedBlock locatedBlock = nameNodeRpc.addBlock(src, clientMachine,
+            null, null, fileStatus.getFileId(), null);
+        doTestLocatedBlock(replication, locatedBlock);
+
+        //test chooseTarget for existing file.
+        LocatedBlock additionalLocatedBlock =
+            nameNodeRpc.getAdditionalDatanode(src, fileStatus.getFileId(),
+                locatedBlock.getBlock(), locatedBlock.getLocations(),
+                locatedBlock.getStorageIDs(), new DatanodeInfo[0],
+                additionalReplication, clientMachine);
+        doTestLocatedBlock(replication + additionalReplication, additionalLocatedBlock);
+      }
+    }
+  }
+
+  /**
+   * Test more randomly. So it covers some special cases.
+   * Like when some racks already have 2 replicas, while some racks have none,
+   * we should choose the racks that have none.
+   */
+  private void doTestChooseTargetSpecialCase() throws Exception {
+    String clientMachine = "client.foo.com";
+    // Test 5 files
+    String src = "/testfile_1_";
+    // Create the file with client machine
+    HdfsFileStatus fileStatus = namesystem.startFile(src, perm,
+        clientMachine, clientMachine, EnumSet.of(CreateFlag.CREATE), true,
+        (short) 20, DEFAULT_BLOCK_SIZE, null, false);
+
+    //test chooseTarget for new file
+    LocatedBlock locatedBlock = nameNodeRpc.addBlock(src, clientMachine,
+        null, null, fileStatus.getFileId(), null);
+    doTestLocatedBlock(20, locatedBlock);
+
+    DatanodeInfo[] locs = locatedBlock.getLocations();
+    String[] storageIDs = locatedBlock.getStorageIDs();
+
+    for (int time = 0; time < 5; time++) {
+      shuffle(locs, storageIDs);
+      for (int i = 1; i < locs.length; i++) {
+        DatanodeInfo[] partLocs = new DatanodeInfo[i];
+        String[] partStorageIDs = new String[i];
+        System.arraycopy(locs, 0, partLocs, 0, i);
+        System.arraycopy(storageIDs, 0, partStorageIDs, 0, i);
+        for (int j = 1; j < 20 - i; j++) {
+          LocatedBlock additionalLocatedBlock =
+              nameNodeRpc.getAdditionalDatanode(src, fileStatus.getFileId(),
+                  locatedBlock.getBlock(), partLocs,
+                  partStorageIDs, new DatanodeInfo[0],
+                  j, clientMachine);
+          doTestLocatedBlock(i + j, additionalLocatedBlock);
+        }
+      }
+    }
+  }
+
+  private void shuffle(DatanodeInfo[] locs, String[] storageIDs) {
+    int length = locs.length;
+    Object[][] pairs = new Object[length][];
+    for (int i = 0; i < length; i++) {
+      pairs[i] = new Object[]{locs[i], storageIDs[i]};
+    }
+    DFSUtil.shuffle(pairs);
+    for (int i = 0; i < length; i++) {
+      locs[i] = (DatanodeInfo) pairs[i][0];
+      storageIDs[i] = (String) pairs[i][1];
+    }
+  }
+
+  private void doTestLocatedBlock(int replication, LocatedBlock locatedBlock) {
+    assertEquals(replication, locatedBlock.getLocations().length);
+
+    HashMap<String, Integer> racksCount = new HashMap<String, Integer>();
+    for (DatanodeInfo node :
+        locatedBlock.getLocations()) {
+      addToRacksCount(node.getNetworkLocation(), racksCount);
+    }
+
+    int minCount = Integer.MAX_VALUE;
+    int maxCount = Integer.MIN_VALUE;
+    for (Integer rackCount : racksCount.values()) {
+      minCount = Math.min(minCount, rackCount);
+      maxCount = Math.max(maxCount, rackCount);
+    }
+    assertTrue(maxCount - minCount <= 1);
+  }
+
+  private void addToRacksCount(String rack, HashMap<String, Integer> racksCount) {
+    Integer count = racksCount.get(rack);
+    if (count == null) {
+      racksCount.put(rack, 1);
+    } else {
+      racksCount.put(rack, count + 1);
+    }
+  }
+}


[23/50] [abbrv] hadoop git commit: HADOOP-11819. HttpServerFunctionalTest#prepareTestWebapp should create web app directory if it does not exist. (Contributed by Rohith)

Posted by zj...@apache.org.
HADOOP-11819. HttpServerFunctionalTest#prepareTestWebapp should create web app directory if it does not exist. (Contributed by Rohith)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ef4baa35
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ef4baa35
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ef4baa35

Branch: refs/heads/YARN-2928
Commit: ef4baa358ac9505380d7e4fefc30a58583628f78
Parents: 737b437
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 14 11:58:12 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:43 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt         |  3 +++
 .../apache/hadoop/http/HttpServerFunctionalTest.java    | 12 +++++++-----
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef4baa35/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 68913bc..bb2eca5 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -494,6 +494,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-7713. dfs -count -q should label output column (Jonathan Allen
     via aw)
 
+    HADOOP-11819. HttpServerFunctionalTest#prepareTestWebapp should create web
+    app directory if it does not exist. (Rohith via vinayakumarb)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef4baa35/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
index 4a4de41..faa27a7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
@@ -123,11 +123,13 @@ public class HttpServerFunctionalTest extends Assert {
     File testWebappDir = new File(webapps +
         File.separatorChar + TEST);
     try {
-    if (!testWebappDir.exists()) {
-      fail("Test webapp dir " + testWebappDir.getCanonicalPath() + " missing");
-    }
-    }
-    catch (IOException e) {
+      if (!testWebappDir.exists()) {
+        if (!testWebappDir.mkdirs()) {
+          fail("Test webapp dir " + testWebappDir.getCanonicalPath()
+              + " can not be created");
+        }
+      }
+    } catch (IOException e) {
     }
   }
 


[31/50] [abbrv] hadoop git commit: YARN-3318. Create Initial OrderingPolicy Framework and FifoOrderingPolicy. (Craig Welch via wangda)

Posted by zj...@apache.org.
YARN-3318. Create Initial OrderingPolicy Framework and FifoOrderingPolicy. (Craig Welch via wangda)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e2cd8ed0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e2cd8ed0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e2cd8ed0

Branch: refs/heads/YARN-2928
Commit: e2cd8ed076c001de83b6f271792855bf682bdcca
Parents: 3980eec
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Apr 15 09:56:32 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:44 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../dev-support/findbugs-exclude.xml            |   4 +
 .../scheduler/ResourceUsage.java                |  55 ++++++++-
 .../AbstractComparatorOrderingPolicy.java       | 119 +++++++++++++++++++
 .../scheduler/policy/FifoComparator.java        |  37 ++++++
 .../scheduler/policy/FifoOrderingPolicy.java    |  54 +++++++++
 .../scheduler/policy/OrderingPolicy.java        | 109 +++++++++++++++++
 .../scheduler/policy/SchedulableEntity.java     |  51 ++++++++
 .../scheduler/policy/MockSchedulableEntity.java |  78 ++++++++++++
 .../policy/TestFifoOrderingPolicy.java          |  83 +++++++++++++
 10 files changed, 592 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f416ab2..e4755f6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -134,6 +134,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3361. CapacityScheduler side changes to support non-exclusive node
     labels. (Wangda Tan via jianhe)
 
+    YARN-3318. Create Initial OrderingPolicy Framework and FifoOrderingPolicy.
+    (Craig Welch via wangda)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 375d19c..4b01a4d 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -141,6 +141,10 @@
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.resource.Priority$Comparator" />
     <Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE" />
   </Match>
+    <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoComparator" />
+    <Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE" />
+  </Match>
   <!-- Ignore some irrelevant class name warning -->
   <Match>
     <Class name="org.apache.hadoop.yarn.api.records.SerializedException" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
index 5169b78..2f7e19d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
@@ -57,7 +57,10 @@ public class ResourceUsage {
 
   // Usage enum here to make implement cleaner
   private enum ResourceType {
-    USED(0), PENDING(1), AMUSED(2), RESERVED(3);
+    //CACHED_USED and CACHED_PENDING may be read by anyone, but must only
+    //be written by ordering policies
+    USED(0), PENDING(1), AMUSED(2), RESERVED(3), CACHED_USED(4),
+      CACHED_PENDING(5);
 
     private int idx;
 
@@ -102,6 +105,14 @@ public class ResourceUsage {
   public Resource getUsed(String label) {
     return _get(label, ResourceType.USED);
   }
+  
+  public Resource getCachedUsed(String label) {
+    return _get(label, ResourceType.CACHED_USED);
+  }
+  
+  public Resource getCachedPending(String label) {
+    return _get(label, ResourceType.CACHED_PENDING);
+  }
 
   public void incUsed(String label, Resource res) {
     _inc(label, ResourceType.USED, res);
@@ -137,6 +148,14 @@ public class ResourceUsage {
   public void setUsed(String label, Resource res) {
     _set(label, ResourceType.USED, res);
   }
+  
+  public void setCachedUsed(String label, Resource res) {
+    _set(label, ResourceType.CACHED_USED, res);
+  }
+  
+  public void setCachedPending(String label, Resource res) {
+    _set(label, ResourceType.CACHED_PENDING, res);
+  }
 
   /*
    * Pending
@@ -266,6 +285,28 @@ public class ResourceUsage {
       readLock.unlock();
     }
   }
+  
+  private Resource _getAll(ResourceType type) {
+    try {
+      readLock.lock();
+      Resource allOfType = Resources.createResource(0);
+      for (Map.Entry<String, UsageByLabel> usageEntry : usages.entrySet()) {
+        //all usages types are initialized
+        Resources.addTo(allOfType, usageEntry.getValue().resArr[type.idx]);
+      }
+      return allOfType;
+    } finally {
+      readLock.unlock();
+    }
+  }
+  
+  public Resource getAllPending() {
+    return _getAll(ResourceType.PENDING);
+  }
+  
+  public Resource getAllUsed() {
+    return _getAll(ResourceType.USED);
+  }
 
   private UsageByLabel getAndAddIfMissing(String label) {
     if (label == null) {
@@ -309,6 +350,18 @@ public class ResourceUsage {
       writeLock.unlock();
     }
   }
+
+  public Resource getCachedDemand(String label) {
+    try {
+      readLock.lock();
+      Resource demand = Resources.createResource(0);
+      Resources.addTo(demand, getCachedUsed(label));
+      Resources.addTo(demand, getCachedPending(label));
+      return demand;
+    } finally {
+      readLock.unlock();
+    }
+  }
   
   @Override
   public String toString() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java
new file mode 100644
index 0000000..5b32b3d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
+
+import java.util.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import com.google.common.annotations.VisibleForTesting;
+
+
+/**
+ * An OrderingPolicy which can serve as a baseclass for policies which can be
+ * expressed in terms of comparators
+ */
+public abstract class AbstractComparatorOrderingPolicy<S extends SchedulableEntity> implements OrderingPolicy<S> {
+  
+  private static final Log LOG = LogFactory.getLog(OrderingPolicy.class);
+                                            
+  protected TreeSet<S> schedulableEntities;
+  protected Comparator<SchedulableEntity> comparator;
+  
+  public AbstractComparatorOrderingPolicy() { }
+  
+  @Override
+  public Collection<S> getSchedulableEntities() {
+    return schedulableEntities;
+  }
+  
+  @Override
+  public Iterator<S> getAssignmentIterator() {
+    return schedulableEntities.iterator();
+  }
+  
+  @Override
+  public Iterator<S> getPreemptionIterator() {
+    return schedulableEntities.descendingIterator();
+  }
+  
+  public static void updateSchedulingResourceUsage(ResourceUsage ru) {
+    ru.setCachedUsed(CommonNodeLabelsManager.ANY, ru.getAllUsed());
+    ru.setCachedPending(CommonNodeLabelsManager.ANY, ru.getAllPending());
+  }
+  
+  protected void reorderSchedulableEntity(S schedulableEntity) {
+    //remove, update comparable data, and reinsert to update position in order
+    schedulableEntities.remove(schedulableEntity);
+    updateSchedulingResourceUsage(
+      schedulableEntity.getSchedulingResourceUsage());
+    schedulableEntities.add(schedulableEntity);
+  }
+  
+  public void setComparator(Comparator<SchedulableEntity> comparator) {
+    this.comparator = comparator;
+    TreeSet<S> schedulableEntities = new TreeSet<S>(comparator);
+    if (this.schedulableEntities != null) {
+      schedulableEntities.addAll(this.schedulableEntities);
+    }
+    this.schedulableEntities = schedulableEntities;
+  }
+  
+  @VisibleForTesting
+  public Comparator<SchedulableEntity> getComparator() {
+    return comparator; 
+  }
+  
+  @Override
+  public void addSchedulableEntity(S s) {
+    schedulableEntities.add(s); 
+  }
+  
+  @Override
+  public boolean removeSchedulableEntity(S s) {
+    return schedulableEntities.remove(s); 
+  }
+  
+  @Override
+  public void addAllSchedulableEntities(Collection<S> sc) {
+    schedulableEntities.addAll(sc);
+  }
+  
+  @Override
+  public int getNumSchedulableEntities() {
+    return schedulableEntities.size(); 
+  }
+  
+  @Override
+  public abstract void configure(String conf);
+  
+  @Override
+  public abstract void containerAllocated(S schedulableEntity, 
+    RMContainer r);
+  
+  @Override
+  public abstract void containerReleased(S schedulableEntity, 
+    RMContainer r);
+  
+  @Override
+  public abstract String getStatusMessage();
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoComparator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoComparator.java
new file mode 100644
index 0000000..b92b264
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoComparator.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
+
+import java.util.*;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
+
+/**
+ * A Comparator which orders SchedulableEntities by input order
+ */
+public class FifoComparator 
+    implements Comparator<SchedulableEntity> {
+      
+    @Override
+    public int compare(SchedulableEntity r1, SchedulableEntity r2) {
+      int res = r1.compareInputOrderTo(r2);
+      return res;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicy.java
new file mode 100644
index 0000000..8a2f5ba
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicy.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.yarn.server.resourcemanager.scheduler.policy;
+
+import java.util.*;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
+
+/**
+ * An OrderingPolicy which orders SchedulableEntities by input order
+ */
+public class FifoOrderingPolicy<S extends SchedulableEntity> extends AbstractComparatorOrderingPolicy<S> {
+  
+  public FifoOrderingPolicy() {
+    setComparator(new FifoComparator());
+  }
+  
+  @Override
+  public void configure(String conf) {
+    
+  }
+  
+  @Override
+  public void containerAllocated(S schedulableEntity, 
+    RMContainer r) {
+    }
+
+  @Override
+  public void containerReleased(S schedulableEntity, 
+    RMContainer r) {
+    }
+  
+  @Override
+  public String getStatusMessage() {
+    return "FifoOrderingPolicy";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java
new file mode 100644
index 0000000..f907cea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
+
+import java.util.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
+
+
+/**
+ * OrderingPolicy is used by the scheduler to order SchedulableEntities for
+ * container assignment and preemption
+ */
+public interface OrderingPolicy<S extends SchedulableEntity> {
+  /*
+   * Note: OrderingPolicy depends upon external
+   * synchronization of all use of the SchedulableEntity Collection and
+   * Iterators for correctness and to avoid concurrent modification issues
+   */
+  
+  /**
+   * Get the collection of SchedulableEntities which are managed by this
+   * OrderingPolicy - should include processes returned by the Assignment and
+   * Preemption iterator with no guarantees regarding order
+   */
+  public Collection<S> getSchedulableEntities();
+  
+  /**
+   * Return an iterator over the collection of SchedulableEntities which orders
+   * them for container assignment
+   */
+  public Iterator<S> getAssignmentIterator();
+  
+  /**
+   * Return an iterator over the collection of SchedulableEntities which orders
+   * them for preemption
+   */
+  public Iterator<S> getPreemptionIterator();
+  
+  /**
+   * Add a SchedulableEntity to be managed for allocation and preemption 
+   * ordering
+   */
+  public void addSchedulableEntity(S s);
+  
+  /**
+   * Remove a SchedulableEntity from management for allocation and preemption 
+   * ordering
+   */
+  public boolean removeSchedulableEntity(S s);
+  
+  /**
+   * Add a collection of SchedulableEntities to be managed for allocation 
+   * and preemption ordering
+   */
+  public void addAllSchedulableEntities(Collection<S> sc);
+  
+  /**
+   * Get the number of SchedulableEntities managed for allocation and
+   * preemption ordering
+   */
+  public int getNumSchedulableEntities();
+  
+  /**
+   * Provides configuration information for the policy from the scheduler
+   * configuration
+   */
+  public void configure(String conf);
+  
+  /**
+   * The passed SchedulableEntity has been allocated the passed Container,
+   * take appropriate action (depending on comparator, a reordering of the
+   * SchedulableEntity may be required)
+   */
+  public void containerAllocated(S schedulableEntity, 
+    RMContainer r);
+  
+  /**
+   * The passed SchedulableEntity has released the passed Container,
+   * take appropriate action (depending on comparator, a reordering of the
+   * SchedulableEntity may be required)
+   */
+  public void containerReleased(S schedulableEntity, 
+    RMContainer r);
+  
+  /**
+   * Display information regarding configuration & status
+   */
+  public String getStatusMessage();
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/SchedulableEntity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/SchedulableEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/SchedulableEntity.java
new file mode 100644
index 0000000..9b9d73d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/SchedulableEntity.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
+
+import java.util.*;
+
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
+
+
+/**
+ * A SchedulableEntity is a process to be scheduled, 
+ * for example, an application / application attempt
+ */
+public interface SchedulableEntity {
+  
+  /**
+   * Id - each entity must have a unique id
+   */
+  public String getId();
+  
+  /**
+   * Compare the passed SchedulableEntity to this one for input order.
+   * Input order is implementation defined and should reflect the 
+   * correct ordering for first-in first-out processing
+   */
+  public int compareInputOrderTo(SchedulableEntity other);
+  
+  /**
+   * View of Resources wanted and consumed by the entity
+   */
+  public ResourceUsage getSchedulingResourceUsage();
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/MockSchedulableEntity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/MockSchedulableEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/MockSchedulableEntity.java
new file mode 100644
index 0000000..fe8c455
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/MockSchedulableEntity.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
+
+import java.util.*;
+
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+
+
+public class MockSchedulableEntity implements SchedulableEntity {
+  
+  private String id;
+  private long serial = 0;
+  
+  public MockSchedulableEntity() { }
+  
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  public String getId() {
+    return id;
+  }
+  
+  public void setSerial(long serial) {
+    this.serial = serial;
+  }
+  
+  public long getSerial() {
+    return serial; 
+  }
+  
+  public void setUsed(Resource value) {
+    schedulingResourceUsage.setUsed(CommonNodeLabelsManager.ANY, value);
+  }
+  
+  public void setPending(Resource value) {
+    schedulingResourceUsage.setPending(CommonNodeLabelsManager.ANY, value);
+  }
+  
+  private ResourceUsage schedulingResourceUsage = new ResourceUsage();
+  
+  @Override
+  public ResourceUsage getSchedulingResourceUsage() {
+    return schedulingResourceUsage;
+  }
+  
+  @Override
+  public int compareInputOrderTo(SchedulableEntity other) {
+    if (other instanceof MockSchedulableEntity) {
+      MockSchedulableEntity r2 = (MockSchedulableEntity) other;
+      int res = (int) Math.signum(getSerial() - r2.getSerial());
+      return res;
+    }
+    return 1;//let other types go before this, if any
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2cd8ed0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java
new file mode 100644
index 0000000..e05ef26
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
+
+import java.util.*;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+
+public class TestFifoOrderingPolicy {
+  
+  @Test
+  public void testFifoOrderingPolicy() {
+    FifoOrderingPolicy<MockSchedulableEntity> policy = 
+      new FifoOrderingPolicy<MockSchedulableEntity>();
+    MockSchedulableEntity r1 = new MockSchedulableEntity();
+    MockSchedulableEntity r2 = new MockSchedulableEntity();
+    
+    Assert.assertEquals(policy.getComparator().compare(r1, r2), 0);
+    
+    r1.setSerial(1);
+    Assert.assertEquals(policy.getComparator().compare(r1, r2), 1);
+    
+    r2.setSerial(2);
+    Assert.assertEquals(policy.getComparator().compare(r1, r2), -1);
+  }
+  
+  @Test
+  public void testIterators() {
+    OrderingPolicy<MockSchedulableEntity> schedOrder =
+     new FifoOrderingPolicy<MockSchedulableEntity>();
+    
+    MockSchedulableEntity msp1 = new MockSchedulableEntity();
+    MockSchedulableEntity msp2 = new MockSchedulableEntity();
+    MockSchedulableEntity msp3 = new MockSchedulableEntity();
+    
+    msp1.setSerial(3);
+    msp2.setSerial(2);
+    msp3.setSerial(1);
+    
+    schedOrder.addSchedulableEntity(msp1);
+    schedOrder.addSchedulableEntity(msp2);
+    schedOrder.addSchedulableEntity(msp3);
+    
+    //Assignment, oldest to youngest
+    checkSerials(schedOrder.getAssignmentIterator(), new long[]{1, 2, 3});
+    
+    //Preemption, youngest to oldest
+    checkSerials(schedOrder.getPreemptionIterator(), new long[]{3, 2, 1});
+  }
+  
+  public void checkSerials(Iterator<MockSchedulableEntity> si, 
+      long[] serials) {
+    for (int i = 0;i < serials.length;i++) {
+      Assert.assertEquals(si.next().getSerial(), 
+        serials[i]);
+    }
+  }
+  
+}


[22/50] [abbrv] hadoop git commit: HDFS-8055. NullPointerException when topology script is missing. Contributed by Anu Engineer.

Posted by zj...@apache.org.
HDFS-8055. NullPointerException when topology script is missing. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/793bc2fb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/793bc2fb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/793bc2fb

Branch: refs/heads/YARN-2928
Commit: 793bc2fb4d193872480a4e74fcd1fe61d8a73878
Parents: 904cf42
Author: cnauroth <cn...@apache.org>
Authored: Tue Apr 14 10:19:30 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Apr 17 15:29:43 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../server/blockmanagement/DatanodeManager.java | 14 ++-
 .../blockmanagement/TestDatanodeManager.java    | 94 +++++++++++++++-----
 .../test/resources/topology-broken-script.cmd   | 22 +++++
 .../test/resources/topology-broken-script.sh    | 23 +++++
 .../src/test/resources/topology-script.cmd      | 18 ++++
 .../src/test/resources/topology-script.sh       | 21 +++++
 7 files changed, 172 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/793bc2fb/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index f9b27da..274e9cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -497,6 +497,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-6666. Abort NameNode and DataNode startup if security is enabled but
     block access token is not enabled. (Vijay Bhat via cnauroth)
 
+    HDFS-8055. NullPointerException when topology script is missing.
+    (Anu Engineer via cnauroth)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/793bc2fb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index f68c4fd..65c5747 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -372,9 +372,17 @@ public class DatanodeManager {
     if (client == null) {
       List<String> hosts = new ArrayList<String> (1);
       hosts.add(targethost);
-      String rName = dnsToSwitchMapping.resolve(hosts).get(0);
-      if (rName != null)
-        client = new NodeBase(rName + NodeBase.PATH_SEPARATOR_STR + targethost);
+      List<String> resolvedHosts = dnsToSwitchMapping.resolve(hosts);
+      if (resolvedHosts != null && !resolvedHosts.isEmpty()) {
+        String rName = resolvedHosts.get(0);
+        if (rName != null) {
+          client = new NodeBase(rName + NodeBase.PATH_SEPARATOR_STR +
+            targethost);
+        }
+      } else {
+        LOG.error("Node Resolution failed. Please make sure that rack " +
+          "awareness scripts are functional.");
+      }
     }
     
     Comparator<DatanodeInfo> comparator = avoidStaleDataNodesForRead ?

http://git-wip-us.apache.org/repos/asf/hadoop/blob/793bc2fb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
index a4a6263..bf167a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
@@ -19,6 +19,10 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -31,6 +35,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -40,10 +45,10 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.util.Shell;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
-
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.*;
 
@@ -224,26 +229,74 @@ public class TestDatanodeManager {
    * with the storage ids and storage types.
    */
   @Test
-  public void testSortLocatedBlocks() throws IOException {
+  public void testSortLocatedBlocks() throws IOException, URISyntaxException {
+    HelperFunction(null);
+  }
+
+  /**
+   * Execute a functional topology script and make sure that helper
+   * function works correctly
+   *
+   * @throws IOException
+   * @throws URISyntaxException
+   */
+  @Test
+  public void testgoodScript() throws IOException, URISyntaxException {
+    HelperFunction("/" + Shell.appendScriptExtension("topology-script"));
+  }
+
+
+  /**
+   * Run a broken script and verify that helper function is able to
+   * ignore the broken script and work correctly
+   *
+   * @throws IOException
+   * @throws URISyntaxException
+   */
+  @Test
+  public void testBadScript() throws IOException, URISyntaxException {
+    HelperFunction("/"+ Shell.appendScriptExtension("topology-broken-script"));
+  }
+
+
+  /**
+   * Helper function that tests the DatanodeManagers SortedBlock function
+   * we invoke this function with and without topology scripts
+   *
+   * @param scriptFileName - Script Name or null
+   *
+   * @throws URISyntaxException
+   * @throws IOException
+   */
+  public void HelperFunction(String scriptFileName)
+    throws URISyntaxException, IOException {
     // create the DatanodeManager which will be tested
+    Configuration conf = new Configuration();
     FSNamesystem fsn = Mockito.mock(FSNamesystem.class);
     Mockito.when(fsn.hasWriteLock()).thenReturn(true);
+    if (scriptFileName != null && !scriptFileName.isEmpty()) {
+      URL shellScript = getClass().getResource(scriptFileName);
+      Path resourcePath = Paths.get(shellScript.toURI());
+      FileUtil.setExecutable(resourcePath.toFile(), true);
+      conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY,
+        resourcePath.toString());
+    }
     DatanodeManager dm = new DatanodeManager(Mockito.mock(BlockManager.class),
-        fsn, new Configuration());
+      fsn, conf);
 
     // register 5 datanodes, each with different storage ID and type
     DatanodeInfo[] locs = new DatanodeInfo[5];
     String[] storageIDs = new String[5];
     StorageType[] storageTypes = new StorageType[]{
-        StorageType.ARCHIVE,
-        StorageType.DEFAULT,
-        StorageType.DISK,
-        StorageType.RAM_DISK,
-        StorageType.SSD
+      StorageType.ARCHIVE,
+      StorageType.DEFAULT,
+      StorageType.DISK,
+      StorageType.RAM_DISK,
+      StorageType.SSD
     };
-    for(int i = 0; i < 5; i++) {
+    for (int i = 0; i < 5; i++) {
       // register new datanode
-      String uuid = "UUID-"+i;
+      String uuid = "UUID-" + i;
       String ip = "IP-" + i;
       DatanodeRegistration dr = Mockito.mock(DatanodeRegistration.class);
       Mockito.when(dr.getDatanodeUuid()).thenReturn(uuid);
@@ -255,7 +308,7 @@ public class TestDatanodeManager {
 
       // get location and storage information
       locs[i] = dm.getDatanode(uuid);
-      storageIDs[i] = "storageID-"+i;
+      storageIDs[i] = "storageID-" + i;
     }
 
     // set first 2 locations as decomissioned
@@ -280,18 +333,19 @@ public class TestDatanodeManager {
     assertThat(sortedLocs.length, is(5));
     assertThat(storageIDs.length, is(5));
     assertThat(storageTypes.length, is(5));
-    for(int i = 0; i < sortedLocs.length; i++) {
-      assertThat(((DatanodeInfoWithStorage)sortedLocs[i]).getStorageID(), is(storageIDs[i]));
-      assertThat(((DatanodeInfoWithStorage)sortedLocs[i]).getStorageType(), is(storageTypes[i]));
+    for (int i = 0; i < sortedLocs.length; i++) {
+      assertThat(((DatanodeInfoWithStorage) sortedLocs[i]).getStorageID(),
+        is(storageIDs[i]));
+      assertThat(((DatanodeInfoWithStorage) sortedLocs[i]).getStorageType(),
+        is(storageTypes[i]));
     }
-
     // Ensure the local node is first.
     assertThat(sortedLocs[0].getIpAddr(), is(targetIp));
-
     // Ensure the two decommissioned DNs were moved to the end.
-    assertThat(sortedLocs[sortedLocs.length-1].getAdminState(),
-        is(DatanodeInfo.AdminStates.DECOMMISSIONED));
-    assertThat(sortedLocs[sortedLocs.length-2].getAdminState(),
-        is(DatanodeInfo.AdminStates.DECOMMISSIONED));
+    assertThat(sortedLocs[sortedLocs.length - 1].getAdminState(),
+      is(DatanodeInfo.AdminStates.DECOMMISSIONED));
+    assertThat(sortedLocs[sortedLocs.length - 2].getAdminState(),
+      is(DatanodeInfo.AdminStates.DECOMMISSIONED));
   }
 }
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/793bc2fb/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-broken-script.cmd
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-broken-script.cmd b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-broken-script.cmd
new file mode 100644
index 0000000..ec4f4ca5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-broken-script.cmd
@@ -0,0 +1,22 @@
+@echo off
+@rem Licensed to the Apache Software Foundation (ASF) under one or more
+@rem contributor license agreements.  See the NOTICE file distributed with
+@rem this work for additional information regarding copyright ownership.
+@rem The ASF licenses this file to You under the Apache License, Version 2.0
+@rem (the "License"); you may not use this file except in compliance with
+@rem the License.  You may obtain a copy of the License at
+@rem
+@rem     http://www.apache.org/licenses/LICENSE-2.0
+@rem
+@rem Unless required by applicable law or agreed to in writing, software
+@rem distributed under the License is distributed on an "AS IS" BASIS,
+@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@rem See the License for the specific language governing permissions and
+@rem limitations under the License.
+@rem
+
+@rem yes, this is a broken script, please don't fix this.
+@rem This is used in a test case to verify that we can handle broken
+@rem topology scripts.
+
+exit 1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/793bc2fb/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-broken-script.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-broken-script.sh b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-broken-script.sh
new file mode 100644
index 0000000..8e5cf00
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-broken-script.sh
@@ -0,0 +1,23 @@
+#!/usr/bin/env bash
+# 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.
+
+## yes, this is a broken script, please don't fix this.
+## This is used in a test case to verify that we can handle broken
+## topology scripts.
+
+exit 1
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/793bc2fb/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-script.cmd
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-script.cmd b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-script.cmd
new file mode 100644
index 0000000..145df47
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-script.cmd
@@ -0,0 +1,18 @@
+@echo off
+@rem Licensed to the Apache Software Foundation (ASF) under one or more
+@rem contributor license agreements.  See the NOTICE file distributed with
+@rem this work for additional information regarding copyright ownership.
+@rem The ASF licenses this file to You under the Apache License, Version 2.0
+@rem (the "License"); you may not use this file except in compliance with
+@rem the License.  You may obtain a copy of the License at
+@rem
+@rem     http://www.apache.org/licenses/LICENSE-2.0
+@rem
+@rem Unless required by applicable law or agreed to in writing, software
+@rem distributed under the License is distributed on an "AS IS" BASIS,
+@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@rem See the License for the specific language governing permissions and
+@rem limitations under the License.
+@rem
+
+for /F "delims=- tokens=2" %%A in ("%1") do echo /rackID-%%A

http://git-wip-us.apache.org/repos/asf/hadoop/blob/793bc2fb/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-script.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-script.sh b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-script.sh
new file mode 100644
index 0000000..2a308e7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/topology-script.sh
@@ -0,0 +1,21 @@
+#!/usr/bin/env bash
+
+# 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.
+
+
+echo $1 | awk -F'-' '{printf("/rackID-%s",$2)}'
+